From 89c98a4c7068734e322d335cb7c9f22379ff00e8 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 13 Jun 2020 12:20:29 -0500 Subject: [PATCH 001/384] [SPARK-31944] Add instance weight support in LinearRegressionSummary ### What changes were proposed in this pull request? Add instance weight support in LinearRegressionSummary ### Why are the changes needed? LinearRegression and RegressionMetrics support instance weight. We should support instance weight in LinearRegressionSummary too. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? add new test Closes #28772 from huaxingao/lir_weight_summary. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../ml/regression/LinearRegression.scala | 39 +++++++----------- .../ml/regression/LinearRegressionSuite.scala | 41 +++++++++++++++++++ 2 files changed, 56 insertions(+), 24 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index bcf9b7c0426cd..8b6ede3bb362c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -961,21 +961,27 @@ class LinearRegressionSummary private[regression] ( private val privateModel: LinearRegressionModel, private val diagInvAtWA: Array[Double]) extends Serializable { - @transient private val metrics = new RegressionMetrics( - predictions - .select(col(predictionCol), col(labelCol).cast(DoubleType)) - .rdd - .map { case Row(pred: Double, label: Double) => (pred, label) }, - !privateModel.getFitIntercept) + @transient private val metrics = { + val weightCol = + if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { + lit(1.0) + } else { + col(privateModel.getWeightCol).cast(DoubleType) + } + + new RegressionMetrics( + predictions + .select(col(predictionCol), col(labelCol).cast(DoubleType), weightCol) + .rdd + .map { case Row(pred: Double, label: Double, weight: Double) => (pred, label, weight) }, + !privateModel.getFitIntercept) + } /** * Returns the explained variance regression score. * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) * Reference: * Wikipedia explain variation - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. */ @Since("1.5.0") val explainedVariance: Double = metrics.explainedVariance @@ -983,9 +989,6 @@ class LinearRegressionSummary private[regression] ( /** * Returns the mean absolute error, which is a risk function corresponding to the * expected value of the absolute error loss or l1-norm loss. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. */ @Since("1.5.0") val meanAbsoluteError: Double = metrics.meanAbsoluteError @@ -993,9 +996,6 @@ class LinearRegressionSummary private[regression] ( /** * Returns the mean squared error, which is a risk function corresponding to the * expected value of the squared error loss or quadratic loss. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. */ @Since("1.5.0") val meanSquaredError: Double = metrics.meanSquaredError @@ -1003,9 +1003,6 @@ class LinearRegressionSummary private[regression] ( /** * Returns the root mean squared error, which is defined as the square root of * the mean squared error. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. */ @Since("1.5.0") val rootMeanSquaredError: Double = metrics.rootMeanSquaredError @@ -1014,9 +1011,6 @@ class LinearRegressionSummary private[regression] ( * Returns R^2^, the coefficient of determination. * Reference: * Wikipedia coefficient of determination - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. */ @Since("1.5.0") val r2: Double = metrics.r2 @@ -1025,9 +1019,6 @@ class LinearRegressionSummary private[regression] ( * Returns Adjusted R^2^, the adjusted coefficient of determination. * Reference: * Wikipedia coefficient of determination - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. */ @Since("2.3.0") val r2adj: Double = { diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index df9a66b49fe48..c4a94ff2d6f44 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.LinearDataGenerator import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions.lit class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTest { @@ -899,6 +900,46 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe } } + test("linear regression model training summary with weighted samples") { + Seq("auto", "l-bfgs", "normal").foreach { solver => + val trainer1 = new LinearRegression().setSolver(solver) + val trainer2 = new LinearRegression().setSolver(solver).setWeightCol("weight") + + Seq(0.25, 1.0, 10.0, 50.00).foreach { w => + val model1 = trainer1.fit(datasetWithDenseFeature) + val model2 = trainer2.fit(datasetWithDenseFeature.withColumn("weight", lit(w))) + assert(model1.summary.explainedVariance ~== model2.summary.explainedVariance relTol 1e-6) + assert(model1.summary.meanAbsoluteError ~== model2.summary.meanAbsoluteError relTol 1e-6) + assert(model1.summary.meanSquaredError ~== model2.summary.meanSquaredError relTol 1e-6) + assert(model1.summary.rootMeanSquaredError ~== + model2.summary.rootMeanSquaredError relTol 1e-6) + assert(model1.summary.r2 ~== model2.summary.r2 relTol 1e-6) + assert(model1.summary.r2adj ~== model2.summary.r2adj relTol 1e-6) + } + } + } + + test("linear regression model testset evaluation summary with weighted samples") { + Seq("auto", "l-bfgs", "normal").foreach { solver => + val trainer1 = new LinearRegression().setSolver(solver) + val trainer2 = new LinearRegression().setSolver(solver).setWeightCol("weight") + + Seq(0.25, 1.0, 10.0, 50.00).foreach { w => + val model1 = trainer1.fit(datasetWithDenseFeature) + val model2 = trainer2.fit(datasetWithDenseFeature.withColumn("weight", lit(w))) + val testSummary1 = model1.evaluate(datasetWithDenseFeature) + val testSummary2 = model2.evaluate(datasetWithDenseFeature.withColumn("weight", lit(w))) + assert(testSummary1.explainedVariance ~== testSummary2.explainedVariance relTol 1e-6) + assert(testSummary1.meanAbsoluteError ~== testSummary2.meanAbsoluteError relTol 1e-6) + assert(testSummary1.meanSquaredError ~== testSummary2.meanSquaredError relTol 1e-6) + assert(testSummary1.rootMeanSquaredError ~== + testSummary2.rootMeanSquaredError relTol 1e-6) + assert(testSummary1.r2 ~== testSummary2.r2 relTol 1e-6) + assert(testSummary1.r2adj ~== testSummary2.r2adj relTol 1e-6) + } + } + } + test("linear regression with weighted samples") { val sqlContext = spark.sqlContext import sqlContext.implicits._ From 610acb2fe4e2d8f76c4a863e3872d94299b88c4c Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 13 Jun 2020 19:04:33 -0700 Subject: [PATCH 002/384] [SPARK-31644][BUILD][FOLLOWUP] Make Spark's guava version configurable from the command line for sbt ### What changes were proposed in this pull request? This PR proposes to support guava version configurable from command line for sbt. ### Why are the changes needed? #28455 added the configurability for Maven but not for sbt. sbt is usually faster than Maven so it's useful for developers. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I confirmed the guava version is changed with the following commands. ``` $ build/sbt "inspect tree clean" | grep guava [info] +-spark/*:dependencyOverrides = Set(com.google.guava:guava:14.0.1, xerces:xercesImpl:2.12.0, jline:jline:2.14.6, org.apache.avro:avro:1.8.2) ``` ``` $ build/sbt -Dguava.version=25.0-jre "inspect tree clean" | grep guava [info] +-spark/*:dependencyOverrides = Set(com.google.guava:guava:25.0-jre, xerces:xercesImpl:2.12.0, jline:jline:2.14.6, org.apache.avro:avro:1.8.2) ``` Closes #28822 from sarutak/guava-version-for-sbt. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index eb12f2f1f6ab7..0035f1d95a90d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -621,8 +621,9 @@ object KubernetesIntegrationTests { * Overrides to work around sbt's dependency resolution being different from Maven's. */ object DependencyOverrides { + lazy val guavaVersion = sys.props.get("guava.version").getOrElse("14.0.1") lazy val settings = Seq( - dependencyOverrides += "com.google.guava" % "guava" % "14.0.1", + dependencyOverrides += "com.google.guava" % "guava" % guavaVersion, dependencyOverrides += "xerces" % "xercesImpl" % "2.12.0", dependencyOverrides += "jline" % "jline" % "2.14.6", dependencyOverrides += "org.apache.avro" % "avro" % "1.8.2") From c2e5012a0a76734acf94b8716ee293ba2f58ccb4 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 14 Jun 2020 14:17:16 +0900 Subject: [PATCH 003/384] [SPARK-31632][CORE][WEBUI][FOLLOWUP] Enrich the exception message when application summary is unavailable ### What changes were proposed in this pull request? This PR enriches the exception message when application summary is not available. #28444 covers the case when application information is not available but the case application summary is not available is not covered. ### Why are the changes needed? To complement #28444 . ### Does this PR introduce _any_ user-facing change? Yes. Before this change, we can get the following error message when we access to `/jobs` if application summary is not available. no-such-element-exception-error-message After this change, we can get the following error message. It's like #28444 does. enriched-errorm-message ### How was this patch tested? I checked with the following procedure. 1. Set breakpoint in the line of `kvstore.write(appSummary)` in `AppStatusListener#onStartApplicatin`. Only the thread reaching this line should be suspended. 2. Start spark-shell and wait few seconds. 3. Access to `/jobs` Closes #28820 from sarutak/fix-no-such-element. Authored-by: Kousuke Saruta Signed-off-by: HyukjinKwon --- .../scala/org/apache/spark/status/AppStatusStore.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index bd19c9522f3df..31a6f7d901131 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -525,7 +525,13 @@ private[spark] class AppStatusStore( } def appSummary(): AppSummary = { - store.read(classOf[AppSummary], classOf[AppSummary].getName()) + try { + store.read(classOf[AppSummary], classOf[AppSummary].getName()) + } catch { + case _: NoSuchElementException => + throw new NoSuchElementException("Failed to get the application summary. " + + "If you are starting up Spark, please wait a while until it's ready.") + } } def close(): Unit = { From a4ea599b1b9b8ebaae0100b54e6ac1d7576c6d8c Mon Sep 17 00:00:00 2001 From: TJX2014 Date: Sat, 13 Jun 2020 22:21:35 -0700 Subject: [PATCH 004/384] [SPARK-31968][SQL] Duplicate partition columns check when writing data MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? A unit test is added Partition duplicate check added in `org.apache.spark.sql.execution.datasources.PartitioningUtils#validatePartitionColumn` ### Why are the changes needed? When people write data with duplicate partition column, it will cause a `org.apache.spark.sql.AnalysisException: Found duplicate column ...` in loading data from the writted. ### Does this PR introduce _any_ user-facing change? Yes. It will prevent people from using duplicate partition columns to write data. 1. Before the PR: It will look ok at `df.write.partitionBy("b", "b").csv("file:///tmp/output")`, but get an exception when read: `spark.read.csv("file:///tmp/output").show()` org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the partition schema: `b`; 2. After the PR: `df.write.partitionBy("b", "b").csv("file:///tmp/output")` will trigger the exception: org.apache.spark.sql.AnalysisException: Found duplicate column(s) b, b: `b`; ### How was this patch tested? Unit test. Closes #28814 from TJX2014/master-SPARK-31968. Authored-by: TJX2014 Signed-off-by: Dongjoon Hyun --- .../sql/execution/datasources/PartitioningUtils.scala | 3 +++ .../spark/sql/sources/PartitionedWriteSuite.scala | 10 +++++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index f7e225b0cdc96..5846d46e146ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -545,6 +545,9 @@ object PartitioningUtils { partitionColumns: Seq[String], caseSensitive: Boolean): Unit = { + SchemaUtils.checkColumnNameDuplication( + partitionColumns, partitionColumns.mkString(", "), caseSensitive) + partitionColumnsSchema(schema, partitionColumns, caseSensitive).foreach { field => field.dataType match { case _: AtomicType => // OK diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala index 1c4e2a967b0a2..6df1c5db14c26 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext import org.apache.spark.TestUtils import org.apache.spark.internal.Logging -import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol @@ -156,4 +156,12 @@ class PartitionedWriteSuite extends QueryTest with SharedSparkSession { } } } + + test("SPARK-31968: duplicate partition columns check") { + withTempPath { f => + val e = intercept[AnalysisException]( + Seq((3, 2)).toDF("a", "b").write.partitionBy("b", "b").csv(f.getAbsolutePath)) + assert(e.getMessage.contains("Found duplicate column(s) b, b: `b`;")) + } + } } From 84815d05503460d58b85be52421d5923474aa08b Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sun, 14 Jun 2020 14:37:38 +0900 Subject: [PATCH 005/384] [SPARK-24634][SS] Add a new metric regarding number of inputs later than watermark plus allowed delay ### What changes were proposed in this pull request? Please refer https://issues.apache.org/jira/browse/SPARK-24634 to see rationalization of the issue. This patch adds a new metric to count the number of inputs arrived later than watermark plus allowed delay. To make changes simpler, this patch doesn't count the exact number of input rows which are later than watermark plus allowed delay. Instead, this patch counts the inputs which are dropped in the logic of operator. The difference of twos are shown in streaming aggregation: to optimize the calculation, streaming aggregation "pre-aggregates" the input rows, and later checks the lateness against "pre-aggregated" inputs, hence the number might be reduced. The new metric will be provided via two places: 1. On Spark UI: check the metrics in stateful operator nodes in query execution details page in SQL tab 2. On Streaming Query Listener: check "numLateInputs" in "stateOperators" in QueryProcessEvent. ### Why are the changes needed? Dropping late inputs means that end users might not get expected outputs. Even end users may indicate the fact and tolerate the result (as that's what allowed lateness is for), but they should be able to observe whether the current value of allowed lateness drops inputs or not so that they can adjust the value. Also, whatever the chance they have multiple of stateful operators in a single query, if Spark drops late inputs "between" these operators, it becomes "correctness" issue. Spark should disallow such possibility, but given we already provided the flexibility, at least we should provide the way to observe the correctness issue and decide whether they should make correction of their query or not. ### Does this PR introduce _any_ user-facing change? Yes. End users will be able to retrieve the information of late inputs via two ways: 1. SQL tab in Spark UI 2. Streaming Query Listener ### How was this patch tested? New UTs added & existing UTs are modified to reflect the change. And ran manual test reproducing SPARK-28094. I've picked the specific case on "B outer C outer D" which is enough to represent the "intermediate late row" issue due to global watermark. https://gist.github.com/jammann/b58bfbe0f4374b89ecea63c1e32c8f17 Spark logs warning message on the query which means SPARK-28074 is working correctly, ``` 20/05/30 17:52:47 WARN UnsupportedOperationChecker: Detected pattern of possible 'correctness' issue due to global watermark. The query contains stateful operation which can emit rows older than the current watermark plus allowed late record delay, which are "late rows" in downstream stateful operations and these rows can be discarded. Please refer the programming guide doc for more details.; Join LeftOuter, ((D_FK#28 = D_ID#87) AND (B_LAST_MOD#26-T30000ms = D_LAST_MOD#88-T30000ms)) :- Join LeftOuter, ((C_FK#27 = C_ID#58) AND (B_LAST_MOD#26-T30000ms = C_LAST_MOD#59-T30000ms)) : :- EventTimeWatermark B_LAST_MOD#26: timestamp, 30 seconds : : +- Project [v#23.B_ID AS B_ID#25, v#23.B_LAST_MOD AS B_LAST_MOD#26, v#23.C_FK AS C_FK#27, v#23.D_FK AS D_FK#28] : : +- Project [from_json(StructField(B_ID,StringType,false), StructField(B_LAST_MOD,TimestampType,false), StructField(C_FK,StringType,true), StructField(D_FK,StringType,true), value#21, Some(UTC)) AS v#23] : : +- Project [cast(value#8 as string) AS value#21] : : +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider3a7fd18c, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable396d2958, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee61a, [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> B, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#0, value#1, topic#2, partition#3, offset#4L, timestamp#5, timestampType#6] : +- EventTimeWatermark C_LAST_MOD#59: timestamp, 30 seconds : +- Project [v#56.C_ID AS C_ID#58, v#56.C_LAST_MOD AS C_LAST_MOD#59] : +- Project [from_json(StructField(C_ID,StringType,false), StructField(C_LAST_MOD,TimestampType,false), value#54, Some(UTC)) AS v#56] : +- Project [cast(value#41 as string) AS value#54] : +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider3f507373, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable7b6736a4, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee61b, [key#40, value#41, topic#42, partition#43, offset#44L, timestamp#45, timestampType#46], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> C, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#33, value#34, topic#35, partition#36, offset#37L, timestamp#38, timestampType#39] +- EventTimeWatermark D_LAST_MOD#88: timestamp, 30 seconds +- Project [v#85.D_ID AS D_ID#87, v#85.D_LAST_MOD AS D_LAST_MOD#88] +- Project [from_json(StructField(D_ID,StringType,false), StructField(D_LAST_MOD,TimestampType,false), value#83, Some(UTC)) AS v#85] +- Project [cast(value#70 as string) AS value#83] +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider2b90e779, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable36f8cd29, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee620, [key#69, value#70, topic#71, partition#72, offset#73L, timestamp#74, timestampType#75], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> D, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#62, value#63, topic#64, partition#65, offset#66L, timestamp#67, timestampType#68] ``` and we can find the late inputs from the batch 4 as follows: ![Screen Shot 2020-05-30 at 18 02 53](https://user-images.githubusercontent.com/1317309/83324401-058fd200-a2a0-11ea-8bf6-89cf777e9326.png) which represents intermediate inputs are being lost, ended up with correctness issue. Closes #28607 from HeartSaVioR/SPARK-24634-v3. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../structured-streaming-programming-guide.md | 10 ++++++++ project/MimaExcludes.scala | 5 ++-- .../FlatMapGroupsWithStateExec.scala | 2 +- .../streaming/ProgressReporter.scala | 2 +- .../StreamingSymmetricHashJoinExec.scala | 2 +- .../streaming/statefulOperators.scala | 20 +++++++++++++--- .../apache/spark/sql/streaming/progress.scala | 9 +++++-- .../streaming/EventTimeWatermarkSuite.scala | 21 ++++++++++++++-- .../sql/streaming/StateStoreMetricsTest.scala | 24 +++++++++++++++---- .../StreamingDeduplicationSuite.scala | 10 ++++---- .../sql/streaming/StreamingJoinSuite.scala | 10 ++++---- ...StreamingQueryStatusAndProgressSuite.scala | 8 ++++--- 12 files changed, 94 insertions(+), 29 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 69d744d09e1e5..53b7a3a8e46f6 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1675,6 +1675,16 @@ Any of the stateful operation(s) after any of below stateful operations can have As Spark cannot check the state function of `mapGroupsWithState`/`flatMapGroupsWithState`, Spark assumes that the state function emits late rows if the operator uses Append mode. +Spark provides two ways to check the number of late rows on stateful operators which would help you identify the issue: + +1. On Spark UI: check the metrics in stateful operator nodes in query execution details page in SQL tab +2. On Streaming Query Listener: check "numLateInputs" in "stateOperators" in QueryProcessEvent. + +Please note that the definition of "input" is relative: it doesn't always mean "input rows" for the operator. +Streaming aggregation does pre-aggregate input rows and checks the late inputs against pre-aggregated inputs, +hence the number is not same as the number of original input rows. You'd like to check the fact whether the value is zero +or non-zero. + There's a known workaround: split your streaming query into multiple queries per stateful operator, and ensure end-to-end exactly once per query. Ensuring end-to-end exactly once for the last query is optional. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 57fbb125dc470..addb2d8152189 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -46,10 +46,11 @@ object MimaExcludes { // false positive, no binary incompatibility ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel"), ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.ChiSqSelector"), - //[SPARK-31840] Add instance weight support in LogisticRegressionSummary // weightCol in org.apache.spark.ml.classification.LogisticRegressionSummary is present only in current version - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightCol") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightCol"), + // [SPARK-24634] Add a new metric regarding number of inputs later than watermark plus allowed delay + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StateOperatorProgress.$default$4") ) // Exclude rules for 3.0.x diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala index d65c4ffbb7a24..eb8b8af7950b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala @@ -122,7 +122,7 @@ case class FlatMapGroupsWithStateExec( // If timeout is based on event time, then filter late data based on watermark val filteredIter = watermarkPredicateForData match { case Some(predicate) if timeoutConf == EventTimeTimeout => - iter.filter(row => !predicate.eval(row)) + applyRemovingRowsOlderThanWatermark(iter, predicate) case _ => iter } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 0dff1c2fe5768..bdfdc31e508cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -222,7 +222,7 @@ trait ProgressReporter extends Logging { lastExecution.executedPlan.collect { case p if p.isInstanceOf[StateStoreWriter] => val progress = p.asInstanceOf[StateStoreWriter].getProgress() - if (hasExecuted) progress else progress.copy(newNumRowsUpdated = 0) + if (hasExecuted) progress else progress.copy(newNumRowsUpdated = 0, newNumLateInputs = 0) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index 198e17db419a7..dc5fc2e43143d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -462,7 +462,7 @@ case class StreamingSymmetricHashJoinExec( WatermarkSupport.watermarkExpression(watermarkAttribute, eventTimeWatermark) match { case Some(watermarkExpr) => val predicate = Predicate.create(watermarkExpr, inputAttributes) - inputIter.filter { row => !predicate.eval(row) } + applyRemovingRowsOlderThanWatermark(inputIter, predicate) case None => inputIter } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index 1bec924ba219a..073266bd621d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -77,6 +77,8 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numLateInputs" -> SQLMetrics.createMetric(sparkContext, + "number of inputs which are later than watermark ('inputs' are relative to operators)"), "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows"), "allUpdatesTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to update"), @@ -100,6 +102,7 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => numRowsTotal = longMetric("numTotalStateRows").value, numRowsUpdated = longMetric("numUpdatedStateRows").value, memoryUsedBytes = longMetric("stateMemory").value, + numLateInputs = longMetric("numLateInputs").value, javaConvertedCustomMetrics ) } @@ -132,6 +135,16 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => }.toMap } + protected def applyRemovingRowsOlderThanWatermark( + iter: Iterator[InternalRow], + predicateFilterOutLateInput: BasePredicate): Iterator[InternalRow] = { + iter.filterNot { row => + val lateInput = predicateFilterOutLateInput.eval(row) + if (lateInput) longMetric("numLateInputs") += 1 + lateInput + } + } + /** * Should the MicroBatchExecution run another batch based on this stateful operator and the * current updated metadata. @@ -328,7 +341,8 @@ case class StateStoreSaveExec( // Assumption: watermark predicates must be non-empty if append mode is allowed case Some(Append) => allUpdatesTimeMs += timeTakenMs { - val filteredIter = iter.filter(row => !watermarkPredicateForData.get.eval(row)) + val filteredIter = applyRemovingRowsOlderThanWatermark(iter, + watermarkPredicateForData.get) while (filteredIter.hasNext) { val row = filteredIter.next().asInstanceOf[UnsafeRow] stateManager.put(store, row) @@ -371,7 +385,7 @@ case class StateStoreSaveExec( new NextIterator[InternalRow] { // Filter late date using watermark if specified private[this] val baseIterator = watermarkPredicateForData match { - case Some(predicate) => iter.filter((row: InternalRow) => !predicate.eval(row)) + case Some(predicate) => applyRemovingRowsOlderThanWatermark(iter, predicate) case None => iter } private val updatesStartTimeNs = System.nanoTime @@ -456,7 +470,7 @@ case class StreamingDeduplicateExec( val commitTimeMs = longMetric("commitTimeMs") val baseIterator = watermarkPredicateForData match { - case Some(predicate) => iter.filter(row => !predicate.eval(row)) + case Some(predicate) => applyRemovingRowsOlderThanWatermark(iter, predicate) case None => iter } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index 13b506b60a126..22bae76ef4222 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -43,6 +43,7 @@ class StateOperatorProgress private[sql]( val numRowsTotal: Long, val numRowsUpdated: Long, val memoryUsedBytes: Long, + val numLateInputs: Long, val customMetrics: ju.Map[String, JLong] = new ju.HashMap() ) extends Serializable { @@ -52,13 +53,17 @@ class StateOperatorProgress private[sql]( /** The pretty (i.e. indented) JSON representation of this progress. */ def prettyJson: String = pretty(render(jsonValue)) - private[sql] def copy(newNumRowsUpdated: Long): StateOperatorProgress = - new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, customMetrics) + private[sql] def copy( + newNumRowsUpdated: Long, + newNumLateInputs: Long): StateOperatorProgress = + new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, newNumLateInputs, + customMetrics) private[sql] def jsonValue: JValue = { ("numRowsTotal" -> JInt(numRowsTotal)) ~ ("numRowsUpdated" -> JInt(numRowsUpdated)) ~ ("memoryUsedBytes" -> JInt(memoryUsedBytes)) ~ + ("numLateInputs" -> JInt(numLateInputs)) ~ ("customMetrics" -> { if (!customMetrics.isEmpty) { val keys = customMetrics.keySet.asScala.toSeq.sorted diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 6486e1aee8649..329196a5cfef7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -298,9 +298,11 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche AddData(inputData, 25), // Advance watermark to 15 seconds CheckNewAnswer((10, 5)), assertNumStateRows(2), + assertNumLateInputs(0), AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), - assertNumStateRows(2) + assertNumStateRows(2), + assertNumLateInputs(1) ) } @@ -321,12 +323,15 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche AddData(inputData, 25), // Advance watermark to 15 seconds CheckNewAnswer((25, 1)), assertNumStateRows(2), + assertNumLateInputs(0), AddData(inputData, 10, 25), // Ignore 10 as its less than watermark CheckNewAnswer((25, 2)), assertNumStateRows(2), + assertNumLateInputs(1), AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), - assertNumStateRows(2) + assertNumStateRows(2), + assertNumLateInputs(1) ) } @@ -783,6 +788,18 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche true } + private def assertNumLateInputs(numLateInputs: Long): AssertOnQuery = AssertOnQuery { q => + q.processAllAvailable() + val progressWithData = q.recentProgress.filterNot { p => + // filter out batches which are falling into one of types: + // 1) doesn't execute the batch run + // 2) empty input batch + p.inputRowsPerSecond == 0 + }.lastOption.get + assert(progressWithData.stateOperators(0).numLateInputs === numLateInputs) + true + } + /** Assert event stats generated on that last batch with data in it */ private def assertEventStats(body: ju.Map[String, String] => Unit): AssertOnQuery = { Execute("AssertEventStats") { q => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala index fb5d13d09fb0e..640f5181aa526 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala @@ -29,8 +29,12 @@ trait StateStoreMetricsTest extends StreamTest { lastCheckedRecentProgressIndex = -1 } - def assertNumStateRows(total: Seq[Long], updated: Seq[Long]): AssertOnQuery = - AssertOnQuery(s"Check total state rows = $total, updated state rows = $updated") { q => + def assertNumStateRows( + total: Seq[Long], + updated: Seq[Long], + lateInputs: Seq[Long]): AssertOnQuery = + AssertOnQuery(s"Check total state rows = $total, updated state rows = $updated" + + s", late inputs = $lateInputs") { q => // This assumes that the streaming query will not make any progress while the eventually // is being executed. eventually(timeout(streamingTimeout)) { @@ -51,6 +55,9 @@ trait StateStoreMetricsTest extends StreamTest { val allNumUpdatedRowsSinceLastCheck = progressesSinceLastCheck.map(_.stateOperators.map(_.numRowsUpdated)) + val allNumLateInputsSinceLastCheck = + progressesSinceLastCheck.map(_.stateOperators.map(_.numLateInputs)) + lazy val debugString = "recent progresses:\n" + progressesSinceLastCheck.map(_.prettyJson).mkString("\n\n") @@ -60,13 +67,22 @@ trait StateStoreMetricsTest extends StreamTest { val numUpdatedRows = arraySum(allNumUpdatedRowsSinceLastCheck, numStateOperators) assert(numUpdatedRows === updated, s"incorrect updates rows, $debugString") + val numLateInputs = arraySum(allNumLateInputsSinceLastCheck, numStateOperators) + assert(numLateInputs === lateInputs, s"incorrect late inputs, $debugString") + lastCheckedRecentProgressIndex = recentProgress.length - 1 } true } - def assertNumStateRows(total: Long, updated: Long): AssertOnQuery = - assertNumStateRows(Seq(total), Seq(updated)) + def assertNumStateRows(total: Seq[Long], updated: Seq[Long]): AssertOnQuery = { + assert(total.length === updated.length) + assertNumStateRows(total, updated, lateInputs = (0 until total.length).map(_ => 0L)) + } + + def assertNumStateRows(total: Long, updated: Long, lateInput: Long = 0): AssertOnQuery = { + assertNumStateRows(Seq(total), Seq(updated), Seq(lateInput)) + } def arraySum(arraySeq: Seq[Array[Long]], arrayLength: Int): Seq[Long] = { if (arraySeq.isEmpty) return Seq.fill(arrayLength)(0L) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index f63778aef5a7f..ceb6775f79cc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -54,13 +54,13 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { testStream(result, Append)( AddData(inputData, "a" -> 1), CheckLastBatch("a" -> 1), - assertNumStateRows(total = 1, updated = 1), + assertNumStateRows(total = 1, updated = 1, lateInput = 0), AddData(inputData, "a" -> 2), // Dropped CheckLastBatch(), - assertNumStateRows(total = 1, updated = 0), + assertNumStateRows(total = 1, updated = 0, lateInput = 0), AddData(inputData, "b" -> 1), CheckLastBatch("b" -> 1), - assertNumStateRows(total = 2, updated = 1) + assertNumStateRows(total = 2, updated = 1, lateInput = 0) ) } @@ -102,7 +102,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), - assertNumStateRows(total = 1, updated = 0), + assertNumStateRows(total = 1, updated = 0, lateInput = 1), AddData(inputData, 45), // Advance watermark to 35 seconds, no-data-batch drops row 25 CheckNewAnswer(45), @@ -136,7 +136,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { AddData(inputData, 10), // Should not emit anything as data less than watermark CheckLastBatch(), - assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L)), + assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L), lateInputs = Seq(0L, 1L)), AddData(inputData, 40), // Advance watermark to 30 seconds CheckLastBatch((15 -> 1), (25 -> 1)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 3f218c9cb7fd9..1f6d0a9945682 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -166,7 +166,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(input1, 5), CheckNewAnswer(), // Same reason as above - assertNumStateRows(total = 2, updated = 0) + assertNumStateRows(total = 2, updated = 0, lateInput = 1) ) } @@ -219,12 +219,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with // (1, 28) ==> passed filter, matched with left (1, 3) and (1, 5), added to state AddData(rightInput, (1, 20), (1, 21), (1, 28)), CheckNewAnswer((1, 3, 21), (1, 5, 21), (1, 3, 28), (1, 5, 28)), - assertNumStateRows(total = 5, updated = 1), + assertNumStateRows(total = 5, updated = 1, lateInput = 1), // New data to left input with leftTime <= 20 should be filtered due to event time watermark AddData(leftInput, (1, 20), (1, 21)), CheckNewAnswer((1, 21, 28)), - assertNumStateRows(total = 6, updated = 1) + assertNumStateRows(total = 6, updated = 1, lateInput = 1) ) } @@ -293,7 +293,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(leftInput, (1, 30), (1, 31)), // 30 should not be processed or added to state CheckNewAnswer((1, 31, 26), (1, 31, 30), (1, 31, 31)), - assertNumStateRows(total = 11, updated = 1), // only 31 added + assertNumStateRows(total = 11, updated = 1, lateInput = 1), // only 31 added // Advance the watermark AddData(rightInput, (1, 80)), @@ -307,7 +307,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(rightInput, (1, 46), (1, 50)), // 46 should not be processed or added to state CheckNewAnswer((1, 49, 50), (1, 50, 50)), - assertNumStateRows(total = 7, updated = 1) // 50 added + assertNumStateRows(total = 7, updated = 1, lateInput = 1) // 50 added ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index 08b3644745f9a..79028a6c442d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -64,6 +64,7 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "numRowsTotal" : 0, | "numRowsUpdated" : 1, | "memoryUsedBytes" : 3, + | "numLateInputs" : 0, | "customMetrics" : { | "loadedMapCacheHitCount" : 1, | "loadedMapCacheMissCount" : 0, @@ -113,7 +114,8 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "stateOperators" : [ { | "numRowsTotal" : 0, | "numRowsUpdated" : 1, - | "memoryUsedBytes" : 2 + | "memoryUsedBytes" : 2, + | "numLateInputs" : 0 | } ], | "sources" : [ { | "description" : "source", @@ -321,7 +323,7 @@ object StreamingQueryStatusAndProgressSuite { "avg" -> "2016-12-05T20:54:20.827Z", "watermark" -> "2016-12-05T20:54:20.827Z").asJava), stateOperators = Array(new StateOperatorProgress( - numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 3, + numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 3, numLateInputs = 0, customMetrics = new java.util.HashMap(Map("stateOnCurrentVersionSizeBytes" -> 2L, "loadedMapCacheHitCount" -> 1L, "loadedMapCacheMissCount" -> 0L) .mapValues(long2Long).asJava) @@ -353,7 +355,7 @@ object StreamingQueryStatusAndProgressSuite { // empty maps should be handled correctly eventTime = new java.util.HashMap(Map.empty[String, String].asJava), stateOperators = Array(new StateOperatorProgress( - numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 2)), + numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 2, numLateInputs = 0)), sources = Array( new SourceProgress( description = "source", From 1e40bccf447dccad9d31bccc75d21b8fca77ba52 Mon Sep 17 00:00:00 2001 From: uncleGen Date: Sun, 14 Jun 2020 14:49:01 +0900 Subject: [PATCH 006/384] [SPARK-31593][SS] Remove unnecessary streaming query progress update ### What changes were proposed in this pull request? Structured Streaming progress reporter will always report an `empty` progress when there is no new data. As design, we should provide progress updates every 10s (default) when there is no new data. Before PR: ![20200428175008](https://user-images.githubusercontent.com/7402327/80474832-88a8ca00-897a-11ea-820b-d4be6127d2fe.jpg) ![20200428175037](https://user-images.githubusercontent.com/7402327/80474844-8ba3ba80-897a-11ea-873c-b7137bd4a804.jpg) ![20200428175102](https://user-images.githubusercontent.com/7402327/80474848-8e061480-897a-11ea-806e-28c6bbf1fe03.jpg) After PR: ![image](https://user-images.githubusercontent.com/7402327/80475099-f35a0580-897a-11ea-8fb3-53f343df2c3f.png) ### Why are the changes needed? Fixes a bug around incorrect progress report ### Does this PR introduce any user-facing change? Fixes a bug around incorrect progress report ### How was this patch tested? existing ut and manual test Closes #28391 from uncleGen/SPARK-31593. Authored-by: uncleGen Signed-off-by: HyukjinKwon --- .../streaming/ProgressReporter.scala | 2 +- .../StreamingDeduplicationSuite.scala | 7 ++- .../StreamingQueryListenerSuite.scala | 56 +++++++++++++++++-- 3 files changed, 58 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index bdfdc31e508cb..55491d96f9b1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -201,7 +201,7 @@ trait ProgressReporter extends Logging { if (hasExecuted) { // Reset noDataEventTimestamp if we processed any data - lastNoExecutionProgressEventTime = Long.MinValue + lastNoExecutionProgressEventTime = triggerClock.getTimeMillis() updateProgress(newProgress) } else { val now = triggerClock.getTimeMillis() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index ceb6775f79cc4..ee0b980a9d87b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -281,7 +281,12 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { if (flag) assertNumStateRows(total = 1, updated = 1) else assertNumStateRows(total = 7, updated = 1) }, - AssertOnQuery(q => q.lastProgress.sink.numOutputRows == 0L) + AssertOnQuery { q => + eventually(timeout(streamingTimeout)) { + q.lastProgress.sink.numOutputRows == 0L + true + } + } ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index e585b8a885c9c..6e08b88f538df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -389,7 +389,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Structured Streaming in Spark 2.0.0. Because we renamed the classes, // SparkListenerApplicationEnd is the only valid event and it's the last event. We use it // to verify that we can skip broken jsons generated by Structured Streaming. - testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.0.txt", 1) + testReplayListenerBusWithBrokenEventJsons("query-event-logs-version-2.0.0.txt", 1) } testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2_0_1") { @@ -397,14 +397,14 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Structured Streaming in Spark 2.0.1. Because we renamed the classes, // SparkListenerApplicationEnd is the only valid event and it's the last event. We use it // to verify that we can skip broken jsons generated by Structured Streaming. - testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.1.txt", 1) + testReplayListenerBusWithBrokenEventJsons("query-event-logs-version-2.0.1.txt", 1) } testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2_0_2") { // query-event-logs-version-2.0.2.txt has all types of events generated by // Structured Streaming in Spark 2.0.2. SPARK-18516 refactored Structured Streaming query events // in 2.1.0. This test is to verify we are able to load events generated by Spark 2.0.2. - testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.2.txt", 5) + testReplayListenerBusWithBrokenEventJsons("query-event-logs-version-2.0.2.txt", 5) } test("listener propagates observable metrics") { @@ -433,9 +433,13 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } try { + val noDataProgressIntervalKey = SQLConf.STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL.key spark.streams.addListener(listener) testStream(df, OutputMode.Append)( - StartStream(Trigger.ProcessingTime(100), triggerClock = clock), + StartStream( + Trigger.ProcessingTime(100), + triggerClock = clock, + Map(noDataProgressIntervalKey -> "100")), // Batch 1 AddData(inputData, 1, 2), AdvanceManualClock(100), @@ -464,7 +468,49 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } } - private def testReplayListenerBusWithBorkenEventJsons( + test("SPARK-31593: remove unnecessary streaming query progress update") { + withSQLConf(SQLConf.STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL.key -> "100") { + @volatile var numProgressEvent = 0 + val listener = new StreamingQueryListener { + override def onQueryStarted(event: QueryStartedEvent): Unit = {} + override def onQueryProgress(event: QueryProgressEvent): Unit = { + numProgressEvent += 1 + } + override def onQueryTerminated(event: QueryTerminatedEvent): Unit = {} + } + spark.streams.addListener(listener) + + def checkProgressEvent(count: Int): StreamAction = { + AssertOnQuery { _ => + eventually(Timeout(streamingTimeout)) { + assert(numProgressEvent == count) + } + true + } + } + + try { + val input = new MemoryStream[Int](0, sqlContext) + val clock = new StreamManualClock() + val result = input.toDF().select("value") + testStream(result)( + StartStream(trigger = Trigger.ProcessingTime(10), triggerClock = clock), + AddData(input, 10), + checkProgressEvent(1), + AdvanceManualClock(10), + checkProgressEvent(2), + AdvanceManualClock(90), + checkProgressEvent(2), + AdvanceManualClock(10), + checkProgressEvent(3) + ) + } finally { + spark.streams.removeListener(listener) + } + } + } + + private def testReplayListenerBusWithBrokenEventJsons( fileName: String, expectedEventSize: Int): Unit = { val input = getClass.getResourceAsStream(s"/structured-streaming/$fileName") From 54e702c0dd585e5d6e929b427e31a34614ae257c Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Sun, 14 Jun 2020 14:26:11 -0700 Subject: [PATCH 007/384] [SPARK-31970][CORE] Make MDC configuration step be consistent between setLocalProperty and log4j.properties ### What changes were proposed in this pull request? This PR proposes to use "mdc.XXX" as the consistent key for both `sc.setLocalProperty` and `log4j.properties` when setting up configurations for MDC. ### Why are the changes needed? It's weird that we use "mdc.XXX" as key to set MDC value via `sc.setLocalProperty` while we use "XXX" as key to set MDC pattern in log4j.properties. It could also bring extra burden to the user. ### Does this PR introduce _any_ user-facing change? No, as MDC feature is added in version 3.1, which hasn't been released. ### How was this patch tested? Tested manually. Closes #28801 from Ngone51/consistent-mdc. Authored-by: yi.wu Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/executor/Executor.scala | 7 ++----- docs/configuration.md | 8 ++++---- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 93d1acdd2d156..c8b1afeebac0d 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -323,10 +323,7 @@ private[spark] class Executor( val threadName = s"Executor task launch worker for task $taskId" val taskName = taskDescription.name val mdcProperties = taskDescription.properties.asScala - .filter(_._1.startsWith("mdc.")).map { item => - val key = item._1.substring(4) - (key, item._2) - }.toSeq + .filter(_._1.startsWith("mdc.")).toSeq /** If specified, this task has been killed and this option contains the reason. */ @volatile private var reasonIfKilled: Option[String] = None @@ -705,7 +702,7 @@ private[spark] class Executor( MDC.clear() mdc.foreach { case (key, value) => MDC.put(key, value) } // avoid overriding the takName by the user - MDC.put("taskName", taskName) + MDC.put("mdc.taskName", taskName) } /** diff --git a/docs/configuration.md b/docs/configuration.md index 420942f7b7bbb..706c2552b1d17 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2955,11 +2955,11 @@ Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can config `log4j.properties` file in the `conf` directory. One way to start is to copy the existing `log4j.properties.template` located there. -By default, Spark adds 1 record to the MDC (Mapped Diagnostic Context): `taskName`, which shows something -like `task 1.0 in stage 0.0`. You can add `%X{taskName}` to your patternLayout in +By default, Spark adds 1 record to the MDC (Mapped Diagnostic Context): `mdc.taskName`, which shows something +like `task 1.0 in stage 0.0`. You can add `%X{mdc.taskName}` to your patternLayout in order to print it in the logs. -Moreover, you can use `spark.sparkContext.setLocalProperty("mdc." + name, "value")` to add user specific data into MDC. -The key in MDC will be the string after the `mdc.` prefix. +Moreover, you can use `spark.sparkContext.setLocalProperty(s"mdc.$name", "value")` to add user specific data into MDC. +The key in MDC will be the string of "mdc.$name". # Overriding configuration directory From f5f6eee3045e90e02fc7e999f616b5a021d7c724 Mon Sep 17 00:00:00 2001 From: iRakson Date: Sun, 14 Jun 2020 16:41:59 -0500 Subject: [PATCH 008/384] [SPARK-31642][FOLLOWUP] Fix Sorting for duration column and make Status column sortable ### What changes were proposed in this pull request? In #28485 pagination support for tables of Structured Streaming Tab was added. It missed 2 things: * For sorting duration column, `String` was used which sometimes gives wrong results(consider `"3 ms"` and `"12 ms"`). Now we first sort the duration column and then convert it to readable String * Status column was not made sortable. ### Why are the changes needed? To fix the wrong result for sorting and making Status column sortable. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? After changes: Screenshot 2020-06-08 at 2 18 48 PM Closes #28752 from iRakson/ssTests. Authored-by: iRakson Signed-off-by: Sean Owen --- .../spark/sql/streaming/ui/StreamingQueryPage.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala index b969e41e4e55c..b98fdf16eef31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala @@ -152,7 +152,7 @@ class StreamingQueryPagedTable( val headerAndCss: Seq[(String, Boolean, Option[String])] = { Seq( ("Name", true, None), - ("Status", false, None), + ("Status", true, None), ("ID", true, None), ("Run ID", true, None), ("Start Time", true, None), @@ -197,7 +197,7 @@ class StreamingQueryPagedTable( {streamingQuery.id} {streamingQuery.runId} {SparkUIUtils.formatDate(streamingQuery.startTimestamp)} - {query.duration} + {SparkUIUtils.formatDurationVerbose(query.duration)} {withNoProgress(streamingQuery, {query.avgInput.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {query.avgProcess.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {streamingQuery.lastProgress.batchId}, "NaN")} @@ -207,7 +207,7 @@ class StreamingQueryPagedTable( } case class StructuredStreamingRow( - duration: String, + duration: Long, avgInput: Double, avgProcess: Double, streamingUIData: StreamingQueryUIData) @@ -224,12 +224,12 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St private def streamingRow(query: StreamingQueryUIData): StructuredStreamingRow = { val duration = if (isActive) { - SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - query.startTimestamp) + System.currentTimeMillis() - query.startTimestamp } else { withNoProgress(query, { val endTimeMs = query.lastProgress.timestamp - SparkUIUtils.formatDurationVerbose(parseProgressTimestamp(endTimeMs) - query.startTimestamp) - }, "-") + parseProgressTimestamp(endTimeMs) - query.startTimestamp + }, 0) } val avgInput = (query.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / @@ -244,6 +244,7 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St private def ordering(sortColumn: String, desc: Boolean): Ordering[StructuredStreamingRow] = { val ordering: Ordering[StructuredStreamingRow] = sortColumn match { case "Name" => Ordering.by(q => UIUtils.getQueryName(q.streamingUIData)) + case "Status" => Ordering.by(q => UIUtils.getQueryStatus(q.streamingUIData)) case "ID" => Ordering.by(_.streamingUIData.id) case "Run ID" => Ordering.by(_.streamingUIData.runId) case "Start Time" => Ordering.by(_.streamingUIData.startTimestamp) From 8282bbf12d4e174986a649023ce3984aae7d7755 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 15 Jun 2020 11:01:56 +0900 Subject: [PATCH 009/384] [SPARK-27633][SQL] Remove redundant aliases in NestedColumnAliasing ## What changes were proposed in this pull request? In NestedColumnAliasing rule, we create aliases for nested field access in project list. We considered that top level parent field and nested fields under it were both accessed. In the case, we don't create the aliases because they are redundant. There is another case, where a nested parent field and nested fields under it were both accessed, which we don't consider now. We don't need to create aliases in this case too. ## How was this patch tested? Added test. Closes #24525 from viirya/SPARK-27633. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: HyukjinKwon --- .../optimizer/NestedColumnAliasing.scala | 13 +++++++- .../optimizer/NestedColumnAliasingSuite.scala | 31 +++++++++++++++++++ 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index 7f49d8db4852d..118f41f9cd232 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -154,9 +154,20 @@ object NestedColumnAliasing { .filter(!_.references.subsetOf(exclusiveAttrSet)) .groupBy(_.references.head) .flatMap { case (attr, nestedFields: Seq[ExtractValue]) => + // Remove redundant `ExtractValue`s if they share the same parent nest field. + // For example, when `a.b` and `a.b.c` are in project list, we only need to alias `a.b`. + // We only need to deal with two `ExtractValue`: `GetArrayStructFields` and + // `GetStructField`. Please refer to the method `collectRootReferenceAndExtractValue`. + val dedupNestedFields = nestedFields.filter { + case e @ (_: GetStructField | _: GetArrayStructFields) => + val child = e.children.head + nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty) + case _ => true + } + // Each expression can contain multiple nested fields. // Note that we keep the original names to deliver to parquet in a case-sensitive way. - val nestedFieldToAlias = nestedFields.distinct.map { f => + val nestedFieldToAlias = dedupNestedFields.distinct.map { f => val exprId = NamedExpression.newExprId (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index 7b1735a6f04ab..0425846637f30 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -261,6 +261,37 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { comparePlans(optimized, expected) } + test("SPARK-27633: Do not generate redundant aliases if parent nested field is aliased too") { + val nestedRelation = LocalRelation('a.struct('b.struct('c.int, + 'd.struct('f.int, 'g.int)), 'e.int)) + + // `a.b` + val first = 'a.getField("b") + // `a.b.c` + 1 + val second = 'a.getField("b").getField("c") + Literal(1) + // `a.b.d.f` + val last = 'a.getField("b").getField("d").getField("f") + + val query = nestedRelation + .limit(5) + .select(first, second, last) + .analyze + + val optimized = Optimize.execute(query) + + val aliases = collectGeneratedAliases(optimized) + + val expected = nestedRelation + .select(first.as(aliases(0))) + .limit(5) + .select($"${aliases(0)}".as("a.b"), + ($"${aliases(0)}".getField("c") + Literal(1)).as("(a.b.c + 1)"), + $"${aliases(0)}".getField("d").getField("f").as("a.b.d.f")) + .analyze + + comparePlans(optimized, expected) + } + test("Nested field pruning for Project and Generate") { val query = contact .generate(Explode('friends.getField("first")), outputNames = Seq("explode")) From a0187cd6b59a6b6bb2cadc6711bb663d4d35a844 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 15 Jun 2020 06:10:24 +0000 Subject: [PATCH 010/384] [SPARK-31926][SQL][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber ### What changes were proposed in this pull request? This PR brings 02f32cfae4082696a6c013fb6beb150040be3228 back which reverted by 4a25200cd74561152e42fb92b02e860f2d5c7d4e because of maven test failure diffs newly made: 1. add a missing log4j file to test resources 2. Call `SessionState.detachSession()` to clean the thread local one in `afterAll`. 3. Not use dedicated JVMs for sbt test runner too ### Why are the changes needed? fix the maven test ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? add new tests Closes #28797 from yaooqinn/SPARK-31926-NEW. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- project/SparkBuild.scala | 1 - .../src/test/resources/log4j.properties | 65 +++++++++++++++++++ .../thriftserver/SharedThriftServer.scala | 50 ++++++++++---- .../ThriftServerQueryTestSuite.scala | 3 + .../ThriftServerWithSparkContextSuite.scala | 11 +++- .../cli/thrift/ThriftBinaryCLIService.java | 11 +++- .../service/cli/thrift/ThriftCLIService.java | 3 + .../cli/thrift/ThriftHttpCLIService.java | 21 ++++-- .../cli/thrift/ThriftBinaryCLIService.java | 11 +++- .../service/cli/thrift/ThriftCLIService.java | 3 + .../cli/thrift/ThriftHttpCLIService.java | 21 ++++-- 11 files changed, 170 insertions(+), 30 deletions(-) create mode 100644 sql/hive-thriftserver/src/test/resources/log4j.properties diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0035f1d95a90d..04a3fc4b63050 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -480,7 +480,6 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" ) diff --git a/sql/hive-thriftserver/src/test/resources/log4j.properties b/sql/hive-thriftserver/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..21975ba818142 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/log4j.properties @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file hive-thriftserver/target/unit-tests.log +log4j.rootLogger=DEBUG, CA, FA + +#Console Appender +log4j.appender.CA=org.apache.log4j.ConsoleAppender +log4j.appender.CA.layout=org.apache.log4j.PatternLayout +log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n +log4j.appender.CA.Threshold = WARN + + +#File Appender +log4j.appender.FA=org.apache.log4j.FileAppender +log4j.appender.FA.append=false +log4j.appender.FA.file=target/unit-tests.log +log4j.appender.FA.layout=org.apache.log4j.PatternLayout +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Set the logger level of File Appender to WARN +log4j.appender.FA.Threshold = DEBUG + +# Some packages are noisy for no good reason. +log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false +log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF + +log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false +log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF + +log4j.additivity.hive.log=false +log4j.logger.hive.log=OFF + +log4j.additivity.parquet.hadoop.ParquetRecordReader=false +log4j.logger.parquet.hadoop.ParquetRecordReader=OFF + +log4j.additivity.org.apache.parquet.hadoop.ParquetRecordReader=false +log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF + +log4j.additivity.org.apache.parquet.hadoop.ParquetOutputCommitter=false +log4j.logger.org.apache.parquet.hadoop.ParquetOutputCommitter=OFF + +log4j.additivity.hive.ql.metadata.Hive=false +log4j.logger.hive.ql.metadata.Hive=OFF + +log4j.additivity.org.apache.hadoop.hive.ql.io.RCFile=false +log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR + +# Parquet related logging +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index e002bc0117c8b..1c33abff0780c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -24,6 +24,7 @@ import scala.concurrent.duration._ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftCLIService import org.apache.spark.sql.test.SharedSparkSession @@ -33,6 +34,8 @@ trait SharedThriftServer extends SharedSparkSession { private var hiveServer2: HiveThriftServer2 = _ private var serverPort: Int = 0 + def mode: ServerMode.Value + override def beforeAll(): Unit = { super.beforeAll() // Retries up to 3 times with different port numbers if the server fails to start @@ -50,14 +53,21 @@ trait SharedThriftServer extends SharedSparkSession { hiveServer2.stop() } finally { super.afterAll() + SessionState.detachSession() } } + protected def jdbcUri: String = if (mode == ServerMode.http) { + s"jdbc:hive2://localhost:$serverPort/default;transportMode=http;httpPath=cliservice" + } else { + s"jdbc:hive2://localhost:$serverPort/" + } + protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") val connections = - fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } + fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } val statements = connections.map(_.createStatement()) try { @@ -69,23 +79,35 @@ trait SharedThriftServer extends SharedSparkSession { } private def startThriftServer(attempt: Int): Unit = { - logInfo(s"Trying to start HiveThriftServer2:, attempt=$attempt") + logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt") val sqlContext = spark.newSession().sqlContext - // Set the HIVE_SERVER2_THRIFT_PORT to 0, so it could randomly pick any free port to use. + // Set the HIVE_SERVER2_THRIFT_PORT and HIVE_SERVER2_THRIFT_HTTP_PORT to 0, so it could + // randomly pick any free port to use. // It's much more robust than set a random port generated by ourselves ahead sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") - hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) - hiveServer2.getServices.asScala.foreach { - case t: ThriftCLIService if t.getPortNumber != 0 => - serverPort = t.getPortNumber - logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt") - case _ => - } + sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") + sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) + + try { + hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) + hiveServer2.getServices.asScala.foreach { + case t: ThriftCLIService => + serverPort = t.getPortNumber + logInfo(s"Started HiveThriftServer2: mode=$mode, port=$serverPort, attempt=$attempt") + case _ => + } - // Wait for thrift server to be ready to serve the query, via executing simple query - // till the query succeeds. See SPARK-30345 for more details. - eventually(timeout(30.seconds), interval(1.seconds)) { - withJdbcStatement { _.execute("SELECT 1") } + // Wait for thrift server to be ready to serve the query, via executing simple query + // till the query succeeds. See SPARK-30345 for more details. + eventually(timeout(30.seconds), interval(1.seconds)) { + withJdbcStatement { _.execute("SELECT 1") } + } + } catch { + case e: Exception => + logError("Error start hive server with Context ", e) + if (hiveServer2 != null) { + hiveServer2.stop() + } } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 15cc3109da3f7..553f10a275bce 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -54,6 +54,9 @@ import org.apache.spark.sql.types._ */ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer { + + override def mode: ServerMode.Value = ServerMode.binary + override protected def testFile(fileName: String): String = { val url = Thread.currentThread().getContextClassLoader.getResource(fileName) // Copy to avoid URISyntaxException during accessing the resources in `sql/core` diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 3e1fce78ae71c..d6420dee41adb 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -class ThriftServerWithSparkContextSuite extends SharedThriftServer { +trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("SPARK-29911: Uncache cached tables when session closed") { val cacheManager = spark.sharedState.cacheManager @@ -42,3 +42,12 @@ class ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } + + +class ThriftServerWithSparkContextInBinarySuite extends ThriftServerWithSparkContextSuite { + override def mode: ServerMode.Value = ServerMode.binary +} + +class ThriftServerWithSparkContextInHttpSuite extends ThriftServerWithSparkContextSuite { + override def mode: ServerMode.Value = ServerMode.http +} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index e1ee503b81209..00bdf7e19126e 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -45,7 +46,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - public void run() { + protected void initializeServer() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -100,6 +101,14 @@ public void run() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + @Override + public void run() { + try { server.serve(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 8fce9d9383438..783e5795aca76 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -175,6 +175,7 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { + initializeServer(); new Thread(this).start(); isStarted = true; } @@ -633,6 +634,8 @@ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { return resp; } + protected abstract void initializeServer(); + @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 1099a00b67eb7..bd64c777c1d76 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.cli.thrift.TCLIService.Iface; @@ -53,13 +54,8 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ @Override - public void run() { + protected void initializeServer() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -150,6 +146,19 @@ public void run() { + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { httpServer.join(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index a7de9c0f3d0d2..ce79e3c8228a6 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -46,7 +47,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - public void run() { + protected void initializeServer() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -101,6 +102,14 @@ public void run() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + @Override + public void run() { + try { server.serve(); } catch (Throwable t) { LOG.error( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index d41c3b493bb47..e46799a1c427d 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -176,6 +176,7 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { + initializeServer(); new Thread(this).start(); isStarted = true; } @@ -670,6 +671,8 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) return resp; } + protected abstract void initializeServer(); + @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 73d5f84476af0..ab9ed5b1f371e 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.rpc.thrift.TCLIService; @@ -54,13 +55,8 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ @Override - public void run() { + protected void initializeServer() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -151,6 +147,19 @@ public void run() { + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { httpServer.join(); } catch (Throwable t) { LOG.error( From 9d95f1b010293c63eff14ed28773c00c35207dee Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 15 Jun 2020 07:25:56 +0000 Subject: [PATCH 011/384] [SPARK-31992][SQL] Benchmark the EXCEPTION rebase mode ### What changes were proposed in this pull request? - Modify `DateTimeRebaseBenchmark` to benchmark the default date-time rebasing mode - `EXCEPTION` for saving/loading dates/timestamps from/to parquet files. The mode is benchmarked for modern timestamps after 1900-01-01 00:00:00Z and dates after 1582-10-15. - Regenerate benchmark results 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 | ### Why are the changes needed? The `EXCEPTION` rebasing mode is the default mode of the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.datetimeRebaseModeInWrite`. The changes are needed to improve benchmark coverage for default settings. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running the benchmark and check results manually. Closes #28829 from MaxGekk/benchmark-exception-mode. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../DateTimeRebaseBenchmark-jdk11-results.txt | 156 ++++++++++-------- .../DateTimeRebaseBenchmark-results.txt | 156 ++++++++++-------- .../benchmark/DateTimeRebaseBenchmark.scala | 28 ++-- 3 files changed, 183 insertions(+), 157 deletions(-) diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt index 5d107c1e50fd8..05896a4d69b47 100644 --- a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt @@ -6,97 +6,109 @@ 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 Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 20802 20802 0 4.8 208.0 1.0X -before 1582, noop 10728 10728 0 9.3 107.3 1.9X -after 1582, rebase off 32924 32924 0 3.0 329.2 0.6X -after 1582, rebase on 32627 32627 0 3.1 326.3 0.6X -before 1582, rebase off 21576 21576 0 4.6 215.8 1.0X -before 1582, rebase on 23115 23115 0 4.3 231.2 0.9X +after 1582, noop 20023 20023 0 5.0 200.2 1.0X +before 1582, noop 10729 10729 0 9.3 107.3 1.9X +after 1582, rebase EXCEPTION 31834 31834 0 3.1 318.3 0.6X +after 1582, rebase LEGACY 31997 31997 0 3.1 320.0 0.6X +after 1582, rebase CORRECTED 31712 31712 0 3.2 317.1 0.6X +before 1582, rebase LEGACY 23663 23663 0 4.2 236.6 0.8X +before 1582, rebase CORRECTED 22749 22749 0 4.4 227.5 0.9X 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 Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off, rebase off 12880 12984 178 7.8 128.8 1.0X -after 1582, vec off, rebase on 13118 13255 174 7.6 131.2 1.0X -after 1582, vec on, rebase off 3645 3698 76 27.4 36.4 3.5X -after 1582, vec on, rebase on 3709 3727 15 27.0 37.1 3.5X -before 1582, vec off, rebase off 13014 13051 36 7.7 130.1 1.0X -before 1582, vec off, rebase on 14195 14242 48 7.0 142.0 0.9X -before 1582, vec on, rebase off 3680 3773 92 27.2 36.8 3.5X -before 1582, vec on, rebase on 4310 4381 87 23.2 43.1 3.0X +after 1582, vec off, rebase EXCEPTION 12984 13262 257 7.7 129.8 1.0X +after 1582, vec off, rebase LEGACY 13278 13330 50 7.5 132.8 1.0X +after 1582, vec off, rebase CORRECTED 13202 13255 50 7.6 132.0 1.0X +after 1582, vec on, rebase EXCEPTION 3823 3853 40 26.2 38.2 3.4X +after 1582, vec on, rebase LEGACY 3846 3876 27 26.0 38.5 3.4X +after 1582, vec on, rebase CORRECTED 3775 3838 62 26.5 37.7 3.4X +before 1582, vec off, rebase LEGACY 13671 13692 26 7.3 136.7 0.9X +before 1582, vec off, rebase CORRECTED 13387 13476 106 7.5 133.9 1.0X +before 1582, vec on, rebase LEGACY 4477 4484 7 22.3 44.8 2.9X +before 1582, vec on, rebase CORRECTED 3729 3773 50 26.8 37.3 3.5X 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 Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 3026 3026 0 33.1 30.3 1.0X -before 1900, noop 2995 2995 0 33.4 30.0 1.0X -after 1900, rebase off 24294 24294 0 4.1 242.9 0.1X -after 1900, rebase on 24480 24480 0 4.1 244.8 0.1X -before 1900, rebase off 31120 31120 0 3.2 311.2 0.1X -before 1900, rebase on 31201 31201 0 3.2 312.0 0.1X +after 1900, noop 3020 3020 0 33.1 30.2 1.0X +before 1900, noop 3013 3013 0 33.2 30.1 1.0X +after 1900, rebase EXCEPTION 28796 28796 0 3.5 288.0 0.1X +after 1900, rebase LEGACY 28869 28869 0 3.5 288.7 0.1X +after 1900, rebase CORRECTED 28522 28522 0 3.5 285.2 0.1X +before 1900, rebase LEGACY 30594 30594 0 3.3 305.9 0.1X +before 1900, rebase CORRECTED 30743 30743 0 3.3 307.4 0.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 Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 18283 18309 39 5.5 182.8 1.0X -after 1900, vec off, rebase on 18235 18269 53 5.5 182.4 1.0X -after 1900, vec on, rebase off 9563 9589 27 10.5 95.6 1.9X -after 1900, vec on, rebase on 9463 9554 81 10.6 94.6 1.9X -before 1900, vec off, rebase off 21377 21469 118 4.7 213.8 0.9X -before 1900, vec off, rebase on 21265 21422 156 4.7 212.7 0.9X -before 1900, vec on, rebase off 12481 12524 46 8.0 124.8 1.5X -before 1900, vec on, rebase on 12360 12482 105 8.1 123.6 1.5X +after 1900, vec off, rebase EXCEPTION 19325 19468 135 5.2 193.3 1.0X +after 1900, vec off, rebase LEGACY 19568 19602 30 5.1 195.7 1.0X +after 1900, vec off, rebase CORRECTED 19532 19538 6 5.1 195.3 1.0X +after 1900, vec on, rebase EXCEPTION 9884 9990 94 10.1 98.8 2.0X +after 1900, vec on, rebase LEGACY 9933 9985 49 10.1 99.3 1.9X +after 1900, vec on, rebase CORRECTED 9967 10043 76 10.0 99.7 1.9X +before 1900, vec off, rebase LEGACY 24162 24198 37 4.1 241.6 0.8X +before 1900, vec off, rebase CORRECTED 24034 24056 20 4.2 240.3 0.8X +before 1900, vec on, rebase LEGACY 12548 12625 72 8.0 125.5 1.5X +before 1900, vec on, rebase CORRECTED 12580 12660 115 7.9 125.8 1.5X 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 Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2984 2984 0 33.5 29.8 1.0X -before 1900, noop 3003 3003 0 33.3 30.0 1.0X -after 1900, rebase off 15814 15814 0 6.3 158.1 0.2X -after 1900, rebase on 16250 16250 0 6.2 162.5 0.2X -before 1900, rebase off 16026 16026 0 6.2 160.3 0.2X -before 1900, rebase on 19735 19735 0 5.1 197.3 0.2X +after 1900, noop 3159 3159 0 31.7 31.6 1.0X +before 1900, noop 3038 3038 0 32.9 30.4 1.0X +after 1900, rebase EXCEPTION 16885 16885 0 5.9 168.8 0.2X +after 1900, rebase LEGACY 17171 17171 0 5.8 171.7 0.2X +after 1900, rebase CORRECTED 17353 17353 0 5.8 173.5 0.2X +before 1900, rebase LEGACY 20579 20579 0 4.9 205.8 0.2X +before 1900, rebase CORRECTED 17544 17544 0 5.7 175.4 0.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 Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 15292 15351 57 6.5 152.9 1.0X -after 1900, vec off, rebase on 15753 15886 173 6.3 157.5 1.0X -after 1900, vec on, rebase off 4879 4923 52 20.5 48.8 3.1X -after 1900, vec on, rebase on 5018 5038 18 19.9 50.2 3.0X -before 1900, vec off, rebase off 15257 15311 53 6.6 152.6 1.0X -before 1900, vec off, rebase on 18459 18537 90 5.4 184.6 0.8X -before 1900, vec on, rebase off 4929 4946 15 20.3 49.3 3.1X -before 1900, vec on, rebase on 8254 8339 93 12.1 82.5 1.9X +after 1900, vec off, rebase EXCEPTION 16304 16345 58 6.1 163.0 1.0X +after 1900, vec off, rebase LEGACY 16503 16585 75 6.1 165.0 1.0X +after 1900, vec off, rebase CORRECTED 16413 16463 44 6.1 164.1 1.0X +after 1900, vec on, rebase EXCEPTION 5017 5034 29 19.9 50.2 3.2X +after 1900, vec on, rebase LEGACY 5060 5094 30 19.8 50.6 3.2X +after 1900, vec on, rebase CORRECTED 4969 4971 1 20.1 49.7 3.3X +before 1900, vec off, rebase LEGACY 19767 20001 203 5.1 197.7 0.8X +before 1900, vec off, rebase CORRECTED 16421 16465 38 6.1 164.2 1.0X +before 1900, vec on, rebase LEGACY 8535 8608 64 11.7 85.4 1.9X +before 1900, vec on, rebase CORRECTED 5044 5077 32 19.8 50.4 3.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 Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2987 2987 0 33.5 29.9 1.0X -before 1900, noop 3002 3002 0 33.3 30.0 1.0X -after 1900, rebase off 15215 15215 0 6.6 152.1 0.2X -after 1900, rebase on 15577 15577 0 6.4 155.8 0.2X -before 1900, rebase off 15505 15505 0 6.4 155.1 0.2X -before 1900, rebase on 19143 19143 0 5.2 191.4 0.2X +after 1900, noop 2995 2995 0 33.4 29.9 1.0X +before 1900, noop 2981 2981 0 33.5 29.8 1.0X +after 1900, rebase EXCEPTION 16196 16196 0 6.2 162.0 0.2X +after 1900, rebase LEGACY 16550 16550 0 6.0 165.5 0.2X +after 1900, rebase CORRECTED 16908 16908 0 5.9 169.1 0.2X +before 1900, rebase LEGACY 20087 20087 0 5.0 200.9 0.1X +before 1900, rebase CORRECTED 17171 17171 0 5.8 171.7 0.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 Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 15330 15436 113 6.5 153.3 1.0X -after 1900, vec off, rebase on 15515 15549 30 6.4 155.1 1.0X -after 1900, vec on, rebase off 6056 6074 19 16.5 60.6 2.5X -after 1900, vec on, rebase on 6376 6390 14 15.7 63.8 2.4X -before 1900, vec off, rebase off 15490 15523 36 6.5 154.9 1.0X -before 1900, vec off, rebase on 18613 18685 118 5.4 186.1 0.8X -before 1900, vec on, rebase off 6065 6109 41 16.5 60.6 2.5X -before 1900, vec on, rebase on 9052 9082 32 11.0 90.5 1.7X +after 1900, vec off, rebase EXCEPTION 16688 16787 88 6.0 166.9 1.0X +after 1900, vec off, rebase LEGACY 17383 17462 73 5.8 173.8 1.0X +after 1900, vec off, rebase CORRECTED 17317 17329 11 5.8 173.2 1.0X +after 1900, vec on, rebase EXCEPTION 6342 6348 6 15.8 63.4 2.6X +after 1900, vec on, rebase LEGACY 6500 6521 18 15.4 65.0 2.6X +after 1900, vec on, rebase CORRECTED 6164 6172 11 16.2 61.6 2.7X +before 1900, vec off, rebase LEGACY 20575 20665 81 4.9 205.7 0.8X +before 1900, vec off, rebase CORRECTED 17239 17290 61 5.8 172.4 1.0X +before 1900, vec on, rebase LEGACY 9310 9373 60 10.7 93.1 1.8X +before 1900, vec on, rebase CORRECTED 6091 6105 16 16.4 60.9 2.7X ================================================================================================ @@ -107,36 +119,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 Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 20653 20653 0 4.8 206.5 1.0X -before 1582, noop 10707 10707 0 9.3 107.1 1.9X -after 1582 28288 28288 0 3.5 282.9 0.7X -before 1582 19196 19196 0 5.2 192.0 1.1X +after 1582, noop 19583 19583 0 5.1 195.8 1.0X +before 1582, noop 10711 10711 0 9.3 107.1 1.8X +after 1582 27864 27864 0 3.6 278.6 0.7X +before 1582 19648 19648 0 5.1 196.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 Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off 10596 10621 37 9.4 106.0 1.0X -after 1582, vec on 3886 3938 61 25.7 38.9 2.7X -before 1582, vec off 10955 10984 26 9.1 109.6 1.0X -before 1582, vec on 4236 4258 24 23.6 42.4 2.5X +after 1582, vec off 10383 10560 192 9.6 103.8 1.0X +after 1582, vec on 3844 3864 33 26.0 38.4 2.7X +before 1582, vec off 10867 10916 48 9.2 108.7 1.0X +before 1582, vec on 4158 4170 12 24.0 41.6 2.5X 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 Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2988 2988 0 33.5 29.9 1.0X -before 1900, noop 3007 3007 0 33.3 30.1 1.0X -after 1900 18082 18082 0 5.5 180.8 0.2X -before 1900 22669 22669 0 4.4 226.7 0.1X +after 1900, noop 2989 2989 0 33.5 29.9 1.0X +before 1900, noop 3000 3000 0 33.3 30.0 1.0X +after 1900 19426 19426 0 5.1 194.3 0.2X +before 1900 23282 23282 0 4.3 232.8 0.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 Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off 12029 12035 9 8.3 120.3 1.0X -after 1900, vec on 5194 5197 3 19.3 51.9 2.3X -before 1900, vec off 14853 14875 23 6.7 148.5 0.8X -before 1900, vec on 7797 7836 60 12.8 78.0 1.5X +after 1900, vec off 12089 12102 15 8.3 120.9 1.0X +after 1900, vec on 5210 5325 100 19.2 52.1 2.3X +before 1900, vec off 15320 15373 46 6.5 153.2 0.8X +before 1900, vec on 7937 7970 48 12.6 79.4 1.5X diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt index f2af3cc000f04..3e94d6c6fcfa7 100644 --- a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt @@ -6,97 +6,109 @@ 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 Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 23567 23567 0 4.2 235.7 1.0X -before 1582, noop 10570 10570 0 9.5 105.7 2.2X -after 1582, rebase off 35335 35335 0 2.8 353.3 0.7X -after 1582, rebase on 35645 35645 0 2.8 356.5 0.7X -before 1582, rebase off 21824 21824 0 4.6 218.2 1.1X -before 1582, rebase on 22532 22532 0 4.4 225.3 1.0X +after 1582, noop 23300 23300 0 4.3 233.0 1.0X +before 1582, noop 10585 10585 0 9.4 105.9 2.2X +after 1582, rebase EXCEPTION 35215 35215 0 2.8 352.1 0.7X +after 1582, rebase LEGACY 34927 34927 0 2.9 349.3 0.7X +after 1582, rebase CORRECTED 35479 35479 0 2.8 354.8 0.7X +before 1582, rebase LEGACY 22767 22767 0 4.4 227.7 1.0X +before 1582, rebase CORRECTED 22527 22527 0 4.4 225.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 Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off, rebase off 13194 13266 81 7.6 131.9 1.0X -after 1582, vec off, rebase on 13402 13466 89 7.5 134.0 1.0X -after 1582, vec on, rebase off 3627 3657 29 27.6 36.3 3.6X -after 1582, vec on, rebase on 3818 3839 26 26.2 38.2 3.5X -before 1582, vec off, rebase off 13075 13146 115 7.6 130.7 1.0X -before 1582, vec off, rebase on 13794 13804 13 7.2 137.9 1.0X -before 1582, vec on, rebase off 3655 3675 21 27.4 36.6 3.6X -before 1582, vec on, rebase on 4579 4634 72 21.8 45.8 2.9X +after 1582, vec off, rebase EXCEPTION 13480 13577 94 7.4 134.8 1.0X +after 1582, vec off, rebase LEGACY 13466 13586 118 7.4 134.7 1.0X +after 1582, vec off, rebase CORRECTED 13526 13558 41 7.4 135.3 1.0X +after 1582, vec on, rebase EXCEPTION 3759 3778 28 26.6 37.6 3.6X +after 1582, vec on, rebase LEGACY 3957 4004 57 25.3 39.6 3.4X +after 1582, vec on, rebase CORRECTED 3739 3755 25 26.7 37.4 3.6X +before 1582, vec off, rebase LEGACY 13986 14038 67 7.1 139.9 1.0X +before 1582, vec off, rebase CORRECTED 13453 13491 49 7.4 134.5 1.0X +before 1582, vec on, rebase LEGACY 4716 4724 10 21.2 47.2 2.9X +before 1582, vec on, rebase CORRECTED 3701 3750 50 27.0 37.0 3.6X 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 Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2671 2671 0 37.4 26.7 1.0X -before 1900, noop 2685 2685 0 37.2 26.8 1.0X -after 1900, rebase off 23899 23899 0 4.2 239.0 0.1X -after 1900, rebase on 24030 24030 0 4.2 240.3 0.1X -before 1900, rebase off 30178 30178 0 3.3 301.8 0.1X -before 1900, rebase on 30127 30127 0 3.3 301.3 0.1X +after 1900, noop 2790 2790 0 35.8 27.9 1.0X +before 1900, noop 2812 2812 0 35.6 28.1 1.0X +after 1900, rebase EXCEPTION 24789 24789 0 4.0 247.9 0.1X +after 1900, rebase LEGACY 24539 24539 0 4.1 245.4 0.1X +after 1900, rebase CORRECTED 24543 24543 0 4.1 245.4 0.1X +before 1900, rebase LEGACY 30496 30496 0 3.3 305.0 0.1X +before 1900, rebase CORRECTED 30428 30428 0 3.3 304.3 0.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 Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 16613 16685 75 6.0 166.1 1.0X -after 1900, vec off, rebase on 16487 16541 47 6.1 164.9 1.0X -after 1900, vec on, rebase off 8840 8870 49 11.3 88.4 1.9X -after 1900, vec on, rebase on 8795 8813 20 11.4 87.9 1.9X -before 1900, vec off, rebase off 20400 20441 62 4.9 204.0 0.8X -before 1900, vec off, rebase on 20430 20481 60 4.9 204.3 0.8X -before 1900, vec on, rebase off 12211 12290 73 8.2 122.1 1.4X -before 1900, vec on, rebase on 12231 12321 95 8.2 122.3 1.4X +after 1900, vec off, rebase EXCEPTION 17106 17192 75 5.8 171.1 1.0X +after 1900, vec off, rebase LEGACY 17273 17337 55 5.8 172.7 1.0X +after 1900, vec off, rebase CORRECTED 17073 17215 128 5.9 170.7 1.0X +after 1900, vec on, rebase EXCEPTION 8903 8976 117 11.2 89.0 1.9X +after 1900, vec on, rebase LEGACY 8793 8876 84 11.4 87.9 1.9X +after 1900, vec on, rebase CORRECTED 8820 8878 53 11.3 88.2 1.9X +before 1900, vec off, rebase LEGACY 20997 21069 82 4.8 210.0 0.8X +before 1900, vec off, rebase CORRECTED 20874 20946 90 4.8 208.7 0.8X +before 1900, vec on, rebase LEGACY 12024 12090 58 8.3 120.2 1.4X +before 1900, vec on, rebase CORRECTED 12020 12069 64 8.3 120.2 1.4X 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 Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2836 2836 0 35.3 28.4 1.0X -before 1900, noop 2812 2812 0 35.6 28.1 1.0X -after 1900, rebase off 15976 15976 0 6.3 159.8 0.2X -after 1900, rebase on 16197 16197 0 6.2 162.0 0.2X -before 1900, rebase off 16140 16140 0 6.2 161.4 0.2X -before 1900, rebase on 20410 20410 0 4.9 204.1 0.1X +after 1900, noop 2939 2939 0 34.0 29.4 1.0X +before 1900, noop 2917 2917 0 34.3 29.2 1.0X +after 1900, rebase EXCEPTION 15954 15954 0 6.3 159.5 0.2X +after 1900, rebase LEGACY 16402 16402 0 6.1 164.0 0.2X +after 1900, rebase CORRECTED 16541 16541 0 6.0 165.4 0.2X +before 1900, rebase LEGACY 20500 20500 0 4.9 205.0 0.1X +before 1900, rebase CORRECTED 16764 16764 0 6.0 167.6 0.2X 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 Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 15297 15324 40 6.5 153.0 1.0X -after 1900, vec off, rebase on 15771 15832 59 6.3 157.7 1.0X -after 1900, vec on, rebase off 4922 4949 32 20.3 49.2 3.1X -after 1900, vec on, rebase on 5392 5411 17 18.5 53.9 2.8X -before 1900, vec off, rebase off 15227 15385 141 6.6 152.3 1.0X -before 1900, vec off, rebase on 19611 19658 41 5.1 196.1 0.8X -before 1900, vec on, rebase off 4965 5013 54 20.1 49.6 3.1X -before 1900, vec on, rebase on 9847 9873 43 10.2 98.5 1.6X +after 1900, vec off, rebase EXCEPTION 15607 15655 81 6.4 156.1 1.0X +after 1900, vec off, rebase LEGACY 15616 15676 54 6.4 156.2 1.0X +after 1900, vec off, rebase CORRECTED 15634 15732 108 6.4 156.3 1.0X +after 1900, vec on, rebase EXCEPTION 5041 5057 16 19.8 50.4 3.1X +after 1900, vec on, rebase LEGACY 5516 5539 29 18.1 55.2 2.8X +after 1900, vec on, rebase CORRECTED 5087 5104 28 19.7 50.9 3.1X +before 1900, vec off, rebase LEGACY 19262 19338 79 5.2 192.6 0.8X +before 1900, vec off, rebase CORRECTED 15718 15755 53 6.4 157.2 1.0X +before 1900, vec on, rebase LEGACY 10147 10240 114 9.9 101.5 1.5X +before 1900, vec on, rebase CORRECTED 5062 5080 21 19.8 50.6 3.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 Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2818 2818 0 35.5 28.2 1.0X -before 1900, noop 2805 2805 0 35.6 28.1 1.0X -after 1900, rebase off 15182 15182 0 6.6 151.8 0.2X -after 1900, rebase on 15614 15614 0 6.4 156.1 0.2X -before 1900, rebase off 15404 15404 0 6.5 154.0 0.2X -before 1900, rebase on 19747 19747 0 5.1 197.5 0.1X +after 1900, noop 2915 2915 0 34.3 29.2 1.0X +before 1900, noop 2894 2894 0 34.6 28.9 1.0X +after 1900, rebase EXCEPTION 15545 15545 0 6.4 155.4 0.2X +after 1900, rebase LEGACY 15840 15840 0 6.3 158.4 0.2X +after 1900, rebase CORRECTED 16324 16324 0 6.1 163.2 0.2X +before 1900, rebase LEGACY 20359 20359 0 4.9 203.6 0.1X +before 1900, rebase CORRECTED 16292 16292 0 6.1 162.9 0.2X 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 Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 15622 15649 24 6.4 156.2 1.0X -after 1900, vec off, rebase on 15572 15677 119 6.4 155.7 1.0X -after 1900, vec on, rebase off 6345 6358 15 15.8 63.5 2.5X -after 1900, vec on, rebase on 6780 6834 92 14.8 67.8 2.3X -before 1900, vec off, rebase off 15540 15584 38 6.4 155.4 1.0X -before 1900, vec off, rebase on 19590 19653 55 5.1 195.9 0.8X -before 1900, vec on, rebase off 6374 6381 10 15.7 63.7 2.5X -before 1900, vec on, rebase on 10530 10544 25 9.5 105.3 1.5X +after 1900, vec off, rebase EXCEPTION 15857 16015 223 6.3 158.6 1.0X +after 1900, vec off, rebase LEGACY 16174 16231 63 6.2 161.7 1.0X +after 1900, vec off, rebase CORRECTED 16353 16400 67 6.1 163.5 1.0X +after 1900, vec on, rebase EXCEPTION 6449 6459 9 15.5 64.5 2.5X +after 1900, vec on, rebase LEGACY 7028 7035 6 14.2 70.3 2.3X +after 1900, vec on, rebase CORRECTED 6585 6623 37 15.2 65.8 2.4X +before 1900, vec off, rebase LEGACY 19929 20027 95 5.0 199.3 0.8X +before 1900, vec off, rebase CORRECTED 16401 16451 49 6.1 164.0 1.0X +before 1900, vec on, rebase LEGACY 10517 10563 40 9.5 105.2 1.5X +before 1900, vec on, rebase CORRECTED 6659 6675 26 15.0 66.6 2.4X ================================================================================================ @@ -107,36 +119,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 Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 23825 23825 0 4.2 238.2 1.0X -before 1582, noop 10501 10501 0 9.5 105.0 2.3X -after 1582 32134 32134 0 3.1 321.3 0.7X -before 1582 19947 19947 0 5.0 199.5 1.2X +after 1582, noop 22782 22782 0 4.4 227.8 1.0X +before 1582, noop 10555 10555 0 9.5 105.6 2.2X +after 1582 31497 31497 0 3.2 315.0 0.7X +before 1582 19803 19803 0 5.0 198.0 1.2X 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 Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off 10034 10056 22 10.0 100.3 1.0X -after 1582, vec on 3664 3698 30 27.3 36.6 2.7X -before 1582, vec off 10472 10502 30 9.5 104.7 1.0X -before 1582, vec on 4052 4098 42 24.7 40.5 2.5X +after 1582, vec off 10180 10214 44 9.8 101.8 1.0X +after 1582, vec on 3785 3804 24 26.4 37.8 2.7X +before 1582, vec off 10537 10582 39 9.5 105.4 1.0X +before 1582, vec on 4117 4146 25 24.3 41.2 2.5X 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 Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2812 2812 0 35.6 28.1 1.0X -before 1900, noop 2801 2801 0 35.7 28.0 1.0X -after 1900 18290 18290 0 5.5 182.9 0.2X -before 1900 22344 22344 0 4.5 223.4 0.1X +after 1900, noop 2853 2853 0 35.1 28.5 1.0X +before 1900, noop 2999 2999 0 33.3 30.0 1.0X +after 1900 16757 16757 0 6.0 167.6 0.2X +before 1900 21542 21542 0 4.6 215.4 0.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 Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off 11257 11279 32 8.9 112.6 1.0X -after 1900, vec on 5296 5310 15 18.9 53.0 2.1X -before 1900, vec off 14700 14758 72 6.8 147.0 0.8X -before 1900, vec on 8576 8665 150 11.7 85.8 1.3X +after 1900, vec off 12212 12254 39 8.2 122.1 1.0X +after 1900, vec on 5369 5390 35 18.6 53.7 2.3X +before 1900, vec off 15661 15705 73 6.4 156.6 0.8X +before 1900, vec on 8720 8744 29 11.5 87.2 1.4X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala index d6167f98b5a51..7caaa5376db7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala @@ -25,8 +25,7 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.util.DateTimeConstants.SECONDS_PER_DAY import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, LA} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._ -import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType +import org.apache.spark.sql.internal.SQLConf.{LegacyBehaviorPolicy, ParquetOutputTimestampType} object DateTime extends Enumeration { type DateTime = Value @@ -118,12 +117,12 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { private def caseName( modernDates: Boolean, dateTime: DateTime, - rebase: Option[Boolean] = None, + mode: Option[LegacyBehaviorPolicy.Value] = None, vec: Option[Boolean] = None): String = { val period = if (modernDates) "after" else "before" val year = if (dateTime == DATE) 1582 else 1900 val vecFlag = vec.map(flagToStr).map(flag => s", vec $flag").getOrElse("") - val rebaseFlag = rebase.map(flagToStr).map(flag => s", rebase $flag").getOrElse("") + val rebaseFlag = mode.map(_.toString).map(m => s", rebase $m").getOrElse("") s"$period $year$vecFlag$rebaseFlag" } @@ -131,10 +130,10 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { basePath: File, dateTime: DateTime, modernDates: Boolean, - rebase: Option[Boolean] = None): String = { + mode: Option[LegacyBehaviorPolicy.Value] = None): String = { val period = if (modernDates) "after" else "before" val year = if (dateTime == DATE) 1582 else 1900 - val rebaseFlag = rebase.map(flagToStr).map(flag => s"_$flag").getOrElse("") + val rebaseFlag = mode.map(_.toString).map(m => s"_$m").getOrElse("") basePath.getAbsolutePath + s"/${dateTime}_${period}_$year$rebaseFlag" } @@ -160,9 +159,10 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { output = output) benchmarkInputs(benchmark, rowsNum, dateTime) Seq(true, false).foreach { modernDates => - Seq(false, true).foreach { rebase => - benchmark.addCase(caseName(modernDates, dateTime, Some(rebase)), 1) { _ => - val mode = if (rebase) LEGACY else CORRECTED + LegacyBehaviorPolicy.values + .filterNot(v => !modernDates && v == LegacyBehaviorPolicy.EXCEPTION) + .foreach { mode => + benchmark.addCase(caseName(modernDates, dateTime, Some(mode)), 1) { _ => withSQLConf( SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> getOutputType(dateTime), SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> mode.toString) { @@ -170,7 +170,7 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { .write .mode("overwrite") .format("parquet") - .save(getPath(path, dateTime, modernDates, Some(rebase))) + .save(getPath(path, dateTime, modernDates, Some(mode))) } } } @@ -181,13 +181,15 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { s"Load $dateTime from parquet", rowsNum, output = output) Seq(true, false).foreach { modernDates => Seq(false, true).foreach { vec => - Seq(false, true).foreach { rebase => - val name = caseName(modernDates, dateTime, Some(rebase), Some(vec)) + LegacyBehaviorPolicy.values + .filterNot(v => !modernDates && v == LegacyBehaviorPolicy.EXCEPTION) + .foreach { mode => + val name = caseName(modernDates, dateTime, Some(mode), Some(vec)) benchmark2.addCase(name, 3) { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vec.toString) { spark.read .format("parquet") - .load(getPath(path, dateTime, modernDates, Some(rebase))) + .load(getPath(path, dateTime, modernDates, Some(mode))) .noop() } } From f83cb3cbb3ce3f22fd122bce620917bfd0699ce7 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 15 Jun 2020 08:49:03 -0500 Subject: [PATCH 012/384] [SPARK-31925][ML] Summary.totalIterations greater than maxIters ### What changes were proposed in this pull request? In LogisticRegression and LinearRegression, if set maxIter=n, the model.summary.totalIterations returns n+1 if the training procedure does not drop out. This is because we use ```objectiveHistory.length``` as totalIterations, but ```objectiveHistory``` contains init sate, thus ```objectiveHistory.length``` is 1 larger than number of training iterations. ### Why are the changes needed? correctness ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? add new tests and also modify existing tests Closes #28786 from huaxingao/summary_iter. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../ml/classification/LogisticRegression.scala | 12 +++++++++--- .../spark/ml/regression/LinearRegression.scala | 8 +++++--- .../JavaLogisticRegressionSuite.java | 2 +- .../classification/LogisticRegressionSuite.scala | 13 ++++++++++--- .../ml/regression/LinearRegressionSuite.scala | 15 +++++++++++++++ python/pyspark/ml/classification.py | 3 ++- python/pyspark/ml/tests/test_training_summary.py | 2 +- 7 files changed, 43 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 0d1350640c74a..1f5976c59235b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -594,7 +594,7 @@ class LogisticRegression @Since("1.2.0") ( Vectors.dense(if (numClasses == 2) Double.PositiveInfinity else Double.NegativeInfinity) } if (instances.getStorageLevel != StorageLevel.NONE) instances.unpersist() - return createModel(dataset, numClasses, coefMatrix, interceptVec, Array.empty) + return createModel(dataset, numClasses, coefMatrix, interceptVec, Array(0.0)) } if (!$(fitIntercept) && isConstantLabel) { @@ -1545,13 +1545,19 @@ sealed trait LogisticRegressionSummary extends Serializable { */ sealed trait LogisticRegressionTrainingSummary extends LogisticRegressionSummary { - /** objective function (scaled loss + regularization) at each iteration. */ + /** + * objective function (scaled loss + regularization) at each iteration. + * It contains one more element, the initial state, than number of iterations. + */ @Since("1.5.0") def objectiveHistory: Array[Double] /** Number of training iterations. */ @Since("1.5.0") - def totalIterations: Int = objectiveHistory.length + def totalIterations: Int = { + assert(objectiveHistory.length > 0, s"objectiveHistory length should be greater than 1.") + objectiveHistory.length - 1 + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 8b6ede3bb362c..d9f09c097292a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -433,7 +433,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String Vectors.dense(Array.fill(dim)(1.0)) } - val (parameters, objectiveHistory) = if ($(blockSize) == 1) { + val (parameters, objectiveHistory) = if ($(blockSize) == 1) { trainOnRows(instances, yMean, yStd, featuresMean, featuresStd, initialValues, regularization, optimizer) } else { @@ -939,8 +939,10 @@ class LinearRegressionTrainingSummary private[regression] ( * @see `LinearRegression.solver` */ @Since("1.5.0") - val totalIterations = objectiveHistory.length - + val totalIterations = { + assert(objectiveHistory.length > 0, s"objectiveHistory length should be greater than 1.") + objectiveHistory.length - 1 + } } /** diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index 49ac49339415a..7c63a8755b4f3 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -143,6 +143,6 @@ public void logisticRegressionTrainingSummary() { LogisticRegressionModel model = lr.fit(dataset); LogisticRegressionTrainingSummary summary = model.summary(); - Assert.assertEquals(summary.totalIterations(), summary.objectiveHistory().length); + Assert.assertEquals(summary.totalIterations(), summary.objectiveHistory().length - 1); } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 25e9697d64855..30c21d8b06670 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -266,6 +266,8 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(blorModel.summary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) assert(blorModel.summary.asBinary.isInstanceOf[BinaryLogisticRegressionSummary]) assert(blorModel.binarySummary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) + assert(blorModel.summary.totalIterations == 1) + assert(blorModel.binarySummary.totalIterations == 1) val mlorModel = lr.setFamily("multinomial").fit(smallMultinomialDataset) assert(mlorModel.summary.isInstanceOf[LogisticRegressionTrainingSummary]) @@ -279,6 +281,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { mlorModel.summary.asBinary } } + assert(mlorModel.summary.totalIterations == 1) val mlorBinaryModel = lr.setFamily("multinomial").fit(smallBinaryDataset) assert(mlorBinaryModel.summary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) @@ -2570,7 +2573,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { rows.map(_.getDouble(0)).toArray === binaryExpected } } - assert(model2.summary.totalIterations === 1) + assert(model2.summary.totalIterations === 0) val lr3 = new LogisticRegression().setFamily("multinomial") val model3 = lr3.fit(smallMultinomialDataset) @@ -2585,7 +2588,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { rows.map(_.getDouble(0)).toArray === multinomialExpected } } - assert(model4.summary.totalIterations === 1) + assert(model4.summary.totalIterations === 0) } test("binary logistic regression with all labels the same") { @@ -2605,6 +2608,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(allZeroInterceptModel.coefficients ~== Vectors.dense(0.0) absTol 1E-3) assert(allZeroInterceptModel.intercept === Double.NegativeInfinity) assert(allZeroInterceptModel.summary.totalIterations === 0) + assert(allZeroInterceptModel.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) val allOneInterceptModel = lrIntercept .setLabelCol("oneLabel") @@ -2612,6 +2616,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(allOneInterceptModel.coefficients ~== Vectors.dense(0.0) absTol 1E-3) assert(allOneInterceptModel.intercept === Double.PositiveInfinity) assert(allOneInterceptModel.summary.totalIterations === 0) + assert(allOneInterceptModel.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) // fitIntercept=false val lrNoIntercept = new LogisticRegression() @@ -2647,6 +2652,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(pred === 4.0) } assert(model.summary.totalIterations === 0) + assert(model.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) // force the model to be trained with only one class val constantZeroData = Seq( @@ -2660,7 +2666,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(prob === Vectors.dense(Array(1.0))) assert(pred === 0.0) } - assert(modelZeroLabel.summary.totalIterations > 0) + assert(modelZeroLabel.summary.totalIterations === 0) // ensure that the correct value is predicted when numClasses passed through metadata val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(6).toMetadata() @@ -2675,6 +2681,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(pred === 4.0) } require(modelWithMetadata.summary.totalIterations === 0) + assert(model.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) } test("compressed storage for constant label") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index c4a94ff2d6f44..fb70883bffc5f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -761,6 +761,7 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe .fit(datasetWithWeightConstantLabel) if (fitIntercept) { assert(model1.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) + assert(model1.summary.totalIterations === 0) } val model2 = new LinearRegression() .setFitIntercept(fitIntercept) @@ -768,6 +769,7 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe .setSolver("l-bfgs") .fit(datasetWithWeightZeroLabel) assert(model2.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) + assert(model2.summary.totalIterations === 0) } } } @@ -940,6 +942,19 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe } } + test("linear regression training summary totalIterations") { + Seq(1, 5, 10, 20).foreach { maxIter => + val trainer = new LinearRegression().setSolver("l-bfgs").setMaxIter(maxIter) + val model = trainer.fit(datasetWithDenseFeature) + assert(model.summary.totalIterations <= maxIter) + } + Seq("auto", "normal").foreach { solver => + val trainer = new LinearRegression().setSolver(solver) + val model = trainer.fit(datasetWithDenseFeature) + assert(model.summary.totalIterations === 0) + } + } + test("linear regression with weighted samples") { val sqlContext = spark.sqlContext import sqlContext.implicits._ diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 734c393db2a26..3f3699ce53b51 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1119,7 +1119,8 @@ class LogisticRegressionTrainingSummary(LogisticRegressionSummary): def objectiveHistory(self): """ Objective function (scaled loss + regularization) at each - iteration. + iteration. It contains one more element, the initial state, + than number of iterations. """ return self._call_java("objectiveHistory") diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index b5054095d190b..ac944d8397a86 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -42,7 +42,7 @@ def test_linear_regression_summary(self): self.assertTrue(model.hasSummary) s = model.summary # test that api is callable and returns expected types - self.assertGreater(s.totalIterations, 0) + self.assertEqual(s.totalIterations, 0) self.assertTrue(isinstance(s.predictions, DataFrame)) self.assertEqual(s.predictionCol, "prediction") self.assertEqual(s.labelCol, "label") From 7f7b4dd5199e7c185aedf51fccc400c7072bed05 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 15 Jun 2020 07:48:48 -0700 Subject: [PATCH 013/384] [SPARK-31990][SS] Use toSet.toSeq in Dataset.dropDuplicates ### What changes were proposed in this pull request? This PR partially revert SPARK-31292 in order to provide a hot-fix for a bug in `Dataset.dropDuplicates`; we must preserve the input order of `colNames` for `groupCols` because the Streaming's state store depends on the `groupCols` order. ### Why are the changes needed? Bug fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests in `DataFrameSuite`. Closes #28830 from maropu/SPARK-31990. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index ae8d33d8558ba..524e231eb7eb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2541,7 +2541,9 @@ class Dataset[T] private[sql]( def dropDuplicates(colNames: Seq[String]): Dataset[T] = withTypedPlan { val resolver = sparkSession.sessionState.analyzer.resolver val allColumns = queryExecution.analyzed.output - val groupCols = colNames.distinct.flatMap { (colName: String) => + // SPARK-31990: We must keep `toSet.toSeq` here because of the backward compatibility issue + // (the Streaming's state store depends on the `groupCols` order). + val groupCols = colNames.toSet.toSeq.flatMap { (colName: String) => // It is possibly there are more than one columns with the same name, // so we call filter instead of find. val cols = allColumns.filter(col => resolver(col.name, colName)) From eae1747b663adaa612681d799dcaa11915cd1692 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 15 Jun 2020 08:09:07 -0700 Subject: [PATCH 014/384] [SPARK-31959][SQL][TESTS][FOLLOWUP] Adopt the test "SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945" to outdated tzdb ### What changes were proposed in this pull request? Old JDK can have outdated time zone database in which `Asia/Hong_Kong` doesn't have timestamp overlapping in 1946 at all. This PR changes the test "SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945" in `RebaseDateTimeSuite`, and makes it tolerant to the case. ### Why are the changes needed? To fix the test failures on old JDK w/ outdated tzdb like on Jenkins machine `research-jenkins-worker-09`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running the test on old JDK Closes #28832 from MaxGekk/HongKong-tz-1945-followup. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/util/RebaseDateTimeSuite.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index e3af64a562e23..8a51f158a9429 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -421,7 +421,8 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { var ldt = LocalDateTime.of(1945, 11, 18, 1, 30, 0) var earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) var laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) - if (earlierMicros + MICROS_PER_HOUR != laterMicros) { + var overlapInterval = MICROS_PER_HOUR + if (earlierMicros + overlapInterval != laterMicros) { // Old JDK might have an outdated time zone database. // See https://bugs.openjdk.java.net/browse/JDK-8228469: "Hong Kong ... Its 1945 transition // from JST to HKT was on 11-18 at 02:00, not 09-15 at 00:00" @@ -429,14 +430,15 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { ldt = LocalDateTime.of(1945, 9, 14, 23, 30, 0) earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) - assert(earlierMicros + MICROS_PER_HOUR === laterMicros) + // If time zone db doesn't have overlapping at all, set the overlap interval to zero. + overlapInterval = laterMicros - earlierMicros } val rebasedEarlierMicros = rebaseGregorianToJulianMicros(hkZid, earlierMicros) val rebasedLaterMicros = rebaseGregorianToJulianMicros(hkZid, laterMicros) def toTsStr(micros: Long): String = toJavaTimestamp(micros).toString assert(toTsStr(rebasedEarlierMicros) === expected) assert(toTsStr(rebasedLaterMicros) === expected) - assert(rebasedEarlierMicros + MICROS_PER_HOUR === rebasedLaterMicros) + assert(rebasedEarlierMicros + overlapInterval === rebasedLaterMicros) // Check optimized rebasing assert(rebaseGregorianToJulianMicros(earlierMicros) === rebasedEarlierMicros) assert(rebaseGregorianToJulianMicros(laterMicros) === rebasedLaterMicros) From 3698a14204dd861ea3ee3c14aa923123b52caba1 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 16 Jun 2020 00:27:45 +0900 Subject: [PATCH 015/384] [SPARK-26905][SQL] Follow the SQL:2016 reserved keywords ### What changes were proposed in this pull request? This PR intends to move keywords `ANTI`, `SEMI`, and `MINUS` from reserved to non-reserved. ### Why are the changes needed? To comply with the ANSI/SQL standard. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests. Closes #28807 from maropu/SPARK-26905-2. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- docs/sql-ref-ansi-compliance.md | 6 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 + .../ansi-sql-2016-reserved-keywords.txt | 401 ++++++++++++++++++ .../parser/TableIdentifierParserSuite.scala | 24 +- 4 files changed, 429 insertions(+), 5 deletions(-) create mode 100644 sql/catalyst/src/test/resources/ansi-sql-2016-reserved-keywords.txt diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index eab194c71ec79..e5ca7e9d10d59 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -135,7 +135,7 @@ Below is a list of all the keywords in Spark SQL. |ALTER|non-reserved|non-reserved|reserved| |ANALYZE|non-reserved|non-reserved|non-reserved| |AND|reserved|non-reserved|reserved| -|ANTI|reserved|strict-non-reserved|non-reserved| +|ANTI|non-reserved|strict-non-reserved|non-reserved| |ANY|reserved|non-reserved|reserved| |ARCHIVE|non-reserved|non-reserved|non-reserved| |ARRAY|non-reserved|non-reserved|reserved| @@ -264,7 +264,7 @@ Below is a list of all the keywords in Spark SQL. |MAP|non-reserved|non-reserved|non-reserved| |MATCHED|non-reserved|non-reserved|non-reserved| |MERGE|non-reserved|non-reserved|non-reserved| -|MINUS|reserved|strict-non-reserved|non-reserved| +|MINUS|not-reserved|strict-non-reserved|non-reserved| |MINUTE|reserved|non-reserved|reserved| |MONTH|reserved|non-reserved|reserved| |MSCK|non-reserved|non-reserved|non-reserved| @@ -325,7 +325,7 @@ Below is a list of all the keywords in Spark SQL. |SCHEMA|non-reserved|non-reserved|non-reserved| |SECOND|reserved|non-reserved|reserved| |SELECT|reserved|non-reserved|reserved| -|SEMI|reserved|strict-non-reserved|non-reserved| +|SEMI|non-reserved|strict-non-reserved|non-reserved| |SEPARATED|non-reserved|non-reserved|non-reserved| |SERDE|non-reserved|non-reserved|non-reserved| |SERDEPROPERTIES|non-reserved|non-reserved|non-reserved| diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 61aeafdc27f11..691fde8d48f94 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -1014,6 +1014,7 @@ ansiNonReserved | AFTER | ALTER | ANALYZE + | ANTI | ARCHIVE | ARRAY | ASC @@ -1146,10 +1147,12 @@ ansiNonReserved | ROW | ROWS | SCHEMA + | SEMI | SEPARATED | SERDE | SERDEPROPERTIES | SET + | SETMINUS | SETS | SHOW | SKEWED diff --git a/sql/catalyst/src/test/resources/ansi-sql-2016-reserved-keywords.txt b/sql/catalyst/src/test/resources/ansi-sql-2016-reserved-keywords.txt new file mode 100644 index 0000000000000..921491a4a4761 --- /dev/null +++ b/sql/catalyst/src/test/resources/ansi-sql-2016-reserved-keywords.txt @@ -0,0 +1,401 @@ +-- This file comes from: https://github.com/postgres/postgres/tree/master/doc/src/sgml/keywords +ABS +ACOS +ALL +ALLOCATE +ALTER +AND +ANY +ARE +ARRAY +ARRAY_AGG +ARRAY_MAX_CARDINALITY +AS +ASENSITIVE +ASIN +ASYMMETRIC +AT +ATAN +ATOMIC +AUTHORIZATION +AVG +BEGIN +BEGIN_FRAME +BEGIN_PARTITION +BETWEEN +BIGINT +BINARY +BLOB +BOOLEAN +BOTH +BY +CALL +CALLED +CARDINALITY +CASCADED +CASE +CAST +CEIL +CEILING +CHAR +CHAR_LENGTH +CHARACTER +CHARACTER_LENGTH +CHECK +CLASSIFIER +CLOB +CLOSE +COALESCE +COLLATE +COLLECT +COLUMN +COMMIT +CONDITION +CONNECT +CONSTRAINT +CONTAINS +CONVERT +COPY +CORR +CORRESPONDING +COS +COSH +COUNT +COVAR_POP +COVAR_SAMP +CREATE +CROSS +CUBE +CUME_DIST +CURRENT +CURRENT_CATALOG +CURRENT_DATE +CURRENT_DEFAULT_TRANSFORM_GROUP +CURRENT_PATH +CURRENT_ROLE +CURRENT_ROW +CURRENT_SCHEMA +CURRENT_TIME +CURRENT_TIMESTAMP +CURRENT_TRANSFORM_GROUP_FOR_TYPE +CURRENT_USER +CURSOR +CYCLE +DATE +DAY +DEALLOCATE +DEC +DECIMAL +DECFLOAT +DECLARE +DEFAULT +DEFINE +DELETE +DENSE_RANK +DEREF +DESCRIBE +DETERMINISTIC +DISCONNECT +DISTINCT +DOUBLE +DROP +DYNAMIC +EACH +ELEMENT +ELSE +EMPTY +END +END_FRAME +END_PARTITION +END-EXEC +EQUALS +ESCAPE +EVERY +EXCEPT +EXEC +EXECUTE +EXISTS +EXP +EXTERNAL +EXTRACT +FALSE +FETCH +FILTER +FIRST_VALUE +FLOAT +FLOOR +FOR +FOREIGN +FRAME_ROW +FREE +FROM +FULL +FUNCTION +FUSION +GET +GLOBAL +GRANT +GROUP +GROUPING +GROUPS +HAVING +HOLD +HOUR +IDENTITY +IN +INDICATOR +INITIAL +INNER +INOUT +INSENSITIVE +INSERT +INT +INTEGER +INTERSECT +INTERSECTION +INTERVAL +INTO +IS +JOIN +JSON_ARRAY +JSON_ARRAYAGG +JSON_EXISTS +JSON_OBJECT +JSON_OBJECTAGG +JSON_QUERY +JSON_TABLE +JSON_TABLE_PRIMITIVE +JSON_VALUE +LAG +LANGUAGE +LARGE +LAST_VALUE +LATERAL +LEAD +LEADING +LEFT +LIKE +LIKE_REGEX +LISTAGG +LN +LOCAL +LOCALTIME +LOCALTIMESTAMP +LOG +LOG10 +LOWER +MATCH +MATCH_NUMBER +MATCH_RECOGNIZE +MATCHES +MAX +MEASURES +MEMBER +MERGE +METHOD +MIN +MINUTE +MOD +MODIFIES +MODULE +MONTH +MULTISET +NATIONAL +NATURAL +NCHAR +NCLOB +NEW +NO +NONE +NORMALIZE +NOT +NTH_VALUE +NTILE +NULL +NULLIF +NUMERIC +OCTET_LENGTH +OCCURRENCES_REGEX +OF +OFFSET +OLD +OMIT +ON +ONE +ONLY +OPEN +OR +ORDER +OUT +OUTER +OVER +OVERLAPS +OVERLAY +PARAMETER +PARTITION +PATTERN +PER +PERCENT +PERCENT_RANK +PERCENTILE_CONT +PERCENTILE_DISC +PERIOD +PERMUTE +PORTION +POSITION +POSITION_REGEX +POWER +PRECEDES +PRECISION +PREPARE +PRIMARY +PROCEDURE +PTF +RANGE +RANK +READS +REAL +RECURSIVE +REF +REFERENCES +REFERENCING +REGR_AVGX +REGR_AVGY +REGR_COUNT +REGR_INTERCEPT +REGR_R2 +REGR_SLOPE +REGR_SXX +REGR_SXY +REGR_SYY +RELEASE +RESULT +RETURN +RETURNS +REVOKE +RIGHT +ROLLBACK +ROLLUP +ROW +ROW_NUMBER +ROWS +RUNNING +SAVEPOINT +SCOPE +SCROLL +SEARCH +SECOND +SEEK +SELECT +SENSITIVE +SESSION_USER +SET +SHOW +SIMILAR +SIN +SINH +SKIP +SMALLINT +SOME +SPECIFIC +SPECIFICTYPE +SQL +SQLEXCEPTION +SQLSTATE +SQLWARNING +SQRT +START +STATIC +STDDEV_POP +STDDEV_SAMP +SUBMULTISET +SUBSET +SUBSTRING +SUBSTRING_REGEX +SUCCEEDS +SUM +SYMMETRIC +SYSTEM +SYSTEM_TIME +SYSTEM_USER +TABLE +TABLESAMPLE +TAN +TANH +THEN +TIME +TIMESTAMP +TIMEZONE_HOUR +TIMEZONE_MINUTE +TO +TRAILING +TRANSLATE +TRANSLATE_REGEX +TRANSLATION +TREAT +TRIGGER +TRIM +TRIM_ARRAY +TRUE +TRUNCATE +UESCAPE +UNION +UNIQUE +UNKNOWN +UNMATCHED +UNNEST +UPDATE +UPPER +USER +USING +VALUE +VALUES +VALUE_OF +VAR_POP +VAR_SAMP +VARBINARY +VARCHAR +VARYING +VERSIONING +WHEN +WHENEVER +WHERE +WIDTH_BUCKET +WINDOW +WITH +WITHIN +WITHOUT +YEAR +DATALINK +DLNEWCOPY +DLPREVIOUSCOPY +DLURLCOMPLETE +DLURLCOMPLETEWRITE +DLURLCOMPLETEONLY +DLURLPATH +DLURLPATHWRITE +DLURLPATHONLY +DLURLSCHEME +DLURLSERVER +DLVALUE +IMPORT +XML +XMLAGG +XMLATTRIBUTES +XMLBINARY +XMLCAST +XMLCOMMENT +XMLCONCAT +XMLDOCUMENT +XMLELEMENT +XMLEXISTS +XMLFOREST +XMLITERATE +XMLNAMESPACES +XMLPARSE +XMLPI +XMLQUERY +XMLSERIALIZE +XMLTABLE +XMLTEXT +XMLVALIDATE diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 04969e34fb841..04c427da3ec87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -16,8 +16,11 @@ */ package org.apache.spark.sql.catalyst.parser +import java.io.File +import java.nio.file.Files import java.util.Locale +import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.SparkFunSuite @@ -340,7 +343,12 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { // The case where a symbol has multiple literal definitions, // e.g., `DATABASES: 'DATABASES' | 'SCHEMAS';`. if (hasMultipleLiterals) { - val literals = splitDefs.map(_.replaceAll("'", "").trim).toSeq + // Filters out inappropriate entries, e.g., `!` in `NOT: 'NOT' | '!';` + val litDef = """([A-Z_]+)""".r + val literals = splitDefs.map(_.replaceAll("'", "").trim).toSeq.flatMap { + case litDef(lit) => Some(lit) + case _ => None + } (symbol, literals) :: Nil } else { val literal = literalDef.replaceAll("'", "").trim @@ -388,12 +396,24 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { val reservedKeywordsInAnsiMode = allCandidateKeywords -- nonReservedKeywordsInAnsiMode test("check # of reserved keywords") { - val numReservedKeywords = 78 + val numReservedKeywords = 74 assert(reservedKeywordsInAnsiMode.size == numReservedKeywords, s"The expected number of reserved keywords is $numReservedKeywords, but " + s"${reservedKeywordsInAnsiMode.size} found.") } + test("reserved keywords in Spark are also reserved in SQL 2016") { + withTempDir { dir => + val tmpFile = new File(dir, "tmp") + val is = Thread.currentThread().getContextClassLoader + .getResourceAsStream("ansi-sql-2016-reserved-keywords.txt") + Files.copy(is, tmpFile.toPath) + val reservedKeywordsInSql2016 = Files.readAllLines(tmpFile.toPath) + .asScala.filterNot(_.startsWith("--")).map(_.trim).toSet + assert((reservedKeywordsInAnsiMode -- reservedKeywordsInSql2016).isEmpty) + } + } + test("table identifier") { // Regular names. assert(TableIdentifier("q") === parseTableIdentifier("q")) From a7d0d353cd745391cf285039a4630b37754a56a0 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jun 2020 11:26:03 -0700 Subject: [PATCH 016/384] [SPARK-31994][K8S] Docker image should use `https` urls for only deb.debian.org mirrors ### What changes were proposed in this pull request? At the moment, we switch to `https` urls for all the debian mirrors, but turns out some of the mirrors do not support. In this patch, we turn on https mode only for `deb.debian.org` mirror (as it supports SSL). ### Why are the changes needed? It appears, that security.debian.org does not support https. ``` curl https://security.debian.org curl: (35) LibreSSL SSL_connect: SSL_ERROR_SYSCALL in connection to security.debian.org:443 ``` While building the image, it fails in the following way. ``` MacBook-Pro:spark prashantsharma$ bin/docker-image-tool.sh -r scrapcodes -t v3.1.0-1 build Sending build context to Docker daemon 222.1MB Step 1/18 : ARG java_image_tag=8-jre-slim Step 2/18 : FROM openjdk:${java_image_tag} ---> 381b20190cf7 Step 3/18 : ARG spark_uid=185 ---> Using cache ---> 65c06f86753c Step 4/18 : RUN set -ex && sed -i 's/http:/https:/g' /etc/apt/sources.list && apt-get update && ln -s /lib /lib64 && apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps && mkdir -p /opt/spark && mkdir -p /opt/spark/examples && mkdir -p /opt/spark/work-dir && touch /opt/spark/RELEASE && rm /bin/sh && ln -sv /bin/bash /bin/sh && echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su && chgrp root /etc/passwd && chmod ug+rw /etc/passwd && rm -rf /var/cache/apt/* ---> Running in a3461dadd6eb + sed -i s/http:/https:/g /etc/apt/sources.list + apt-get update Ign:1 https://security.debian.org/debian-security buster/updates InRelease Err:2 https://security.debian.org/debian-security buster/updates Release Could not handshake: The TLS connection was non-properly terminated. [IP: 151.101.0.204 443] Get:3 https://deb.debian.org/debian buster InRelease [121 kB] Get:4 https://deb.debian.org/debian buster-updates InRelease [51.9 kB] Get:5 https://deb.debian.org/debian buster/main amd64 Packages [7905 kB] Get:6 https://deb.debian.org/debian buster-updates/main amd64 Packages [7868 B] Reading package lists... E: The repository 'https://security.debian.org/debian-security buster/updates Release' does not have a Release file. The command '/bin/sh -c set -ex && sed -i 's/http:/https:/g' /etc/apt/sources.list && apt-get update && ln -s /lib /lib64 && apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps && mkdir -p /opt/spark && mkdir -p /opt/spark/examples && mkdir -p /opt/spark/work-dir && touch /opt/spark/RELEASE && rm /bin/sh && ln -sv /bin/bash /bin/sh && echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su && chgrp root /etc/passwd && chmod ug+rw /etc/passwd && rm -rf /var/cache/apt/*' returned a non-zero code: 100 Failed to build Spark JVM Docker image, please refer to Docker build output for details. ``` So, if we limit the `https` support to only deb.debian.org, does the trick. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually, by building an image and testing it by running spark shell against it locally using kubernetes. Closes #28834 from ScrapCodes/spark-31994/debian_mirror_fix. Authored-by: Prashant Sharma Signed-off-by: Dongjoon Hyun --- .../kubernetes/docker/src/main/dockerfiles/spark/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index b722123195dc2..a13fe67c9eb72 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -28,7 +28,7 @@ ARG spark_uid=185 # docker build -t spark:latest -f kubernetes/dockerfiles/spark/Dockerfile . RUN set -ex && \ - sed -i 's/http:/https:/g' /etc/apt/sources.list && \ + sed -i 's/http:\/\/deb.\(.*\)/https:\/\/deb.\1/g' /etc/apt/sources.list && \ apt-get update && \ ln -s /lib /lib64 && \ apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps && \ From 5e89fbe44e945ceeb674c5b9a5b536da92866a56 Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Mon, 15 Jun 2020 14:15:54 -0700 Subject: [PATCH 017/384] [SPARK-31824][CORE][TESTS] DAGSchedulerSuite: Improve and reuse completeShuffleMapStageSuccessfully ### What changes were proposed in this pull request? `DAGSchedulerSuite `provides `completeShuffleMapStageSuccessfully `to make `ShuffleMapStage `successfully. But many test case uses complete directly as follows: `complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1))))` We need to improve `completeShuffleMapStageSuccessfully `and reuse it. `completeShuffleMapStageSuccessfully(0, 0, 1, Some(0), Seq("hostA"))` ### Why are the changes needed? Improve and reuse completeShuffleMapStageSuccessfully ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28641 from beliefer/improve-and-reuse-method. Authored-by: gengjiaan Signed-off-by: Xingbo Jiang --- .../spark/scheduler/DAGSchedulerSuite.scala | 197 +++++++----------- 1 file changed, 77 insertions(+), 120 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 4c6033edff247..9d412f2dba3ce 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -458,9 +458,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mapStageC.parents === List(mapStageA, mapStageB)) assert(finalStage.parents === List(mapStageC)) - complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1)))) - complete(taskSets(1), Seq((Success, makeMapStatus("hostA", 1)))) - complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) + completeShuffleMapStageSuccessfully(1, 0, 1) + completeShuffleMapStageSuccessfully(2, 0, 1) complete(taskSets(3), Seq((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -473,9 +473,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") conf.set("spark.files.fetchFailure.unRegisterOutputOnHost", "true") init(conf) - runEvent(ExecutorAdded("exec-hostA1", "hostA")) - runEvent(ExecutorAdded("exec-hostA2", "hostA")) - runEvent(ExecutorAdded("exec-hostB", "hostB")) + runEvent(ExecutorAdded("hostA-exec1", "hostA")) + runEvent(ExecutorAdded("hostA-exec2", "hostA")) + runEvent(ExecutorAdded("hostB-exec", "hostB")) val firstRDD = new MyRDD(sc, 3, Nil) val firstShuffleDep = new ShuffleDependency(firstRDD, new HashPartitioner(3)) val firstShuffleId = firstShuffleDep.shuffleId @@ -488,20 +488,20 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, MapStatus( - BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 5)), + BlockManagerId("hostA-exec1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 5)), (Success, MapStatus( - BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 6)), + BlockManagerId("hostA-exec2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 6)), (Success, makeMapStatus("hostB", 1, mapTaskId = 7)) )) // map stage2 completes successfully, with one task on each executor complete(taskSets(1), Seq( (Success, MapStatus( - BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 8)), + BlockManagerId("hostA-exec1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 8)), (Success, MapStatus( - BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 9)), + BlockManagerId("hostA-exec2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 9)), (Success, makeMapStatus("hostB", 1, mapTaskId = 10)) )) // make sure our test setup is correct @@ -509,19 +509,19 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get assert(initialMapStatus1.count(_ != null) === 3) assert(initialMapStatus1.map{_.location.executorId}.toSet === - Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + Set("hostA-exec1", "hostA-exec2", "hostB-exec")) assert(initialMapStatus1.map{_.mapId}.toSet === Set(5, 6, 7)) val initialMapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get assert(initialMapStatus2.count(_ != null) === 3) assert(initialMapStatus2.map{_.location.executorId}.toSet === - Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + Set("hostA-exec1", "hostA-exec2", "hostB-exec")) assert(initialMapStatus2.map{_.mapId}.toSet === Set(8, 9, 10)) // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( - (FetchFailed(BlockManagerId("exec-hostA2", "hostA", 12345), + (FetchFailed(BlockManagerId("hostA-exec2", "hostA", 12345), firstShuffleId, 0L, 0, 0, "ignored"), null) )) @@ -531,12 +531,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val mapStatus1 = mapOutputTracker.shuffleStatuses(firstShuffleId).mapStatuses assert(mapStatus1.count(_ != null) === 1) - assert(mapStatus1(2).location.executorId === "exec-hostB") + assert(mapStatus1(2).location.executorId === "hostB-exec") assert(mapStatus1(2).location.host === "hostB") val mapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses assert(mapStatus2.count(_ != null) === 1) - assert(mapStatus2(2).location.executorId === "exec-hostB") + assert(mapStatus2(2).location.executorId === "hostB-exec") assert(mapStatus2(2).location.host === "hostB") } @@ -743,9 +743,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) @@ -759,15 +757,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, reduceRdd.partitions.length) // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null))) // this will get called - // blockManagerMaster.removeExecutor("exec-hostA") + // blockManagerMaster.removeExecutor("hostA-exec") // ask the scheduler to try it again scheduler.resubmitFailedStages() // have the 2nd attempt pass @@ -806,10 +802,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) - runEvent(ExecutorLost("exec-hostA", event)) + completeShuffleMapStageSuccessfully(0, 0, 1) + runEvent(ExecutorLost("hostA-exec", event)) if (expectFileLoss) { intercept[MetadataFetchFailedException] { mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0) @@ -889,16 +883,28 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi * @param stageId - The current stageId * @param attemptIdx - The current attempt count * @param numShufflePartitions - The number of partitions in the next stage + * @param hostNames - Host on which each task in the task set is executed */ private def completeShuffleMapStageSuccessfully( stageId: Int, attemptIdx: Int, - numShufflePartitions: Int): Unit = { - val stageAttempt = taskSets.last - checkStageId(stageId, attemptIdx, stageAttempt) + numShufflePartitions: Int, + hostNames: Seq[String] = Seq.empty[String]): Unit = { + def compareStageAttempt(taskSet: TaskSet): Boolean = { + taskSet.stageId == stageId && taskSet.stageAttemptId == attemptIdx + } + + val stageAttemptOpt = taskSets.find(compareStageAttempt(_)) + assert(stageAttemptOpt.isDefined) + val stageAttempt = stageAttemptOpt.get complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) => - (Success, makeMapStatus("host" + ('A' + idx).toChar, numShufflePartitions)) + val hostName = if (idx < hostNames.size) { + hostNames(idx) + } else { + s"host${('A' + idx).toChar}" + } + (Success, makeMapStatus(hostName, numShufflePartitions)) }.toSeq) } @@ -1140,9 +1146,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, reduceRdd.partitions.length) // The MapOutputTracker should know about both map output locations. assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === HashSet("hostA", "hostB")) @@ -1169,9 +1173,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, reduceRdd.partitions.length) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) // The first result task fails, with a fetch failure for the output from the first mapper. @@ -1231,9 +1233,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0, 1)) // Complete the map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2, hostNames = Seq("hostA", "hostA")) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) // The first ResultTask fails @@ -1267,9 +1267,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The map stage should have been submitted. assert(countSubmittedMapStageAttempts() === 1) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) // The MapOutputTracker should know about both map output locations. assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === HashSet("hostA", "hostB")) @@ -1328,9 +1326,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(countSubmittedMapStageAttempts() === 1) // Complete the map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) // The reduce stage should have been submitted. assert(countSubmittedReduceStageAttempts() === 1) @@ -1415,7 +1411,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // pretend we were told hostA went away val oldEpoch = mapOutputTracker.getEpoch - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) @@ -1538,14 +1534,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0)) // things start out smoothly, stage 0 completes with no issues - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostB", shuffleMapRdd.partitions.length)), - (Success, makeMapStatus("hostB", shuffleMapRdd.partitions.length)), - (Success, makeMapStatus("hostA", shuffleMapRdd.partitions.length)) - )) + completeShuffleMapStageSuccessfully( + 0, 0, shuffleMapRdd.partitions.length, Seq("hostB", "hostB", "hostA")) // then one executor dies, and a task fails in stage 1 - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(null, firstShuffleId, 2L, 2, 0, "Fetch failed"), @@ -1643,18 +1636,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi makeMapStatus("hostA", reduceRdd.partitions.length))) // now that host goes down - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) // so we resubmit those tasks runEvent(makeCompletionEvent(taskSets(0).tasks(0), Resubmitted, null)) runEvent(makeCompletionEvent(taskSets(0).tasks(1), Resubmitted, null)) // now complete everything on a different host - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)) - )) + completeShuffleMapStageSuccessfully( + 0, 0, reduceRdd.partitions.length, Seq("hostB", "hostB", "hostB")) // now we should submit stage 1, and the map output from stage 0 should be registered @@ -1768,7 +1758,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // correct behavior. val job1Id = 0 // TaskSet priority for Stages run with "job1" as the ActiveJob checkJobPropertiesAndPriority(taskSets(0), "job1", job1Id) - complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) shuffleDep1 } @@ -1785,7 +1775,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // the stage. checkJobPropertiesAndPriority(taskSets(1), "job2", 1) - complete(taskSets(1), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(1, 0, 1) assert(taskSets(2).properties != null) complete(taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -1817,9 +1807,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi checkJobPropertiesAndPriority(taskSets(2), "job2", job2Id) // run the rest of the stages normally, checking that they have the correct properties - complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 1, 1) checkJobPropertiesAndPriority(taskSets(3), "job2", job2Id) - complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(1, 1, 1) checkJobPropertiesAndPriority(taskSets(4), "job2", job2Id) complete(taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -1841,10 +1831,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) // Tell the DAGScheduler that hostA was lost. - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) + completeShuffleMapStageSuccessfully(0, 0, 1) // At this point, no more tasks are running for the stage (and the TaskSetManager considers the // stage complete), but the tasks that ran on HostA need to be re-run, so the DAGScheduler @@ -1880,19 +1868,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo), tracker = mapOutputTracker) submit(finalRdd, Array(0)) // have the first stage complete normally - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) // have the second stage complete normally - complete(taskSets(1), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostC", 1)))) + completeShuffleMapStageSuccessfully(1, 0, 1, Seq("hostA", "hostC")) // fail the third stage because hostA went down complete(taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) // TODO assert this: - // blockManagerMaster.removeExecutor("exec-hostA") + // blockManagerMaster.removeExecutor("hostA-exec") // have DAGScheduler try again scheduler.resubmitFailedStages() complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 2)))) @@ -1912,19 +1896,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) // complete stage 0 - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) // complete stage 1 - complete(taskSets(1), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + completeShuffleMapStageSuccessfully(1, 0, 1) // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) // TODO assert this: - // blockManagerMaster.removeExecutor("exec-hostA") + // blockManagerMaster.removeExecutor("hostA-exec") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. scheduler.resubmitFailedStages() assertLocations(taskSets(3), Seq(Seq("hostD"))) @@ -2169,8 +2149,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"))) @@ -2215,8 +2194,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep, narrowDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"))) @@ -2319,9 +2297,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Submit a map stage by itself submitMapStage(shuffleDep) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, reduceRdd.partitions.length) assert(results.size === 1) results.clear() assertDataStructuresEmpty() @@ -2378,9 +2354,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete the first stage assert(taskSets(0).stageId === 0) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", rdd1.partitions.length)), - (Success, makeMapStatus("hostB", rdd1.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, rdd1.partitions.length) assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) assert(listener1.results.size === 1) @@ -2442,9 +2416,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete the stage0. assert(taskSets(0).stageId === 0) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", rdd1.partitions.length)), - (Success, makeMapStatus("hostB", rdd1.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, rdd1.partitions.length) assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) assert(listener1.results.size === 1) @@ -2503,7 +2475,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Pretend host A was lost. This will cause the TaskSetManager to resubmit task 0, because it // completed on hostA. val oldEpoch = mapOutputTracker.getEpoch - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) @@ -2651,9 +2623,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete both tasks in rddA. assert(taskSets(0).stageId === 0 && taskSets(0).stageAttemptId === 0) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA")) // Fetch failed for task(stageId=1, stageAttemptId=0, partitionId=0) running on hostA // and task(stageId=1, stageAttemptId=0, partitionId=1) is still running. @@ -2827,15 +2797,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(finalRdd, Array(0, 1)) // Finish the first shuffle map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) // Finish the second shuffle map stage. - complete(taskSets(1), Seq( - (Success, makeMapStatus("hostC", 2)), - (Success, makeMapStatus("hostD", 2)))) + completeShuffleMapStageSuccessfully(1, 0, 2, Seq("hostC", "hostD")) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) // The first task of the final stage failed with fetch failure @@ -2894,14 +2860,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(4).tasks.length == 2) // Finish all stage. - complete(taskSets(4), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 1, 2) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) - complete(taskSets(5), Seq( - (Success, makeMapStatus("hostC", 2)), - (Success, makeMapStatus("hostD", 2)))) + completeShuffleMapStageSuccessfully(1, 2, 2, Seq("hostC", "hostD")) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) complete(taskSets(6), Seq((Success, 11), (Success, 12))) @@ -2932,14 +2894,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(finalRdd, Array(0, 1), properties = new Properties()) // Finish the first 2 shuffle map stages. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) - complete(taskSets(1), Seq( - (Success, makeMapStatus("hostB", 2)), - (Success, makeMapStatus("hostD", 2)))) + completeShuffleMapStageSuccessfully(1, 0, 2, Seq("hostB", "hostD")) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) // Executor lost on hostB, both of stage 0 and 1 should be reran. @@ -2959,9 +2917,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(taskSetIndex).stageId == stageId) assert(taskSets(taskSetIndex).stageAttemptId == 1) assert(taskSets(taskSetIndex).tasks.length == 2) - complete(taskSets(taskSetIndex), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(stageId, 1, 2) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) } @@ -3434,8 +3390,9 @@ object DAGSchedulerSuite { def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2, mapTaskId: Long = -1): MapStatus = MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes), mapTaskId) - def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) + def makeBlockManagerId(host: String): BlockManagerId = { + BlockManagerId(host + "-exec", host, 12345) + } } object FailThisAttempt { From 75afd889044c789b259b9e609aea34b542490f53 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 15 Jun 2020 19:04:23 -0700 Subject: [PATCH 018/384] Revert "[SPARK-31926][SQL][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber" This reverts commit a0187cd6b59a6b6bb2cadc6711bb663d4d35a844. --- project/SparkBuild.scala | 1 + .../src/test/resources/log4j.properties | 65 ------------------- .../thriftserver/SharedThriftServer.scala | 50 ++++---------- .../ThriftServerQueryTestSuite.scala | 3 - .../ThriftServerWithSparkContextSuite.scala | 11 +--- .../cli/thrift/ThriftBinaryCLIService.java | 11 +--- .../service/cli/thrift/ThriftCLIService.java | 3 - .../cli/thrift/ThriftHttpCLIService.java | 21 ++---- .../cli/thrift/ThriftBinaryCLIService.java | 11 +--- .../service/cli/thrift/ThriftCLIService.java | 3 - .../cli/thrift/ThriftHttpCLIService.java | 21 ++---- 11 files changed, 30 insertions(+), 170 deletions(-) delete mode 100644 sql/hive-thriftserver/src/test/resources/log4j.properties diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 04a3fc4b63050..0035f1d95a90d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -480,6 +480,7 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", + "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" ) diff --git a/sql/hive-thriftserver/src/test/resources/log4j.properties b/sql/hive-thriftserver/src/test/resources/log4j.properties deleted file mode 100644 index 21975ba818142..0000000000000 --- a/sql/hive-thriftserver/src/test/resources/log4j.properties +++ /dev/null @@ -1,65 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Set everything to be logged to the file hive-thriftserver/target/unit-tests.log -log4j.rootLogger=DEBUG, CA, FA - -#Console Appender -log4j.appender.CA=org.apache.log4j.ConsoleAppender -log4j.appender.CA.layout=org.apache.log4j.PatternLayout -log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n -log4j.appender.CA.Threshold = WARN - - -#File Appender -log4j.appender.FA=org.apache.log4j.FileAppender -log4j.appender.FA.append=false -log4j.appender.FA.file=target/unit-tests.log -log4j.appender.FA.layout=org.apache.log4j.PatternLayout -log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Set the logger level of File Appender to WARN -log4j.appender.FA.Threshold = DEBUG - -# Some packages are noisy for no good reason. -log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false -log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF - -log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false -log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF - -log4j.additivity.hive.log=false -log4j.logger.hive.log=OFF - -log4j.additivity.parquet.hadoop.ParquetRecordReader=false -log4j.logger.parquet.hadoop.ParquetRecordReader=OFF - -log4j.additivity.org.apache.parquet.hadoop.ParquetRecordReader=false -log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF - -log4j.additivity.org.apache.parquet.hadoop.ParquetOutputCommitter=false -log4j.logger.org.apache.parquet.hadoop.ParquetOutputCommitter=OFF - -log4j.additivity.hive.ql.metadata.Hive=false -log4j.logger.hive.ql.metadata.Hive=OFF - -log4j.additivity.org.apache.hadoop.hive.ql.io.RCFile=false -log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR - -# Parquet related logging -log4j.logger.org.apache.parquet.CorruptStatistics=ERROR -log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index 1c33abff0780c..e002bc0117c8b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -24,7 +24,6 @@ import scala.concurrent.duration._ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftCLIService import org.apache.spark.sql.test.SharedSparkSession @@ -34,8 +33,6 @@ trait SharedThriftServer extends SharedSparkSession { private var hiveServer2: HiveThriftServer2 = _ private var serverPort: Int = 0 - def mode: ServerMode.Value - override def beforeAll(): Unit = { super.beforeAll() // Retries up to 3 times with different port numbers if the server fails to start @@ -53,21 +50,14 @@ trait SharedThriftServer extends SharedSparkSession { hiveServer2.stop() } finally { super.afterAll() - SessionState.detachSession() } } - protected def jdbcUri: String = if (mode == ServerMode.http) { - s"jdbc:hive2://localhost:$serverPort/default;transportMode=http;httpPath=cliservice" - } else { - s"jdbc:hive2://localhost:$serverPort/" - } - protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") val connections = - fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } + fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } val statements = connections.map(_.createStatement()) try { @@ -79,35 +69,23 @@ trait SharedThriftServer extends SharedSparkSession { } private def startThriftServer(attempt: Int): Unit = { - logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt") + logInfo(s"Trying to start HiveThriftServer2:, attempt=$attempt") val sqlContext = spark.newSession().sqlContext - // Set the HIVE_SERVER2_THRIFT_PORT and HIVE_SERVER2_THRIFT_HTTP_PORT to 0, so it could - // randomly pick any free port to use. + // Set the HIVE_SERVER2_THRIFT_PORT to 0, so it could randomly pick any free port to use. // It's much more robust than set a random port generated by ourselves ahead sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") - sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") - sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) - - try { - hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) - hiveServer2.getServices.asScala.foreach { - case t: ThriftCLIService => - serverPort = t.getPortNumber - logInfo(s"Started HiveThriftServer2: mode=$mode, port=$serverPort, attempt=$attempt") - case _ => - } + hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) + hiveServer2.getServices.asScala.foreach { + case t: ThriftCLIService if t.getPortNumber != 0 => + serverPort = t.getPortNumber + logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt") + case _ => + } - // Wait for thrift server to be ready to serve the query, via executing simple query - // till the query succeeds. See SPARK-30345 for more details. - eventually(timeout(30.seconds), interval(1.seconds)) { - withJdbcStatement { _.execute("SELECT 1") } - } - } catch { - case e: Exception => - logError("Error start hive server with Context ", e) - if (hiveServer2 != null) { - hiveServer2.stop() - } + // Wait for thrift server to be ready to serve the query, via executing simple query + // till the query succeeds. See SPARK-30345 for more details. + eventually(timeout(30.seconds), interval(1.seconds)) { + withJdbcStatement { _.execute("SELECT 1") } } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 553f10a275bce..15cc3109da3f7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -54,9 +54,6 @@ import org.apache.spark.sql.types._ */ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer { - - override def mode: ServerMode.Value = ServerMode.binary - override protected def testFile(fileName: String): String = { val url = Thread.currentThread().getContextClassLoader.getResource(fileName) // Copy to avoid URISyntaxException during accessing the resources in `sql/core` diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index d6420dee41adb..3e1fce78ae71c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -trait ThriftServerWithSparkContextSuite extends SharedThriftServer { +class ThriftServerWithSparkContextSuite extends SharedThriftServer { test("SPARK-29911: Uncache cached tables when session closed") { val cacheManager = spark.sharedState.cacheManager @@ -42,12 +42,3 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } - - -class ThriftServerWithSparkContextInBinarySuite extends ThriftServerWithSparkContextSuite { - override def mode: ServerMode.Value = ServerMode.binary -} - -class ThriftServerWithSparkContextInHttpSuite extends ThriftServerWithSparkContextSuite { - override def mode: ServerMode.Value = ServerMode.http -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index 00bdf7e19126e..e1ee503b81209 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -46,7 +45,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - protected void initializeServer() { + public void run() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -101,14 +100,6 @@ protected void initializeServer() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - @Override - public void run() { - try { server.serve(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 783e5795aca76..8fce9d9383438 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -175,7 +175,6 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { - initializeServer(); new Thread(this).start(); isStarted = true; } @@ -634,8 +633,6 @@ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { return resp; } - protected abstract void initializeServer(); - @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index bd64c777c1d76..1099a00b67eb7 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.cli.thrift.TCLIService.Iface; @@ -54,8 +53,13 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ @Override - protected void initializeServer() { + public void run() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -146,19 +150,6 @@ protected void initializeServer() { + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ - @Override - public void run() { - try { httpServer.join(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index ce79e3c8228a6..a7de9c0f3d0d2 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -47,7 +46,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - protected void initializeServer() { + public void run() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -102,14 +101,6 @@ protected void initializeServer() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - @Override - public void run() { - try { server.serve(); } catch (Throwable t) { LOG.error( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index e46799a1c427d..d41c3b493bb47 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -176,7 +176,6 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { - initializeServer(); new Thread(this).start(); isStarted = true; } @@ -671,8 +670,6 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) return resp; } - protected abstract void initializeServer(); - @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index ab9ed5b1f371e..73d5f84476af0 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.rpc.thrift.TCLIService; @@ -55,8 +54,13 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ @Override - protected void initializeServer() { + public void run() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -147,19 +151,6 @@ protected void initializeServer() { + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ - @Override - public void run() { - try { httpServer.join(); } catch (Throwable t) { LOG.error( From e9145d41f3eae53dcee7d298ee1ae9d065645019 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 16 Jun 2020 06:00:05 +0000 Subject: [PATCH 019/384] [SPARK-31986][SQL] Fix Julian-Gregorian micros rebasing of overlapping local timestamps ### What changes were proposed in this pull request? It fixes microseconds rebasing from the hybrid calendar (Julian + Gregorian) to Proleptic Gregorian calendar in the function `RebaseDateTime`.`rebaseJulianToGregorianMicros(zoneId: ZoneId, micros: Long): Long` in the case of local timestamp overlapping. In the case of overlapping, we look ahead of 1 day to determinate which instant we should take - earlier or later zoned timestamp. If our current standard zone and DST offsets are equal to zone offset of the next date, we choose the later timestamp otherwise the earlier one. For example, the local timestamp **1945-11-18 01:30:00.0** can be mapped to two instants (microseconds since 1970-01-01 00:00:00Z): -761211000000000 or -761207400000000. If the first one is passed to `rebaseJulianToGregorianMicros()`, we take the earlier instant in Proleptic Gregorian calendar while rebasing **1945-11-18T01:30+09:00[Asia/Hong_Kong]** otherwise the later one **1945-11-18T01:30+08:00[Asia/Hong_Kong]**. Note: The fix assumes that only one transition of standard or DST offsets can occur during a day. ### Why are the changes needed? Current implementation of `rebaseJulianToGregorianMicros()` handles timestamps overlapping only during daylight saving time but overlapping can happen also during transition from one standard time zone to another one. For example in the case of `Asia/Hong_Kong`, the time zone switched from `Japan Standard Time` (UTC+9) to `Hong Kong Time` (UTC+8) on _Sunday, 18 November, 1945 01:59:59 AM_. The changes allow to handle the special case as well. ### Does this PR introduce _any_ user-facing change? There is no behaviour change for timestamps of CE after 0001-01-01. The PR might affects timestamps of BCE for which the modified `rebaseJulianToGregorianMicros()` is called directly. ### How was this patch tested? 1. By existing tests in `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite` and `TimestampFormatterSuite`. 2. Added new checks to `RebaseDateTimeSuite`.`SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945`: ```scala assert(rebaseJulianToGregorianMicros(hkZid, rebasedEarlierMicros) === earlierMicros) assert(rebaseJulianToGregorianMicros(hkZid, rebasedLaterMicros) === laterMicros) ``` 3. Regenerated `julian-gregorian-rebase-micros.json` with the step of 30 minutes, and got the same JSON file. The JSON file isn't affected because previously it was generated with the step of 1 week. And the spike in diffs/switch points during 1 hour of timestamp overlapping wasn't detected. Closes #28816 from MaxGekk/fix-overlap-julian-2-grep. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/RebaseDateTime.scala | 57 ++++++++++--------- .../catalyst/util/RebaseDateTimeSuite.scala | 3 + 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index c31dc624b0611..8b23bed6d2788 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.util import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneId} import java.time.temporal.ChronoField import java.util.{Calendar, TimeZone} +import java.util.Calendar.{DAY_OF_MONTH, DST_OFFSET, ERA, HOUR_OF_DAY, MINUTE, MONTH, SECOND, YEAR, ZONE_OFFSET} import scala.collection.mutable.AnyRefMap @@ -102,15 +103,15 @@ object RebaseDateTime { .setInstant(Math.multiplyExact(days, MILLIS_PER_DAY)) .build() val localDate = LocalDate.of( - utcCal.get(Calendar.YEAR), - utcCal.get(Calendar.MONTH) + 1, + utcCal.get(YEAR), + utcCal.get(MONTH) + 1, // The number of days will be added later to handle non-existing // Julian dates in Proleptic Gregorian calendar. // For example, 1000-02-29 exists in Julian calendar because 1000 // is a leap year but it is not a leap year in Gregorian calendar. 1) - .`with`(ChronoField.ERA, utcCal.get(Calendar.ERA)) - .plusDays(utcCal.get(Calendar.DAY_OF_MONTH) - 1) + .`with`(ChronoField.ERA, utcCal.get(ERA)) + .plusDays(utcCal.get(DAY_OF_MONTH) - 1) Math.toIntExact(localDate.toEpochDay) } @@ -350,9 +351,9 @@ object RebaseDateTime { // If so, we will take zone offsets from the previous day otherwise from the next day. // This assumes that transitions cannot happen often than once per 2 days. val shift = if (trans.getOffsetBefore == zonedDateTime.getOffset) -1 else 1 - cloned.add(Calendar.DAY_OF_MONTH, shift) - cal.set(Calendar.ZONE_OFFSET, cloned.get(Calendar.ZONE_OFFSET)) - cal.set(Calendar.DST_OFFSET, cloned.get(Calendar.DST_OFFSET)) + cloned.add(DAY_OF_MONTH, shift) + cal.set(ZONE_OFFSET, cloned.get(ZONE_OFFSET)) + cal.set(DST_OFFSET, cloned.get(DST_OFFSET)) } millisToMicros(cal.getTimeInMillis) + ldt.get(ChronoField.MICRO_OF_SECOND) } @@ -413,32 +414,36 @@ object RebaseDateTime { .setInstant(microsToMillis(micros)) .build() val localDateTime = LocalDateTime.of( - cal.get(Calendar.YEAR), - cal.get(Calendar.MONTH) + 1, + cal.get(YEAR), + cal.get(MONTH) + 1, // The number of days will be added later to handle non-existing // Julian dates in Proleptic Gregorian calendar. // For example, 1000-02-29 exists in Julian calendar because 1000 // is a leap year but it is not a leap year in Gregorian calendar. 1, - cal.get(Calendar.HOUR_OF_DAY), - cal.get(Calendar.MINUTE), - cal.get(Calendar.SECOND), + cal.get(HOUR_OF_DAY), + cal.get(MINUTE), + cal.get(SECOND), (Math.floorMod(micros, MICROS_PER_SECOND) * NANOS_PER_MICROS).toInt) - .`with`(ChronoField.ERA, cal.get(Calendar.ERA)) - .plusDays(cal.get(Calendar.DAY_OF_MONTH) - 1) + .`with`(ChronoField.ERA, cal.get(ERA)) + .plusDays(cal.get(DAY_OF_MONTH) - 1) val zonedDateTime = localDateTime.atZone(zoneId) - // Assuming the daylight saving switchover time is 2:00, the local clock will go back to - // 2:00 after hitting 2:59. This means the local time between [2:00, 3:00) appears twice, and - // can map to two different physical times (seconds from the UTC epoch). - // Java 8 time API resolves the ambiguity by picking the earlier physical time. This is the same - // as Java 7 time API, except for 2:00 where Java 7 picks the later physical time. - // Here we detect the "2:00" case and pick the latter physical time, to be compatible with the - // Java 7 date-time. - val adjustedZdt = if (cal.get(Calendar.DST_OFFSET) == 0) { - zonedDateTime.withLaterOffsetAtOverlap() - } else { - zonedDateTime - } + // In the case of local timestamp overlapping, we need to choose the correct time instant + // which is related to the original local timestamp. We look ahead of 1 day, and if the next + // date has the same standard zone and DST offsets, the current local timestamp belongs to + // the period after the transition. In that case, we take the later zoned date time otherwise + // earlier one. Here, we assume that transitions happen not often than once per day. + val trans = zoneId.getRules.getTransition(localDateTime) + val adjustedZdt = if (trans != null && trans.isOverlap) { + val dstOffset = cal.get(DST_OFFSET) + val zoneOffset = cal.get(ZONE_OFFSET) + cal.add(DAY_OF_MONTH, 1) + if (zoneOffset == cal.get(ZONE_OFFSET) && dstOffset == cal.get(DST_OFFSET)) { + zonedDateTime.withLaterOffsetAtOverlap() + } else { + zonedDateTime.withEarlierOffsetAtOverlap() + } + } else zonedDateTime instantToMicros(adjustedZdt.toInstant) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index 8a51f158a9429..b3363169b391e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -445,6 +445,9 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { // Check reverse rebasing assert(rebaseJulianToGregorianMicros(rebasedEarlierMicros) === earlierMicros) assert(rebaseJulianToGregorianMicros(rebasedLaterMicros) === laterMicros) + // Check reverse not-optimized rebasing + assert(rebaseJulianToGregorianMicros(hkZid, rebasedEarlierMicros) === earlierMicros) + assert(rebaseJulianToGregorianMicros(hkZid, rebasedLaterMicros) === laterMicros) } } } From fe68e95a5a5a2358c4e66a4f5b7b5cc6d3f5aaba Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 16 Jun 2020 16:41:08 +0900 Subject: [PATCH 020/384] [SPARK-24634][SS][FOLLOWUP] Rename the variable from "numLateInputs" to "numRowsDroppedByWatermark" ### What changes were proposed in this pull request? This PR renames the variable from "numLateInputs" to "numRowsDroppedByWatermark" so that it becomes self-explanation. ### Why are the changes needed? This is originated from post-review, see https://github.com/apache/spark/pull/28607#discussion_r439853232 ### Does this PR introduce _any_ user-facing change? No, as SPARK-24634 is not introduced in any release yet. ### How was this patch tested? Existing UTs. Closes #28828 from HeartSaVioR/SPARK-24634-v3-followup. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../structured-streaming-programming-guide.md | 9 ++++---- .../streaming/ProgressReporter.scala | 6 ++++- .../streaming/statefulOperators.scala | 14 +++++------ .../apache/spark/sql/streaming/progress.scala | 10 ++++---- .../streaming/EventTimeWatermarkSuite.scala | 16 +++++++------ .../sql/streaming/StateStoreMetricsTest.scala | 23 +++++++++++-------- .../StreamingDeduplicationSuite.scala | 11 +++++---- .../sql/streaming/StreamingJoinSuite.scala | 10 ++++---- ...StreamingQueryStatusAndProgressSuite.scala | 8 +++---- 9 files changed, 59 insertions(+), 48 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 53b7a3a8e46f6..a371f4f50f9f0 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1678,12 +1678,11 @@ emits late rows if the operator uses Append mode. Spark provides two ways to check the number of late rows on stateful operators which would help you identify the issue: 1. On Spark UI: check the metrics in stateful operator nodes in query execution details page in SQL tab -2. On Streaming Query Listener: check "numLateInputs" in "stateOperators" in QueryProcessEvent. +2. On Streaming Query Listener: check "numRowsDroppedByWatermark" in "stateOperators" in QueryProcessEvent. -Please note that the definition of "input" is relative: it doesn't always mean "input rows" for the operator. -Streaming aggregation does pre-aggregate input rows and checks the late inputs against pre-aggregated inputs, -hence the number is not same as the number of original input rows. You'd like to check the fact whether the value is zero -or non-zero. +Please note that "numRowsDroppedByWatermark" represents the number of "dropped" rows by watermark, which is not always same as the count of "late input rows" for the operator. +It depends on the implementation of the operator - e.g. streaming aggregation does pre-aggregate input rows and checks the late inputs against pre-aggregated inputs, +hence the number is not same as the number of original input rows. You'd like to just check the fact whether the value is zero or non-zero. There's a known workaround: split your streaming query into multiple queries per stateful operator, and ensure end-to-end exactly once per query. Ensuring end-to-end exactly once for the last query is optional. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 55491d96f9b1a..2c737206dd2d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -222,7 +222,11 @@ trait ProgressReporter extends Logging { lastExecution.executedPlan.collect { case p if p.isInstanceOf[StateStoreWriter] => val progress = p.asInstanceOf[StateStoreWriter].getProgress() - if (hasExecuted) progress else progress.copy(newNumRowsUpdated = 0, newNumLateInputs = 0) + if (hasExecuted) { + progress + } else { + progress.copy(newNumRowsUpdated = 0, newNumRowsDroppedByWatermark = 0) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index 073266bd621d2..74daaf80b10e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -77,8 +77,8 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numLateInputs" -> SQLMetrics.createMetric(sparkContext, - "number of inputs which are later than watermark ('inputs' are relative to operators)"), + "numRowsDroppedByWatermark" -> SQLMetrics.createMetric(sparkContext, + "number of rows which are dropped by watermark"), "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows"), "allUpdatesTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to update"), @@ -102,7 +102,7 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => numRowsTotal = longMetric("numTotalStateRows").value, numRowsUpdated = longMetric("numUpdatedStateRows").value, memoryUsedBytes = longMetric("stateMemory").value, - numLateInputs = longMetric("numLateInputs").value, + numRowsDroppedByWatermark = longMetric("numRowsDroppedByWatermark").value, javaConvertedCustomMetrics ) } @@ -137,11 +137,11 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => protected def applyRemovingRowsOlderThanWatermark( iter: Iterator[InternalRow], - predicateFilterOutLateInput: BasePredicate): Iterator[InternalRow] = { + predicateDropRowByWatermark: BasePredicate): Iterator[InternalRow] = { iter.filterNot { row => - val lateInput = predicateFilterOutLateInput.eval(row) - if (lateInput) longMetric("numLateInputs") += 1 - lateInput + val shouldDrop = predicateDropRowByWatermark.eval(row) + if (shouldDrop) longMetric("numRowsDroppedByWatermark") += 1 + shouldDrop } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index 22bae76ef4222..482f2b4bf4ed7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -43,7 +43,7 @@ class StateOperatorProgress private[sql]( val numRowsTotal: Long, val numRowsUpdated: Long, val memoryUsedBytes: Long, - val numLateInputs: Long, + val numRowsDroppedByWatermark: Long, val customMetrics: ju.Map[String, JLong] = new ju.HashMap() ) extends Serializable { @@ -55,15 +55,15 @@ class StateOperatorProgress private[sql]( private[sql] def copy( newNumRowsUpdated: Long, - newNumLateInputs: Long): StateOperatorProgress = - new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, newNumLateInputs, - customMetrics) + newNumRowsDroppedByWatermark: Long): StateOperatorProgress = + new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, + newNumRowsDroppedByWatermark, customMetrics) private[sql] def jsonValue: JValue = { ("numRowsTotal" -> JInt(numRowsTotal)) ~ ("numRowsUpdated" -> JInt(numRowsUpdated)) ~ ("memoryUsedBytes" -> JInt(memoryUsedBytes)) ~ - ("numLateInputs" -> JInt(numLateInputs)) ~ + ("numRowsDroppedByWatermark" -> JInt(numRowsDroppedByWatermark)) ~ ("customMetrics" -> { if (!customMetrics.isEmpty) { val keys = customMetrics.keySet.asScala.toSeq.sorted diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 329196a5cfef7..b5e313d2e107c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -298,11 +298,11 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche AddData(inputData, 25), // Advance watermark to 15 seconds CheckNewAnswer((10, 5)), assertNumStateRows(2), - assertNumLateInputs(0), + assertNumRowsDroppedByWatermark(0), AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), assertNumStateRows(2), - assertNumLateInputs(1) + assertNumRowsDroppedByWatermark(1) ) } @@ -323,15 +323,15 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche AddData(inputData, 25), // Advance watermark to 15 seconds CheckNewAnswer((25, 1)), assertNumStateRows(2), - assertNumLateInputs(0), + assertNumRowsDroppedByWatermark(0), AddData(inputData, 10, 25), // Ignore 10 as its less than watermark CheckNewAnswer((25, 2)), assertNumStateRows(2), - assertNumLateInputs(1), + assertNumRowsDroppedByWatermark(1), AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), assertNumStateRows(2), - assertNumLateInputs(1) + assertNumRowsDroppedByWatermark(1) ) } @@ -788,7 +788,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche true } - private def assertNumLateInputs(numLateInputs: Long): AssertOnQuery = AssertOnQuery { q => + private def assertNumRowsDroppedByWatermark( + numRowsDroppedByWatermark: Long): AssertOnQuery = AssertOnQuery { q => q.processAllAvailable() val progressWithData = q.recentProgress.filterNot { p => // filter out batches which are falling into one of types: @@ -796,7 +797,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // 2) empty input batch p.inputRowsPerSecond == 0 }.lastOption.get - assert(progressWithData.stateOperators(0).numLateInputs === numLateInputs) + assert(progressWithData.stateOperators(0).numRowsDroppedByWatermark + === numRowsDroppedByWatermark) true } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala index 640f5181aa526..be83f0ee776da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala @@ -32,9 +32,9 @@ trait StateStoreMetricsTest extends StreamTest { def assertNumStateRows( total: Seq[Long], updated: Seq[Long], - lateInputs: Seq[Long]): AssertOnQuery = + droppedByWatermark: Seq[Long]): AssertOnQuery = AssertOnQuery(s"Check total state rows = $total, updated state rows = $updated" + - s", late inputs = $lateInputs") { q => + s", rows dropped by watermark = $droppedByWatermark") { q => // This assumes that the streaming query will not make any progress while the eventually // is being executed. eventually(timeout(streamingTimeout)) { @@ -55,8 +55,8 @@ trait StateStoreMetricsTest extends StreamTest { val allNumUpdatedRowsSinceLastCheck = progressesSinceLastCheck.map(_.stateOperators.map(_.numRowsUpdated)) - val allNumLateInputsSinceLastCheck = - progressesSinceLastCheck.map(_.stateOperators.map(_.numLateInputs)) + val allNumRowsDroppedByWatermarkSinceLastCheck = + progressesSinceLastCheck.map(_.stateOperators.map(_.numRowsDroppedByWatermark)) lazy val debugString = "recent progresses:\n" + progressesSinceLastCheck.map(_.prettyJson).mkString("\n\n") @@ -67,8 +67,10 @@ trait StateStoreMetricsTest extends StreamTest { val numUpdatedRows = arraySum(allNumUpdatedRowsSinceLastCheck, numStateOperators) assert(numUpdatedRows === updated, s"incorrect updates rows, $debugString") - val numLateInputs = arraySum(allNumLateInputsSinceLastCheck, numStateOperators) - assert(numLateInputs === lateInputs, s"incorrect late inputs, $debugString") + val numRowsDroppedByWatermark = arraySum(allNumRowsDroppedByWatermarkSinceLastCheck, + numStateOperators) + assert(numRowsDroppedByWatermark === droppedByWatermark, + s"incorrect dropped rows by watermark, $debugString") lastCheckedRecentProgressIndex = recentProgress.length - 1 } @@ -77,11 +79,14 @@ trait StateStoreMetricsTest extends StreamTest { def assertNumStateRows(total: Seq[Long], updated: Seq[Long]): AssertOnQuery = { assert(total.length === updated.length) - assertNumStateRows(total, updated, lateInputs = (0 until total.length).map(_ => 0L)) + assertNumStateRows(total, updated, droppedByWatermark = (0 until total.length).map(_ => 0L)) } - def assertNumStateRows(total: Long, updated: Long, lateInput: Long = 0): AssertOnQuery = { - assertNumStateRows(Seq(total), Seq(updated), Seq(lateInput)) + def assertNumStateRows( + total: Long, + updated: Long, + droppedByWatermark: Long = 0): AssertOnQuery = { + assertNumStateRows(Seq(total), Seq(updated), Seq(droppedByWatermark)) } def arraySum(arraySeq: Seq[Array[Long]], arrayLength: Int): Seq[Long] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index ee0b980a9d87b..f6f150e7bbbb1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -54,13 +54,13 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { testStream(result, Append)( AddData(inputData, "a" -> 1), CheckLastBatch("a" -> 1), - assertNumStateRows(total = 1, updated = 1, lateInput = 0), + assertNumStateRows(total = 1, updated = 1, droppedByWatermark = 0), AddData(inputData, "a" -> 2), // Dropped CheckLastBatch(), - assertNumStateRows(total = 1, updated = 0, lateInput = 0), + assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 0), AddData(inputData, "b" -> 1), CheckLastBatch("b" -> 1), - assertNumStateRows(total = 2, updated = 1, lateInput = 0) + assertNumStateRows(total = 2, updated = 1, droppedByWatermark = 0) ) } @@ -102,7 +102,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), - assertNumStateRows(total = 1, updated = 0, lateInput = 1), + assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1), AddData(inputData, 45), // Advance watermark to 35 seconds, no-data-batch drops row 25 CheckNewAnswer(45), @@ -136,7 +136,8 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { AddData(inputData, 10), // Should not emit anything as data less than watermark CheckLastBatch(), - assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L), lateInputs = Seq(0L, 1L)), + assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L), + droppedByWatermark = Seq(0L, 1L)), AddData(inputData, 40), // Advance watermark to 30 seconds CheckLastBatch((15 -> 1), (25 -> 1)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 1f6d0a9945682..833fc24e8b66d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -166,7 +166,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(input1, 5), CheckNewAnswer(), // Same reason as above - assertNumStateRows(total = 2, updated = 0, lateInput = 1) + assertNumStateRows(total = 2, updated = 0, droppedByWatermark = 1) ) } @@ -219,12 +219,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with // (1, 28) ==> passed filter, matched with left (1, 3) and (1, 5), added to state AddData(rightInput, (1, 20), (1, 21), (1, 28)), CheckNewAnswer((1, 3, 21), (1, 5, 21), (1, 3, 28), (1, 5, 28)), - assertNumStateRows(total = 5, updated = 1, lateInput = 1), + assertNumStateRows(total = 5, updated = 1, droppedByWatermark = 1), // New data to left input with leftTime <= 20 should be filtered due to event time watermark AddData(leftInput, (1, 20), (1, 21)), CheckNewAnswer((1, 21, 28)), - assertNumStateRows(total = 6, updated = 1, lateInput = 1) + assertNumStateRows(total = 6, updated = 1, droppedByWatermark = 1) ) } @@ -293,7 +293,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(leftInput, (1, 30), (1, 31)), // 30 should not be processed or added to state CheckNewAnswer((1, 31, 26), (1, 31, 30), (1, 31, 31)), - assertNumStateRows(total = 11, updated = 1, lateInput = 1), // only 31 added + assertNumStateRows(total = 11, updated = 1, droppedByWatermark = 1), // only 31 added // Advance the watermark AddData(rightInput, (1, 80)), @@ -307,7 +307,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(rightInput, (1, 46), (1, 50)), // 46 should not be processed or added to state CheckNewAnswer((1, 49, 50), (1, 50, 50)), - assertNumStateRows(total = 7, updated = 1, lateInput = 1) // 50 added + assertNumStateRows(total = 7, updated = 1, droppedByWatermark = 1) // 50 added ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index 79028a6c442d0..98e2342c78e56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -64,7 +64,7 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "numRowsTotal" : 0, | "numRowsUpdated" : 1, | "memoryUsedBytes" : 3, - | "numLateInputs" : 0, + | "numRowsDroppedByWatermark" : 0, | "customMetrics" : { | "loadedMapCacheHitCount" : 1, | "loadedMapCacheMissCount" : 0, @@ -115,7 +115,7 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "numRowsTotal" : 0, | "numRowsUpdated" : 1, | "memoryUsedBytes" : 2, - | "numLateInputs" : 0 + | "numRowsDroppedByWatermark" : 0 | } ], | "sources" : [ { | "description" : "source", @@ -323,7 +323,7 @@ object StreamingQueryStatusAndProgressSuite { "avg" -> "2016-12-05T20:54:20.827Z", "watermark" -> "2016-12-05T20:54:20.827Z").asJava), stateOperators = Array(new StateOperatorProgress( - numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 3, numLateInputs = 0, + numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 3, numRowsDroppedByWatermark = 0, customMetrics = new java.util.HashMap(Map("stateOnCurrentVersionSizeBytes" -> 2L, "loadedMapCacheHitCount" -> 1L, "loadedMapCacheMissCount" -> 0L) .mapValues(long2Long).asJava) @@ -355,7 +355,7 @@ object StreamingQueryStatusAndProgressSuite { // empty maps should be handled correctly eventTime = new java.util.HashMap(Map.empty[String, String].asJava), stateOperators = Array(new StateOperatorProgress( - numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 2, numLateInputs = 0)), + numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 2, numRowsDroppedByWatermark = 0)), sources = Array( new SourceProgress( description = "source", From f0e6d0ec13d9cdadf341d1b976623345bcdb1028 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 16 Jun 2020 08:35:35 +0000 Subject: [PATCH 021/384] [SPARK-31710][SQL] Fail casting numeric to timestamp by default ## What changes were proposed in this pull request? we fail casting from numeric to timestamp by default. ## Why are the changes needed? casting from numeric to timestamp is not a non-standard,meanwhile it may generate different result between spark and other systems,for example hive ## Does this PR introduce any user-facing change? Yes,user cannot cast numeric to timestamp directly,user have to use the following function to achieve the same effect:TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS ## How was this patch tested? unit test added Closes #28593 from GuoPhilipse/31710-fix-compatibility. Lead-authored-by: GuoPhilipse Co-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + python/pyspark/sql/dataframe.py | 5 +- python/pyspark/sql/functions.py | 13 +++ python/pyspark/sql/tests/test_dataframe.py | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 13 ++- .../expressions/datetimeExpressions.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 12 +++ .../sql/catalyst/expressions/CastSuite.scala | 99 ++++++++++++------- .../org/apache/spark/sql/functions.scala | 9 ++ .../sql-tests/inputs/udf/udf-window.sql | 16 +-- .../resources/sql-tests/inputs/window.sql | 16 +-- .../sql-tests/results/udf/udf-window.sql.out | 16 +-- .../sql-tests/results/window.sql.out | 16 +-- .../apache/spark/sql/DateFunctionsSuite.scala | 4 +- .../spark/sql/StatisticsCollectionSuite.scala | 3 +- .../streaming/MicroBatchExecutionSuite.scala | 4 +- .../sources/ForeachWriterSuite.scala | 6 +- .../streaming/EventTimeWatermarkSuite.scala | 40 ++++---- .../sql/streaming/FileStreamSinkSuite.scala | 2 +- .../FlatMapGroupsWithStateSuite.scala | 7 +- .../streaming/StreamingAggregationSuite.scala | 2 +- .../StreamingDeduplicationSuite.scala | 10 +- .../sql/streaming/StreamingJoinSuite.scala | 35 ++++--- .../execution/HiveCompatibilitySuite.scala | 6 ++ ...esting-0-6a01a94ef1b0d29152c88cd3083fd70b} | 0 ...cast #3-0-732ed232ac592c5e7f7c913a88874fd2 | 1 - ...cast #4-0-6d2da5cfada03605834e38bc4075bc79 | 1 - .../sql/hive/execution/HiveQuerySuite.scala | 27 +++-- .../sql/hive/execution/HiveUDFSuite.scala | 4 +- .../sql/hive/execution/SQLQuerySuite.scala | 10 +- 30 files changed, 233 insertions(+), 150 deletions(-) rename sql/hive/src/test/resources/golden/{constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 => constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 0130923e694b1..d6550c30b9553 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -30,6 +30,8 @@ license: | - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. + - In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can set `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710. + ## Upgrading from Spark SQL 2.4 to 3.0 ### Dataset/DataFrame APIs diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 03e3b9ca4bd05..3ad899bcc3670 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -534,7 +534,10 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving - >>> sdf.select('name', sdf.time.cast('timestamp')).withWatermark('time', '10 minutes') + >>> from pyspark.sql.functions import timestamp_seconds + >>> sdf.select( + ... 'name', + ... timestamp_seconds(sdf.time).alias('time')).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index de0d38e2aed12..0c8c34dd87996 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1427,6 +1427,19 @@ def to_utc_timestamp(timestamp, tz): return Column(sc._jvm.functions.to_utc_timestamp(_to_java_column(timestamp), tz)) +@since(3.1) +def timestamp_seconds(col): + """ + >>> from pyspark.sql.functions import timestamp_seconds + >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) + >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() + [Row(ts=datetime.datetime(2008, 12, 25, 7, 30))] + """ + + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.timestamp_seconds(_to_java_column(col))) + + @since(2.0) @ignore_unicode_prefix def window(timeColumn, windowDuration, slideDuration=None, startTime=None): diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 062e61663a332..30c3fd4c8d167 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -644,7 +644,7 @@ def test_to_pandas_from_mixed_dataframe(self): CAST(col6 AS DOUBLE) AS double, CAST(col7 AS BOOLEAN) AS boolean, CAST(col8 AS STRING) AS string, - CAST(col9 AS TIMESTAMP) AS timestamp + timestamp_seconds(col9) AS timestamp FROM VALUES (1, 1, 1, 1, 1, 1, 1, 1, 1), (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL) """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index ef70915a5c969..5576e71b57024 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -59,7 +59,8 @@ object Cast { case (StringType, TimestampType) => true case (BooleanType, TimestampType) => true case (DateType, TimestampType) => true - case (_: NumericType, TimestampType) => true + case (_: NumericType, TimestampType) => + SQLConf.get.getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP) case (StringType, DateType) => true case (TimestampType, DateType) => true @@ -266,7 +267,15 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit TypeCheckResult.TypeCheckSuccess } else { TypeCheckResult.TypeCheckFailure( - s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}") + if (child.dataType.isInstanceOf[NumericType] && dataType.isInstanceOf[TimestampType]) { + s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," + + "you can enable the casting by setting " + + s"${SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," + + "but we strongly recommend using function " + + "TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead." + } else { + s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}" + }) } } 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 c5cf447c103b7..b46c3fb349ee1 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 @@ -1549,7 +1549,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr def this(left: Expression, format: Expression) { this(left, Option(format), - Cast(Cast(UnixTimestamp(left, format), TimestampType), DateType)) + Cast(SecondsToTimestamp(UnixTimestamp(left, format)), DateType)) } def this(left: Expression) = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 33f40b47d072b..7f63d79a21ed6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2608,6 +2608,15 @@ object SQLConf { .checkValue(_ > 0, "The timeout value must be positive") .createWithDefault(10L) + val LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP = + buildConf("spark.sql.legacy.allowCastNumericToTimestamp") + .internal() + .doc("When true, allow casting numeric to timestamp," + + "when false, forbid the cast, more details in SPARK-31710") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * @@ -3196,6 +3205,9 @@ class SQLConf extends Serializable with Logging { def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID) + def legacyAllowCastNumericToTimestamp: Boolean = + getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 6af995cab64fe..35b4017980138 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -50,7 +50,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } protected def checkNullCast(from: DataType, to: DataType): Unit = { - checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) + } } test("null cast") { @@ -239,7 +241,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkCast(1.5, 1.5f) checkCast(1.5, "1.5") - checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) + } } test("cast from string") { @@ -305,17 +309,20 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(cast(cast(cast( cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), LongType), 5.toLong) - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - 5.toShort) - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - null) - checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), - ByteType), TimestampType), LongType), StringType), ShortType), - 5.toShort) + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + 5.toShort) + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + null) + checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), + ByteType), TimestampType), LongType), StringType), ShortType), + 5.toShort) + } checkEvaluation(cast("23", DoubleType), 23d) checkEvaluation(cast("23", IntegerType), 23) @@ -376,29 +383,32 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(ts, LongType), 15.toLong) checkEvaluation(cast(ts, FloatType), 15.003f) checkEvaluation(cast(ts, DoubleType), 15.003) - checkEvaluation(cast(cast(tss, ShortType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation(cast(cast(tss, IntegerType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation(cast(cast(tss, LongType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), - millis.toFloat / MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), - millis.toDouble / MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), - Decimal(1)) - // A test for higher precision than millis - checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(tss, ShortType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation(cast(cast(tss, IntegerType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation(cast(cast(tss, LongType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), + millis.toFloat / MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), + millis.toDouble / MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), + Decimal(1)) - checkEvaluation(cast(Double.NaN, TimestampType), null) - checkEvaluation(cast(1.0 / 0.0, TimestampType), null) - checkEvaluation(cast(Float.NaN, TimestampType), null) - checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) + // A test for higher precision than millis + checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) + + checkEvaluation(cast(Double.NaN, TimestampType), null) + checkEvaluation(cast(1.0 / 0.0, TimestampType), null) + checkEvaluation(cast(Float.NaN, TimestampType), null) + checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) + } } test("cast from array") { @@ -1026,8 +1036,11 @@ class CastSuite extends CastSuiteBase { test("cast from int 2") { checkEvaluation(cast(1, LongType), 1.toLong) - checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) - checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) + checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) + } checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) @@ -1310,6 +1323,20 @@ class CastSuite extends CastSuiteBase { checkEvaluation(cast(negativeTs, LongType), expectedSecs) } } + + test("SPARK-31710:fail casting from numeric to timestamp by default") { + Seq(true, false).foreach { enable => + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) { + assert(cast(2.toByte, TimestampType).resolved == enable) + assert(cast(10.toShort, TimestampType).resolved == enable) + assert(cast(3, TimestampType).resolved == enable) + assert(cast(10L, TimestampType).resolved == enable) + assert(cast(Decimal(1.2), TimestampType).resolved == enable) + assert(cast(1.7f, TimestampType).resolved == enable) + assert(cast(2.3d, TimestampType).resolved == enable) + } + } + } } /** 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 0cca3e7b47c56..62ad5ea9b5935 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 @@ -3358,6 +3358,15 @@ object functions { window(timeColumn, windowDuration, windowDuration, "0 second") } + /** + * Creates timestamp from the number of seconds since UTC epoch. + * @group = datetime_funcs + * @since = 3.1.0 + */ + def timestamp_seconds(e: Column): Column = withExpr { + SecondsToTimestamp(e.expr) + } + ////////////////////////////////////////////////////////////////////////////////////////////// // Collection functions ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql index bcbf87f8a04c5..1659f1c819592 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql @@ -1,15 +1,15 @@ --This test file was converted from window.sql. -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate); -- RowsBetween diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index 3d05dfda6c3fa..72d812d6a4e49 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -5,15 +5,15 @@ -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate); -- RowsBetween diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index a915c1bd6c717..a84070535b658 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -4,15 +4,15 @@ -- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 625088f90ced9..ede044a44fdaa 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -4,15 +4,15 @@ -- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> 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 5cc9e156db1b5..9caa4c0377009 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 @@ -639,7 +639,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val now = sql("select unix_timestamp()").collect().head.getLong(0) checkAnswer( - sql(s"select cast ($now as timestamp)"), + sql(s"select timestamp_seconds($now)"), Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) } } @@ -716,7 +716,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") checkAnswer(df.select(to_timestamp(col("ss"))), - df.select(unix_timestamp(col("ss")).cast("timestamp"))) + df.select(timestamp_seconds(unix_timestamp(col("ss"))))) checkAnswer(df.select(to_timestamp(col("ss"))), Seq( Row(ts1), Row(ts2))) if (legacyParserPolicy == "legacy") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 708b98e8fe15a..91ec1b5ab2937 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} +import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.ArrayData @@ -467,7 +468,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared withTable(table) { TimeZone.setDefault(srcTimeZone) spark.range(start, end) - .select('id.cast(TimestampType).cast(t).as(column)) + .select(timestamp_seconds($"id").cast(t).as(column)) .write.saveAsTable(table) sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala index c228740df07c8..c0f25e3a5053f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.streaming import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.streaming.StreamTest class MicroBatchExecutionSuite extends StreamTest with BeforeAndAfter { @@ -33,7 +33,7 @@ class MicroBatchExecutionSuite extends StreamTest with BeforeAndAfter { test("SPARK-24156: do not plan a no-data batch again after it has already been planned") { val inputData = MemoryStream[Int] val df = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index 0c17320acade9..e87bd11f0dca5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.ForeachWriter import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSparkSession @@ -163,7 +163,7 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -197,7 +197,7 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index b5e313d2e107c..705e980df86a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.MemorySink -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode._ import org.apache.spark.util.Utils @@ -129,7 +129,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // No event time metrics when there is no watermarking val inputData1 = MemoryStream[Int] val aggWithoutWatermark = inputData1.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) @@ -146,7 +146,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData2 = MemoryStream[Int] val aggWithWatermark = inputData2.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -169,7 +169,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -224,7 +224,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -286,7 +286,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -311,7 +311,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "10") val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -341,7 +341,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val input = MemoryStream[Long] val aggWithWatermark = input.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "2 years 5 months") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -373,7 +373,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("recovery") { val inputData = MemoryStream[Int] val df = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -408,14 +408,14 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val first = MemoryStream[Int] val firstDf = first.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .select('value) val second = MemoryStream[Int] val secondDf = second.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "5 seconds") .select('value) @@ -485,7 +485,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -510,7 +510,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy($"eventTime") .agg(count("*") as 'count) @@ -549,8 +549,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("the new watermark should override the old one") { val df = MemoryStream[(Long, Long)].toDF() - .withColumn("first", $"_1".cast("timestamp")) - .withColumn("second", $"_2".cast("timestamp")) + .withColumn("first", timestamp_seconds($"_1")) + .withColumn("second", timestamp_seconds($"_2")) .withWatermark("first", "1 minute") .withWatermark("second", "2 minutes") @@ -562,7 +562,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("EventTime watermark should be ignored in batch query.") { val df = testData - .withColumn("eventTime", $"key".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"key")) .withWatermark("eventTime", "1 minute") .select("eventTime") .as[Long] @@ -601,7 +601,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("SPARK-27340 Alias on TimeWindow expression cause watermark metadata lost") { val inputData = MemoryStream[Int] val aliasWindow = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .select(window($"eventTime", "5 seconds") as 'aliasWindow) // Check the eventTime metadata is kept in the top level alias. @@ -631,7 +631,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -767,10 +767,10 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche input1: MemoryStream[Int], input2: MemoryStream[Int]): Dataset[_] = { val df1 = input1.toDF - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") val df2 = input2.toDF - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "15 seconds") df1.union(df2).select($"eventTime".cast("int")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 877965100f018..a25451bef62fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -210,7 +210,7 @@ abstract class FileStreamSinkSuite extends StreamTest { val inputData = MemoryStream[Long] val inputDF = inputData.toDF.toDF("time") val outputDf = inputDF - .selectExpr("CAST(time AS timestamp) AS timestamp") + .selectExpr("timestamp_seconds(time) AS timestamp") .withWatermark("timestamp", "10 seconds") .groupBy(window($"timestamp", "5 seconds")) .count() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala index b04f8b0d4d174..e2887e78b0508 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution.RDDScanExec import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, MemoryStateStore, StateStore, StateStoreId, StateStoreMetrics, UnsafeRowPair} +import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.types.{DataType, IntegerType} @@ -826,7 +827,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { val inputData = MemoryStream[(String, Int)] val result = inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime")) .withWatermark("eventTime", "10 seconds") .as[(String, Long)] .groupByKey(_._1) @@ -901,7 +902,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { val inputData = MemoryStream[(String, Int)] val result = inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime")) .withWatermark("eventTime", "10 seconds") .as[(String, Long)] .groupByKey(_._1) @@ -1111,7 +1112,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { val inputData = MemoryStream[(String, Long)] val result = inputData.toDF().toDF("key", "time") - .selectExpr("key", "cast(time as timestamp) as timestamp") + .selectExpr("key", "timestamp_seconds(time) as timestamp") .withWatermark("timestamp", "10 second") .as[(String, Long)] .groupByKey(x => x._1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 85e1b85b84d26..cb69460ca1580 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -188,7 +188,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { testWithAllStateVersions("state metrics - append mode") { val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index f6f150e7bbbb1..1f346aac8d2c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -86,7 +86,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { test("deduplicate with watermark") { val inputData = MemoryStream[Int] val result = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .select($"eventTime".cast("long").as[Long]) @@ -113,7 +113,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { test("deduplicate with aggregate - append mode") { val inputData = MemoryStream[Int] val windowedaggregate = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .withWatermark("eventTime", "10 seconds") @@ -230,7 +230,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { test("SPARK-19841: watermarkPredicate should filter based on keys") { val input = MemoryStream[(Int, Int)] val df = input.toDS.toDF("time", "id") - .withColumn("time", $"time".cast("timestamp")) + .withColumn("time", timestamp_seconds($"time")) .withWatermark("time", "1 second") .dropDuplicates("id", "time") // Change the column positions .select($"id") @@ -249,7 +249,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { test("SPARK-21546: dropDuplicates should ignore watermark when it's not a key") { val input = MemoryStream[(Int, Int)] val df = input.toDS.toDF("id", "time") - .withColumn("time", $"time".cast("timestamp")) + .withColumn("time", timestamp_seconds($"time")) .withWatermark("time", "1 second") .dropDuplicates("id") .select($"id", $"time".cast("long")) @@ -265,7 +265,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { val inputData = MemoryStream[Int] val result = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .select($"eventTime".cast("long").as[Long]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 833fc24e8b66d..caca749f9dd1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -87,11 +87,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val input2 = MemoryStream[Int] val df1 = input1.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", ('value * 2) as "leftValue") + .select('value as "key", timestamp_seconds($"value") as "timestamp", + ('value * 2) as "leftValue") .select('key, window('timestamp, "10 second"), 'leftValue) val df2 = input2.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", + .select('value as "key", timestamp_seconds($"value") as "timestamp", ('value * 3) as "rightValue") .select('key, window('timestamp, "10 second"), 'rightValue) @@ -127,12 +128,13 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val input2 = MemoryStream[Int] val df1 = input1.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", ('value * 2) as "leftValue") + .select('value as "key", timestamp_seconds($"value") as "timestamp", + ('value * 2) as "leftValue") .withWatermark("timestamp", "10 seconds") .select('key, window('timestamp, "10 second"), 'leftValue) val df2 = input2.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", + .select('value as "key", timestamp_seconds($"value") as "timestamp", ('value * 3) as "rightValue") .select('key, window('timestamp, "10 second"), 'rightValue) @@ -177,11 +179,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "10 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "10 seconds") val joined = @@ -235,11 +238,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "20 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "30 seconds") val condition = expr( @@ -425,7 +429,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with test("SPARK-26187 restore the stream-stream inner join query from Spark 2.4") { val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) @@ -500,7 +504,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val df = input.toDF .select( 'value as "key", - 'value.cast("timestamp") as s"${prefix}Time", + timestamp_seconds($"value") as s"${prefix}Time", ('value * multiplier) as s"${prefix}Value") .withWatermark(s"${prefix}Time", "10 seconds") @@ -682,11 +686,12 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "10 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "10 seconds") val joined = @@ -777,7 +782,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) @@ -840,7 +845,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) // we're just flipping "left" and "right" from left outer join and apply right outer join @@ -883,7 +888,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with test("SPARK-26187 restore the stream-stream outer join query from Spark 2.4") { val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index db1f6fbd97d90..82af7dceb27f2 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -39,6 +39,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled private val originalSessionLocalTimeZone = TestHive.conf.sessionLocalTimeZone + private val originalLegacyAllowCastNumericToTimestamp = + TestHive.conf.legacyAllowCastNumericToTimestamp def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -58,6 +60,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests // (timestamp_*) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") + // Ensures that cast numeric to timestamp enabled so that we can test them + TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) RuleExecutor.resetMetrics() } @@ -68,6 +72,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, originalSessionLocalTimeZone) + TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + originalLegacyAllowCastNumericToTimestamp) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 b/sql/hive/src/test/resources/golden/constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b similarity index 100% rename from sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 rename to sql/hive/src/test/resources/golden/constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 deleted file mode 100644 index 5625e59da8873..0000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 +++ /dev/null @@ -1 +0,0 @@ -1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 deleted file mode 100644 index 1d94c8a014fb4..0000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 +++ /dev/null @@ -1 +0,0 @@ --1.2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index b10a8cb8bf2bf..2b42444ceeaa1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -201,14 +201,17 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, - |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21, + |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST('1969-12-31 16:00:01' AS TIMESTAMP)) AS COL21, |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) test("constant null testing timestamp") { - val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL20") - .collect().head + var r1 = sql( + """ + |SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), + |CAST('1969-12-31 16:00:01' AS TIMESTAMP)) AS COL20 + """.stripMargin).collect().head assert(new Timestamp(1000) == r1.getTimestamp(0)) } @@ -552,28 +555,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd // Jdk version leads to different query output for double, so not use createQueryTest here test("timestamp cast #1") { - val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(1 == res.getDouble(0)) } test("timestamp cast #2") { - val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(-1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(-1 == res.get(0)) } - createQueryTest("timestamp cast #3", - "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - - createQueryTest("timestamp cast #4", - "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - - test("timestamp cast #5") { - val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + test("timestamp cast #3") { + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(1200) AS INT) FROM src LIMIT 1").collect().head assert(1200 == res.getInt(0)) } - test("timestamp cast #6") { - val res = sql("SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + test("timestamp cast #4") { + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(-1200) AS INT) FROM src LIMIT 1").collect().head assert(-1200 == res.getInt(0)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 7bca2af379934..057f2f4ce01be 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -434,8 +434,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { withTempView("tab1") { Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") - val count = sql("select testUDFToDate(cast(test_date as timestamp))" + - " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count() + val count = sql("select testUDFToDate(timestamp_seconds(test_date))" + + " from tab1 group by testUDFToDate(timestamp_seconds(test_date))").count() sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToDate") assert(count == 1) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index d12eae0e410b1..2fe6a59a27c1b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -1172,7 +1172,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi test("SPARK-6785: HiveQuerySuite - Date comparison test 2") { checkAnswer( - sql("SELECT CAST(CAST(0 AS timestamp) AS date) > CAST(0 AS timestamp) FROM src LIMIT 1"), + sql("SELECT CAST(timestamp_seconds(0) AS date) > timestamp_seconds(0) FROM src LIMIT 1"), Row(false)) } @@ -1182,10 +1182,10 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi sql( """ | SELECT - | CAST(CAST(0 AS timestamp) AS date), - | CAST(CAST(CAST(0 AS timestamp) AS date) AS string), - | CAST(0 AS timestamp), - | CAST(CAST(0 AS timestamp) AS string), + | CAST(timestamp_seconds(0) AS date), + | CAST(CAST(timestamp_seconds(0) AS date) AS string), + | timestamp_seconds(0), + | CAST(timestamp_seconds(0) AS string), | CAST(CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) AS timestamp) | FROM src LIMIT 1 """.stripMargin), From d24d27f1bc39e915df23d65f8fda0d83e716b308 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 16 Jun 2020 19:20:44 +0900 Subject: [PATCH 022/384] [SPARK-31997][SQL][TESTS] Drop test_udtf table when SingleSessionSuite test completed MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? `SingleSessionSuite` not do `DROP TABLE IF EXISTS test_udtf` when test completed, then if we do mvn test `HiveThriftBinaryServerSuite`, the test case named `SPARK-11595 ADD JAR with input path having URL scheme` will FAILED because it want to re-create an exists table test_udtf. ### Why are the changes needed? test suite shouldn't rely on their execution order ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Manual test,mvn test SingleSessionSuite and HiveThriftBinaryServerSuite in order Closes #28838 from LuciferYang/drop-test-table. Authored-by: yangjie01 Signed-off-by: HyukjinKwon --- .../spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index ff54879cb5084..3fd46dc82f03f 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -883,7 +883,7 @@ class SingleSessionSuite extends HiveThriftJdbcTest { s"--conf ${HIVE_THRIFT_SERVER_SINGLESESSION.key}=true" :: Nil test("share the temporary functions across JDBC connections") { - withMultipleConnectionJdbcStatement()( + withMultipleConnectionJdbcStatement("test_udtf")( { statement => val jarPath = "../hive/src/test/resources/TestUDTF.jar" val jarURL = s"file://${System.getProperty("user.dir")}/$jarPath" From 6e9ff72195e52c4fb61630900a545864c2be7772 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 16 Jun 2020 12:56:27 +0000 Subject: [PATCH 023/384] [SPARK-31984][SQL] Make micros rebasing functions via local timestamps pure ### What changes were proposed in this pull request? 1. Set the given time zone as the first parameter of `RebaseDateTime`.`rebaseJulianToGregorianMicros()` and `rebaseGregorianToJulianMicros()` to Java 7 `GregorianCalendar`. ```scala val cal = new Calendar.Builder() // `gregory` is a hybrid calendar that supports both the Julian and Gregorian calendar systems .setCalendarType("gregory") ... .setTimeZone(tz) .build() ``` This makes the instance of the calendar independent from the default JVM time zone. 2. Change type of the first parameter from `ZoneId` to `TimeZone`. This allows to avoid unnecessary conversion from `TimeZone` to `ZoneId`, for example in ```scala def rebaseJulianToGregorianMicros(micros: Long): Long = { ... if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { rebaseJulianToGregorianMicros(timeZone.toZoneId, micros) ``` and back to `TimeZone` inside of `rebaseJulianToGregorianMicros(zoneId: ZoneId, ...)` 3. Modify tests in `RebaseDateTimeSuite`, and set the default JVM time zone only for functions that depend on it. ### Why are the changes needed? 1. Ignoring passed parameter and using a global variable is bad practice. 2. Dependency from the global state doesn't allow to run the functions in parallel otherwise there is non-zero probability that the functions may return wrong result if the default JVM is changed during their execution. 3. This open opportunity for parallelisation of JSON files generation `gregorian-julian-rebase-micros.json` and `julian-gregorian-rebase-micros.json`. Currently, the tests `generate 'gregorian-julian-rebase-micros.json'` and `generate 'julian-gregorian-rebase-micros.json'` generate the JSON files by iterating over all time zones sequentially w/ step of 1 week. Due to the large step, we can miss some spikes in diffs between 2 calendars (Java 8 Gregorian and Java 7 hybrid calendars) as the PR https://github.com/apache/spark/pull/28787 fixed and https://github.com/apache/spark/pull/28816 should fix. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running existing tests from `RebaseDateTimeSuite`. Closes #28824 from MaxGekk/pure-micros-rebasing. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/RebaseDateTime.scala | 19 +- .../catalyst/util/RebaseDateTimeSuite.scala | 163 +++++++++--------- 2 files changed, 94 insertions(+), 88 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index 8b23bed6d2788..24fe9fa42c19f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -320,13 +320,14 @@ object RebaseDateTime { * Julian calendar: 1582-01-01 00:00:00.123456 -> -12243196799876544 * The code below converts -12244061221876544 to -12243196799876544. * - * @param zoneId The time zone ID at which the rebasing should be performed. + * @param tz The time zone at which the rebasing should be performed. * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' * in Proleptic Gregorian calendar. It can be negative. * @return The rebased microseconds since the epoch in Julian calendar. */ - private[sql] def rebaseGregorianToJulianMicros(zoneId: ZoneId, micros: Long): Long = { + private[sql] def rebaseGregorianToJulianMicros(tz: TimeZone, micros: Long): Long = { val instant = microsToInstant(micros) + val zoneId = tz.toZoneId val zonedDateTime = instant.atZone(zoneId) var ldt = zonedDateTime.toLocalDateTime if (ldt.isAfter(julianEndTs) && ldt.isBefore(gregorianStartTs)) { @@ -337,6 +338,7 @@ object RebaseDateTime { .setCalendarType("gregory") .setDate(ldt.getYear, ldt.getMonthValue - 1, ldt.getDayOfMonth) .setTimeOfDay(ldt.getHour, ldt.getMinute, ldt.getSecond) + .setTimeZone(tz) .build() // A local timestamp can have 2 instants in the cases of switching from: // 1. Summer to winter time. @@ -379,7 +381,7 @@ object RebaseDateTime { val tzId = timeZone.getID val rebaseRecord = gregJulianRebaseMap.getOrNull(tzId) if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { - rebaseGregorianToJulianMicros(timeZone.toZoneId, micros) + rebaseGregorianToJulianMicros(timeZone, micros) } else { rebaseMicros(rebaseRecord, micros) } @@ -401,17 +403,17 @@ object RebaseDateTime { * Proleptic Gregorian calendar: 1582-01-01 00:00:00.123456 -> -12244061221876544 * The code below converts -12243196799876544 to -12244061221876544. * - * @param zoneId The time zone ID at which the rebasing should be performed. + * @param tz The time zone at which the rebasing should be performed. * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' * in the Julian calendar. It can be negative. * @return The rebased microseconds since the epoch in Proleptic Gregorian calendar. */ - private[sql] def rebaseJulianToGregorianMicros(zoneId: ZoneId, micros: Long): Long = { + private[sql] def rebaseJulianToGregorianMicros(tz: TimeZone, micros: Long): Long = { val cal = new Calendar.Builder() - // `gregory` is a hybrid calendar that supports both - // the Julian and Gregorian calendar systems + // `gregory` is a hybrid calendar that supports both the Julian and Gregorian calendar systems .setCalendarType("gregory") .setInstant(microsToMillis(micros)) + .setTimeZone(tz) .build() val localDateTime = LocalDateTime.of( cal.get(YEAR), @@ -427,6 +429,7 @@ object RebaseDateTime { (Math.floorMod(micros, MICROS_PER_SECOND) * NANOS_PER_MICROS).toInt) .`with`(ChronoField.ERA, cal.get(ERA)) .plusDays(cal.get(DAY_OF_MONTH) - 1) + val zoneId = tz.toZoneId val zonedDateTime = localDateTime.atZone(zoneId) // In the case of local timestamp overlapping, we need to choose the correct time instant // which is related to the original local timestamp. We look ahead of 1 day, and if the next @@ -479,7 +482,7 @@ object RebaseDateTime { val tzId = timeZone.getID val rebaseRecord = julianGregRebaseMap.getOrNull(tzId) if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { - rebaseJulianToGregorianMicros(timeZone.toZoneId, micros) + rebaseJulianToGregorianMicros(timeZone, micros) } else { rebaseMicros(rebaseRecord, micros) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index b3363169b391e..254bf01c89b4f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -201,21 +201,21 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { test("optimization of micros rebasing - Gregorian to Julian") { outstandingZoneIds.foreach { zid => withClue(s"zone id = $zid") { - withDefaultTimeZone(zid) { - val start = instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0) - .atZone(zid) - .toInstant) - val end = instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0) - .atZone(zid) - .toInstant) - var micros = start - do { - val rebased = rebaseGregorianToJulianMicros(zid, micros) - val rebasedAndOptimized = rebaseGregorianToJulianMicros(micros) - assert(rebasedAndOptimized === rebased) - micros += (MICROS_PER_DAY * 30 * (0.5 + Math.random())).toLong - } while (micros <= end) - } + val start = instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0) + .atZone(zid) + .toInstant) + val end = instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0) + .atZone(zid) + .toInstant) + var micros = start + do { + val rebased = rebaseGregorianToJulianMicros(TimeZone.getTimeZone(zid), micros) + val rebasedAndOptimized = withDefaultTimeZone(zid) { + rebaseGregorianToJulianMicros(micros) + } + assert(rebasedAndOptimized === rebased) + micros += (MICROS_PER_DAY * 30 * (0.5 + Math.random())).toLong + } while (micros <= end) } } } @@ -223,26 +223,26 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { test("optimization of micros rebasing - Julian to Gregorian") { outstandingZoneIds.foreach { zid => withClue(s"zone id = $zid") { - withDefaultTimeZone(zid) { - val start = rebaseGregorianToJulianMicros( - instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0).atZone(zid).toInstant)) - val end = rebaseGregorianToJulianMicros( - instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0).atZone(zid).toInstant)) - var micros = start - do { - val rebased = rebaseJulianToGregorianMicros(zid, micros) - val rebasedAndOptimized = rebaseJulianToGregorianMicros(micros) - assert(rebasedAndOptimized === rebased) - micros += (MICROS_PER_DAY * 30 * (0.5 + Math.random())).toLong - } while (micros <= end) - } + val start = rebaseGregorianToJulianMicros( + instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0).atZone(zid).toInstant)) + val end = rebaseGregorianToJulianMicros( + instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0).atZone(zid).toInstant)) + var micros = start + do { + val rebased = rebaseJulianToGregorianMicros(TimeZone.getTimeZone(zid), micros) + val rebasedAndOptimized = withDefaultTimeZone(zid) { + rebaseJulianToGregorianMicros(micros) + } + assert(rebasedAndOptimized === rebased) + micros += (MICROS_PER_DAY * 30 * (0.5 + Math.random())).toLong + } while (micros <= end) } } } private def generateRebaseJson( - adjustFunc: Long => Long, - rebaseFunc: (ZoneId, Long) => Long, + adjustFunc: (TimeZone, Long) => Long, + rebaseFunc: (TimeZone, Long) => Long, dir: String, fileName: String): Unit = { import java.nio.file.{Files, Paths} @@ -260,14 +260,15 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { .sortBy(_.getId) .foreach { zid => withDefaultTimeZone(zid) { - val start = adjustFunc(instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0) - .atZone(zid) - .toInstant)) + val tz = TimeZone.getTimeZone(zid) + val start = adjustFunc( + tz, + instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0).atZone(zid).toInstant)) // sun.util.calendar.ZoneInfo resolves DST after 2037 year incorrectly. // See https://bugs.java.com/bugdatabase/view_bug.do?bug_id=8073446 - val end = adjustFunc(instantToMicros(LocalDateTime.of(2037, 1, 1, 0, 0, 0) - .atZone(zid) - .toInstant)) + val end = adjustFunc( + tz, + instantToMicros(LocalDateTime.of(2037, 1, 1, 0, 0, 0).atZone(zid).toInstant)) var micros = start var diff = Long.MaxValue @@ -276,7 +277,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { val switches = new ArrayBuffer[Long]() val diffs = new ArrayBuffer[Long]() while (micros < end) { - val rebased = rebaseFunc(zid, micros) + val rebased = rebaseFunc(tz, micros) val curDiff = rebased - micros if (curDiff != diff) { if (step > MICROS_PER_SECOND) { @@ -308,7 +309,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { ignore("generate 'gregorian-julian-rebase-micros.json'") { generateRebaseJson( - adjustFunc = identity[Long], + adjustFunc = (_: TimeZone, micros: Long) => micros, rebaseFunc = rebaseGregorianToJulianMicros, dir = "/Users/maximgekk/tmp", fileName = "gregorian-julian-rebase-micros.json") @@ -383,26 +384,27 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { test("rebase not-existed timestamps in the hybrid calendar") { outstandingZoneIds.foreach { zid => - withDefaultTimeZone(zid) { - Seq( - "1582-10-04T23:59:59.999999" -> "1582-10-04 23:59:59.999999", - "1582-10-05T00:00:00.000000" -> "1582-10-15 00:00:00.000000", - "1582-10-06T01:02:03.000001" -> "1582-10-15 01:02:03.000001", - "1582-10-07T00:00:00.000000" -> "1582-10-15 00:00:00.000000", - "1582-10-08T23:59:59.999999" -> "1582-10-15 23:59:59.999999", - "1582-10-09T23:59:59.001001" -> "1582-10-15 23:59:59.001001", - "1582-10-10T00:11:22.334455" -> "1582-10-15 00:11:22.334455", - "1582-10-11T11:12:13.111111" -> "1582-10-15 11:12:13.111111", - "1582-10-12T10:11:12.131415" -> "1582-10-15 10:11:12.131415", - "1582-10-13T00:00:00.123321" -> "1582-10-15 00:00:00.123321", - "1582-10-14T23:59:59.999999" -> "1582-10-15 23:59:59.999999", - "1582-10-15T00:00:00.000000" -> "1582-10-15 00:00:00.000000" - ).foreach { case (gregTs, hybridTs) => - withClue(s"tz = ${zid.getId} greg ts = $gregTs hybrid ts = $hybridTs") { - val hybridMicros = parseToJulianMicros(hybridTs) - val gregorianMicros = parseToGregMicros(gregTs, zid) - - assert(rebaseGregorianToJulianMicros(zid, gregorianMicros) === hybridMicros) + Seq( + "1582-10-04T23:59:59.999999" -> "1582-10-04 23:59:59.999999", + "1582-10-05T00:00:00.000000" -> "1582-10-15 00:00:00.000000", + "1582-10-06T01:02:03.000001" -> "1582-10-15 01:02:03.000001", + "1582-10-07T00:00:00.000000" -> "1582-10-15 00:00:00.000000", + "1582-10-08T23:59:59.999999" -> "1582-10-15 23:59:59.999999", + "1582-10-09T23:59:59.001001" -> "1582-10-15 23:59:59.001001", + "1582-10-10T00:11:22.334455" -> "1582-10-15 00:11:22.334455", + "1582-10-11T11:12:13.111111" -> "1582-10-15 11:12:13.111111", + "1582-10-12T10:11:12.131415" -> "1582-10-15 10:11:12.131415", + "1582-10-13T00:00:00.123321" -> "1582-10-15 00:00:00.123321", + "1582-10-14T23:59:59.999999" -> "1582-10-15 23:59:59.999999", + "1582-10-15T00:00:00.000000" -> "1582-10-15 00:00:00.000000" + ).foreach { case (gregTs, hybridTs) => + withClue(s"tz = ${zid.getId} greg ts = $gregTs hybrid ts = $hybridTs") { + val hybridMicros = withDefaultTimeZone(zid) { parseToJulianMicros(hybridTs) } + val gregorianMicros = parseToGregMicros(gregTs, zid) + + val tz = TimeZone.getTimeZone(zid) + assert(rebaseGregorianToJulianMicros(tz, gregorianMicros) === hybridMicros) + withDefaultTimeZone(zid) { assert(rebaseGregorianToJulianMicros(gregorianMicros) === hybridMicros) } } @@ -416,38 +418,39 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { // clocks were moved backward to become Sunday, 18 November, 1945 01:00:00 AM. // In this way, the overlap happened w/o Daylight Saving Time. val hkZid = getZoneId("Asia/Hong_Kong") + var expected = "1945-11-18 01:30:00.0" + var ldt = LocalDateTime.of(1945, 11, 18, 1, 30, 0) + var earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) + var laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) + var overlapInterval = MICROS_PER_HOUR + if (earlierMicros + overlapInterval != laterMicros) { + // Old JDK might have an outdated time zone database. + // See https://bugs.openjdk.java.net/browse/JDK-8228469: "Hong Kong ... Its 1945 transition + // from JST to HKT was on 11-18 at 02:00, not 09-15 at 00:00" + expected = "1945-09-14 23:30:00.0" + ldt = LocalDateTime.of(1945, 9, 14, 23, 30, 0) + earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) + laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) + // If time zone db doesn't have overlapping at all, set the overlap interval to zero. + overlapInterval = laterMicros - earlierMicros + } + val hkTz = TimeZone.getTimeZone(hkZid) + val rebasedEarlierMicros = rebaseGregorianToJulianMicros(hkTz, earlierMicros) + val rebasedLaterMicros = rebaseGregorianToJulianMicros(hkTz, laterMicros) + assert(rebasedEarlierMicros + overlapInterval === rebasedLaterMicros) withDefaultTimeZone(hkZid) { - var expected = "1945-11-18 01:30:00.0" - var ldt = LocalDateTime.of(1945, 11, 18, 1, 30, 0) - var earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) - var laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) - var overlapInterval = MICROS_PER_HOUR - if (earlierMicros + overlapInterval != laterMicros) { - // Old JDK might have an outdated time zone database. - // See https://bugs.openjdk.java.net/browse/JDK-8228469: "Hong Kong ... Its 1945 transition - // from JST to HKT was on 11-18 at 02:00, not 09-15 at 00:00" - expected = "1945-09-14 23:30:00.0" - ldt = LocalDateTime.of(1945, 9, 14, 23, 30, 0) - earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) - laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) - // If time zone db doesn't have overlapping at all, set the overlap interval to zero. - overlapInterval = laterMicros - earlierMicros - } - val rebasedEarlierMicros = rebaseGregorianToJulianMicros(hkZid, earlierMicros) - val rebasedLaterMicros = rebaseGregorianToJulianMicros(hkZid, laterMicros) def toTsStr(micros: Long): String = toJavaTimestamp(micros).toString assert(toTsStr(rebasedEarlierMicros) === expected) assert(toTsStr(rebasedLaterMicros) === expected) - assert(rebasedEarlierMicros + overlapInterval === rebasedLaterMicros) // Check optimized rebasing assert(rebaseGregorianToJulianMicros(earlierMicros) === rebasedEarlierMicros) assert(rebaseGregorianToJulianMicros(laterMicros) === rebasedLaterMicros) // Check reverse rebasing assert(rebaseJulianToGregorianMicros(rebasedEarlierMicros) === earlierMicros) assert(rebaseJulianToGregorianMicros(rebasedLaterMicros) === laterMicros) - // Check reverse not-optimized rebasing - assert(rebaseJulianToGregorianMicros(hkZid, rebasedEarlierMicros) === earlierMicros) - assert(rebaseJulianToGregorianMicros(hkZid, rebasedLaterMicros) === laterMicros) } + // Check reverse not-optimized rebasing + assert(rebaseJulianToGregorianMicros(hkTz, rebasedEarlierMicros) === earlierMicros) + assert(rebaseJulianToGregorianMicros(hkTz, rebasedLaterMicros) === laterMicros) } } From 36435658b116e5bf9b3bfc3e276a068406eddb30 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 16 Jun 2020 14:07:03 +0000 Subject: [PATCH 024/384] [SPARK-31710][SQL][FOLLOWUP] Replace CAST by TIMESTAMP_SECONDS in benchmarks ### What changes were proposed in this pull request? Replace `CAST(... AS TIMESTAMP` by `TIMESTAMP_SECONDS` in the following benchmarks: - ExtractBenchmark - DateTimeBenchmark - FilterPushdownBenchmark - InExpressionBenchmark ### Why are the changes needed? The benchmarks fail w/o the changes: ``` [info] Running benchmark: datetime +/- interval [info] Running case: date + interval(m) [error] Exception in thread "main" org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(`id` AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommend using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 5; [error] 'Project [(cast(cast(id#0L as timestamp) as date) + 1 months) AS (CAST(CAST(id AS TIMESTAMP) AS DATE) + INTERVAL '1 months')#2] [error] +- Range (0, 10000000, step=1, splits=Some(1)) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running the affected benchmarks. Closes #28843 from MaxGekk/GuoPhilipse-31710-fix-compatibility-followup. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../benchmark/DateTimeBenchmark.scala | 18 +++++++++--------- .../execution/benchmark/ExtractBenchmark.scala | 8 ++++---- .../benchmark/FilterPushdownBenchmark.scala | 10 +++++----- .../benchmark/InExpressionBenchmark.scala | 10 +++++----- 4 files changed, 23 insertions(+), 23 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala index c7b8737b7a753..b06ca71b04ecc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala @@ -54,7 +54,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { private def run(cardinality: Int, func: String): Unit = { codegenBenchmark(s"$func of timestamp", cardinality) { - doBenchmark(cardinality, s"$func(cast(id as timestamp))") + doBenchmark(cardinality, s"$func(timestamp_seconds(id))") } } @@ -64,7 +64,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { val N = 10000000 runBenchmark("datetime +/- interval") { val benchmark = new Benchmark("datetime +/- interval", N, output = output) - val ts = "cast(id as timestamp)" + val ts = "timestamp_seconds(id)" val dt = s"cast($ts as date)" benchmark.addCase("date + interval(m)") { _ => doBenchmark(N, s"$dt + interval 1 month") @@ -105,7 +105,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { benchmark.run() } runBenchmark("Extract components") { - run(N, "cast to timestamp", "cast(id as timestamp)") + run(N, "cast to timestamp", "timestamp_seconds(id)") run(N, "year") run(N, "quarter") run(N, "month") @@ -124,7 +124,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { run(N, "current_timestamp", "current_timestamp") } runBenchmark("Date arithmetic") { - val dateExpr = "cast(cast(id as timestamp) as date)" + val dateExpr = "cast(timestamp_seconds(id) as date)" run(N, "cast to date", dateExpr) run(N, "last_day", s"last_day($dateExpr)") run(N, "next_day", s"next_day($dateExpr, 'TU')") @@ -133,31 +133,31 @@ object DateTimeBenchmark extends SqlBasedBenchmark { run(N, "add_months", s"add_months($dateExpr, 10)") } runBenchmark("Formatting dates") { - val dateExpr = "cast(cast(id as timestamp) as date)" + val dateExpr = "cast(timestamp_seconds(id) as date)" run(N, "format date", s"date_format($dateExpr, 'MMM yyyy')") } runBenchmark("Formatting timestamps") { run(N, "from_unixtime", "from_unixtime(id, 'yyyy-MM-dd HH:mm:ss.SSSSSS')") } runBenchmark("Convert timestamps") { - val timestampExpr = "cast(id as timestamp)" + val timestampExpr = "timestamp_seconds(id)" run(N, "from_utc_timestamp", s"from_utc_timestamp($timestampExpr, 'CET')") run(N, "to_utc_timestamp", s"to_utc_timestamp($timestampExpr, 'CET')") } runBenchmark("Intervals") { - val (start, end) = ("cast(id as timestamp)", "cast((id+8640000) as timestamp)") + val (start, end) = ("timestamp_seconds(id)", "timestamp_seconds(id+8640000)") run(N, "cast interval", start, end) run(N, "datediff", s"datediff($start, $end)") run(N, "months_between", s"months_between($start, $end)") run(1000000, "window", s"window($start, 100, 10, 1)") } runBenchmark("Truncation") { - val timestampExpr = "cast(id as timestamp)" + val timestampExpr = "timestamp_seconds(id)" Seq("YEAR", "YYYY", "YY", "MON", "MONTH", "MM", "DAY", "DD", "HOUR", "MINUTE", "SECOND", "WEEK", "QUARTER").foreach { level => run(N, s"date_trunc $level", s"date_trunc('$level', $timestampExpr)") } - val dateExpr = "cast(cast(id as timestamp) as date)" + val dateExpr = "cast(timestamp_seconds(id) as date)" Seq("year", "yyyy", "yy", "mon", "month", "mm").foreach { level => run(N, s"trunc $level", s"trunc('$level', $dateExpr)") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index 287854dc3646c..8372698fb47ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -59,10 +59,10 @@ object ExtractBenchmark extends SqlBasedBenchmark { } private def castExpr(from: String): String = from match { - case "timestamp" => "cast(id as timestamp)" - case "date" => "cast(cast(id as timestamp) as date)" - case "interval" => "(cast(cast(id as timestamp) as date) - date'0001-01-01') + " + - "(cast(id as timestamp) - timestamp'1000-01-01 01:02:03.123456')" + case "timestamp" => "timestamp_seconds(id)" + case "date" => "cast(timestamp_seconds(id) as date)" + case "interval" => "(cast(timestamp_seconds(id) as date) - date'0001-01-01') + " + + "(timestamp_seconds(id) - timestamp'1000-01-01 01:02:03.123456')" case other => throw new IllegalArgumentException( s"Unsupported column type $other. Valid column types are 'timestamp' and 'date'") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index b3f65d40ad95b..9ade8b14f59b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.functions.monotonically_increasing_id +import org.apache.spark.sql.functions.{monotonically_increasing_id, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType} @@ -332,11 +332,11 @@ object FilterPushdownBenchmark extends SqlBasedBenchmark { withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> fileType) { val columns = (1 to width).map(i => s"CAST(id AS string) c$i") val df = spark.range(numRows).selectExpr(columns: _*) - .withColumn("value", monotonically_increasing_id().cast(TimestampType)) + .withColumn("value", timestamp_seconds(monotonically_increasing_id())) withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) - Seq(s"value = CAST($mid AS timestamp)").foreach { whereExpr => + Seq(s"value = timestamp_seconds($mid)").foreach { whereExpr => val title = s"Select 1 timestamp stored as $fileType row ($whereExpr)" .replace("value AND value", "value") filterPushDownBenchmark(numRows, title, whereExpr) @@ -348,8 +348,8 @@ object FilterPushdownBenchmark extends SqlBasedBenchmark { filterPushDownBenchmark( numRows, s"Select $percent% timestamp stored as $fileType rows " + - s"(value < CAST(${numRows * percent / 100} AS timestamp))", - s"value < CAST(${numRows * percent / 100} as timestamp)", + s"(value < timestamp_seconds(${numRows * percent / 100}))", + s"value < timestamp_seconds(${numRows * percent / 100})", selectExpr ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala index caf3387875813..704227e4b4db4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.functions.{array, struct} +import org.apache.spark.sql.functions.{array, struct, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -128,15 +128,15 @@ object InExpressionBenchmark extends SqlBasedBenchmark { private def runTimestampBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { val name = s"$numItems timestamps" - val values = (1 to numItems).map(m => s"CAST('1970-01-01 01:00:00.$m' AS timestamp)") - val df = spark.range(0, numRows).select($"id".cast(TimestampType)) + val values = (1 to numItems).map(m => s"timestamp'1970-01-01 01:00:00.$m'") + val df = spark.range(0, numRows).select(timestamp_seconds($"id").as("id")) runBenchmark(name, df, values, numRows, minNumIters) } private def runDateBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { val name = s"$numItems dates" - val values = (1 to numItems).map(n => 1970 + n).map(y => s"CAST('$y-01-01' AS date)") - val df = spark.range(0, numRows).select($"id".cast(TimestampType).cast(DateType)) + val values = (1 to numItems).map(n => 1970 + n).map(y => s"date'$y-01-01'") + val df = spark.range(0, numRows).select(timestamp_seconds($"id").cast(DateType).as("id")) runBenchmark(name, df, values, numRows, minNumIters) } From 8d577092eda6a0d6a632419d7ee463db011aca2c Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 16 Jun 2020 09:13:00 -0700 Subject: [PATCH 025/384] [SPARK-31705][SQL][FOLLOWUP] Avoid the unnecessary CNF computation for full-outer joins ### What changes were proposed in this pull request? To avoid the unnecessary CNF computation for full-outer joins, this PR fixes code for filtering out full-outer joins at the entrance of the rule. ### Why are the changes needed? To mitigate optimizer overhead. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #28810 from maropu/SPARK-31705. Authored-by: Takeshi Yamamuro Signed-off-by: Yuming Wang --- .../sql/catalyst/expressions/predicates.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 22 ++++++++++++------- .../PushCNFPredicateThroughJoin.scala | 14 ++++++++---- 3 files changed, 25 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index c9b57367e0f44..05a5ff45b8fb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -211,7 +211,7 @@ trait PredicateHelper extends Logging { * @return the CNF result as sequence of disjunctive expressions. If the number of expressions * exceeds threshold on converting `Or`, `Seq.empty` is returned. */ - def conjunctiveNormalForm(condition: Expression): Seq[Expression] = { + protected def conjunctiveNormalForm(condition: Expression): Seq[Expression] = { val postOrderNodes = postOrderTraversal(condition) val resultStack = new mutable.Stack[Seq[Expression]] val maxCnfNodeCount = SQLConf.get.maxCnfNodeCount 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 8e57e9737c73f..e800ee3b93f51 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 @@ -1289,11 +1289,17 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { (leftEvaluateCondition, rightEvaluateCondition, commonCondition ++ nonDeterministic) } + private def canPushThrough(joinType: JoinType): Boolean = joinType match { + case _: InnerLike | LeftSemi | RightOuter | LeftOuter | LeftAnti | ExistenceJoin(_) => true + case _ => false + } + def apply(plan: LogicalPlan): LogicalPlan = plan transform applyLocally val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = { // push the where condition down into join filter - case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition, hint)) => + case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition, hint)) + if canPushThrough(joinType) => val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = split(splitConjunctivePredicates(filterCondition), left, right) joinType match { @@ -1333,13 +1339,13 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { (rightFilterConditions ++ commonFilterCondition). reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) - case FullOuter => f // DO Nothing for Full Outer Join - case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") - case UsingJoin(_, _) => sys.error("Untransformed Using join node") + + case other => + throw new IllegalStateException(s"Unexpected join type: $other") } // push down the join filter into sub query scanning if applicable - case j @ Join(left, right, joinType, joinCondition, hint) => + case j @ Join(left, right, joinType, joinCondition, hint) if canPushThrough(joinType) => val (leftJoinConditions, rightJoinConditions, commonJoinCondition) = split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) @@ -1369,9 +1375,9 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newJoinCond = (leftJoinConditions ++ commonJoinCondition).reduceLeftOption(And) Join(newLeft, newRight, joinType, newJoinCond, hint) - case FullOuter => j - case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") - case UsingJoin(_, _) => sys.error("Untransformed Using join node") + + case other => + throw new IllegalStateException(s"Unexpected join type: $other") } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala index f406b7d77ab63..109e5f993c02e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala @@ -29,8 +29,15 @@ import org.apache.spark.sql.catalyst.rules.Rule * when predicate pushdown happens. */ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { + + private def canPushThrough(joinType: JoinType): Boolean = joinType match { + case _: InnerLike | LeftSemi | RightOuter | LeftOuter | LeftAnti | ExistenceJoin(_) => true + case _ => false + } + def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case j @ Join(left, right, joinType, Some(joinCondition), hint) => + case j @ Join(left, right, joinType, Some(joinCondition), hint) + if canPushThrough(joinType) => val predicates = conjunctiveNormalForm(joinCondition) if (predicates.isEmpty) { j @@ -53,9 +60,8 @@ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelpe Join(newLeft, right, RightOuter, Some(joinCondition), hint) case LeftOuter | LeftAnti | ExistenceJoin(_) => Join(left, newRight, joinType, Some(joinCondition), hint) - case FullOuter => j - case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") - case UsingJoin(_, _) => sys.error("Untransformed Using join node") + case other => + throw new IllegalStateException(s"Unexpected join type: $other") } } } From 2ec9b866285fc059cae6816033babca64b4da7ec Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Tue, 16 Jun 2020 12:59:57 -0500 Subject: [PATCH 026/384] [SPARK-31929][WEBUI] Close leveldbiterator when leveldb.close ### What changes were proposed in this pull request? Close LevelDBIterator when LevelDB.close() is called. ### Why are the changes needed? This pull request would prevent JNI resources leaking from Level DB instance and its' iterators. In before implementation JNI resources from LevelDBIterator are cleaned by finalize() function. This behavior is also mentioned in comments of ["LevelDBIterator.java"](https://github.com/apache/spark/blob/master/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java) by squito . But if DB instance is already closed, then iterator's close method would be ignored. LevelDB's iterator would keep level db files opened (for the case table cache is filled up), till iterator.close() is called. Then these JNI resources (file handle) would be leaked. This JNI resource leaking issue would cause the problem described in [SPARK-31929](https://issues.apache.org/jira/browse/SPARK-31929) on Windows: in spark history server, leaked file handle for level db files would trigger "IOException" when HistoryServerDiskManager try to remove them for releasing disk space. ![IOException](https://user-images.githubusercontent.com/10524738/84134659-7c388680-aa7b-11ea-807f-04dcfa7886a0.JPG) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add unit test and manually tested it. Closes #28769 from zhli1142015/close-leveldbiterator-when-leveldb.close. Authored-by: Zhen Li Signed-off-by: Sean Owen --- .../apache/spark/util/kvstore/LevelDB.java | 32 ++++++++++++++++- .../spark/util/kvstore/LevelDBIterator.java | 1 + .../spark/util/kvstore/LevelDBSuite.java | 36 +++++++++++++++++++ .../apache/spark/status/AppStatusStore.scala | 8 ++++- 4 files changed, 75 insertions(+), 2 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java index 2ca4b0b2cb9f9..98f33b70fea23 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java @@ -19,8 +19,10 @@ import java.io.File; import java.io.IOException; +import java.lang.ref.SoftReference; import java.util.*; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; import static java.nio.charset.StandardCharsets.UTF_8; @@ -64,6 +66,13 @@ public class LevelDB implements KVStore { private final ConcurrentMap typeAliases; private final ConcurrentMap, LevelDBTypeInfo> types; + /** + * Trying to close a JNI LevelDB handle with a closed DB causes JVM crashes. This is used to + * ensure that all iterators are correctly closed before LevelDB is closed. Use soft reference + * to ensure that the iterator can be GCed, when it is only referenced here. + */ + private final ConcurrentLinkedQueue>> iteratorTracker; + public LevelDB(File path) throws Exception { this(path, new KVStoreSerializer()); } @@ -94,6 +103,8 @@ public LevelDB(File path, KVStoreSerializer serializer) throws Exception { aliases = new HashMap<>(); } typeAliases = new ConcurrentHashMap<>(aliases); + + iteratorTracker = new ConcurrentLinkedQueue<>(); } @Override @@ -189,7 +200,9 @@ public KVStoreView view(Class type) throws Exception { @Override public Iterator iterator() { try { - return new LevelDBIterator<>(type, LevelDB.this, this); + LevelDBIterator it = new LevelDBIterator<>(type, LevelDB.this, this); + iteratorTracker.add(new SoftReference<>(it)); + return it; } catch (Exception e) { throw Throwables.propagate(e); } @@ -238,6 +251,14 @@ public void close() throws IOException { } try { + if (iteratorTracker != null) { + for (SoftReference> ref: iteratorTracker) { + LevelDBIterator it = ref.get(); + if (it != null) { + it.close(); + } + } + } _db.close(); } catch (IOException ioe) { throw ioe; @@ -252,6 +273,7 @@ public void close() throws IOException { * with a closed DB can cause JVM crashes, so this ensures that situation does not happen. */ void closeIterator(LevelDBIterator it) throws IOException { + notifyIteratorClosed(it); synchronized (this._db) { DB _db = this._db.get(); if (_db != null) { @@ -260,6 +282,14 @@ void closeIterator(LevelDBIterator it) throws IOException { } } + /** + * Remove iterator from iterator tracker. `LevelDBIterator` calls it to notify + * iterator is closed. + */ + void notifyIteratorClosed(LevelDBIterator it) { + iteratorTracker.removeIf(ref -> it.equals(ref.get())); + } + /** Returns metadata about indices for the given type. */ LevelDBTypeInfo getTypeInfo(Class type) throws Exception { LevelDBTypeInfo ti = types.get(type); diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java index 94e8c9fc5796c..e8fb4fac5ba17 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java @@ -185,6 +185,7 @@ public boolean skip(long n) { @Override public synchronized void close() throws IOException { + db.notifyIteratorClosed(this); if (!closed) { it.close(); closed = true; diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java index 0b755ba0e8000..f6566617765d4 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -19,6 +19,7 @@ import java.io.File; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.stream.Collectors; @@ -276,6 +277,41 @@ public void testNegativeIndexValues() throws Exception { assertEquals(expected, results); } + @Test + public void testCloseLevelDBIterator() throws Exception { + // SPARK-31929: test when LevelDB.close() is called, related LevelDBIterators + // are closed. And files opened by iterators are also closed. + File dbPathForCloseTest = File + .createTempFile( + "test_db_close.", + ".ldb"); + dbPathForCloseTest.delete(); + LevelDB dbForCloseTest = new LevelDB(dbPathForCloseTest); + for (int i = 0; i < 8192; i++) { + dbForCloseTest.write(createCustomType1(i)); + } + String key = dbForCloseTest + .view(CustomType1.class).iterator().next().key; + assertEquals("key0", key); + Iterator it0 = dbForCloseTest + .view(CustomType1.class).max(1).iterator(); + while (it0.hasNext()) { + it0.next(); + } + System.gc(); + Iterator it1 = dbForCloseTest + .view(CustomType1.class).iterator(); + assertEquals("key0", it1.next().key); + try (KVStoreIterator it2 = dbForCloseTest + .view(CustomType1.class).closeableIterator()) { + assertEquals("key0", it2.next().key); + } + dbForCloseTest.close(); + assertTrue(dbPathForCloseTest.exists()); + FileUtils.deleteQuietly(dbPathForCloseTest); + assertTrue(!dbPathForCloseTest.exists()); + } + private CustomType1 createCustomType1(int i) { CustomType1 t = new CustomType1(); t.key = "key" + i; diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 31a6f7d901131..106d272948b9f 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -39,7 +39,13 @@ private[spark] class AppStatusStore( def applicationInfo(): v1.ApplicationInfo = { try { // The ApplicationInfo may not be available when Spark is starting up. - store.view(classOf[ApplicationInfoWrapper]).max(1).iterator().next().info + Utils.tryWithResource( + store.view(classOf[ApplicationInfoWrapper]) + .max(1) + .closeableIterator() + ) { it => + it.next().info + } } catch { case _: NoSuchElementException => throw new NoSuchElementException("Failed to get the application information. " + From 7f6a8ab16656533f460f2421dd2381cc91c6cbf4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 16 Jun 2020 16:46:32 -0700 Subject: [PATCH 027/384] [SPARK-31777][ML][PYSPARK] Add user-specified fold column to CrossValidator ### What changes were proposed in this pull request? This patch adds user-specified fold column support to `CrossValidator`. User can assign fold numbers to dataset instead of letting Spark do random splits. ### Why are the changes needed? This gives `CrossValidator` users more flexibility in splitting folds. ### Does this PR introduce _any_ user-facing change? Yes, a new `foldCol` param is added to `CrossValidator`. User can use it to specify custom fold splitting. ### How was this patch tested? Added unit tests. Closes #28704 from viirya/SPARK-31777. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- .../spark/ml/tuning/CrossValidator.scala | 43 +++++++-- .../org/apache/spark/mllib/util/MLUtils.scala | 34 ++++++- .../spark/ml/tuning/CrossValidatorSuite.scala | 64 ++++++++++++++ .../spark/mllib/util/MLUtilsSuite.scala | 30 +++++++ python/pyspark/ml/tests/test_tuning.py | 74 ++++++++++++++++ python/pyspark/ml/tuning.py | 88 +++++++++++++++---- 6 files changed, 308 insertions(+), 25 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 858cc57982a3f..e99c55b0cdd85 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -30,13 +30,13 @@ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator -import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.{HasCollectSubModels, HasParallelism} import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.util.ThreadUtils /** @@ -56,6 +56,19 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { def getNumFolds: Int = $(numFolds) setDefault(numFolds -> 3) + + /** + * Param for the column name of user specified fold number. Once this is specified, + * `CrossValidator` won't do random k-fold split. Note that this column should be + * integer type with range [0, numFolds) and Spark will throw exception on out-of-range + * fold numbers. + */ + val foldCol: Param[String] = new Param[String](this, "foldCol", + "the column name of user specified fold number") + + def getFoldCol: String = $(foldCol) + + setDefault(foldCol, "") } /** @@ -94,6 +107,10 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) @Since("2.0.0") def setSeed(value: Long): this.type = set(seed, value) + /** @group setParam */ + @Since("3.1.0") + def setFoldCol(value: String): this.type = set(foldCol, value) + /** * Set the maximum level of parallelism to evaluate models in parallel. * Default is 1 for serial evaluation @@ -132,7 +149,7 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, numFolds, seed, parallelism) + instr.logParams(this, numFolds, seed, parallelism, foldCol) logTuningParams(instr) val collectSubModelsParam = $(collectSubModels) @@ -142,10 +159,15 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) } else None // Compute metrics for each model over each split - val splits = MLUtils.kFold(dataset.toDF.rdd, $(numFolds), $(seed)) + val (splits, schemaWithoutFold) = if ($(foldCol) == "") { + (MLUtils.kFold(dataset.toDF.rdd, $(numFolds), $(seed)), schema) + } else { + val filteredSchema = StructType(schema.filter(_.name != $(foldCol)).toArray) + (MLUtils.kFold(dataset.toDF, $(numFolds), $(foldCol)), filteredSchema) + } val metrics = splits.zipWithIndex.map { case ((training, validation), splitIndex) => - val trainingDataset = sparkSession.createDataFrame(training, schema).cache() - val validationDataset = sparkSession.createDataFrame(validation, schema).cache() + val trainingDataset = sparkSession.createDataFrame(training, schemaWithoutFold).cache() + val validationDataset = sparkSession.createDataFrame(validation, schemaWithoutFold).cache() instr.logDebug(s"Train split $splitIndex with multiple sets of parameters.") // Fit models in a Future for training in parallel @@ -183,7 +205,14 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) } @Since("1.4.0") - override def transformSchema(schema: StructType): StructType = transformSchemaImpl(schema) + override def transformSchema(schema: StructType): StructType = { + if ($(foldCol) != "") { + val foldColDt = schema.apply($(foldCol)).dataType + require(foldColDt.isInstanceOf[IntegerType], + s"The specified `foldCol` column ${$(foldCol)} must be integer type, but got $foldColDt.") + } + transformSchemaImpl(schema) + } @Since("1.4.0") override def copy(extra: ParamMap): CrossValidator = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 9198334ba02a1..d177364d012f4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.util import scala.annotation.varargs import scala.reflect.ClassTag -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.linalg.{MatrixUDT => MLMatrixUDT, VectorUDT => MLVectorUDT} @@ -28,7 +28,7 @@ import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.{PartitionwiseSampledRDD, RDD} -import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.functions._ @@ -248,6 +248,36 @@ object MLUtils extends Logging { }.toArray } + /** + * Version of `kFold()` taking a fold column name. + */ + @Since("3.1.0") + def kFold(df: DataFrame, numFolds: Int, foldColName: String): Array[(RDD[Row], RDD[Row])] = { + val foldCol = df.col(foldColName) + val checker = udf { foldNum: Int => + // Valid fold number is in range [0, numFolds). + if (foldNum < 0 || foldNum >= numFolds) { + throw new SparkException(s"Fold number must be in range [0, $numFolds), but got $foldNum.") + } + true + } + (0 until numFolds).map { fold => + val training = df + .filter(checker(foldCol) && foldCol =!= fold) + .drop(foldColName).rdd + val validation = df + .filter(checker(foldCol) && foldCol === fold) + .drop(foldColName).rdd + if (training.isEmpty()) { + throw new SparkException(s"The training data at fold $fold is empty.") + } + if (validation.isEmpty()) { + throw new SparkException(s"The validation data at fold $fold is empty.") + } + (training, validation) + }.toArray + } + /** * Returns a new vector with `1.0` (bias) appended to the input vector. */ diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index a30428ec2d283..d7cbfa8e2e2c5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.mllib.util.LinearDataGenerator import org.apache.spark.sql.Dataset +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType class CrossValidatorSuite @@ -40,10 +41,14 @@ class CrossValidatorSuite import testImplicits._ @transient var dataset: Dataset[_] = _ + @transient var datasetWithFold: Dataset[_] = _ override def beforeAll(): Unit = { super.beforeAll() dataset = sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2).toDF() + val dfWithRandom = dataset.repartition(1).withColumn("random", rand(100L)) + val foldCol = when(col("random") < 0.33, 0).when(col("random") < 0.66, 1).otherwise(2) + datasetWithFold = dfWithRandom.withColumn("fold", foldCol).drop("random").repartition(2) } test("cross validation with logistic regression") { @@ -75,6 +80,65 @@ class CrossValidatorSuite } } + test("cross validation with logistic regression with fold col") { + val lr = new LogisticRegression + val lrParamMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.001, 1000.0)) + .addGrid(lr.maxIter, Array(0, 10)) + .build() + val eval = new BinaryClassificationEvaluator + val cv = new CrossValidator() + .setEstimator(lr) + .setEstimatorParamMaps(lrParamMaps) + .setEvaluator(eval) + .setNumFolds(3) + .setFoldCol("fold") + val cvModel = cv.fit(datasetWithFold) + + MLTestingUtils.checkCopyAndUids(cv, cvModel) + + val parent = cvModel.bestModel.parent.asInstanceOf[LogisticRegression] + assert(parent.getRegParam === 0.001) + assert(parent.getMaxIter === 10) + assert(cvModel.avgMetrics.length === lrParamMaps.length) + + val result = cvModel.transform(dataset).select("prediction").as[Double].collect() + testTransformerByGlobalCheckFunc[(Double, Vector)](dataset.toDF(), cvModel, "prediction") { + rows => + val result2 = rows.map(_.getDouble(0)) + assert(result === result2) + } + } + + test("cross validation with logistic regression with wrong fold col") { + val lr = new LogisticRegression + val lrParamMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.001, 1000.0)) + .addGrid(lr.maxIter, Array(0, 10)) + .build() + val eval = new BinaryClassificationEvaluator + val cv = new CrossValidator() + .setEstimator(lr) + .setEstimatorParamMaps(lrParamMaps) + .setEvaluator(eval) + .setNumFolds(3) + .setFoldCol("fold1") + val err1 = intercept[IllegalArgumentException] { + cv.fit(datasetWithFold) + } + assert(err1.getMessage.contains("fold1 does not exist. Available: label, features, fold")) + + // Fold column must be integer type. + val foldCol = udf(() => 1L) + val datasetWithWrongFoldType = dataset.withColumn("fold1", foldCol()) + val err2 = intercept[IllegalArgumentException] { + cv.fit(datasetWithWrongFoldType) + } + assert(err2 + .getMessage + .contains("The specified `foldCol` column fold1 must be integer type, but got LongType.")) + } + test("cross validation with linear regression") { val dataset = sc.parallelize( LinearDataGenerator.generateLinearInput( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 665708a780c48..fb3bc9f798490 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -353,4 +353,34 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { convertMatrixColumnsFromML(df, "p._2") } } + + test("kFold with fold column") { + val data = sc.parallelize(1 to 100, 2).map(x => (x, if (x <= 50) 0 else 1)).toDF("i", "fold") + val collectedData = data.collect().map(_.getInt(0)).sorted + val twoFoldedRdd = kFold(data, 2, "fold") + assert(twoFoldedRdd(0)._1.collect().map(_.getInt(0)).sorted === + twoFoldedRdd(1)._2.collect().map(_.getInt(0)).sorted) + assert(twoFoldedRdd(0)._2.collect().map(_.getInt(0)).sorted === + twoFoldedRdd(1)._1.collect().map(_.getInt(0)).sorted) + + val result1 = twoFoldedRdd(0)._1.union(twoFoldedRdd(0)._2).collect().map(_.getInt(0)).sorted + assert(result1 === collectedData, + "Each training+validation set combined should contain all of the data.") + val result2 = twoFoldedRdd(1)._1.union(twoFoldedRdd(1)._2).collect().map(_.getInt(0)).sorted + assert(result2 === collectedData, + "Each training+validation set combined should contain all of the data.") + } + + test("kFold with fold column: invalid fold numbers") { + val data = sc.parallelize(Seq(0, 1, 2), 2).toDF( "fold") + val err1 = intercept[SparkException] { + kFold(data, 2, "fold")(0)._1.collect() + } + assert(err1.getMessage.contains("Fold number must be in range [0, 2), but got 2.")) + + val err2 = intercept[SparkException] { + kFold(data, 4, "fold")(0)._1.collect() + } + assert(err2.getMessage.contains("The validation data at fold 3 is empty.")) + } } diff --git a/python/pyspark/ml/tests/test_tuning.py b/python/pyspark/ml/tests/test_tuning.py index 6bcc3f93e1b3b..0aa5d47ca6bc4 100644 --- a/python/pyspark/ml/tests/test_tuning.py +++ b/python/pyspark/ml/tests/test_tuning.py @@ -380,6 +380,80 @@ def test_save_load_pipeline_estimator(self): original_nested_pipeline_model.stages): self.assertEqual(loadedStage.uid, originalStage.uid) + def test_user_specified_folds(self): + from pyspark.sql import functions as F + + dataset = self.spark.createDataFrame( + [(Vectors.dense([0.0]), 0.0), + (Vectors.dense([0.4]), 1.0), + (Vectors.dense([0.5]), 0.0), + (Vectors.dense([0.6]), 1.0), + (Vectors.dense([1.0]), 1.0)] * 10, + ["features", "label"]).repartition(2, "features") + + dataset_with_folds = dataset.repartition(1).withColumn("random", rand(100)) \ + .withColumn("fold", F.when(F.col("random") < 0.33, 0) + .when(F.col("random") < 0.66, 1) + .otherwise(2)).repartition(2, "features") + + lr = LogisticRegression() + grid = ParamGridBuilder().addGrid(lr.maxIter, [20]).build() + evaluator = BinaryClassificationEvaluator() + + cv = CrossValidator(estimator=lr, estimatorParamMaps=grid, evaluator=evaluator, numFolds=3) + cv_with_user_folds = CrossValidator(estimator=lr, + estimatorParamMaps=grid, + evaluator=evaluator, + numFolds=3, + foldCol="fold") + + self.assertEqual(cv.getEstimator().uid, cv_with_user_folds.getEstimator().uid) + + cvModel1 = cv.fit(dataset) + cvModel2 = cv_with_user_folds.fit(dataset_with_folds) + for index in range(len(cvModel1.avgMetrics)): + print(abs(cvModel1.avgMetrics[index] - cvModel2.avgMetrics[index])) + self.assertTrue(abs(cvModel1.avgMetrics[index] - cvModel2.avgMetrics[index]) + < 0.1) + + # test save/load of CrossValidator + temp_path = tempfile.mkdtemp() + cvPath = temp_path + "/cv" + cv_with_user_folds.save(cvPath) + loadedCV = CrossValidator.load(cvPath) + self.assertEqual(loadedCV.getFoldCol(), cv_with_user_folds.getFoldCol()) + + def test_invalid_user_specified_folds(self): + from pyspark.sql import functions as F + + dataset_with_folds = self.spark.createDataFrame( + [(Vectors.dense([0.0]), 0.0, 0), + (Vectors.dense([0.4]), 1.0, 1), + (Vectors.dense([0.5]), 0.0, 2), + (Vectors.dense([0.6]), 1.0, 0), + (Vectors.dense([1.0]), 1.0, 1)] * 10, + ["features", "label", "fold"]) + + lr = LogisticRegression() + grid = ParamGridBuilder().addGrid(lr.maxIter, [20]).build() + evaluator = BinaryClassificationEvaluator() + + cv = CrossValidator(estimator=lr, + estimatorParamMaps=grid, + evaluator=evaluator, + numFolds=2, + foldCol="fold") + with self.assertRaisesRegexp(Exception, "Fold number must be in range"): + cv.fit(dataset_with_folds) + + cv = CrossValidator(estimator=lr, + estimatorParamMaps=grid, + evaluator=evaluator, + numFolds=4, + foldCol="fold") + with self.assertRaisesRegexp(Exception, "The validation data at fold 3 is empty"): + cv.fit(dataset_with_folds) + class TrainValidationSplitTests(SparkSessionTestCase): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index cb4542c5d025f..e00753b2ffc20 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -27,7 +27,8 @@ from pyspark.ml.param.shared import HasCollectSubModels, HasParallelism, HasSeed from pyspark.ml.util import * from pyspark.ml.wrapper import JavaParams -from pyspark.sql.functions import rand +from pyspark.sql.functions import col, lit, rand, UserDefinedFunction +from pyspark.sql.types import BooleanType __all__ = ['ParamGridBuilder', 'CrossValidator', 'CrossValidatorModel', 'TrainValidationSplit', 'TrainValidationSplitModel'] @@ -200,6 +201,12 @@ class _CrossValidatorParams(_ValidatorParams): numFolds = Param(Params._dummy(), "numFolds", "number of folds for cross validation", typeConverter=TypeConverters.toInt) + foldCol = Param(Params._dummy(), "foldCol", "Param for the column name of user " + + "specified fold number. Once this is specified, :py:class:`CrossValidator` " + + "won't do random k-fold split. Note that this column should be integer type " + + "with range [0, numFolds) and Spark will throw exception on out-of-range " + + "fold numbers.", typeConverter=TypeConverters.toString) + @since("1.4.0") def getNumFolds(self): """ @@ -207,6 +214,13 @@ def getNumFolds(self): """ return self.getOrDefault(self.numFolds) + @since("3.1.0") + def getFoldCol(self): + """ + Gets the value of foldCol or its default value. + """ + return self.getOrDefault(self.foldCol) + class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollectSubModels, MLReadable, MLWritable): @@ -255,23 +269,23 @@ class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollec @keyword_only def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3, - seed=None, parallelism=1, collectSubModels=False): + seed=None, parallelism=1, collectSubModels=False, foldCol=""): """ __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3,\ - seed=None, parallelism=1, collectSubModels=False) + seed=None, parallelism=1, collectSubModels=False, foldCol="") """ super(CrossValidator, self).__init__() - self._setDefault(numFolds=3, parallelism=1) + self._setDefault(numFolds=3, parallelism=1, foldCol="") kwargs = self._input_kwargs self._set(**kwargs) @keyword_only @since("1.4.0") def setParams(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3, - seed=None, parallelism=1, collectSubModels=False): + seed=None, parallelism=1, collectSubModels=False, foldCol=""): """ setParams(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3,\ - seed=None, parallelism=1, collectSubModels=False): + seed=None, parallelism=1, collectSubModels=False, foldCol=""): Sets params for cross validator. """ kwargs = self._input_kwargs @@ -305,6 +319,13 @@ def setNumFolds(self, value): """ return self._set(numFolds=value) + @since("3.1.0") + def setFoldCol(self, value): + """ + Sets the value of :py:attr:`foldCol`. + """ + return self._set(foldCol=value) + def setSeed(self, value): """ Sets the value of :py:attr:`seed`. @@ -329,10 +350,6 @@ def _fit(self, dataset): numModels = len(epm) eva = self.getOrDefault(self.evaluator) nFolds = self.getOrDefault(self.numFolds) - seed = self.getOrDefault(self.seed) - h = 1.0 / nFolds - randCol = self.uid + "_rand" - df = dataset.select("*", rand(seed).alias(randCol)) metrics = [0.0] * numModels pool = ThreadPool(processes=min(self.getParallelism(), numModels)) @@ -341,12 +358,10 @@ def _fit(self, dataset): if collectSubModelsParam: subModels = [[None for j in range(numModels)] for i in range(nFolds)] + datasets = self._kFold(dataset) for i in range(nFolds): - validateLB = i * h - validateUB = (i + 1) * h - condition = (df[randCol] >= validateLB) & (df[randCol] < validateUB) - validation = df.filter(condition).cache() - train = df.filter(~condition).cache() + validation = datasets[i][1].cache() + train = datasets[i][0].cache() tasks = _parallelFitTasks(est, train, eva, validation, epm, collectSubModelsParam) for j, metric, subModel in pool.imap_unordered(lambda f: f(), tasks): @@ -364,6 +379,45 @@ def _fit(self, dataset): bestModel = est.fit(dataset, epm[bestIndex]) return self._copyValues(CrossValidatorModel(bestModel, metrics, subModels)) + def _kFold(self, dataset): + nFolds = self.getOrDefault(self.numFolds) + foldCol = self.getOrDefault(self.foldCol) + + datasets = [] + if not foldCol: + # Do random k-fold split. + seed = self.getOrDefault(self.seed) + h = 1.0 / nFolds + randCol = self.uid + "_rand" + df = dataset.select("*", rand(seed).alias(randCol)) + for i in range(nFolds): + validateLB = i * h + validateUB = (i + 1) * h + condition = (df[randCol] >= validateLB) & (df[randCol] < validateUB) + validation = df.filter(condition) + train = df.filter(~condition) + datasets.append((train, validation)) + else: + # Use user-specified fold numbers. + def checker(foldNum): + if foldNum < 0 or foldNum >= nFolds: + raise ValueError( + "Fold number must be in range [0, %s), but got %s." % (nFolds, foldNum)) + return True + + checker_udf = UserDefinedFunction(checker, BooleanType()) + for i in range(nFolds): + training = dataset.filter(checker_udf(dataset[foldCol]) & (col(foldCol) != lit(i))) + validation = dataset.filter( + checker_udf(dataset[foldCol]) & (col(foldCol) == lit(i))) + if training.rdd.getNumPartitions() == 0 or len(training.take(1)) == 0: + raise ValueError("The training data at fold %s is empty." % i) + if validation.rdd.getNumPartitions() == 0 or len(validation.take(1)) == 0: + raise ValueError("The validation data at fold %s is empty." % i) + datasets.append((training, validation)) + + return datasets + @since("1.4.0") def copy(self, extra=None): """ @@ -407,10 +461,11 @@ def _from_java(cls, java_stage): seed = java_stage.getSeed() parallelism = java_stage.getParallelism() collectSubModels = java_stage.getCollectSubModels() + foldCol = java_stage.getFoldCol() # Create a new instance of this stage. py_stage = cls(estimator=estimator, estimatorParamMaps=epms, evaluator=evaluator, numFolds=numFolds, seed=seed, parallelism=parallelism, - collectSubModels=collectSubModels) + collectSubModels=collectSubModels, foldCol=foldCol) py_stage._resetUid(java_stage.uid()) return py_stage @@ -431,6 +486,7 @@ def _to_java(self): _java_obj.setNumFolds(self.getNumFolds()) _java_obj.setParallelism(self.getParallelism()) _java_obj.setCollectSubModels(self.getCollectSubModels()) + _java_obj.setFoldCol(self.getFoldCol()) return _java_obj From eeb81200e221decca6116a863cd50148e622cc75 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 16 Jun 2020 18:22:12 -0700 Subject: [PATCH 028/384] [SPARK-31337][SQL] Support MS SQL Kerberos login in JDBC connector ### What changes were proposed in this pull request? When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it. This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues. In this PR I've added MS SQL support. What this PR contains: * Added `MSSQLConnectionProvider` * Added `MSSQLConnectionProviderSuite` * Changed MS SQL JDBC driver to use the latest (test scope only) * Changed `MsSqlServerIntegrationSuite` docker image to use the latest * Added a version comment to `MariaDBConnectionProvider` to increase trackability ### Why are the changes needed? Missing JDBC kerberos support. ### Does this PR introduce _any_ user-facing change? Yes, now user is able to connect to MS SQL using kerberos. ### How was this patch tested? * Additional + existing unit tests * Existing integration tests * Test on cluster manually Closes #28635 from gaborgsomogyi/SPARK-31337. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- external/docker-integration-tests/pom.xml | 1 - .../jdbc/MsSqlServerIntegrationSuite.scala | 2 +- pom.xml | 6 ++ sql/core/pom.xml | 5 + .../jdbc/connection/ConnectionProvider.scala | 4 + .../connection/MSSQLConnectionProvider.scala | 97 +++++++++++++++++++ .../MariaDBConnectionProvider.scala | 2 +- .../MSSQLConnectionProviderSuite.scala | 51 ++++++++++ 8 files changed, 165 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 7f9e92f585168..298e3d36c145a 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -166,7 +166,6 @@ com.microsoft.sqlserver mssql-jdbc - 7.2.1.jre8 test diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 42d64873c44d9..6c633af1fde84 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "mcr.microsoft.com/mssql/server:2017-GA-ubuntu" + override val imageName = "mcr.microsoft.com/mssql/server:2019-GA-ubuntu-16.04" override val env = Map( "SA_PASSWORD" -> "Sapass123", "ACCEPT_EULA" -> "Y" diff --git a/pom.xml b/pom.xml index 8d552e08f0cd9..3abbf871ad2de 100644 --- a/pom.xml +++ b/pom.xml @@ -978,6 +978,12 @@ 11.5.0.0 test + + com.microsoft.sqlserver + mssql-jdbc + 8.2.2.jre8 + test + org.apache.curator curator-recipes diff --git a/sql/core/pom.xml b/sql/core/pom.xml index e4ef1467a960e..0855fa13fa79a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -145,6 +145,11 @@ jcc test + + com.microsoft.sqlserver + mssql-jdbc + test + org.apache.parquet parquet-avro diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala index 73e73e59be574..6c310ced37883 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala @@ -60,6 +60,10 @@ private[jdbc] object ConnectionProvider extends Logging { logDebug("DB2 connection provider found") new DB2ConnectionProvider(driver, options) + case MSSQLConnectionProvider.driverClass => + logDebug("MS SQL connection provider found") + new MSSQLConnectionProvider(driver, options) + case _ => throw new IllegalArgumentException(s"Driver ${options.driverClass} does not support " + "Kerberos authentication") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala new file mode 100644 index 0000000000000..2950aa9b4db94 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc.connection + +import java.security.PrivilegedExceptionAction +import java.sql.{Connection, Driver} +import java.util.Properties + +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions + +private[sql] class MSSQLConnectionProvider( + driver: Driver, + options: JDBCOptions, + parserMethod: String = "parseAndMergeProperties" + ) extends SecureConnectionProvider(driver, options) { + override val appEntry: String = { + val configName = "jaasConfigurationName" + val appEntryDefault = "SQLJDBCDriver" + + val parseURL = try { + // The default parser method signature is the following: + // private Properties parseAndMergeProperties(String Url, Properties suppliedProperties) + val m = driver.getClass.getDeclaredMethod(parserMethod, classOf[String], classOf[Properties]) + m.setAccessible(true) + Some(m) + } catch { + case _: NoSuchMethodException => None + } + + parseURL match { + case Some(m) => + logDebug("Property parser method found, using it") + m.invoke(driver, options.url, null).asInstanceOf[Properties] + .getProperty(configName, appEntryDefault) + + case None => + logDebug("Property parser method not found, using custom parsing mechanism") + options.url.split(';').map(_.split('=')) + .find(kv => kv.length == 2 && kv(0) == configName) + .getOrElse(Array(configName, appEntryDefault))(1) + } + } + + override def getConnection(): Connection = { + setAuthenticationConfigIfNeeded() + UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + MSSQLConnectionProvider.super.getConnection() + } + } + ) + } + + override def getAdditionalProperties(): Properties = { + val result = new Properties() + // These props needed to reach internal kerberos authentication in the JDBC driver + result.put("integratedSecurity", "true") + result.put("authenticationScheme", "JavaKerberos") + result + } + + override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { + val (parent, configEntry) = getConfigWithAppEntry() + /** + * Couple of things to mention here (v8.2.2 client): + * 1. MS SQL supports JAAS application name configuration + * 2. MS SQL sets a default JAAS config if "java.security.auth.login.config" is not set + */ + val entryUsesKeytab = configEntry != null && + configEntry.exists(_.getOptions().get("useKeyTab") == "true") + if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { + setAuthenticationConfig(parent) + } + } +} + +private[sql] object MSSQLConnectionProvider { + val driverClass = "com.microsoft.sqlserver.jdbc.SQLServerDriver" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala index 8e3381077cbbf..589f13cf6ad5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala @@ -30,7 +30,7 @@ private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptio override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { val (parent, configEntry) = getConfigWithAppEntry() /** - * Couple of things to mention here: + * Couple of things to mention here (v2.5.4 client): * 1. MariaDB doesn't support JAAS application name configuration * 2. MariaDB sets a default JAAS config if "java.security.auth.login.config" is not set */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala new file mode 100644 index 0000000000000..249f1e36347ed --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc.connection + +class MSSQLConnectionProviderSuite extends ConnectionProviderSuiteBase { + test("setAuthenticationConfigIfNeeded default parser must set authentication if not set") { + val driver = registerDriver(MSSQLConnectionProvider.driverClass) + val defaultProvider = new MSSQLConnectionProvider( + driver, options("jdbc:sqlserver://localhost/mssql")) + val customProvider = new MSSQLConnectionProvider( + driver, options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) + + testProviders(defaultProvider, customProvider) + } + + test("setAuthenticationConfigIfNeeded custom parser must set authentication if not set") { + val parserMethod = "IntentionallyNotExistingMethod" + val driver = registerDriver(MSSQLConnectionProvider.driverClass) + val defaultProvider = new MSSQLConnectionProvider( + driver, options("jdbc:sqlserver://localhost/mssql"), parserMethod) + val customProvider = new MSSQLConnectionProvider( + driver, + options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql"), + parserMethod) + + testProviders(defaultProvider, customProvider) + } + + private def testProviders( + defaultProvider: SecureConnectionProvider, + customProvider: SecureConnectionProvider) = { + assert(defaultProvider.appEntry !== customProvider.appEntry) + testSecureConnectionProvider(defaultProvider) + testSecureConnectionProvider(customProvider) + } +} From afd8a8b964d0d1f46f531571613d2c68fe631d25 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 17 Jun 2020 12:07:36 +0900 Subject: [PATCH 029/384] [SPARK-31989][SQL] Generate JSON rebasing files w/ 30 minutes step ### What changes were proposed in this pull request? 1. Change the max step from 1 week to 30 minutes in the tests `RebaseDateTimeSuite`.`generate 'gregorian-julian-rebase-micros.json'` and `generate 'julian-gregorian-rebase-micros.json'`. 2. Parallelise JSON files generation in the function `generateRebaseJson` by using `ThreadUtils.parmap`. ### Why are the changes needed? 1. To prevent the bugs that are fixed by https://github.com/apache/spark/pull/28787 and https://github.com/apache/spark/pull/28816. 2. The parallelisation speeds up JSON file generation. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? By generating the JSON file `julian-gregorian-rebase-micros.json`. Closes #28827 from MaxGekk/rebase-30-min. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- .../sql/catalyst/util/RebaseDateTimeSuite.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index 254bf01c89b4f..9b43635a70701 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.catalyst.util.RebaseDateTime._ +import org.apache.spark.util.ThreadUtils class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { @@ -254,11 +255,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper} case class RebaseRecord(tz: String, switches: Array[Long], diffs: Array[Long]) - - val result = new ArrayBuffer[RebaseRecord]() - ALL_TIMEZONES - .sortBy(_.getId) - .foreach { zid => + val rebaseRecords = ThreadUtils.parmap(ALL_TIMEZONES, "JSON-rebase-gen", 16) { zid => withDefaultTimeZone(zid) { val tz = TimeZone.getTimeZone(zid) val start = adjustFunc( @@ -272,7 +269,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { var micros = start var diff = Long.MaxValue - val maxStep = DAYS_PER_WEEK * MICROS_PER_DAY + val maxStep = 30 * MICROS_PER_MINUTE var step: Long = MICROS_PER_SECOND val switches = new ArrayBuffer[Long]() val diffs = new ArrayBuffer[Long]() @@ -294,9 +291,11 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { } micros += step } - result.append(RebaseRecord(zid.getId, switches.toArray, diffs.toArray)) + RebaseRecord(zid.getId, switches.toArray, diffs.toArray) } } + val result = new ArrayBuffer[RebaseRecord]() + rebaseRecords.sortBy(_.tz).foreach(result.append(_)) val mapper = (new ObjectMapper() with ScalaObjectMapper) .registerModule(DefaultScalaModule) .writerWithDefaultPrettyPrinter() From feeca63198466640ac461a2a34922493fa6162a8 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 17 Jun 2020 12:10:12 +0900 Subject: [PATCH 030/384] [SPARK-32011][PYTHON][CORE] Remove warnings about pin-thread modes and guide to use collectWithJobGroup ### What changes were proposed in this pull request? This PR proposes to remove the warning about multi-thread in local properties, and change the guide to use `collectWithJobGroup` for multi-threads for now because: - It is too noisy to users who don't use multiple threads - the number of this single thread case is arguably more prevailing. - There was a critical issue found about pin-thread mode SPARK-32010, which will be fixed in Spark 3.1. - To smoothly migrate, `RDD.collectWithJobGroup` was added, which will be deprecated in Spark 3.1 with SPARK-32010 fixed. I will target to deprecate `RDD.collectWithJobGroup`, and make this pin-thread mode stable in Spark 3.1. In the future releases, I plan to make this mode as a default mode, and remove `RDD.collectWithJobGroup` away. ### Why are the changes needed? To avoid guiding users a feature with a critical issue, and provide a proper workaround for now. ### Does this PR introduce _any_ user-facing change? Yes, warning message and documentation. ### How was this patch tested? Manually tested: Before: ``` >>> spark.sparkContext.setLocalProperty("a", "b") /.../spark/python/pyspark/util.py:141: UserWarning: Currently, 'setLocalProperty' (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties for each thread on PVM. To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. ``` After: ``` >>> spark.sparkContext.setLocalProperty("a", "b") ``` Closes #28845 from HyukjinKwon/SPARK-32011. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/context.py | 37 ++++++------------------------------- python/pyspark/util.py | 27 --------------------------- 2 files changed, 6 insertions(+), 58 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 96353bb9228d5..32d69edb171db 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -41,7 +41,6 @@ from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler -from pyspark.util import _warn_pin_thread if sys.version > '3': xrange = range @@ -1026,17 +1025,9 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): .. note:: Currently, setting a group ID (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local - properties for each thread on PVM. - - To work around this, you can set `PYSPARK_PIN_THREAD` to - `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties - from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. - - To work around this, you should manually copy and set the local - properties from the parent thread to the child thread when you create another thread. + properties for each thread on PVM. To work around this, You can use + :meth:`RDD.collectWithJobGroup` for now. """ - _warn_pin_thread("setJobGroup") self._jsc.setJobGroup(groupId, description, interruptOnCancel) def setLocalProperty(self, key, value): @@ -1047,17 +1038,9 @@ def setLocalProperty(self, key, value): .. note:: Currently, setting a local property with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. - - To work around this, you can set `PYSPARK_PIN_THREAD` to - `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties - from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. - - To work around this, you should manually copy and set the local - properties from the parent thread to the child thread when you create another thread. + for each thread on PVM. To work around this, You can use + :meth:`RDD.collectWithJobGroup`. """ - _warn_pin_thread("setLocalProperty") self._jsc.setLocalProperty(key, value) def getLocalProperty(self, key): @@ -1074,17 +1057,9 @@ def setJobDescription(self, value): .. note:: Currently, setting a job description (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate - local properties for each thread on PVM. - - To work around this, you can set `PYSPARK_PIN_THREAD` to - `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties - from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. - - To work around this, you should manually copy and set the local - properties from the parent thread to the child thread when you create another thread. + local properties for each thread on PVM. To work around this, You can use + :meth:`RDD.collectWithJobGroup` for now. """ - _warn_pin_thread("setJobDescription") self._jsc.setJobDescription(value) def sparkUser(self): diff --git a/python/pyspark/util.py b/python/pyspark/util.py index cc614a2c0eaed..d9429372a6bfc 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -114,33 +114,6 @@ def wrapper(*args, **kwargs): return wrapper -def _warn_pin_thread(name): - if os.environ.get("PYSPARK_PIN_THREAD", "false").lower() == "true": - msg = ( - "PYSPARK_PIN_THREAD feature is enabled. " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread.") - else: - msg = ( - "Currently, '%s' (set to local properties) with multiple threads does " - "not properly work. " - "\n" - "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " - "for multiple threads on PVM, which fails to isolate local properties for each " - "thread on PVM. " - "\n" - "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread." % name) - warnings.warn(msg, UserWarning) - - def _print_missing_jar(lib_name, pkg_name, jar_name, spark_version): print(""" ________________________________________________________________________________________________ From 93bb70f3fe65b567cd55cb3b8fb2a748e2a54b24 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 16 Jun 2020 21:05:29 -0700 Subject: [PATCH 031/384] [SPARK-29148][CORE][FOLLOWUP] Fix warning message to show a correct executor id ### What changes were proposed in this pull request? This aims to replace `executorIdsToBeRemoved` with `executorIdToBeRemoved`. ### Why are the changes needed? Since a wrong variable is used currently, `ArrayBuffer()` is always displayed. ``` 20/06/16 19:33:31 WARN ExecutorAllocationManager: Not removing executor ArrayBuffer() because the ResourceProfile was UNKNOWN! ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual. Closes #28847 from dongjoon-hyun/SPARK-29148. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/ExecutorAllocationManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 4df58354c99ca..e0ac2b3e0f4b8 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -518,7 +518,7 @@ private[spark] class ExecutorAllocationManager( if (testing) { throw new SparkException("ResourceProfile Id was UNKNOWN, this is not expected") } - logWarning(s"Not removing executor $executorIdsToBeRemoved because the " + + logWarning(s"Not removing executor $executorIdToBeRemoved because the " + "ResourceProfile was UNKNOWN!") } else { // get the running total as we remove or initialize it to the count - pendingRemoval From 350aa859fe1273fa96effd1c96afdfaca2c1b0d5 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 17 Jun 2020 06:28:47 +0000 Subject: [PATCH 032/384] [SPARK-32006][SQL] Create date/timestamp formatters once before collect in `hiveResultString()` ### What changes were proposed in this pull request? 1. Add method `getTimeFormatters` to `HiveResult` which creates timestamp and date formatters. 2. Move creation of `dateFormatter` and `timestampFormatter` from the constructor of the `HiveResult` object to `HiveResult. hiveResultString()` via `getTimeFormatters`. This allows to resolve time zone ID from Spark's session time zone `spark.sql.session.timeZone` and create date/timestamp formatters only once before collecting `java.sql.Timestamp`/`java.sql.Date` values. 3. Create date/timestamp formatters once in SparkExecuteStatementOperation. ### Why are the changes needed? To fix perf regression comparing to Spark 2.4 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - By existing test suite `HiveResultSuite` and etc. - Re-generate benchmarks results of `DateTimeBenchmark` 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 | Closes #28842 from MaxGekk/opt-toHiveString-oss-master. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../DateTimeBenchmark-jdk11-results.txt | 262 +++++++++--------- .../benchmarks/DateTimeBenchmark-results.txt | 260 ++++++++--------- .../spark/sql/execution/HiveResult.scala | 59 ++-- .../spark/sql/execution/HiveResultSuite.scala | 19 +- .../SparkExecuteStatementOperation.scala | 22 +- .../ThriftServerQueryTestSuite.scala | 17 +- 6 files changed, 325 insertions(+), 314 deletions(-) diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt index 70d888227141d..d84dccbf6c266 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) 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 +date + interval(m) 1550 1609 83 6.5 155.0 1.0X +date + interval(m, d) 1572 1575 5 6.4 157.2 1.0X +date + interval(m, d, ms) 6512 6512 0 1.5 651.2 0.2X +date - interval(m) 1469 1489 28 6.8 146.9 1.1X +date - interval(m, d) 1558 1572 19 6.4 155.8 1.0X +date - interval(m, d, ms) 6602 6605 4 1.5 660.2 0.2X +timestamp + interval(m) 2945 2961 23 3.4 294.5 0.5X +timestamp + interval(m, d) 3075 3083 12 3.3 307.5 0.5X +timestamp + interval(m, d, ms) 3421 3430 13 2.9 342.1 0.5X +timestamp - interval(m) 3050 3061 17 3.3 305.0 0.5X +timestamp - interval(m, d) 3195 3201 8 3.1 319.5 0.5X +timestamp - interval(m, d, ms) 3442 3450 11 2.9 344.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 333 334 0 30.0 33.3 1.0X -cast to timestamp wholestage on 349 368 12 28.6 34.9 1.0X +cast to timestamp wholestage off 320 326 8 31.2 32.0 1.0X +cast to timestamp wholestage on 289 297 5 34.6 28.9 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 year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +year of timestamp wholestage off 1266 1266 1 7.9 126.6 1.0X +year of timestamp wholestage on 1233 1253 15 8.1 123.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 quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +quarter of timestamp wholestage off 1594 1600 8 6.3 159.4 1.0X +quarter of timestamp wholestage on 1529 1532 3 6.5 152.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 month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +month of timestamp wholestage off 1239 1257 25 8.1 123.9 1.0X +month of timestamp wholestage on 1235 1243 5 8.1 123.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 weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +weekofyear of timestamp wholestage off 2209 2216 9 4.5 220.9 1.0X +weekofyear of timestamp wholestage on 1831 1838 9 5.5 183.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 day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +day of timestamp wholestage off 1238 1238 0 8.1 123.8 1.0X +day of timestamp wholestage on 1223 1235 12 8.2 122.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 dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +dayofyear of timestamp wholestage off 1302 1304 3 7.7 130.2 1.0X +dayofyear of timestamp wholestage on 1269 1276 6 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 dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +dayofmonth of timestamp wholestage off 1251 1253 3 8.0 125.1 1.0X +dayofmonth of timestamp wholestage on 1225 1232 9 8.2 122.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 dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +dayofweek of timestamp wholestage off 1424 1424 1 7.0 142.4 1.0X +dayofweek of timestamp wholestage on 1385 1389 4 7.2 138.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 weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +weekday of timestamp wholestage off 1366 1366 0 7.3 136.6 1.0X +weekday of timestamp wholestage on 1320 1325 5 7.6 132.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 hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +hour of timestamp wholestage off 985 986 1 10.2 98.5 1.0X +hour of timestamp wholestage on 974 981 10 10.3 97.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 minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +minute of timestamp wholestage off 1044 1047 5 9.6 104.4 1.0X +minute of timestamp wholestage on 984 994 17 10.2 98.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 997 998 2 10.0 99.7 1.0X -second of timestamp wholestage on 925 935 8 10.8 92.5 1.1X +second of timestamp wholestage off 999 1003 6 10.0 99.9 1.0X +second of timestamp wholestage on 961 974 8 10.4 96.1 1.0X ================================================================================================ @@ -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 297 297 0 33.7 29.7 1.0X -current_date wholestage on 280 282 2 35.7 28.0 1.1X +current_date wholestage off 297 302 7 33.6 29.7 1.0X +current_date wholestage on 270 283 22 37.1 27.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 307 337 43 32.6 30.7 1.0X -current_timestamp wholestage on 260 284 29 38.4 26.0 1.2X +current_timestamp wholestage off 302 310 11 33.1 30.2 1.0X +current_timestamp wholestage on 264 351 98 37.9 26.4 1.1X ================================================================================================ @@ -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 1066 1073 10 9.4 106.6 1.0X -cast to date wholestage on 997 1003 6 10.0 99.7 1.1X +cast to date wholestage off 1083 1083 1 9.2 108.3 1.0X +cast to date wholestage on 1040 1044 5 9.6 104.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 last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +last_day wholestage off 1258 1258 0 7.9 125.8 1.0X +last_day wholestage on 1244 1254 8 8.0 124.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 next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +next_day wholestage off 1133 1135 3 8.8 113.3 1.0X +next_day wholestage on 1093 1100 7 9.1 109.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 date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_add wholestage off 1065 1074 14 9.4 106.5 1.0X +date_add wholestage on 1044 1053 6 9.6 104.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_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_sub wholestage off 1069 1076 9 9.4 106.9 1.0X +date_sub wholestage on 1047 1052 8 9.6 104.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 add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +add_months wholestage off 1417 1430 18 7.1 141.7 1.0X +add_months wholestage on 1439 1445 5 6.9 143.9 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 5293 5296 5 1.9 529.3 1.0X -format date wholestage on 5143 5157 19 1.9 514.3 1.0X +format date wholestage off 5228 5232 6 1.9 522.8 1.0X +format date wholestage on 5172 5193 17 1.9 517.2 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 7136 7136 1 1.4 713.6 1.0X -from_unixtime wholestage on 7049 7068 29 1.4 704.9 1.0X +from_unixtime wholestage off 6941 6952 16 1.4 694.1 1.0X +from_unixtime wholestage on 6898 6926 32 1.4 689.8 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 1325 1329 6 7.5 132.5 1.0X -from_utc_timestamp wholestage on 1269 1273 4 7.9 126.9 1.0X +from_utc_timestamp wholestage off 1339 1342 5 7.5 133.9 1.0X +from_utc_timestamp wholestage on 1285 1292 5 7.8 128.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_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +to_utc_timestamp wholestage off 1697 1717 29 5.9 169.7 1.0X +to_utc_timestamp wholestage on 1656 1665 13 6.0 165.6 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 343 346 4 29.1 34.3 1.0X -cast interval wholestage on 281 282 1 35.6 28.1 1.2X +cast interval wholestage off 333 344 16 30.1 33.3 1.0X +cast interval wholestage on 288 290 2 34.7 28.8 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 1831 1840 13 5.5 183.1 1.0X -datediff wholestage on 1759 1769 15 5.7 175.9 1.0X +datediff wholestage off 1857 1860 4 5.4 185.7 1.0X +datediff wholestage on 1795 1808 10 5.6 179.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 months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +months_between wholestage off 5826 5834 11 1.7 582.6 1.0X +months_between wholestage on 5737 5763 18 1.7 573.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 window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 2183 2189 9 0.5 2182.6 1.0X -window wholestage on 46835 46944 88 0.0 46834.8 0.0X +window wholestage off 2220 2246 36 0.5 2220.4 1.0X +window wholestage on 46696 46794 89 0.0 46696.1 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 2668 2672 5 3.7 266.8 1.0X -date_trunc YEAR wholestage on 2719 2731 9 3.7 271.9 1.0X +date_trunc YEAR wholestage off 2658 2659 1 3.8 265.8 1.0X +date_trunc YEAR wholestage on 2691 2700 8 3.7 269.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 YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc YYYY wholestage off 2671 2679 11 3.7 267.1 1.0X +date_trunc YYYY wholestage on 2700 2706 6 3.7 270.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 2670 2673 4 3.7 267.0 1.0X -date_trunc YY wholestage on 2711 2720 7 3.7 271.1 1.0X +date_trunc YY wholestage off 2674 2689 20 3.7 267.4 1.0X +date_trunc YY wholestage on 2697 2716 17 3.7 269.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 MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc MON wholestage off 2695 2700 7 3.7 269.5 1.0X +date_trunc MON wholestage on 2711 2722 11 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 MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc MONTH wholestage off 2682 2685 4 3.7 268.2 1.0X +date_trunc MONTH wholestage on 2709 2727 15 3.7 270.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 MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc MM wholestage off 2683 2693 14 3.7 268.3 1.0X +date_trunc MM wholestage on 2706 2722 16 3.7 270.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 date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc DAY wholestage off 2292 2299 10 4.4 229.2 1.0X +date_trunc DAY wholestage on 2290 2311 14 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 DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc DD wholestage off 2302 2309 9 4.3 230.2 1.0X +date_trunc DD wholestage on 2282 2292 6 4.4 228.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 HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc HOUR wholestage off 2288 2288 0 4.4 228.8 1.0X +date_trunc HOUR wholestage on 2277 2290 14 4.4 227.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 MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc MINUTE wholestage off 400 419 26 25.0 40.0 1.0X +date_trunc MINUTE wholestage on 401 405 4 24.9 40.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 376 1 26.7 37.5 1.0X -date_trunc SECOND wholestage on 370 376 8 27.0 37.0 1.0X +date_trunc SECOND wholestage off 408 414 9 24.5 40.8 1.0X +date_trunc SECOND wholestage on 408 413 8 24.5 40.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 date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc WEEK wholestage off 2623 2631 12 3.8 262.3 1.0X +date_trunc WEEK wholestage on 2613 2621 8 3.8 261.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 date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc QUARTER wholestage off 3518 3520 3 2.8 351.8 1.0X +date_trunc QUARTER wholestage on 3501 3510 11 2.9 350.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 trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +trunc year wholestage off 315 333 26 31.8 31.5 1.0X +trunc year wholestage on 352 360 7 28.4 35.2 0.9X 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 331 331 0 30.2 33.1 1.0X -trunc yyyy wholestage on 336 339 4 29.8 33.6 1.0X +trunc yyyy wholestage off 321 321 1 31.2 32.1 1.0X +trunc yyyy wholestage on 354 358 5 28.3 35.4 0.9X 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 330 342 17 30.3 33.0 1.0X -trunc yy wholestage on 333 337 3 30.0 33.3 1.0X +trunc yy wholestage off 312 313 1 32.0 31.2 1.0X +trunc yy wholestage on 355 360 5 28.2 35.5 0.9X 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 334 335 1 30.0 33.4 1.0X -trunc mon wholestage on 333 347 9 30.0 33.3 1.0X +trunc mon wholestage off 324 327 4 30.9 32.4 1.0X +trunc mon wholestage on 355 357 2 28.2 35.5 0.9X 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 332 333 1 30.1 33.2 1.0X -trunc month wholestage on 333 340 7 30.0 33.3 1.0X +trunc month wholestage off 313 318 8 32.0 31.3 1.0X +trunc month wholestage on 354 358 5 28.3 35.4 0.9X 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 328 336 11 30.5 32.8 1.0X -trunc mm wholestage on 333 343 11 30.0 33.3 1.0X +trunc mm wholestage off 314 325 15 31.8 31.4 1.0X +trunc mm wholestage on 353 366 17 28.4 35.3 0.9X ================================================================================================ @@ -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 170 171 1 5.9 170.1 1.0X -to timestamp str wholestage on 172 174 2 5.8 171.6 1.0X +to timestamp str wholestage off 168 169 0 5.9 168.4 1.0X +to timestamp str wholestage on 168 173 7 6.0 167.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 1437 1439 3 0.7 1437.0 1.0X -to_timestamp wholestage on 1288 1292 5 0.8 1288.1 1.1X +to_timestamp wholestage off 1390 1390 0 0.7 1389.8 1.0X +to_timestamp wholestage on 1204 1215 11 0.8 1204.2 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 to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +to_unix_timestamp wholestage off 1277 1281 4 0.8 1277.5 1.0X +to_unix_timestamp wholestage on 1203 1213 11 0.8 1202.6 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 date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +to date str wholestage off 218 219 1 4.6 218.2 1.0X +to date str wholestage on 211 214 5 4.7 210.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 to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +to_date wholestage off 3016 3041 35 0.3 3016.1 1.0X +to_date wholestage on 3015 3023 9 0.3 3014.6 1.0X ================================================================================================ @@ -444,18 +444,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 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 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 -java.sql.Date to Hive string 17536 18367 1059 0.3 3507.2 0.0X -java.time.LocalDate to Hive string 12089 12897 725 0.4 2417.8 0.0X -java.sql.Timestamp to Hive string 48014 48625 752 0.1 9602.9 0.0X -java.time.Instant to Hive string 37346 37445 93 0.1 7469.1 0.0X +From java.sql.Date 430 442 18 11.6 86.0 1.0X +From java.time.LocalDate 351 354 3 14.3 70.2 1.2X +Collect java.sql.Date 2095 2853 733 2.4 418.9 0.2X +Collect java.time.LocalDate 1691 1910 209 3.0 338.3 0.3X +From java.sql.Timestamp 276 280 4 18.1 55.2 1.6X +From java.time.Instant 324 328 4 15.4 64.8 1.3X +Collect longs 1348 1450 126 3.7 269.5 0.3X +Collect java.sql.Timestamp 1441 1478 62 3.5 288.3 0.3X +Collect java.time.Instant 1471 1579 100 3.4 294.3 0.3X +java.sql.Date to Hive string 12049 12909 862 0.4 2409.8 0.0X +java.time.LocalDate to Hive string 12045 12130 74 0.4 2408.9 0.0X +java.sql.Timestamp to Hive string 12854 13376 510 0.4 2570.9 0.0X +java.time.Instant to Hive string 15057 15184 115 0.3 3011.4 0.0X diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 0795f11a57f28..ebfcb45f30ce0 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) 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 +date + interval(m) 1636 1653 24 6.1 163.6 1.0X +date + interval(m, d) 1802 1818 23 5.5 180.2 0.9X +date + interval(m, d, ms) 6330 6348 26 1.6 633.0 0.3X +date - interval(m) 1462 1484 32 6.8 146.2 1.1X +date - interval(m, d) 1732 1732 1 5.8 173.2 0.9X +date - interval(m, d, ms) 6494 6505 16 1.5 649.4 0.3X +timestamp + interval(m) 2446 2446 0 4.1 244.6 0.7X +timestamp + interval(m, d) 2670 2703 46 3.7 267.0 0.6X +timestamp + interval(m, d, ms) 2992 3012 29 3.3 299.2 0.5X +timestamp - interval(m) 2447 2449 3 4.1 244.7 0.7X +timestamp - interval(m, d) 2739 2739 0 3.7 273.9 0.6X +timestamp - interval(m, d, ms) 2977 2983 8 3.4 297.7 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 313 320 10 31.9 31.3 1.0X -cast to timestamp wholestage on 325 341 18 30.8 32.5 1.0X +cast to timestamp wholestage off 312 321 13 32.1 31.2 1.0X +cast to timestamp wholestage on 290 311 14 34.5 29.0 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 year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +year of timestamp wholestage off 1226 1228 3 8.2 122.6 1.0X +year of timestamp wholestage on 1214 1222 10 8.2 121.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 quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +quarter of timestamp wholestage off 1437 1447 14 7.0 143.7 1.0X +quarter of timestamp wholestage on 1354 1359 4 7.4 135.4 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 month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +month of timestamp wholestage off 1219 1219 1 8.2 121.9 1.0X +month of timestamp wholestage on 1205 1211 7 8.3 120.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 weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +weekofyear of timestamp wholestage off 1849 1854 7 5.4 184.9 1.0X +weekofyear of timestamp wholestage on 1829 1835 5 5.5 182.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 day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +day of timestamp wholestage off 1224 1230 8 8.2 122.4 1.0X +day of timestamp wholestage on 1204 1215 10 8.3 120.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 dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +dayofyear of timestamp wholestage off 1272 1275 5 7.9 127.2 1.0X +dayofyear of timestamp wholestage on 1246 1256 7 8.0 124.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 dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +dayofmonth of timestamp wholestage off 1226 1233 11 8.2 122.6 1.0X +dayofmonth of timestamp wholestage on 1205 1211 5 8.3 120.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 dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +dayofweek of timestamp wholestage off 1420 1427 9 7.0 142.0 1.0X +dayofweek of timestamp wholestage on 1375 1385 11 7.3 137.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 weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 1344 1353 13 7.4 134.4 1.0X +weekday of timestamp wholestage off 1345 1347 3 7.4 134.5 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 992 1000 10 10.1 99.2 1.0X -hour of timestamp wholestage on 960 962 3 10.4 96.0 1.0X +hour of timestamp wholestage off 983 984 1 10.2 98.3 1.0X +hour of timestamp wholestage on 942 953 8 10.6 94.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 minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +minute of timestamp wholestage off 1008 1010 3 9.9 100.8 1.0X +minute of timestamp wholestage on 942 945 3 10.6 94.2 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 second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +second of timestamp wholestage off 975 976 1 10.3 97.5 1.0X +second of timestamp wholestage on 938 944 4 10.7 93.8 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 281 282 2 35.6 28.1 1.0X -current_date wholestage on 294 300 5 34.0 29.4 1.0X +current_date wholestage off 295 296 2 33.9 29.5 1.0X +current_date wholestage on 267 274 6 37.5 26.7 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 current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +current_timestamp wholestage off 298 303 7 33.5 29.8 1.0X +current_timestamp wholestage on 261 275 12 38.2 26.1 1.1X ================================================================================================ @@ -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 1060 1061 1 9.4 106.0 1.0X -cast to date wholestage on 1021 1026 10 9.8 102.1 1.0X +cast to date wholestage off 1071 1073 3 9.3 107.1 1.0X +cast to date wholestage on 998 1014 31 10.0 99.8 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 last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +last_day wholestage off 1260 1261 1 7.9 126.0 1.0X +last_day wholestage on 1245 1261 17 8.0 124.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 next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +next_day wholestage off 1118 1120 2 8.9 111.8 1.0X +next_day wholestage on 1043 1047 3 9.6 104.3 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_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_add wholestage off 1046 1048 3 9.6 104.6 1.0X +date_add wholestage on 1040 1048 11 9.6 104.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_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_sub wholestage off 1081 1081 0 9.3 108.1 1.0X +date_sub wholestage on 1030 1035 6 9.7 103.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 add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +add_months wholestage off 1393 1400 10 7.2 139.3 1.0X +add_months wholestage on 1391 1396 5 7.2 139.1 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 5482 5803 454 1.8 548.2 1.0X -format date wholestage on 5502 5518 9 1.8 550.2 1.0X +format date wholestage off 5424 5426 2 1.8 542.4 1.0X +format date wholestage on 5408 5448 37 1.8 540.8 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 8538 8553 22 1.2 853.8 1.0X -from_unixtime wholestage on 8545 8552 6 1.2 854.5 1.0X +from_unixtime wholestage off 8839 8841 3 1.1 883.9 1.0X +from_unixtime wholestage on 8788 8826 24 1.1 878.8 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 1094 1099 8 9.1 109.4 1.0X -from_utc_timestamp wholestage on 1109 1114 5 9.0 110.9 1.0X +from_utc_timestamp wholestage off 1105 1111 8 9.0 110.5 1.0X +from_utc_timestamp wholestage on 1073 1081 8 9.3 107.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 to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +to_utc_timestamp wholestage off 1462 1465 4 6.8 146.2 1.0X +to_utc_timestamp wholestage on 1394 1408 13 7.2 139.4 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 332 332 0 30.1 33.2 1.0X -cast interval wholestage on 315 324 10 31.7 31.5 1.1X +cast interval wholestage off 325 328 4 30.8 32.5 1.0X +cast interval wholestage on 286 290 3 35.0 28.6 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 1796 1802 8 5.6 179.6 1.0X -datediff wholestage on 1758 1764 10 5.7 175.8 1.0X +datediff wholestage off 1822 1824 3 5.5 182.2 1.0X +datediff wholestage on 1757 1761 5 5.7 175.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 months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +months_between wholestage off 4886 4893 10 2.0 488.6 1.0X +months_between wholestage on 4785 4799 12 2.1 478.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 window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 1812 1908 136 0.6 1811.7 1.0X -window wholestage on 46279 46376 74 0.0 46278.8 0.0X +window wholestage off 2024 2052 40 0.5 2023.7 1.0X +window wholestage on 46599 46660 45 0.0 46599.0 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 2367 2368 1 4.2 236.7 1.0X -date_trunc YEAR wholestage on 2321 2334 22 4.3 232.1 1.0X +date_trunc YEAR wholestage off 2361 2366 7 4.2 236.1 1.0X +date_trunc YEAR wholestage on 2325 2328 3 4.3 232.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 YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc YYYY wholestage off 2366 2374 12 4.2 236.6 1.0X +date_trunc YYYY wholestage on 2316 2328 13 4.3 231.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 2334 2335 1 4.3 233.4 1.0X -date_trunc YY wholestage on 2315 2324 6 4.3 231.5 1.0X +date_trunc YY wholestage off 2359 2359 0 4.2 235.9 1.0X +date_trunc YY wholestage on 2315 2325 7 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 2327 2330 4 4.3 232.7 1.0X -date_trunc MON wholestage on 2279 2289 12 4.4 227.9 1.0X +date_trunc MON wholestage off 2360 2369 12 4.2 236.0 1.0X +date_trunc MON wholestage on 2306 2314 9 4.3 230.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 MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc MONTH wholestage off 2359 2360 2 4.2 235.9 1.0X +date_trunc MONTH wholestage on 2304 2308 4 4.3 230.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 MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc MM wholestage off 2356 2358 2 4.2 235.6 1.0X +date_trunc MM wholestage on 2302 2309 6 4.3 230.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 date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc DAY wholestage off 1962 1964 3 5.1 196.2 1.0X +date_trunc DAY wholestage on 1916 1921 6 5.2 191.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 DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc DD wholestage off 1956 1957 2 5.1 195.6 1.0X +date_trunc DD wholestage on 1916 1922 6 5.2 191.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 HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc HOUR wholestage off 1968 1970 3 5.1 196.8 1.0X +date_trunc HOUR wholestage on 1949 1961 9 5.1 194.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 MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc MINUTE wholestage off 368 373 7 27.2 36.8 1.0X +date_trunc MINUTE wholestage on 338 343 6 29.6 33.8 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 360 361 1 27.8 36.0 1.0X -date_trunc SECOND wholestage on 335 348 15 29.8 33.5 1.1X +date_trunc SECOND wholestage off 379 379 1 26.4 37.9 1.0X +date_trunc SECOND wholestage on 327 340 13 30.6 32.7 1.2X 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 2232 2236 6 4.5 223.2 1.0X -date_trunc WEEK wholestage on 2225 2232 6 4.5 222.5 1.0X +date_trunc WEEK wholestage off 2227 2242 21 4.5 222.7 1.0X +date_trunc WEEK wholestage on 2231 2241 9 4.5 223.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 QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +date_trunc QUARTER wholestage off 3158 3160 3 3.2 315.8 1.0X +date_trunc QUARTER wholestage on 3150 3163 12 3.2 315.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 trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +trunc year wholestage off 321 323 3 31.2 32.1 1.0X +trunc year wholestage on 302 330 18 33.1 30.2 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 323 327 5 30.9 32.3 1.0X -trunc yyyy wholestage on 299 302 3 33.4 29.9 1.1X +trunc yyyy wholestage off 320 324 6 31.2 32.0 1.0X +trunc yyyy wholestage on 294 329 20 34.0 29.4 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 315 315 1 31.8 31.5 1.0X -trunc yy wholestage on 299 304 4 33.4 29.9 1.1X +trunc yy wholestage off 322 322 0 31.1 32.2 1.0X +trunc yy wholestage on 293 320 37 34.1 29.3 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 320 321 1 31.2 32.0 1.0X -trunc mon wholestage on 299 307 10 33.4 29.9 1.1X +trunc mon wholestage off 320 322 2 31.2 32.0 1.0X +trunc mon wholestage on 291 312 26 34.4 29.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 trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +trunc month wholestage off 318 331 18 31.4 31.8 1.0X +trunc month wholestage on 297 329 28 33.7 29.7 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 313 313 1 32.0 31.3 1.0X -trunc mm wholestage on 298 302 4 33.5 29.8 1.0X +trunc mm wholestage off 318 319 1 31.4 31.8 1.0X +trunc mm wholestage on 312 335 15 32.1 31.2 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 217 217 0 4.6 217.3 1.0X -to timestamp str wholestage on 209 212 2 4.8 209.5 1.0X +to timestamp str wholestage off 217 221 5 4.6 217.5 1.0X +to timestamp str wholestage on 210 214 5 4.8 210.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 to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +to_timestamp wholestage off 1714 1718 5 0.6 1714.4 1.0X +to_timestamp wholestage on 1418 1433 14 0.7 1418.5 1.2X 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 1582 1589 9 0.6 1582.1 1.0X -to_unix_timestamp wholestage on 1634 1637 3 0.6 1633.8 1.0X +to_unix_timestamp wholestage off 1436 1441 6 0.7 1436.2 1.0X +to_unix_timestamp wholestage on 1421 1426 7 0.7 1420.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 to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +to date str wholestage off 267 267 0 3.8 266.6 1.0X +to date str wholestage on 260 262 2 3.8 260.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 to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +to_date wholestage off 3419 3436 25 0.3 3419.0 1.0X +to_date wholestage on 3344 3352 7 0.3 3343.5 1.0X ================================================================================================ @@ -444,18 +444,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 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 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 -java.sql.Date to Hive string 18745 20895 1364 0.3 3749.0 0.0X -java.time.LocalDate to Hive string 15296 15450 143 0.3 3059.2 0.0X -java.sql.Timestamp to Hive string 46421 47210 946 0.1 9284.2 0.0X -java.time.Instant to Hive string 34747 35187 382 0.1 6949.4 0.0X +From java.sql.Date 436 445 8 11.5 87.2 1.0X +From java.time.LocalDate 348 357 11 14.4 69.7 1.3X +Collect java.sql.Date 1723 1917 168 2.9 344.5 0.3X +Collect java.time.LocalDate 1591 1602 18 3.1 318.3 0.3X +From java.sql.Timestamp 248 252 4 20.2 49.6 1.8X +From java.time.Instant 232 238 5 21.5 46.5 1.9X +Collect longs 1398 1455 99 3.6 279.5 0.3X +Collect java.sql.Timestamp 1469 1483 13 3.4 293.9 0.3X +Collect java.time.Instant 1561 1597 40 3.2 312.2 0.3X +java.sql.Date to Hive string 13820 14798 857 0.4 2763.9 0.0X +java.time.LocalDate to Hive string 14374 14779 357 0.3 2874.8 0.0X +java.sql.Timestamp to Hive string 14872 15461 653 0.3 2974.5 0.0X +java.time.Instant to Hive string 17062 17789 759 0.3 3412.4 0.0X diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index 9f99bf5011569..4d388e40fb8bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -33,6 +33,23 @@ import org.apache.spark.unsafe.types.CalendarInterval * Runs a query returning the result in Hive compatible form. */ object HiveResult { + case class TimeFormatters(date: DateFormatter, timestamp: TimestampFormatter) + + def getTimeFormatters: TimeFormatters = { + // The date formatter does not depend on Spark's session time zone controlled by + // the SQL config `spark.sql.session.timeZone`. The `zoneId` parameter is used only in + // parsing of special date values like `now`, `yesterday` and etc. but not in date formatting. + // While formatting of: + // - `java.time.LocalDate`, zone id is not used by `DateTimeFormatter` at all. + // - `java.sql.Date`, the date formatter delegates formatting to the legacy formatter + // which uses the default system time zone `TimeZone.getDefault`. This works correctly + // due to `DateTimeUtils.toJavaDate` which is based on the system time zone too. + val dateFormatter = DateFormatter(ZoneOffset.UTC) + val timestampFormatter = TimestampFormatter.getFractionFormatter( + DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + TimeFormatters(dateFormatter, timestampFormatter) + } + /** * Returns the result as a hive compatible sequence of strings. This is used in tests and * `SparkSQLDriver` for CLI applications. @@ -55,11 +72,12 @@ object HiveResult { case command @ ExecutedCommandExec(_: ShowViewsCommand) => command.executeCollect().map(_.getString(1)) case other => + val timeFormatters = getTimeFormatters val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq // We need the types so we can output struct field names val types = executedPlan.output.map(_.dataType) // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(e => toHiveString(e))) + result.map(_.zip(types).map(e => toHiveString(e, false, timeFormatters))) .map(_.mkString("\t")) } @@ -72,47 +90,32 @@ object HiveResult { } } - // We can create the date formatter only once because it does not depend on Spark's - // session time zone controlled by the SQL config `spark.sql.session.timeZone`. - // The `zoneId` parameter is used only in parsing of special date values like `now`, - // `yesterday` and etc. but not in date formatting. While formatting of: - // - `java.time.LocalDate`, zone id is not used by `DateTimeFormatter` at all. - // - `java.sql.Date`, the date formatter delegates formatting to the legacy formatter - // which uses the default system time zone `TimeZone.getDefault`. This works correctly - // due to `DateTimeUtils.toJavaDate` which is based on the system time zone too. - private val dateFormatter = DateFormatter( - format = DateFormatter.defaultPattern, - // We can set any time zone id. UTC was taken for simplicity. - zoneId = ZoneOffset.UTC, - locale = DateFormatter.defaultLocale, - // Use `FastDateFormat` as the legacy formatter because it is thread-safe. - legacyFormat = LegacyDateFormats.FAST_DATE_FORMAT, - isParsing = false) - private def timestampFormatter = TimestampFormatter.getFractionFormatter( - DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) - /** Formats a datum (based on the given data type) and returns the string representation. */ - def toHiveString(a: (Any, DataType), nested: Boolean = false): String = a match { + def toHiveString( + a: (Any, DataType), + nested: Boolean, + formatters: TimeFormatters): String = a match { case (null, _) => if (nested) "null" else "NULL" case (b, BooleanType) => b.toString - case (d: Date, DateType) => dateFormatter.format(d) - case (ld: LocalDate, DateType) => dateFormatter.format(ld) - case (t: Timestamp, TimestampType) => timestampFormatter.format(t) - case (i: Instant, TimestampType) => timestampFormatter.format(i) + case (d: Date, DateType) => formatters.date.format(d) + case (ld: LocalDate, DateType) => formatters.date.format(ld) + case (t: Timestamp, TimestampType) => formatters.timestamp.format(t) + case (i: Instant, TimestampType) => formatters.timestamp.format(i) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString case (n, _: NumericType) => n.toString case (s: String, StringType) => if (nested) "\"" + s + "\"" else s case (interval: CalendarInterval, CalendarIntervalType) => interval.toString case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(e => toHiveString(e, true)).mkString("[", ",", "]") + seq.map(v => (v, typ)).map(e => toHiveString(e, true, formatters)).mkString("[", ",", "]") case (m: Map[_, _], MapType(kType, vType, _)) => m.map { case (key, value) => - toHiveString((key, kType), true) + ":" + toHiveString((value, vType), true) + toHiveString((key, kType), true, formatters) + ":" + + toHiveString((value, vType), true, formatters) }.toSeq.sorted.mkString("{", ",", "}") case (struct: Row, StructType(fields)) => struct.toSeq.zip(fields).map { case (v, t) => - s""""${t.name}":${toHiveString((v, t.dataType), true)}""" + s""""${t.name}":${toHiveString((v, t.dataType), true, formatters)}""" }.mkString("{", ",", "}") case (other, _: UserDefinedType[_]) => other.toString } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index a0b212d2cf6fd..a49beda2186b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.execution.HiveResult._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} @@ -31,10 +32,10 @@ class HiveResultSuite extends SharedSparkSession { val dates = Seq("2018-12-28", "1582-10-03", "1582-10-04", "1582-10-15") val df = dates.toDF("a").selectExpr("cast(a as date) as b") val executedPlan1 = df.queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan1) + val result = hiveResultString(executedPlan1) assert(result == dates) val executedPlan2 = df.selectExpr("array(b)").queryExecution.executedPlan - val result2 = HiveResult.hiveResultString(executedPlan2) + val result2 = hiveResultString(executedPlan2) assert(result2 == dates.map(x => s"[$x]")) } } @@ -48,17 +49,17 @@ class HiveResultSuite extends SharedSparkSession { "1582-10-15 01:02:03") val df = timestamps.toDF("a").selectExpr("cast(a as timestamp) as b") val executedPlan1 = df.queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan1) + val result = hiveResultString(executedPlan1) assert(result == timestamps) val executedPlan2 = df.selectExpr("array(b)").queryExecution.executedPlan - val result2 = HiveResult.hiveResultString(executedPlan2) + val result2 = hiveResultString(executedPlan2) assert(result2 == timestamps.map(x => s"[$x]")) } test("toHiveString correctly handles UDTs") { val point = new ExamplePoint(50.0, 50.0) val tpe = new ExamplePointUDT() - assert(HiveResult.toHiveString((point, tpe)) === "(50.0, 50.0)") + assert(toHiveString((point, tpe), false, getTimeFormatters) === "(50.0, 50.0)") } test("decimal formatting in hive result") { @@ -66,13 +67,13 @@ class HiveResultSuite extends SharedSparkSession { Seq(2, 6, 18).foreach { scala => val executedPlan = df.selectExpr(s"CAST(value AS decimal(38, $scala))").queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan) + val result = hiveResultString(executedPlan) assert(result.head.split("\\.").last.length === scala) } val executedPlan = Seq(java.math.BigDecimal.ZERO).toDS() .selectExpr(s"CAST(value AS decimal(38, 8))").queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan) + val result = hiveResultString(executedPlan) assert(result.head === "0.00000000") } @@ -84,7 +85,7 @@ class HiveResultSuite extends SharedSparkSession { spark.sql(s"CREATE TABLE $ns.$tbl (id bigint) USING $source") val df = spark.sql(s"SHOW TABLES FROM $ns") val executedPlan = df.queryExecution.executedPlan - assert(HiveResult.hiveResultString(executedPlan).head == tbl) + assert(hiveResultString(executedPlan).head == tbl) } } } @@ -101,7 +102,7 @@ class HiveResultSuite extends SharedSparkSession { val expected = "id " + "\tbigint " + "\tcol1 " - assert(HiveResult.hiveResultString(executedPlan).head == expected) + assert(hiveResultString(executedPlan).head == expected) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index b193c73563ae0..1404ece76449e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -36,7 +36,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext} -import org.apache.spark.sql.execution.HiveResult +import org.apache.spark.sql.execution.HiveResult.{getTimeFormatters, toHiveString, TimeFormatters} import org.apache.spark.sql.execution.command.SetCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -73,7 +73,11 @@ private[hive] class SparkExecuteStatementOperation( } } - def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int): Unit = { + def addNonNullColumnValue( + from: SparkRow, + to: ArrayBuffer[Any], + ordinal: Int, + timeFormatters: TimeFormatters): Unit = { dataTypes(ordinal) match { case StringType => to += from.getString(ordinal) @@ -100,13 +104,14 @@ private[hive] class SparkExecuteStatementOperation( // - work with spark.sql.datetime.java8API.enabled // These types have always been sent over the wire as string, converted later. case _: DateType | _: TimestampType => - val hiveString = HiveResult.toHiveString((from.get(ordinal), dataTypes(ordinal))) - to += hiveString + to += toHiveString((from.get(ordinal), dataTypes(ordinal)), false, timeFormatters) case CalendarIntervalType => - to += HiveResult.toHiveString((from.getAs[CalendarInterval](ordinal), CalendarIntervalType)) + to += toHiveString( + (from.getAs[CalendarInterval](ordinal), CalendarIntervalType), + false, + timeFormatters) case _: ArrayType | _: StructType | _: MapType | _: UserDefinedType[_] => - val hiveString = HiveResult.toHiveString((from.get(ordinal), dataTypes(ordinal))) - to += hiveString + to += toHiveString((from.get(ordinal), dataTypes(ordinal)), false, timeFormatters) } } @@ -159,6 +164,7 @@ private[hive] class SparkExecuteStatementOperation( if (!iter.hasNext) { resultRowSet } else { + val timeFormatters = getTimeFormatters // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int val maxRows = maxRowsL.toInt var curRow = 0 @@ -170,7 +176,7 @@ private[hive] class SparkExecuteStatementOperation( if (sparkRow.isNullAt(curCol)) { row += null } else { - addNonNullColumnValue(sparkRow, row, curCol) + addNonNullColumnValue(sparkRow, row, curCol, timeFormatters) } curCol += 1 } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 15cc3109da3f7..9b68144b52005 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.SQLQueryTestSuite import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.util.fileToString -import org.apache.spark.sql.execution.HiveResult +import org.apache.spark.sql.execution.HiveResult.{getTimeFormatters, toHiveString, TimeFormatters} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -257,8 +257,9 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ private def getNormalizedResult(statement: Statement, sql: String): (String, Seq[String]) = { val rs = statement.executeQuery(sql) val cols = rs.getMetaData.getColumnCount + val timeFormatters = getTimeFormatters val buildStr = () => (for (i <- 1 to cols) yield { - getHiveResult(rs.getObject(i)) + getHiveResult(rs.getObject(i), timeFormatters) }).mkString("\t") val answer = Iterator.continually(rs.next()).takeWhile(identity).map(_ => buildStr()).toSeq @@ -280,18 +281,18 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ upperCase.startsWith("(") } - private def getHiveResult(obj: Object): String = { + private def getHiveResult(obj: Object, timeFormatters: TimeFormatters): String = { obj match { case null => - HiveResult.toHiveString((null, StringType)) + toHiveString((null, StringType), false, timeFormatters) case d: java.sql.Date => - HiveResult.toHiveString((d, DateType)) + toHiveString((d, DateType), false, timeFormatters) case t: Timestamp => - HiveResult.toHiveString((t, TimestampType)) + toHiveString((t, TimestampType), false, timeFormatters) case d: java.math.BigDecimal => - HiveResult.toHiveString((d, DecimalType.fromDecimal(Decimal(d)))) + toHiveString((d, DecimalType.fromDecimal(Decimal(d))), false, timeFormatters) case bin: Array[Byte] => - HiveResult.toHiveString((bin, BinaryType)) + toHiveString((bin, BinaryType), false, timeFormatters) case other => other.toString } From 4badef38a52849b4af0b211523de6b09f73397f1 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 17 Jun 2020 13:28:47 +0000 Subject: [PATCH 033/384] [SPARK-32000][CORE][TESTS] Fix the flaky test for partially launched task in barrier-mode ### What changes were proposed in this pull request? This PR changes the test to get an active executorId and set it as preferred location instead of setting a fixed preferred location. ### Why are the changes needed? The test is flaky. After checking the [log](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124086/artifact/core/), I find the root cause is: Two test cases from different test suites got submitted at the same time because of concurrent execution. In this particular case, the two test cases (from DistributedSuite and BarrierTaskContextSuite) both launch under local-cluster mode. The two applications are submitted at the SAME time so they have the same applications(app-20200615210132-0000). Thus, when the cluster of BarrierTaskContextSuite is launching executors, it failed to create the directory for the executor 0, because the path (/home/jenkins/workspace/work/app-app-20200615210132-0000/0) has been used by the cluster of DistributedSuite. Therefore, it has to launch executor 1 and 2 instead, that lead to non of the tasks can get preferred locality thus they got scheduled together and lead to the test failure. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The test can not be reproduced locally. We can only know it's been fixed when it's no longer flaky on Jenkins. Closes #28849 from Ngone51/fix-spark-32000. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../org/apache/spark/scheduler/BarrierTaskContextSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index 01c82f894cf98..d18ca36f1fa60 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -276,11 +276,12 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext with test("SPARK-31485: barrier stage should fail if only partial tasks are launched") { initLocalClusterSparkContext(2) + val id = sc.getExecutorIds().head val rdd0 = sc.parallelize(Seq(0, 1, 2, 3), 2) val dep = new OneToOneDependency[Int](rdd0) - // set up a barrier stage with 2 tasks and both tasks prefer executor 0 (only 1 core) for + // set up a barrier stage with 2 tasks and both tasks prefer the same executor (only 1 core) for // scheduling. So, one of tasks won't be scheduled in one round of resource offer. - val rdd = new MyRDD(sc, 2, List(dep), Seq(Seq("executor_h_0"), Seq("executor_h_0"))) + val rdd = new MyRDD(sc, 2, List(dep), Seq(Seq(s"executor_h_$id"), Seq(s"executor_h_$id"))) val errorMsg = intercept[SparkException] { rdd.barrier().mapPartitions { iter => BarrierTaskContext.get().barrier() From 9b792518b2d420d1026cfdc38729e30f45a36c91 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 18 Jun 2020 06:08:40 +0000 Subject: [PATCH 034/384] [SPARK-31960][YARN][BUILD] Only populate Hadoop classpath for no-hadoop build ### What changes were proposed in this pull request? If a Spark distribution has built-in hadoop runtime, Spark will not populate the hadoop classpath from `yarn.application.classpath` and `mapreduce.application.classpath` when a job is submitted to Yarn. Users can override this behavior by setting `spark.yarn.populateHadoopClasspath` to `true`. ### Why are the changes needed? Without this, Spark will populate the hadoop classpath from `yarn.application.classpath` and `mapreduce.application.classpath` even Spark distribution has built-in hadoop. This results jar conflict and many unexpected behaviors in runtime. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually test with two builds, with-hadoop and no-hadoop builds. Closes #28788 from dbtsai/yarn-classpath. Authored-by: DB Tsai Signed-off-by: DB Tsai --- dev/.rat-excludes | 1 + docs/running-on-yarn.md | 17 ++++++++++++- resource-managers/yarn/pom.xml | 16 ++++++++++++ .../spark/deploy/yarn/config.properties | 1 + .../org/apache/spark/deploy/yarn/config.scala | 25 ++++++++++++++++--- 5 files changed, 56 insertions(+), 4 deletions(-) create mode 100644 resource-managers/yarn/src/main/resources/org/apache/spark/deploy/yarn/config.properties diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 4540e5feac7c9..326e561529073 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -123,3 +123,4 @@ SessionManager.java SessionHandler.java GangliaReporter.java application_1578436911597_0052 +config.properties diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b58cd24ab051f..eeeb71b022c52 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -82,6 +82,18 @@ In `cluster` mode, the driver runs on a different machine than the client, so `S Running Spark on YARN requires a binary distribution of Spark which is built with YARN support. Binary distributions can be downloaded from the [downloads page](https://spark.apache.org/downloads.html) of the project website. +There are two variants of Spark binary distributions you can download. One is pre-built with a certain +version of Apache Hadoop; this Spark distribution contains built-in Hadoop runtime, so we call it `with-hadoop` Spark +distribution. The other one is pre-built with user-provided Hadoop; since this Spark distribution +doesn't contain a built-in Hadoop runtime, it's smaller, but users have to provide a Hadoop installation separately. +We call this variant `no-hadoop` Spark distribution. For `with-hadoop` Spark distribution, since +it contains a built-in Hadoop runtime already, by default, when a job is submitted to Hadoop Yarn cluster, to prevent jar conflict, it will not +populate Yarn's classpath into Spark. To override this behavior, you can set spark.yarn.populateHadoopClasspath=true. +For `no-hadoop` Spark distribution, Spark will populate Yarn's classpath by default in order to get Hadoop runtime. For `with-hadoop` Spark distribution, +if your application depends on certain library that is only available in the cluster, you can try to populate the Yarn classpath by setting +the property mentioned above. If you run into jar conflict issue by doing so, you will need to turn it off and include this library +in your application jar. + To build Spark yourself, refer to [Building Spark](building-spark.html). To make Spark runtime jars accessible from YARN side, you can specify `spark.yarn.archive` or `spark.yarn.jars`. For details please refer to [Spark Properties](running-on-yarn.html#spark-properties). If neither `spark.yarn.archive` nor `spark.yarn.jars` is specified, Spark will create a zip file with all jars under `$SPARK_HOME/jars` and upload it to the distributed cache. @@ -396,7 +408,10 @@ To use a custom metrics.properties for the application master and executors, upd spark.yarn.populateHadoopClasspath - true + + For with-hadoop Spark distribution, this is set to false; + for no-hadoop distribution, this is set to true. + Whether to populate Hadoop classpath from yarn.application.classpath and mapreduce.application.classpath Note that if this is set to false, diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index b84180abfa200..d081be94ba7ae 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -30,8 +30,18 @@ yarn 1.19 + false + + + hadoop-provided + + true + + + + org.apache.spark @@ -201,6 +211,12 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + src/main/resources + true + + diff --git a/resource-managers/yarn/src/main/resources/org/apache/spark/deploy/yarn/config.properties b/resource-managers/yarn/src/main/resources/org/apache/spark/deploy/yarn/config.properties new file mode 100644 index 0000000000000..b923dbab954f5 --- /dev/null +++ b/resource-managers/yarn/src/main/resources/org/apache/spark/deploy/yarn/config.properties @@ -0,0 +1 @@ +spark.yarn.isHadoopProvided = ${spark.yarn.isHadoopProvided} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index b4257a48664c6..1b0bf295db499 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -17,12 +17,14 @@ package org.apache.spark.deploy.yarn +import java.util.Properties import java.util.concurrent.TimeUnit +import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit -package object config { +package object config extends Logging { /* Common app configuration. */ @@ -74,10 +76,11 @@ package object config { .doc("Whether to populate Hadoop classpath from `yarn.application.classpath` and " + "`mapreduce.application.classpath` Note that if this is set to `false`, it requires " + "a `with-Hadoop` Spark distribution that bundles Hadoop runtime or user has to provide " + - "a Hadoop installation separately.") + "a Hadoop installation separately. By default, for `with-hadoop` Spark distribution, " + + "this is set to `false`; for `no-hadoop` distribution, this is set to `true`.") .version("2.4.6") .booleanConf - .createWithDefault(true) + .createWithDefault(isHadoopProvided()) private[spark] val GATEWAY_ROOT_PATH = ConfigBuilder("spark.yarn.config.gatewayPath") .doc("Root of configuration paths that is present on gateway nodes, and will be replaced " + @@ -394,4 +397,20 @@ package object config { private[yarn] val YARN_DRIVER_RESOURCE_TYPES_PREFIX = "spark.yarn.driver.resource." private[yarn] val YARN_AM_RESOURCE_TYPES_PREFIX = "spark.yarn.am.resource." + def isHadoopProvided(): Boolean = IS_HADOOP_PROVIDED + + private lazy val IS_HADOOP_PROVIDED: Boolean = { + val configPath = "org/apache/spark/deploy/yarn/config.properties" + val propertyKey = "spark.yarn.isHadoopProvided" + try { + val prop = new Properties() + prop.load(ClassLoader.getSystemClassLoader.getResourceAsStream(configPath)) + prop.getProperty(propertyKey).toBoolean + } catch { + case e: Exception => + log.warn(s"Can not load the default value of `$propertyKey` from " + + s"`$configPath` with error, ${e.toString}. Using `false` as a default value.") + false + } + } } From e4f503614625f81aa86455ad5f6bec3b61d525f1 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 18 Jun 2020 22:45:55 +0900 Subject: [PATCH 035/384] [SPARK-32020][SQL] Better error message when SPARK_HOME or spark.test.home is not set ### What changes were proposed in this pull request? Better error message when SPARK_HOME or spark,test.home is not set. ### Why are the changes needed? Currently the error message is not easily consumable as it prints (see below) the real error after printing the current environment which is rather long. **Old output** ` time.name" -> "Java(TM) SE Runtime Environment", "sun.boot.library.path" -> "/Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/lib", "java.vm.version" -> "25.221-b11", . . . . . . . . . ) did not contain key "SPARK_HOME" spark.test.home or SPARK_HOME is not set. at org.scalatest.Assertions.newAssertionFailedExceptio ` **New output** An exception or error caused a run to abort: spark.test.home or SPARK_HOME is not set. org.scalatest.exceptions.TestFailedException: spark.test.home or SPARK_HOME is not set ### Does this PR introduce any user-facing change? ` No. ### How was this patch tested? Ran the tests in intellej manually to see the new error. Closes #28825 from dilipbiswal/minor-spark-31950-followup. Authored-by: Dilip Biswal Signed-off-by: HyukjinKwon --- .../catalyst/parser/TableIdentifierParserSuite.scala | 6 ------ .../apache/spark/sql/catalyst/plans/SQLHelper.scala | 9 +++++++++ .../apache/spark/sql/ExpressionsSchemaSuite.scala | 6 ------ .../apache/spark/sql/IntegratedUDFTestUtils.scala | 9 +-------- .../org/apache/spark/sql/SQLQueryTestSuite.scala | 12 +++--------- 5 files changed, 13 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 04c427da3ec87..a721e17aef02d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.parser import java.io.File import java.nio.file.Files -import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable @@ -295,11 +294,6 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { private val sqlSyntaxDefs = { val sqlBasePath = { - val sparkHome = { - assert(sys.props.contains("spark.test.home") || - sys.env.contains("SPARK_HOME"), "spark.test.home or SPARK_HOME is not set.") - sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME")) - } java.nio.file.Paths.get(sparkHome, "sql", "catalyst", "src", "main", "antlr4", "org", "apache", "spark", "sql", "catalyst", "parser", "SqlBase.g4").toFile } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala index d213743946e76..5deab79b9f67d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala @@ -21,6 +21,8 @@ import java.time.ZoneId import scala.util.control.NonFatal +import org.scalatest.Assertions.fail + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils.getZoneId @@ -83,4 +85,11 @@ trait SQLHelper { } } } + + protected lazy val sparkHome: String = { + if (!(sys.props.contains("spark.test.home") || sys.env.contains("SPARK_HOME"))) { + fail("spark.test.home or SPARK_HOME is not set.") + } + sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala index 4c9ba9455c33f..de1517e001b47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala @@ -72,12 +72,6 @@ class ExpressionsSchemaSuite extends QueryTest with SharedSparkSession { // We use a path based on Spark home for 2 reasons: // 1. Maven can't get correct resource directory when resources in other jars. // 2. We test subclasses in the hive-thriftserver module. - val sparkHome = { - assert(sys.props.contains("spark.test.home") || - sys.env.contains("SPARK_HOME"), "spark.test.home or SPARK_HOME is not set.") - sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME")) - } - java.nio.file.Paths.get(sparkHome, "sql", "core", "src", "test", "resources", "sql-functions").toFile } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index 4a4504a075060..6391d5664d5c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -75,14 +75,7 @@ object IntegratedUDFTestUtils extends SQLHelper { import scala.sys.process._ private lazy val pythonPath = sys.env.getOrElse("PYTHONPATH", "") - private lazy val sparkHome = if (sys.props.contains(Tests.IS_TESTING.key)) { - assert(sys.props.contains("spark.test.home") || - sys.env.contains("SPARK_HOME"), "spark.test.home or SPARK_HOME is not set.") - sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME")) - } else { - assert(sys.env.contains("SPARK_HOME"), "SPARK_HOME is not set.") - sys.env("SPARK_HOME") - } + // Note that we will directly refer pyspark's source, not the zip from a regular build. // It is possible the test is being ran without the build. private lazy val sourcePath = Paths.get(sparkHome, "python").toAbsolutePath diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 92da58c27a141..f0522dfeafaac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -19,14 +19,14 @@ package org.apache.spark.sql import java.io.File import java.util.Locale -import java.util.regex.Pattern -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} @@ -124,7 +124,7 @@ import org.apache.spark.tags.ExtendedSQLTest * different types of UDFs. See 'udf/udf-inner-join.sql' as an example. */ @ExtendedSQLTest -class SQLQueryTestSuite extends QueryTest with SharedSparkSession { +class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper { import IntegratedUDFTestUtils._ @@ -134,12 +134,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // We use a path based on Spark home for 2 reasons: // 1. Maven can't get correct resource directory when resources in other jars. // 2. We test subclasses in the hive-thriftserver module. - val sparkHome = { - assert(sys.props.contains("spark.test.home") || - sys.env.contains("SPARK_HOME"), "spark.test.home or SPARK_HOME is not set.") - sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME")) - } - java.nio.file.Paths.get(sparkHome, "sql", "core", "src", "test", "resources", "sql-tests").toFile } From 8a9ae01e74a2973a678d013c0bc8cceeedd685a2 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 18 Jun 2020 15:32:44 +0000 Subject: [PATCH 036/384] [MINOR] update dev/create-release/known_translations This is auto-updated by running script `translate-contributors.py` Closes #28861 from cloud-fan/update. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- dev/create-release/known_translations | 148 ++++++++++++++++++++++++++ 1 file changed, 148 insertions(+) diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations index 65c00cce8c9c6..376398bc3788a 100644 --- a/dev/create-release/known_translations +++ b/dev/create-release/known_translations @@ -261,3 +261,151 @@ yanlin-Lynn - Yanlin Wang yucai - Yucai Yu zhengruifeng - Ruifeng Zheng zuotingbing - Tingbing Zuo +012huang - Weiyi Huang +07ARB - Ankit Raj Boudh +Andrew-Crosby - Andrew Crosby +AngersZhuuuu - Yi Zhu +Deegue - Yizhong Zhang +Gschiavon - German Schiavon Matteo +GuoPhilipse - Philipse Guo +Hellsen83 - Erik Christiansen +Icysandwich - Icysandwich +JasonWayne - Wenjie Wu +JkSelf - Ke Jia +JoanFM - Joan Fontanals +JulienPeloton - Julien Peloton +Koraseg - Artem Kupchinskiy +KyleLi1985 - Liang Li +LiShuMing - Shuming Li +LinhongLiu - Liu, Linhong +LuciferYang - Yang Jie +MaxGekk - Maxim Gekk +Ngone51 - Yi Wu +PavithraRamachandran - Pavithra Ramachandran +SongYadong - Yadong Song +TigerYang414 - David Yang +TomokoKomiyama - Tomoko Komiyama +TopGunViper - TopGunViper +Udbhav30 - Udbhav Agrawal +WangGuangxin - Guangxin Wang +William1104 - William Wong +YongjinZhou - Yongjin Zhou +aaruna - Aaruna Godthi +adrian555 - Weiqiang Zhuang +ajithme - Ajith S +amanomer - Aman Omer +ancasarb - Anca Sarb +avkgh - Aleksandr Kashkirov +ayudovin - Artsiom Yudovin +bartosz25 - Bartosz Konieczny +beliefer - Jiaan Geng +bettermouse - Chen Hao +bscan - Brian Scannell +cchung100m - Neo Chien +cclauss - Christian Clauss +chakravarthiT - Chakravarthi +chandulal - Chandu Kavar +chitralverma - Chitral Verma +cjn082030 - Jenny +cloud-fan - Wenchen Fan +codeborui - codeborui +colinmjj - Colin Ma +cxzl25 - cxzl25 +cyq89051127 - Yongqiang Chai +darrentirto - Darren Tirto +daviddingly - Xiaoyuan Ding +davidvrba - David Vrba +deepyaman - Deepyaman Datta +denglingang - Lingang Deng +dengziming - dengziming +deshanxiao - deshanxiao +dima-asana - Dima Kamalov +dlindelof - David Lindelof +dongjoon-hyun - Dongjoon Hyun +eatoncys - eatoncys +fan31415 - Yijie Fan +fitermay - Yuli Fiterman +francis0407 - Mingcong Han +fuwhu - Fuwang Hu +gss2002 - Greg Senia +hddong - Dongdong Hong +hehuiyuan - hehuiyuan +helenyugithub - Helen Yu +highmoutain - highmoutain +httfighter - Tiantian Han +huangtianhua - huangtianhua +hvanhovell - Herman Van Hovell +iRakson - Rakesh Raushan +igorcalabria - Igor Calabria +imback82 - Terry Kim +javierivanov - Javier Fuentes +joelgenter - Joel Genter +ketank-new - Ketan Kunde +laskfla - Keith Sun +lcqzte10192193 - Chaoqun Li +leoluan2009 - Xuedong Luan +liangxs - Xuesen Liang +lidinghao - Li Hao +linehrr - Ryne Yang +linzebing - Zebing Lin +lipzhu - Lipeng Zhu +liucht-inspur - liucht-inspur +liupc - Pengcheng Liu +liwensun - Liwen Sun +manuzhang - Manu Zhang +mareksimunek - Marek Simunek +masa3141 - Masahiro Kazama +mdianjun - Dianjun Ma +merrily01 - Ruilei Ma +mob-ai - mob-ai +mu5358271 - Shuheng Dai +mwlon - Martin Loncaric +nandorKollar - Nandor Kollar +nooberfsh - nooberfsh +oleg-smith - Oleg Kuznetsov +ozancicek - Ozan Cicekci +pengbo - Peng Bo +planga82 - Pablo Langa Blanco +praneetsharma - Praneet Sharma +ptkool - Michael Styles +qb-tarushg - Tarush Grover +redsanket - Sanket Reddy +redsk - Nicola Bova +roland1982 - roland1982 +rongma1997 - Rong Ma +rrusso2007 - Rob Russo +samsetegne - Samuel L. Setegne +sangramga - Sangram Gaikwad +sarthfrey - Sarth Frey +seayoun - Haiyang Yu +sev7e0 - Jiaqi Li +shahidki31 - Shahid +sharangk - Sharanabasappa G Keriwaddi +sheepstop - Ting Yang +shivsood - Shiv Prashant Sood +sitegui - Guilherme Souza +slamke - Sun Ke +southernriver - Liang Chen +squito - Imran Rashid +stczwd - Jackey Lee +sujith71955 - Sujith Chacko +suxingfate - Xinglong Wang +teeyog - teeyog +tinhto-000 - Tin Hang To +tools4origins - tools4origins +triplesheep - triplesheep +turboFei - Fei Wang +ulysses-you - ulysses-you +uzadude - Ohad Raviv +wackxu - wackxu +wangjiaochun - wangjiaochun +wangshisan - wangshisan +weixiuli - XiuLi Wei +wenfang6 - wenfang6 +wenxuanguan - wenxuanguan +windpiger - Song Jun +woudygao - Woudy Gao +xianyinxin - Xianyin Xin +yunzoud - Yun Zou +zero323 - Maciej Szymkiewicz +zjf2012 - Jiafu Zhang From ac98a9a07fe3e76c16bd5bb03aca553e80629e01 Mon Sep 17 00:00:00 2001 From: James Yu Date: Thu, 18 Jun 2020 14:36:20 -0700 Subject: [PATCH 037/384] [MINOR][DOCS] Update running-on-kubernetes.md ### What changes were proposed in this pull request? Fix executor container name typo. `executor` should be `spark-kubernetes-executor`. ### Why are the changes needed? The Executor pod container name the users actually get from their Kubernetes clusters is different from that described in the documentation. For example, below is what a user get from an executor pod. ``` Containers: spark-kubernetes-executor: Container ID: docker://aaaabbbbccccddddeeeeffff Image: Image ID: docker-pullable://0000.dkr.ecr.us-east-0.amazonaws.com/spark Port: 7079/TCP Host Port: 0/TCP Args: executor State: Running Started: Thu, 28 May 2020 05:54:04 -0700 Ready: True Restart Count: 0 Limits: memory: 16Gi ``` ### Does this PR introduce _any_ user-facing change? Document change. ### How was this patch tested? N/A Closes #28862 from yuj/patch-1. Authored-by: James Yu Signed-off-by: Dongjoon Hyun --- docs/running-on-kubernetes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index ba735cacd4c38..d6484051653b4 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1341,7 +1341,7 @@ The following affect the driver and executor containers. All other containers in See description The container name will be assigned by spark ("spark-kubernetes-driver" for the driver container, and - "executor" for each executor container) if not defined by the pod template. If the container is defined by the + "spark-kubernetes-executor" for each executor container) if not defined by the pod template. If the container is defined by the template, the template's name will be used. From 8750363c8ddaa649d52e6d9001b9871d379fc4e1 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Fri, 19 Jun 2020 12:17:40 +0900 Subject: [PATCH 038/384] [MINOR][DOCS] Emphasize the Streaming tab is for DStream API ### What changes were proposed in this pull request? Emphasize the Streaming tab is for DStream API. ### Why are the changes needed? Some users reported that it's a little confusing of the streaming tab and structured streaming tab. ### Does this PR introduce _any_ user-facing change? Document change. ### How was this patch tested? N/A Closes #28854 from xuanyuanking/minor-doc. Authored-by: Yuanjian Li Signed-off-by: HyukjinKwon --- docs/web-ui.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/web-ui.md b/docs/web-ui.md index e2e612cef3e54..69c9da6428938 100644 --- a/docs/web-ui.md +++ b/docs/web-ui.md @@ -435,10 +435,10 @@ The tracked operations are listed as follows. 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 -for troubleshooting the streaming application. +## Streaming (DStreams) Tab +The web UI includes a Streaming tab if the application uses Spark Streaming with DStream API. +This tab displays scheduling delay and processing time for each micro-batch in the data stream, +which can be useful for troubleshooting the streaming application. ## JDBC/ODBC Server Tab We can see this tab when Spark is running as a [distributed SQL engine](sql-distributed-sql-engine.html). It shows information about sessions and submitted SQL operations. From 17a5007fd8b9249593a703ba7659847ec09be2e8 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 19 Jun 2020 05:41:09 +0000 Subject: [PATCH 039/384] [SPARK-30865][SQL][SS] Refactor DateTimeUtils ### What changes were proposed in this pull request? 1. Move TimeZoneUTC and TimeZoneGMT to DateTimeTestUtils 2. Remove TimeZoneGMT 3. Use ZoneId.systemDefault() instead of defaultTimeZone().toZoneId 4. Alias SQLDate & SQLTimestamp to internal types of DateType and TimestampType 5. Avoid one `*` `DateTimeUtils`.`in fromJulianDay()` 6. Use toTotalMonths in `DateTimeUtils`.`subtractDates()` 7. Remove `julianCommonEraStart`, `timestampToString()`, `microsToEpochDays()`, `epochDaysToMicros()`, `instantToDays()` from `DateTimeUtils`. 8. Make splitDate() private. 9. Remove `def daysToMicros(days: Int): Long` and `def microsToDays(micros: Long): Int`. ### Why are the changes needed? This simplifies the common code related to date-time operations, and should improve maintainability. In particular: 1. TimeZoneUTC and TimeZoneGMT are moved to DateTimeTestUtils because they are used only in tests 2. TimeZoneGMT can be removed because it is equal to TimeZoneUTC 3. After the PR #27494, Spark expressions and DateTimeUtils functions switched to ZoneId instead of TimeZone completely. `defaultTimeZone()` with `TimeZone` as return type is not needed anymore. 4. SQLDate and SQLTimestamp types can be explicitly aliased to internal types of DateType and and TimestampType instead of declaring this in a comment. 5. Avoid one `*` `DateTimeUtils`.`in fromJulianDay()`. 6. Use toTotalMonths in `DateTimeUtils`.`subtractDates()`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites Closes #27617 from MaxGekk/move-time-zone-consts. Lead-authored-by: Max Gekk Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/Cast.scala | 13 +- .../expressions/datetimeExpressions.scala | 12 +- .../sql/catalyst/util/DateTimeUtils.scala | 393 ++++++++---------- .../catalyst/util/TimestampFormatter.scala | 6 +- .../sql/catalyst/expressions/CastSuite.scala | 13 +- .../expressions/CsvExpressionsSuite.scala | 4 +- .../expressions/DateExpressionsSuite.scala | 5 +- .../expressions/ObjectExpressionsSuite.scala | 4 +- .../catalyst/json/JacksonGeneratorSuite.scala | 8 +- .../catalyst/util/DateTimeUtilsSuite.scala | 10 +- .../util/DatetimeFormatterSuite.scala | 2 +- .../catalyst/util/RebaseDateTimeSuite.scala | 2 +- .../spark/sql/util/ArrowUtilsSuite.scala | 5 +- .../datasources/jdbc/JDBCRelation.scala | 2 +- .../datasources/parquet/ParquetFilters.scala | 3 +- .../parquet/ParquetRowConverter.scala | 3 +- .../spark/sql/DataFrameFunctionsSuite.scala | 6 +- .../spark/sql/StatisticsCollectionSuite.scala | 7 +- .../datasources/json/JsonSuite.scala | 4 +- .../ParquetPartitionDiscoverySuite.scala | 5 +- .../sql/sources/ResolvedDataSourceSuite.scala | 4 +- .../spark/sql/streaming/StreamSuite.scala | 4 +- 22 files changed, 241 insertions(+), 274 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 5576e71b57024..3ce284d5518a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -303,7 +303,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes) case DateType => buildCast[Int](_, d => UTF8String.fromString(dateFormatter.format(d))) case TimestampType => buildCast[Long](_, - t => UTF8String.fromString(DateTimeUtils.timestampToString(timestampFormatter, t))) + t => UTF8String.fromString(timestampFormatter.format(t))) case ArrayType(et, _) => buildCast[ArrayData](_, array => { val builder = new UTF8StringBuilder @@ -443,7 +443,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case ByteType => buildCast[Byte](_, b => longToTimestamp(b.toLong)) case DateType => - buildCast[Int](_, d => epochDaysToMicros(d, zoneId)) + buildCast[Int](_, d => daysToMicros(d, zoneId)) // TimestampWritable.decimalToTimestamp case DecimalType() => buildCast[Decimal](_, d => decimalToTimestamp(d)) @@ -480,7 +480,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. - buildCast[Long](_, t => microsToEpochDays(t, zoneId)) + buildCast[Long](_, t => microsToDays(t, zoneId)) } // IntervalConverter @@ -1034,8 +1034,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val tf = JavaCode.global( ctx.addReferenceObj("timestampFormatter", timestampFormatter), timestampFormatter.getClass) - (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString( - org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($tf, $c));""" + (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString($tf.format($c));""" case CalendarIntervalType => (c, evPrim, _) => code"""$evPrim = UTF8String.fromString($c.toString());""" case ArrayType(et, _) => @@ -1120,7 +1119,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val zid = getZoneId() (c, evPrim, evNull) => code"""$evPrim = - org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToEpochDays($c, $zid);""" + org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToDays($c, $zid);""" case _ => (c, evPrim, evNull) => code"$evNull = true;" } @@ -1247,7 +1246,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit zoneIdClass) (c, evPrim, evNull) => code"""$evPrim = - org.apache.spark.sql.catalyst.util.DateTimeUtils.epochDaysToMicros($c, $zid);""" + org.apache.spark.sql.catalyst.util.DateTimeUtils.daysToMicros($c, $zid);""" case DecimalType() => (c, evPrim, evNull) => code"$evPrim = ${decimalToTimestampCode(c)};" case DoubleType => 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 b46c3fb349ee1..b9ba32b8ee337 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 @@ -895,7 +895,7 @@ abstract class ToTimestamp } else { left.dataType match { case DateType => - epochDaysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor + daysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor case TimestampType => t.asInstanceOf[Long] / downScaleFactor case StringType => @@ -975,7 +975,7 @@ abstract class ToTimestamp boolean ${ev.isNull} = ${eval1.isNull}; $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { - ${ev.value} = $dtu.epochDaysToMicros(${eval1.value}, $zid) / $downScaleFactor; + ${ev.value} = $dtu.daysToMicros(${eval1.value}, $zid) / $downScaleFactor; }""") } } @@ -1242,10 +1242,10 @@ case class DateAddInterval( if (ansiEnabled || itvl.microseconds == 0) { DateTimeUtils.dateAddInterval(start.asInstanceOf[Int], itvl) } else { - val startTs = DateTimeUtils.epochDaysToMicros(start.asInstanceOf[Int], zoneId) + val startTs = DateTimeUtils.daysToMicros(start.asInstanceOf[Int], zoneId) val resultTs = DateTimeUtils.timestampAddInterval( startTs, itvl.months, itvl.days, itvl.microseconds, zoneId) - DateTimeUtils.microsToEpochDays(resultTs, zoneId) + DateTimeUtils.microsToDays(resultTs, zoneId) } } @@ -1261,10 +1261,10 @@ case class DateAddInterval( |if ($i.microseconds == 0) { | ${ev.value} = $dtu.dateAddInterval($sd, $i); |} else { - | long $startTs = $dtu.epochDaysToMicros($sd, $zid); + | long $startTs = $dtu.daysToMicros($sd, $zid); | long $resultTs = | $dtu.timestampAddInterval($startTs, $i.months, $i.days, $i.microseconds, $zid); - | ${ev.value} = $dtu.microsToEpochDays($resultTs, $zid); + | ${ev.value} = $dtu.microsToDays($resultTs, $zid); |} |""".stripMargin }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 41a271b95e83c..c466a60259c7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -42,51 +42,32 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} */ object DateTimeUtils { - // we use Int and Long internally to represent [[DateType]] and [[TimestampType]] - type SQLDate = Int - type SQLTimestamp = Long - - // see http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian - // it's 2440587.5, rounding up to compatible with Hive + // See http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian + // It's 2440587.5, rounding up to be compatible with Hive. final val JULIAN_DAY_OF_EPOCH = 2440588 - final val julianCommonEraStart = Timestamp.valueOf("0001-01-01 00:00:00") - - final val TimeZoneGMT = TimeZone.getTimeZone("GMT") final val TimeZoneUTC = TimeZone.getTimeZone("UTC") val TIMEZONE_OPTION = "timeZone" - def defaultTimeZone(): TimeZone = TimeZone.getDefault() - def getZoneId(timeZoneId: String): ZoneId = ZoneId.of(timeZoneId, ZoneId.SHORT_IDS) - def getTimeZone(timeZoneId: String): TimeZone = { - TimeZone.getTimeZone(getZoneId(timeZoneId)) - } + def getTimeZone(timeZoneId: String): TimeZone = TimeZone.getTimeZone(getZoneId(timeZoneId)) - def microsToDays(timestamp: SQLTimestamp): SQLDate = { - microsToDays(timestamp, defaultTimeZone().toZoneId) - } - - def microsToDays(timestamp: SQLTimestamp, zoneId: ZoneId): SQLDate = { - val instant = microsToInstant(timestamp) - localDateToDays(LocalDateTime.ofInstant(instant, zoneId).toLocalDate) - } - - def daysToMicros(days: SQLDate): SQLTimestamp = { - daysToMicros(days, defaultTimeZone().toZoneId) + /** + * Converts microseconds since 1970-01-01 00:00:00Z to days since 1970-01-01 at the given zone ID. + */ + def microsToDays(micros: Long, zoneId: ZoneId): Int = { + localDateToDays(getLocalDateTime(micros, zoneId).toLocalDate) } - def daysToMicros(days: SQLDate, zoneId: ZoneId): SQLTimestamp = { + /** + * Converts days since 1970-01-01 at the given zone ID to microseconds since 1970-01-01 00:00:00Z. + */ + def daysToMicros(days: Int, zoneId: ZoneId): Long = { val instant = daysToLocalDate(days).atStartOfDay(zoneId).toInstant instantToMicros(instant) } - // Converts Timestamp to string according to Hive TimestampWritable convention. - def timestampToString(tf: TimestampFormatter, us: SQLTimestamp): String = { - tf.format(us) - } - /** * Converts a local date at the default JVM time zone to the number of days since 1970-01-01 * in the hybrid calendar (Julian + Gregorian) by discarding the time part. The resulted days are @@ -103,7 +84,7 @@ object DateTimeUtils { * combines Julian and Gregorian calendars. * @return The number of days since the epoch in Proleptic Gregorian calendar. */ - def fromJavaDate(date: Date): SQLDate = { + def fromJavaDate(date: Date): Int = { val millisUtc = date.getTime val millisLocal = millisUtc + TimeZone.getDefault.getOffset(millisUtc) val julianDays = Math.toIntExact(Math.floorDiv(millisLocal, MILLIS_PER_DAY)) @@ -121,11 +102,11 @@ object DateTimeUtils { * Note: The date is shifted by the offset of the default JVM time zone for backward compatibility * with Spark 2.4 and earlier versions. * - * @param daysSinceEpoch The number of days since 1970-01-01 in Proleptic Gregorian calendar. + * @param days The number of days since 1970-01-01 in Proleptic Gregorian calendar. * @return A local date in the hybrid calendar as `java.sql.Date` from number of days since epoch. */ - def toJavaDate(daysSinceEpoch: SQLDate): Date = { - val rebasedDays = rebaseGregorianToJulianDays(daysSinceEpoch) + def toJavaDate(days: Int): Date = { + val rebasedDays = rebaseGregorianToJulianDays(days) val localMillis = Math.multiplyExact(rebasedDays, MILLIS_PER_DAY) val timeZoneOffset = TimeZone.getDefault match { case zoneInfo: ZoneInfo => zoneInfo.getOffsetsByWall(localMillis, null) @@ -147,11 +128,11 @@ object DateTimeUtils { * representation as `year`, `month`, `day`, ..., `seconds` in the original calendar * and in the target calendar. * - * @param us The number of microseconds since 1970-01-01T00:00:00.000000Z. + * @param micros The number of microseconds since 1970-01-01T00:00:00.000000Z. * @return A `java.sql.Timestamp` from number of micros since epoch. */ - def toJavaTimestamp(us: SQLTimestamp): Timestamp = { - val rebasedMicros = rebaseGregorianToJulianMicros(us) + def toJavaTimestamp(micros: Long): Timestamp = { + val rebasedMicros = rebaseGregorianToJulianMicros(micros) val seconds = Math.floorDiv(rebasedMicros, MICROS_PER_SECOND) val ts = new Timestamp(seconds * MILLIS_PER_SECOND) val nanos = (rebasedMicros - seconds * MICROS_PER_SECOND) * NANOS_PER_MICROS @@ -178,21 +159,18 @@ object DateTimeUtils { * Gregorian calendars. * @return The number of micros since epoch from `java.sql.Timestamp`. */ - def fromJavaTimestamp(t: Timestamp): SQLTimestamp = { + def fromJavaTimestamp(t: Timestamp): Long = { val micros = millisToMicros(t.getTime) + (t.getNanos / NANOS_PER_MICROS) % MICROS_PER_MILLIS rebaseJulianToGregorianMicros(micros) } /** - * Returns the number of microseconds since epoch from Julian day - * and nanoseconds in a day + * Returns the number of microseconds since epoch from Julian day and nanoseconds in a day. */ - def fromJulianDay(day: Int, nanoseconds: Long): SQLTimestamp = { + def fromJulianDay(days: Int, nanos: Long): Long = { // use Long to avoid rounding errors - val seconds = (day - JULIAN_DAY_OF_EPOCH).toLong * SECONDS_PER_DAY - val micros = SECONDS.toMicros(seconds) + NANOSECONDS.toMicros(nanoseconds) - val rebased = rebaseJulianToGregorianMicros(micros) - rebased + val micros = (days - JULIAN_DAY_OF_EPOCH).toLong * MICROS_PER_DAY + nanos / NANOS_PER_MICROS + rebaseJulianToGregorianMicros(micros) } /** @@ -200,44 +178,33 @@ object DateTimeUtils { * * Note: support timestamp since 4717 BC (without negative nanoseconds, compatible with Hive). */ - def toJulianDay(us: SQLTimestamp): (Int, Long) = { - val julian_us = rebaseGregorianToJulianMicros(us) + JULIAN_DAY_OF_EPOCH * MICROS_PER_DAY - val day = julian_us / MICROS_PER_DAY - val micros = julian_us % MICROS_PER_DAY - (day.toInt, MICROSECONDS.toNanos(micros)) + def toJulianDay(micros: Long): (Int, Long) = { + val julianUs = rebaseGregorianToJulianMicros(micros) + JULIAN_DAY_OF_EPOCH * MICROS_PER_DAY + val days = julianUs / MICROS_PER_DAY + val us = julianUs % MICROS_PER_DAY + (days.toInt, MICROSECONDS.toNanos(us)) } - /* - * Converts the timestamp to milliseconds since epoch. In spark timestamp values have microseconds + /** + * Converts the timestamp to milliseconds since epoch. In Spark timestamp values have microseconds * precision, so this conversion is lossy. */ - def microsToMillis(us: SQLTimestamp): Long = { + def microsToMillis(micros: Long): Long = { // When the timestamp is negative i.e before 1970, we need to adjust the millseconds portion. // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision. // In millis precision the above needs to be represented as (-157700927877). - Math.floorDiv(us, MICROS_PER_MILLIS) + Math.floorDiv(micros, MICROS_PER_MILLIS) } - /* - * Converts milliseconds since epoch to SQLTimestamp. + /** + * Converts milliseconds since the epoch to microseconds. */ - def millisToMicros(millis: Long): SQLTimestamp = { + def millisToMicros(millis: Long): Long = { Math.multiplyExact(millis, MICROS_PER_MILLIS) } - def microsToEpochDays(epochMicros: SQLTimestamp, zoneId: ZoneId): SQLDate = { - localDateToDays(microsToInstant(epochMicros).atZone(zoneId).toLocalDate) - } - - def epochDaysToMicros(epochDays: SQLDate, zoneId: ZoneId): SQLTimestamp = { - val localDate = LocalDate.ofEpochDay(epochDays) - val zeroLocalTime = LocalTime.MIDNIGHT - val localDateTime = LocalDateTime.of(localDate, zeroLocalTime) - instantToMicros(localDateTime.atZone(zoneId).toInstant) - } - - // A method called by JSON/CSV parser to clean up the legacy timestamp string by removing the - // "GMT" string. + // The method is called by JSON/CSV parser to clean up the legacy timestamp string by removing + // the "GMT" string. def cleanLegacyTimestampStr(s: String): String = { val indexOfGMT = s.indexOf("GMT") if (indexOfGMT != -1) { @@ -252,8 +219,8 @@ object DateTimeUtils { } /** - * Trim and parse a given UTF8 date string to the corresponding a corresponding [[Long]] value. - * The return type is [[Option]] in order to distinguish between 0L and null. The following + * Trims and parses a given UTF8 timestamp string to the corresponding a corresponding [[Long]] + * value. The return type is [[Option]] in order to distinguish between 0L and null. The following * formats are allowed: * * `yyyy` @@ -277,7 +244,7 @@ object DateTimeUtils { * - +|-hhmmss * - Region-based zone IDs in the form `area/city`, such as `Europe/Paris` */ - def stringToTimestamp(s: UTF8String, timeZoneId: ZoneId): Option[SQLTimestamp] = { + def stringToTimestamp(s: UTF8String, timeZoneId: ZoneId): Option[Long] = { if (s == null) { return None } @@ -408,34 +375,41 @@ object DateTimeUtils { } } + /** + * Gets the number of microseconds since the epoch of 1970-01-01 00:00:00Z from the given + * instance of `java.time.Instant`. The epoch microsecond count is a simple incrementing count of + * microseconds where microsecond 0 is 1970-01-01 00:00:00Z. + */ def instantToMicros(instant: Instant): Long = { val us = Math.multiplyExact(instant.getEpochSecond, MICROS_PER_SECOND) val result = Math.addExact(us, NANOSECONDS.toMicros(instant.getNano)) result } - def microsToInstant(us: Long): Instant = { - val secs = Math.floorDiv(us, MICROS_PER_SECOND) + /** + * Obtains an instance of `java.time.Instant` using microseconds from + * the epoch of 1970-01-01 00:00:00Z. + */ + def microsToInstant(micros: Long): Instant = { + val secs = Math.floorDiv(micros, MICROS_PER_SECOND) // Unfolded Math.floorMod(us, MICROS_PER_SECOND) to reuse the result of // the above calculation of `secs` via `floorDiv`. - val mos = us - secs * MICROS_PER_SECOND + val mos = micros - secs * MICROS_PER_SECOND Instant.ofEpochSecond(secs, mos * NANOS_PER_MICROS) } - def instantToDays(instant: Instant): Int = { - val seconds = instant.getEpochSecond - val days = Math.floorDiv(seconds, SECONDS_PER_DAY) - days.toInt - } - - def localDateToDays(localDate: LocalDate): Int = { - Math.toIntExact(localDate.toEpochDay) - } + /** + * Converts the local date to the number of days since 1970-01-01. + */ + def localDateToDays(localDate: LocalDate): Int = Math.toIntExact(localDate.toEpochDay) + /** + * Obtains an instance of `java.time.LocalDate` from the epoch day count. + */ def daysToLocalDate(days: Int): LocalDate = LocalDate.ofEpochDay(days) /** - * Trim and parse a given UTF8 date string to a corresponding [[Int]] value. + * Trims and parses a given UTF8 date string to a corresponding [[Int]] value. * The return type is [[Option]] in order to distinguish between 0 and null. The following * formats are allowed: * @@ -446,7 +420,7 @@ object DateTimeUtils { * `yyyy-[m]m-[d]d *` * `yyyy-[m]m-[d]dT*` */ - def stringToDate(s: UTF8String, zoneId: ZoneId): Option[SQLDate] = { + def stringToDate(s: UTF8String, zoneId: ZoneId): Option[Int] = { if (s == null) { return None } @@ -494,124 +468,102 @@ object DateTimeUtils { } } - private def localTimestamp(microsec: SQLTimestamp, zoneId: ZoneId): LocalDateTime = { - microsToInstant(microsec).atZone(zoneId).toLocalDateTime + // Gets the local date-time parts (year, month, day and time) of the instant expressed as the + // number of microseconds since the epoch at the given time zone ID. + private def getLocalDateTime(micros: Long, zoneId: ZoneId): LocalDateTime = { + microsToInstant(micros).atZone(zoneId).toLocalDateTime } /** * Returns the hour value of a given timestamp value. The timestamp is expressed in microseconds. */ - def getHours(microsec: SQLTimestamp, zoneId: ZoneId): Int = { - localTimestamp(microsec, zoneId).getHour + def getHours(micros: Long, zoneId: ZoneId): Int = { + getLocalDateTime(micros, zoneId).getHour } /** * Returns the minute value of a given timestamp value. The timestamp is expressed in - * microseconds. + * microseconds since the epoch. */ - def getMinutes(microsec: SQLTimestamp, zoneId: ZoneId): Int = { - localTimestamp(microsec, zoneId).getMinute + def getMinutes(micros: Long, zoneId: ZoneId): Int = { + getLocalDateTime(micros, zoneId).getMinute } /** * Returns the second value of a given timestamp value. The timestamp is expressed in - * microseconds. + * microseconds since the epoch. */ - def getSeconds(microsec: SQLTimestamp, zoneId: ZoneId): Int = { - localTimestamp(microsec, zoneId).getSecond + def getSeconds(micros: Long, zoneId: ZoneId): Int = { + getLocalDateTime(micros, zoneId).getSecond } /** * Returns the seconds part and its fractional part with microseconds. */ - def getSecondsWithFraction(microsec: SQLTimestamp, zoneId: ZoneId): Decimal = { - Decimal(getMicroseconds(microsec, zoneId), 8, 6) + def getSecondsWithFraction(micros: Long, zoneId: ZoneId): Decimal = { + Decimal(getMicroseconds(micros, zoneId), 8, 6) } /** - * Returns seconds, including fractional parts, multiplied by 1000000. The timestamp - * is expressed in microseconds since the epoch. + * Returns local seconds, including fractional parts, multiplied by 1000000. + * + * @param micros The number of microseconds since the epoch. + * @param zoneId The time zone id which milliseconds should be obtained in. */ - def getMicroseconds(timestamp: SQLTimestamp, zoneId: ZoneId): Int = { - val lt = localTimestamp(timestamp, zoneId) + def getMicroseconds(micros: Long, zoneId: ZoneId): Int = { + val lt = getLocalDateTime(micros, zoneId) (lt.getLong(ChronoField.MICRO_OF_SECOND) + lt.getSecond * MICROS_PER_SECOND).toInt } /** - * Returns the 'day in year' value for the given date. The date is expressed in days - * since 1.1.1970. + * Returns the 'day in year' value for the given number of days since 1970-01-01. */ - def getDayInYear(date: SQLDate): Int = { - LocalDate.ofEpochDay(date).getDayOfYear - } + def getDayInYear(days: Int): Int = daysToLocalDate(days).getDayOfYear /** - * Returns the year value for the given date. The date is expressed in days - * since 1.1.1970. + * Returns the year value for the given number of days since 1970-01-01. */ - def getYear(date: SQLDate): Int = { - LocalDate.ofEpochDay(date).getYear - } + def getYear(days: Int): Int = daysToLocalDate(days).getYear /** * Returns the year which conforms to ISO 8601. Each ISO 8601 week-numbering * year begins with the Monday of the week containing the 4th of January. */ - def getWeekBasedYear(date: SQLDate): Int = { - daysToLocalDate(date).get(IsoFields.WEEK_BASED_YEAR) - } + def getWeekBasedYear(days: Int): Int = daysToLocalDate(days).get(IsoFields.WEEK_BASED_YEAR) - /** - * Returns the quarter for the given date. The date is expressed in days - * since 1.1.1970. - */ - def getQuarter(date: SQLDate): Int = { - LocalDate.ofEpochDay(date).get(IsoFields.QUARTER_OF_YEAR) - } + /** Returns the quarter for the given number of days since 1970-01-01. */ + def getQuarter(days: Int): Int = daysToLocalDate(days).get(IsoFields.QUARTER_OF_YEAR) /** - * Split date (expressed in days since 1.1.1970) into four fields: - * year, month (Jan is Month 1), dayInMonth, daysToMonthEnd (0 if it's last day of month). - */ - def splitDate(date: SQLDate): (Int, Int, Int, Int) = { - val ld = LocalDate.ofEpochDay(date) - (ld.getYear, ld.getMonthValue, ld.getDayOfMonth, ld.lengthOfMonth() - ld.getDayOfMonth) - } - - /** - * Returns the month value for the given date. The date is expressed in days - * since 1.1.1970. January is month 1. + * Returns the month value for the given number of days since 1970-01-01. + * January is month 1. */ - def getMonth(date: SQLDate): Int = { - LocalDate.ofEpochDay(date).getMonthValue - } + def getMonth(days: Int): Int = daysToLocalDate(days).getMonthValue /** - * Returns the 'day of month' value for the given date. The date is expressed in days - * since 1.1.1970. + * Returns the 'day of month' value for the given number of days since 1970-01-01. */ - def getDayOfMonth(date: SQLDate): Int = { - LocalDate.ofEpochDay(date).getDayOfMonth - } + def getDayOfMonth(days: Int): Int = daysToLocalDate(days).getDayOfMonth /** - * Add date and year-month interval. - * Returns a date value, expressed in days since 1.1.1970. + * Adds an year-month interval to a date represented as days since 1970-01-01. + * @return a date value, expressed in days since 1970-01-01. */ - def dateAddMonths(days: SQLDate, months: Int): SQLDate = { - LocalDate.ofEpochDay(days).plusMonths(months).toEpochDay.toInt + def dateAddMonths(days: Int, months: Int): Int = { + localDateToDays(daysToLocalDate(days).plusMonths(months)) } /** - * Add timestamp and full interval. - * Returns a timestamp value, expressed in microseconds since 1.1.1970 00:00:00. + * Adds a full interval (months, days, microseconds) a timestamp represented as the number of + * microseconds since 1970-01-01 00:00:00Z. + * @return A timestamp value, expressed in microseconds since 1970-01-01 00:00:00Z. */ def timestampAddInterval( - start: SQLTimestamp, + start: Long, months: Int, days: Int, microseconds: Long, - zoneId: ZoneId): SQLTimestamp = { + zoneId: ZoneId): Long = { val resultTimestamp = microsToInstant(start) .atZone(zoneId) .plusMonths(months) @@ -621,38 +573,47 @@ object DateTimeUtils { } /** - * Add the date and the interval's months and days. - * Returns a date value, expressed in days since 1.1.1970. + * Adds the interval's months and days to a date expressed as days since the epoch. + * @return A date value, expressed in days since 1970-01-01. * * @throws DateTimeException if the result exceeds the supported date range * @throws IllegalArgumentException if the interval has `microseconds` part */ def dateAddInterval( - start: SQLDate, - interval: CalendarInterval): SQLDate = { + start: Int, + interval: CalendarInterval): Int = { require(interval.microseconds == 0, "Cannot add hours, minutes or seconds, milliseconds, microseconds to a date") - val ld = LocalDate.ofEpochDay(start).plusMonths(interval.months).plusDays(interval.days) + val ld = daysToLocalDate(start).plusMonths(interval.months).plusDays(interval.days) localDateToDays(ld) } /** - * Returns number of months between time1 and time2. time1 and time2 are expressed in - * microseconds since 1.1.1970. If time1 is later than time2, the result is positive. + * Splits date (expressed in days since 1970-01-01) into four fields: + * year, month (Jan is Month 1), dayInMonth, daysToMonthEnd (0 if it's last day of month). + */ + private def splitDate(days: Int): (Int, Int, Int, Int) = { + val ld = daysToLocalDate(days) + (ld.getYear, ld.getMonthValue, ld.getDayOfMonth, ld.lengthOfMonth() - ld.getDayOfMonth) + } + + /** + * Returns number of months between micros1 and micros2. micros1 and micros2 are expressed in + * microseconds since 1970-01-01. If micros1 is later than micros2, the result is positive. * - * If time1 and time2 are on the same day of month, or both are the last day of month, + * If micros1 and micros2 are on the same day of month, or both are the last day of month, * returns, time of day will be ignored. * * Otherwise, the difference is calculated based on 31 days per month. * The result is rounded to 8 decimal places if `roundOff` is set to true. */ def monthsBetween( - time1: SQLTimestamp, - time2: SQLTimestamp, + micros1: Long, + micros2: Long, roundOff: Boolean, zoneId: ZoneId): Double = { - val date1 = microsToDays(time1, zoneId) - val date2 = microsToDays(time2, zoneId) + val date1 = microsToDays(micros1, zoneId) + val date2 = microsToDays(micros2, zoneId) val (year1, monthInYear1, dayInMonth1, daysToMonthEnd1) = splitDate(date1) val (year2, monthInYear2, dayInMonth2, daysToMonthEnd2) = splitDate(date2) @@ -666,8 +627,8 @@ object DateTimeUtils { } // using milliseconds can cause precision loss with more than 8 digits // we follow Hive's implementation which uses seconds - val secondsInDay1 = MICROSECONDS.toSeconds(time1 - daysToMicros(date1, zoneId)) - val secondsInDay2 = MICROSECONDS.toSeconds(time2 - daysToMicros(date2, zoneId)) + val secondsInDay1 = MICROSECONDS.toSeconds(micros1 - daysToMicros(date1, zoneId)) + val secondsInDay2 = MICROSECONDS.toSeconds(micros2 - daysToMicros(date2, zoneId)) val secondsDiff = (dayInMonth1 - dayInMonth2) * SECONDS_PER_DAY + secondsInDay1 - secondsInDay2 val secondsInMonth = DAYS.toSeconds(31) val diff = monthDiff + secondsDiff / secondsInMonth.toDouble @@ -710,17 +671,14 @@ object DateTimeUtils { * Returns the first date which is later than startDate and is of the given dayOfWeek. * dayOfWeek is an integer ranges in [0, 6], and 0 is Thu, 1 is Fri, etc,. */ - def getNextDateForDayOfWeek(startDate: SQLDate, dayOfWeek: Int): SQLDate = { - startDate + 1 + ((dayOfWeek - 1 - startDate) % 7 + 7) % 7 + def getNextDateForDayOfWeek(startDay: Int, dayOfWeek: Int): Int = { + startDay + 1 + ((dayOfWeek - 1 - startDay) % 7 + 7) % 7 } - /** - * Returns last day of the month for the given date. The date is expressed in days - * since 1.1.1970. - */ - def getLastDayOfMonth(date: SQLDate): SQLDate = { - val localDate = LocalDate.ofEpochDay(date) - (date - localDate.getDayOfMonth) + localDate.lengthOfMonth() + /** Returns last day of the month for the given number of days since 1970-01-01. */ + def getLastDayOfMonth(days: Int): Int = { + val localDate = daysToLocalDate(days) + (days - localDate.getDayOfMonth) + localDate.lengthOfMonth() } // The constants are visible for testing purpose only. @@ -746,21 +704,21 @@ object DateTimeUtils { * Returns the trunc date from original date and trunc level. * Trunc level should be generated using `parseTruncLevel()`, should be between 6 and 9. */ - def truncDate(d: SQLDate, level: Int): SQLDate = { + def truncDate(days: Int, level: Int): Int = { level match { - case TRUNC_TO_WEEK => getNextDateForDayOfWeek(d - 7, MONDAY) - case TRUNC_TO_MONTH => d - DateTimeUtils.getDayOfMonth(d) + 1 + case TRUNC_TO_WEEK => getNextDateForDayOfWeek(days - 7, MONDAY) + case TRUNC_TO_MONTH => days - getDayOfMonth(days) + 1 case TRUNC_TO_QUARTER => - localDateToDays(daysToLocalDate(d).`with`(IsoFields.DAY_OF_QUARTER, 1L)) - case TRUNC_TO_YEAR => d - DateTimeUtils.getDayInYear(d) + 1 + localDateToDays(daysToLocalDate(days).`with`(IsoFields.DAY_OF_QUARTER, 1L)) + case TRUNC_TO_YEAR => days - getDayInYear(days) + 1 case _ => // caller make sure that this should never be reached sys.error(s"Invalid trunc level: $level") } } - private def truncToUnit(t: SQLTimestamp, zoneId: ZoneId, unit: ChronoUnit): SQLTimestamp = { - val truncated = microsToInstant(t).atZone(zoneId).truncatedTo(unit) + private def truncToUnit(micros: Long, zoneId: ZoneId, unit: ChronoUnit): Long = { + val truncated = microsToInstant(micros).atZone(zoneId).truncatedTo(unit) instantToMicros(truncated.toInstant) } @@ -768,19 +726,19 @@ object DateTimeUtils { * Returns the trunc date time from original date time and trunc level. * Trunc level should be generated using `parseTruncLevel()`, should be between 0 and 9. */ - def truncTimestamp(t: SQLTimestamp, level: Int, zoneId: ZoneId): SQLTimestamp = { + def truncTimestamp(micros: Long, level: Int, zoneId: ZoneId): Long = { level match { - case TRUNC_TO_MICROSECOND => t + case TRUNC_TO_MICROSECOND => micros case TRUNC_TO_MILLISECOND => - t - Math.floorMod(t, MICROS_PER_MILLIS) + micros - Math.floorMod(micros, MICROS_PER_MILLIS) case TRUNC_TO_SECOND => - t - Math.floorMod(t, MICROS_PER_SECOND) + micros - Math.floorMod(micros, MICROS_PER_SECOND) case TRUNC_TO_MINUTE => - t - Math.floorMod(t, MICROS_PER_MINUTE) - case TRUNC_TO_HOUR => truncToUnit(t, zoneId, ChronoUnit.HOURS) - case TRUNC_TO_DAY => truncToUnit(t, zoneId, ChronoUnit.DAYS) + micros - Math.floorMod(micros, MICROS_PER_MINUTE) + case TRUNC_TO_HOUR => truncToUnit(micros, zoneId, ChronoUnit.HOURS) + case TRUNC_TO_DAY => truncToUnit(micros, zoneId, ChronoUnit.DAYS) case _ => // Try to truncate date levels - val dDays = microsToDays(t, zoneId) + val dDays = microsToDays(micros, zoneId) daysToMicros(truncDate(dDays, level), zoneId) } } @@ -810,36 +768,41 @@ object DateTimeUtils { } /** - * Convert the timestamp `ts` from one timezone to another. + * Converts the timestamp `micros` from one timezone to another. * - * TODO: Because of DST, the conversion between UTC and human time is not exactly one-to-one - * mapping, the conversion here may return wrong result, we should make the timestamp - * timezone-aware. + * Time-zone rules, such as daylight savings, mean that not every local date-time + * is valid for the `toZone` time zone, thus the local date-time may be adjusted. */ - def convertTz(ts: SQLTimestamp, fromZone: ZoneId, toZone: ZoneId): SQLTimestamp = { - val rebasedDateTime = microsToInstant(ts).atZone(toZone).toLocalDateTime.atZone(fromZone) + def convertTz(micros: Long, fromZone: ZoneId, toZone: ZoneId): Long = { + val rebasedDateTime = getLocalDateTime(micros, toZone).atZone(fromZone) instantToMicros(rebasedDateTime.toInstant) } /** - * Returns a timestamp of given timezone from utc timestamp, with the same string + * Returns a timestamp of given timezone from UTC timestamp, with the same string * representation in their timezone. */ - def fromUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { - convertTz(time, ZoneOffset.UTC, getZoneId(timeZone)) + def fromUTCTime(micros: Long, timeZone: String): Long = { + convertTz(micros, ZoneOffset.UTC, getZoneId(timeZone)) } /** * Returns a utc timestamp from a given timestamp from a given timezone, with the same * string representation in their timezone. */ - def toUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { - convertTz(time, getZoneId(timeZone), ZoneOffset.UTC) + def toUTCTime(micros: Long, timeZone: String): Long = { + convertTz(micros, getZoneId(timeZone), ZoneOffset.UTC) } - def currentTimestamp(): SQLTimestamp = instantToMicros(Instant.now()) + /** + * Obtains the current instant as microseconds since the epoch at the UTC time zone. + */ + def currentTimestamp(): Long = instantToMicros(Instant.now()) - def currentDate(zoneId: ZoneId): SQLDate = localDateToDays(LocalDate.now(zoneId)) + /** + * Obtains the current date as days since the epoch in the specified time-zone. + */ + def currentDate(zoneId: ZoneId): Int = localDateToDays(LocalDate.now(zoneId)) private def today(zoneId: ZoneId): ZonedDateTime = { Instant.now().atZone(zoneId).`with`(LocalTime.MIDNIGHT) @@ -849,6 +812,7 @@ object DateTimeUtils { /** * Extracts special values from an input string ignoring case. + * * @param input A trimmed string * @param zoneId Zone identifier used to get the current date. * @return Some special value in lower case or None. @@ -878,12 +842,13 @@ object DateTimeUtils { /** * Converts notational shorthands that are converted to ordinary timestamps. + * * @param input A trimmed string * @param zoneId Zone identifier used to get the current date. * @return Some of microseconds since the epoch if the conversion completed * successfully otherwise None. */ - def convertSpecialTimestamp(input: String, zoneId: ZoneId): Option[SQLTimestamp] = { + def convertSpecialTimestamp(input: String, zoneId: ZoneId): Option[Long] = { extractSpecialValue(input, zoneId).flatMap { case "epoch" => Some(0) case "now" => Some(currentTimestamp()) @@ -894,7 +859,7 @@ object DateTimeUtils { } } - private def convertSpecialTimestamp(bytes: Array[Byte], zoneId: ZoneId): Option[SQLTimestamp] = { + private def convertSpecialTimestamp(bytes: Array[Byte], zoneId: ZoneId): Option[Long] = { if (bytes.length > 0 && Character.isAlphabetic(bytes(0))) { convertSpecialTimestamp(new String(bytes, StandardCharsets.UTF_8), zoneId) } else { @@ -904,11 +869,12 @@ object DateTimeUtils { /** * Converts notational shorthands that are converted to ordinary dates. + * * @param input A trimmed string * @param zoneId Zone identifier used to get the current date. * @return Some of days since the epoch if the conversion completed successfully otherwise None. */ - def convertSpecialDate(input: String, zoneId: ZoneId): Option[SQLDate] = { + def convertSpecialDate(input: String, zoneId: ZoneId): Option[Int] = { extractSpecialValue(input, zoneId).flatMap { case "epoch" => Some(0) case "now" | "today" => Some(currentDate(zoneId)) @@ -918,7 +884,7 @@ object DateTimeUtils { } } - private def convertSpecialDate(bytes: Array[Byte], zoneId: ZoneId): Option[SQLDate] = { + private def convertSpecialDate(bytes: Array[Byte], zoneId: ZoneId): Option[Int] = { if (bytes.length > 0 && Character.isAlphabetic(bytes(0))) { convertSpecialDate(new String(bytes, StandardCharsets.UTF_8), zoneId) } else { @@ -927,17 +893,16 @@ object DateTimeUtils { } /** - * Subtracts two dates. - * @param endDate The end date, exclusive - * @param startDate The start date, inclusive + * Subtracts two dates expressed as days since 1970-01-01. + * + * @param endDay The end date, exclusive + * @param startDay The start date, inclusive * @return An interval between two dates. The interval can be negative * if the end date is before the start date. */ - def subtractDates(endDate: SQLDate, startDate: SQLDate): CalendarInterval = { - val period = Period.between( - LocalDate.ofEpochDay(startDate), - LocalDate.ofEpochDay(endDate)) - val months = period.getMonths + 12 * period.getYears + def subtractDates(endDay: Int, startDay: Int): CalendarInterval = { + val period = Period.between(daysToLocalDate(startDay), daysToLocalDate(endDay)) + val months = Math.toIntExact(period.toTotalMonths) val days = period.getDays new CalendarInterval(months, days, 0) } 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 f460404800264..8db95044359c3 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 @@ -180,7 +180,7 @@ class MicrosCalendar(tz: TimeZone, digitsInFraction: Int) // Converts parsed `MILLISECOND` field to seconds fraction in microsecond precision. // For example if the fraction pattern is `SSSS` then `digitsInFraction` = 4, and // if the `MILLISECOND` field was parsed to `1234`. - def getMicros(): SQLTimestamp = { + def getMicros(): Long = { // Append 6 zeros to the field: 1234 -> 1234000000 val d = fields(Calendar.MILLISECOND) * MICROS_PER_SECOND // Take the first 6 digits from `d`: 1234000000 -> 123400 @@ -209,7 +209,7 @@ class LegacyFastTimestampFormatter( fastDateFormat.getTimeZone, fastDateFormat.getPattern.count(_ == 'S')) - override def parse(s: String): SQLTimestamp = { + override def parse(s: String): Long = { cal.clear() // Clear the calendar because it can be re-used many times if (!fastDateFormat.parse(s, new ParsePosition(0), cal)) { throw new IllegalArgumentException(s"'$s' is an invalid timestamp") @@ -220,7 +220,7 @@ class LegacyFastTimestampFormatter( rebaseJulianToGregorianMicros(julianMicros) } - override def format(timestamp: SQLTimestamp): String = { + override def format(timestamp: Long): String = { val julianMicros = rebaseGregorianToJulianMicros(timestamp) cal.setTimeInMillis(Math.floorDiv(julianMicros, MICROS_PER_SECOND) * MILLIS_PER_SECOND) cal.setMicros(Math.floorMod(julianMicros, MICROS_PER_SECOND)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 35b4017980138..76ec450a4d7c6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, Collect import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -271,13 +270,13 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { for (tz <- ALL_TIMEZONES) { val timeZoneId = Option(tz.getId) - var c = Calendar.getInstance(TimeZoneGMT) + var c = Calendar.getInstance(TimeZoneUTC) c.set(2015, 2, 8, 2, 30, 0) checkEvaluation( cast(cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), TimestampType, timeZoneId), millisToMicros(c.getTimeInMillis)) - c = Calendar.getInstance(TimeZoneGMT) + c = Calendar.getInstance(TimeZoneUTC) c.set(2015, 10, 1, 2, 30, 0) checkEvaluation( cast(cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), @@ -294,7 +293,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(cast(nts, TimestampType, UTC_OPT), StringType, UTC_OPT), nts) checkEvaluation( cast(cast(ts, StringType, UTC_OPT), TimestampType, UTC_OPT), - DateTimeUtils.fromJavaTimestamp(ts)) + fromJavaTimestamp(ts)) // all convert to string type to check checkEvaluation( @@ -386,11 +385,11 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { checkEvaluation(cast(cast(tss, ShortType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + fromJavaTimestamp(ts) * MILLIS_PER_SECOND) checkEvaluation(cast(cast(tss, IntegerType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + fromJavaTimestamp(ts) * MILLIS_PER_SECOND) checkEvaluation(cast(cast(tss, LongType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + fromJavaTimestamp(ts) * MILLIS_PER_SECOND) checkEvaluation( cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), millis.toFloat / MILLIS_PER_SECOND) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala index 16d78b0526503..4a19add23fc58 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala @@ -73,7 +73,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P val schema = StructType(StructField("t", TimestampType) :: Nil) val csvData1 = "2016-01-01T00:00:00.123Z" - var c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT) + var c = Calendar.getInstance(DateTimeUtils.TimeZoneUTC) c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 123) checkEvaluation( @@ -184,7 +184,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P test("to_csv with timestamp") { val schema = StructType(StructField("t", TimestampType) :: Nil) - val c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT) + val c = Calendar.getInstance(DateTimeUtils.TimeZoneUTC) c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) val struct = Literal.create(create_row(c.getTimeInMillis * 1000L), schema) 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 f248a3454f39a..4edf95d8f994b 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 @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneUTC import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -803,7 +804,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd" val sdf3 = new SimpleDateFormat(fmt3, Locale.US) - sdf3.setTimeZone(TimeZone.getTimeZone(UTC)) + sdf3.setTimeZone(TimeZoneUTC) withDefaultTimeZone(UTC) { for (zid <- outstandingZoneIds) { @@ -872,7 +873,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd" val sdf3 = new SimpleDateFormat(fmt3, Locale.US) - sdf3.setTimeZone(TimeZone.getTimeZone(UTC)) + sdf3.setTimeZone(TimeZoneUTC) withDefaultTimeZone(UTC) { for (zid <- outstandingZoneIds) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index c40149368b055..0fea84bb183e0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -156,10 +156,10 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { "fromPrimitiveArray", ObjectType(classOf[Array[Int]]), Array[Int](1, 2, 3), UnsafeArrayData.fromPrimitiveArray(Array[Int](1, 2, 3))), (DateTimeUtils.getClass, ObjectType(classOf[Date]), - "toJavaDate", ObjectType(classOf[DateTimeUtils.SQLDate]), 77777, + "toJavaDate", ObjectType(classOf[Int]), 77777, DateTimeUtils.toJavaDate(77777)), (DateTimeUtils.getClass, ObjectType(classOf[Timestamp]), - "toJavaTimestamp", ObjectType(classOf[DateTimeUtils.SQLTimestamp]), + "toJavaTimestamp", ObjectType(classOf[Long]), 88888888.toLong, DateTimeUtils.toJavaTimestamp(88888888)) ).foreach { case (cls, dataType, methodName, argType, arg, expected) => checkObjectExprEvaluation(StaticInvoke(cls, dataType, methodName, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala index 2bb948ec24fb3..4b8693cf7fd53 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala @@ -26,8 +26,8 @@ import org.apache.spark.sql.types._ class JacksonGeneratorSuite extends SparkFunSuite { - val gmtId = DateTimeUtils.TimeZoneGMT.getID - val option = new JSONOptions(Map.empty, gmtId) + val utcId = DateTimeUtils.TimeZoneUTC.getID + val option = new JSONOptions(Map.empty, utcId) test("initial with StructType and write out a row") { val dataType = StructType(StructField("a", IntegerType) :: Nil) @@ -45,7 +45,7 @@ class JacksonGeneratorSuite extends SparkFunSuite { val input = InternalRow(null) val writer = new CharArrayWriter() val allowNullOption = - new JSONOptions(Map("ignoreNullFields" -> "false"), gmtId) + new JSONOptions(Map("ignoreNullFields" -> "false"), utcId) val gen = new JacksonGenerator(dataType, writer, allowNullOption) gen.write(input) gen.flush() @@ -59,7 +59,7 @@ class JacksonGeneratorSuite extends SparkFunSuite { val input = InternalRow(InternalRow(null)) val writer = new CharArrayWriter() val allowNullOption = - new JSONOptions(Map("ignoreNullFields" -> "false"), gmtId) + new JSONOptions(Map("ignoreNullFields" -> "false"), utcId) val gen = new JacksonGenerator(dataType, writer, allowNullOption) gen.write(input) gen.flush() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 4883bef8c0886..caf4b7e16f285 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -37,12 +37,12 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { private def defaultZoneId = ZoneId.systemDefault() test("nanoseconds truncation") { - val tf = TimestampFormatter.getFractionFormatter(DateTimeUtils.defaultTimeZone.toZoneId) + val tf = TimestampFormatter.getFractionFormatter(ZoneId.systemDefault()) def checkStringToTimestamp(originalTime: String, expectedParsedTime: String): Unit = { val parsedTimestampOp = DateTimeUtils.stringToTimestamp( UTF8String.fromString(originalTime), defaultZoneId) assert(parsedTimestampOp.isDefined, "timestamp with nanoseconds was not parsed correctly") - assert(DateTimeUtils.timestampToString(tf, parsedTimestampOp.get) === expectedParsedTime) + assert(tf.format(parsedTimestampOp.get) === expectedParsedTime) } checkStringToTimestamp("2015-01-02 00:00:00.123456789", "2015-01-02 00:00:00.123456") @@ -121,7 +121,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { checkFromToJavaDate(new Date(df2.parse("1776-07-04 18:30:00 UTC").getTime)) } - private def toDate(s: String, zoneId: ZoneId = UTC): Option[SQLDate] = { + private def toDate(s: String, zoneId: ZoneId = UTC): Option[Int] = { stringToDate(UTF8String.fromString(s), zoneId) } @@ -149,7 +149,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { assert(toDate("1999 08").isEmpty) } - private def toTimestamp(str: String, zoneId: ZoneId): Option[SQLTimestamp] = { + private def toTimestamp(str: String, zoneId: ZoneId): Option[Long] = { stringToTimestamp(UTF8String.fromString(str), zoneId) } @@ -520,7 +520,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { def testTrunc( level: Int, expected: String, - inputTS: SQLTimestamp, + inputTS: Long, zoneId: ZoneId = defaultZoneId): Unit = { val truncated = DateTimeUtils.truncTimestamp(inputTS, level, zoneId) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala index b78facd963338..3b9a4ae88d586 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala @@ -45,7 +45,7 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers private def assertEqual(pattern: String, datetimeStr: String, expected: Long): Unit = { if (useDateFormatter) { assert(dateFormatter(pattern).parse(datetimeStr) === - DateTimeUtils.microsToEpochDays(expected, UTC)) + DateTimeUtils.microsToDays(expected, UTC)) } else { assert(timestampFormatter(pattern).parse(datetimeStr) === expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index 9b43635a70701..b74a219540bb2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -78,7 +78,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { Math.floor(millisLocal.toDouble / MILLIS_PER_DAY).toInt } private def fromJavaDateLegacy(date: Date): Int = { - millisToDaysLegacy(date.getTime, defaultTimeZone()) + millisToDaysLegacy(date.getTime, TimeZone.getTimeZone(ZoneId.systemDefault())) } test("rebase gregorian to/from julian days") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala index 0ffb492e702d1..e6565feebf25c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala @@ -17,11 +17,12 @@ package org.apache.spark.sql.util +import java.time.ZoneId + import org.apache.arrow.vector.types.pojo.ArrowType import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.LA -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ class ArrowUtilsSuite extends SparkFunSuite { @@ -63,7 +64,7 @@ class ArrowUtilsSuite extends SparkFunSuite { assert(ArrowUtils.fromArrowSchema(arrowSchema) === schema) } - roundtripWithTz(DateTimeUtils.defaultTimeZone().getID) + roundtripWithTz(ZoneId.systemDefault().getId) roundtripWithTz("Asia/Tokyo") roundtripWithTz("UTC") roundtripWithTz(LA.getId) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index f5a474ddf3904..2f1ee0f23d45a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -203,7 +203,7 @@ private[sql] object JDBCRelation extends Logging { case TimestampType => val timestampFormatter = TimestampFormatter.getFractionFormatter( DateTimeUtils.getZoneId(timeZoneId)) - DateTimeUtils.timestampToString(timestampFormatter, value) + timestampFormatter.format(value) } s"'$dateTimeStr'" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 491977c61d3cb..39bbc60200b86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -34,7 +34,6 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} -import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLDate import org.apache.spark.sql.sources import org.apache.spark.unsafe.types.UTF8String @@ -124,7 +123,7 @@ class ParquetFilters( private val ParquetTimestampMicrosType = ParquetSchemaType(TIMESTAMP_MICROS, INT64, 0, null) private val ParquetTimestampMillisType = ParquetSchemaType(TIMESTAMP_MILLIS, INT64, 0, null) - private def dateToDays(date: Any): SQLDate = date match { + private def dateToDays(date: Any): Int = date match { case d: Date => DateTimeUtils.fromJavaDate(d) case ld: LocalDate => DateTimeUtils.localDateToDays(ld) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 201ee16faeb08..9a010d7192081 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -34,7 +34,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, CaseInsensitiveMap, DateTimeUtils, GenericArrayData} -import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLTimestamp import org.apache.spark.sql.execution.datasources.DataSourceUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy @@ -743,7 +742,7 @@ private[parquet] object ParquetRowConverter { unscaled } - def binaryToSQLTimestamp(binary: Binary): SQLTimestamp = { + def binaryToSQLTimestamp(binary: Binary): Long = { assert(binary.length() == 12, s"Timestamps (with nanoseconds) are expected to be stored in" + s" 12-byte long binaries. Found a ${binary.length()}-byte binary instead.") val buffer = binary.toByteBuffer.order(ByteOrder.LITTLE_ENDIAN) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index ac98d3f0c7095..cb11519497747 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} -import java.util.TimeZone import scala.util.Random @@ -28,8 +27,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation -import org.apache.spark.sql.catalyst.util.DateTimeTestUtils -import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, UTC} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -1029,7 +1027,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Timestamp.valueOf("2018-01-01 12:00:00"), Timestamp.valueOf("2018-01-02 00:00:00"))))) - DateTimeTestUtils.withDefaultTimeZone(UTC) { + withDefaultTimeZone(UTC) { checkAnswer( spark.sql("select sequence(" + " cast('2018-01-01' as date)" + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 91ec1b5ab2937..18356a4de9ef4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -28,7 +28,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneUTC import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -484,11 +485,11 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared DateTimeTestUtils.outstandingZoneIds.foreach { zid => val timeZone = TimeZone.getTimeZone(zid) - checkTimestampStats(DateType, DateTimeUtils.TimeZoneUTC, timeZone) { stats => + checkTimestampStats(DateType, TimeZoneUTC, timeZone) { stats => assert(stats.min.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(start)) assert(stats.max.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(end - 1)) } - checkTimestampStats(TimestampType, DateTimeUtils.TimeZoneUTC, timeZone) { stats => + checkTimestampStats(TimestampType, TimeZoneUTC, timeZone) { stats => assert(stats.min.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(start)) assert(stats.max.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(end - 1)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 3f8ee12f97776..6344ec6be4878 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -21,7 +21,7 @@ import java.io._ import java.nio.charset.{Charset, StandardCharsets, UnsupportedCharsetException} import java.nio.file.Files import java.sql.{Date, Timestamp} -import java.time.LocalDate +import java.time.{LocalDate, ZoneId} import java.util.Locale import com.fasterxml.jackson.core.JsonFactory @@ -125,7 +125,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ssXXX"))) val ISO8601Date = "1970-01-01" - checkTypePromotion(DateTimeUtils.microsToDays(32400000000L), + checkTypePromotion(DateTimeUtils.microsToDays(32400000000L, ZoneId.systemDefault), enforceCorrectType(ISO8601Date, DateType)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 32a9558e91f10..accd04592bec5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -21,7 +21,7 @@ import java.io.File import java.math.BigInteger import java.sql.{Date, Timestamp} import java.time.{ZoneId, ZoneOffset} -import java.util.{Calendar, Locale, TimeZone} +import java.util.{Calendar, Locale} import scala.collection.mutable.ArrayBuffer @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneUTC import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} @@ -88,7 +89,7 @@ abstract class ParquetPartitionDiscoverySuite check("1990-02-24 12:00:30", Literal.create(Timestamp.valueOf("1990-02-24 12:00:30"), TimestampType)) - val c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) + val c = Calendar.getInstance(TimeZoneUTC) c.set(1990, 1, 24, 12, 0, 30) c.set(Calendar.MILLISECOND, 0) check("1990-02-24 12:00:30", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 36fb418b09cb6..818a66eb436cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.sources +import java.time.ZoneId + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.DataSource @@ -27,7 +29,7 @@ class ResolvedDataSourceSuite extends SharedSparkSession { DataSource( sparkSession = spark, className = name, - options = Map(DateTimeUtils.TIMEZONE_OPTION -> DateTimeUtils.defaultTimeZone().getID) + options = Map(DateTimeUtils.TIMEZONE_OPTION -> ZoneId.systemDefault().getId) ).providingClass test("jdbc") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index bbd0220a74f88..030009572deb3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.streaming import java.io.{File, InterruptedIOException, IOException, UncheckedIOException} import java.nio.channels.ClosedByInterruptException +import java.time.ZoneId import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} import scala.concurrent.TimeoutException @@ -1219,7 +1220,8 @@ class StreamSuite extends StreamTest { } var lastTimestamp = System.currentTimeMillis() - val currentDate = DateTimeUtils.microsToDays(DateTimeUtils.millisToMicros(lastTimestamp)) + val currentDate = DateTimeUtils.microsToDays( + DateTimeUtils.millisToMicros(lastTimestamp), ZoneId.systemDefault) testStream(df) ( AddData(input, 1), CheckLastBatch { rows: Seq[Row] => From 86b54f3321e8125cf9d542571bb777b6aea92f9e Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Fri, 19 Jun 2020 05:56:50 +0000 Subject: [PATCH 040/384] [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store ### What changes were proposed in this pull request? Introduce UnsafeRow format validation for streaming state store. ### Why are the changes needed? Currently, Structured Streaming directly puts the UnsafeRow into StateStore without any schema validation. It's a dangerous behavior when users reusing the checkpoint file during migration. Any changes or bug fix related to the aggregate function may cause random exceptions, even the wrong answer, e.g SPARK-28067. ### Does this PR introduce _any_ user-facing change? Yes. If the underlying changes are detected when the checkpoint is reused during migration, the InvalidUnsafeRowException will be thrown. ### How was this patch tested? UT added. Will also add integrated tests for more scenario in another PR separately. Closes #28707 from xuanyuanking/SPARK-31894. Lead-authored-by: Yuanjian Li Co-authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/UnsafeRowUtils.scala | 86 +++++++++++++++++++ .../apache/spark/sql/internal/SQLConf.scala | 16 +++- .../catalyst/util/UnsafeRowUtilsSuite.scala | 55 ++++++++++++ .../state/HDFSBackedStateStoreProvider.scala | 13 +++ .../streaming/state/StateStore.scala | 33 ++++++- .../streaming/state/StateStoreConf.scala | 19 +++- .../streaming/state/StateStoreRDD.scala | 5 +- .../execution/streaming/state/package.scala | 6 +- .../streaming/statefulOperators.scala | 6 +- 9 files changed, 228 insertions(+), 11 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala new file mode 100644 index 0000000000000..37a34fac66364 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.types._ + +object UnsafeRowUtils { + + /** + * Use the following rules to check the integrity of the UnsafeRow: + * - schema.fields.length == row.numFields should always be true + * - UnsafeRow.calculateBitSetWidthInBytes(row.numFields) < row.getSizeInBytes should always be + * true if the expectedSchema contains at least one field. + * - For variable-length fields: if null bit says it's null then don't do anything, else extract + * offset and size: + * 1) 0 <= size < row.getSizeInBytes should always be true. We can be even more precise than + * this, where the upper bound of size can only be as big as the variable length part of + * the row. + * 2) offset should be >= fixed sized part of the row. + * 3) offset + size should be within the row bounds. + * - For fixed-length fields that are narrower than 8 bytes (boolean/byte/short/int/float), if + * null bit says it's null then don't do anything, else: + * check if the unused bits in the field are all zeros. The UnsafeRowWriter's write() methods + * make this guarantee. + * - Check the total length of the row. + */ + def validateStructuralIntegrity(row: UnsafeRow, expectedSchema: StructType): Boolean = { + if (expectedSchema.fields.length != row.numFields) { + return false + } + val bitSetWidthInBytes = UnsafeRow.calculateBitSetWidthInBytes(row.numFields) + val rowSizeInBytes = row.getSizeInBytes + if (expectedSchema.fields.length > 0 && bitSetWidthInBytes >= rowSizeInBytes) { + return false + } + var varLenFieldsSizeInBytes = 0 + expectedSchema.fields.zipWithIndex.foreach { + case (field, index) if !UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) => + val offsetAndSize = row.getLong(index) + val offset = (offsetAndSize >> 32).toInt + val size = offsetAndSize.toInt + if (size < 0 || + offset < bitSetWidthInBytes + 8 * row.numFields || offset + size > rowSizeInBytes) { + return false + } + varLenFieldsSizeInBytes += size + case (field, index) if UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) => + field.dataType match { + case BooleanType => + if ((row.getLong(index) >> 1) != 0L) return false + case ByteType => + if ((row.getLong(index) >> 8) != 0L) return false + case ShortType => + if ((row.getLong(index) >> 16) != 0L) return false + case IntegerType => + if ((row.getLong(index) >> 32) != 0L) return false + case FloatType => + if ((row.getLong(index) >> 32) != 0L) return false + case _ => + } + case (_, index) if row.isNullAt(index) => + if (row.getLong(index) != 0L) return false + case _ => + } + if (bitSetWidthInBytes + 8 * row.numFields + varLenFieldsSizeInBytes > rowSizeInBytes) { + return false + } + true + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7f63d79a21ed6..6bbeb2de7538c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1237,6 +1237,16 @@ object SQLConf { .intConf .createWithDefault(10) + val STATE_STORE_FORMAT_VALIDATION_ENABLED = + buildConf("spark.sql.streaming.stateStore.formatValidation.enabled") + .internal() + .doc("When true, check if the UnsafeRow from the state store is valid or not when running " + + "streaming queries. This can happen if the state store format has been changed. Note, " + + "the feature is only effective in the build-in HDFS state store provider now.") + .version("3.1.0") + .booleanConf + .createWithDefault(true) + val FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION = buildConf("spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion") .internal() @@ -1543,18 +1553,18 @@ object SQLConf { val STREAMING_CHECKPOINT_FILE_MANAGER_CLASS = buildConf("spark.sql.streaming.checkpointFileManagerClass") + .internal() .doc("The class used to write checkpoint files atomically. This class must be a subclass " + "of the interface CheckpointFileManager.") .version("2.4.0") - .internal() .stringConf val STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED = buildConf("spark.sql.streaming.checkpoint.escapedPathCheck.enabled") + .internal() .doc("Whether to detect a streaming query may pick up an incorrect checkpoint path due " + "to SPARK-26824.") .version("3.0.0") - .internal() .booleanConf .createWithDefault(true) @@ -2746,6 +2756,8 @@ class SQLConf extends Serializable with Logging { def stateStoreMinDeltasForSnapshot: Int = getConf(STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT) + def stateStoreFormatValidationEnabled: Boolean = getConf(STATE_STORE_FORMAT_VALIDATION_ENABLED) + def checkpointLocation: Option[String] = getConf(CHECKPOINT_LOCATION) def isUnsupportedOperationCheckEnabled: Boolean = getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala new file mode 100644 index 0000000000000..4b6a3cfafd894 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + +class UnsafeRowUtilsSuite extends SparkFunSuite { + + val testKeys: Seq[String] = Seq("key1", "key2") + val testValues: Seq[String] = Seq("sum(key1)", "sum(key2)") + + val testOutputSchema: StructType = StructType( + testKeys.map(createIntegerField) ++ testValues.map(createIntegerField)) + + val testRow: UnsafeRow = { + val unsafeRowProjection = UnsafeProjection.create(testOutputSchema) + val row = unsafeRowProjection(new SpecificInternalRow(testOutputSchema)) + (testKeys ++ testValues).zipWithIndex.foreach { case (_, index) => row.setInt(index, index) } + row + } + + private def createIntegerField(name: String): StructField = { + StructField(name, IntegerType, nullable = false) + } + + test("UnsafeRow format invalidation") { + // Pass the checking + UnsafeRowUtils.validateStructuralIntegrity(testRow, testOutputSchema) + // Fail for fields number not match + assert(!UnsafeRowUtils.validateStructuralIntegrity( + testRow, StructType(testKeys.map(createIntegerField)))) + // Fail for invalid schema + val invalidSchema = StructType(testKeys.map(createIntegerField) ++ + Seq(StructField("struct", StructType(Seq(StructField("value1", StringType, true))), true), + StructField("value2", IntegerType, false))) + assert(!UnsafeRowUtils.validateStructuralIntegrity(testRow, invalidSchema)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 05c651f9951b9..31618922e44cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -259,6 +259,9 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit @volatile private var storeConf: StateStoreConf = _ @volatile private var hadoopConf: Configuration = _ @volatile private var numberOfVersionsToRetainInMemory: Int = _ + // TODO: The validation should be moved to a higher level so that it works for all state store + // implementations + @volatile private var isValidated = false private lazy val loadedMaps = new util.TreeMap[Long, MapType](Ordering[Long].reverse) private lazy val baseDir = stateStoreId.storeCheckpointLocation() @@ -457,6 +460,11 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit // Prior to Spark 2.3 mistakenly append 4 bytes to the value row in // `RowBasedKeyValueBatch`, which gets persisted into the checkpoint data valueRow.pointTo(valueRowBuffer, (valueSize / 8) * 8) + if (!isValidated) { + StateStoreProvider.validateStateRowFormat( + keyRow, keySchema, valueRow, valueSchema, storeConf) + isValidated = true + } map.put(keyRow, valueRow) } } @@ -551,6 +559,11 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit // Prior to Spark 2.3 mistakenly append 4 bytes to the value row in // `RowBasedKeyValueBatch`, which gets persisted into the checkpoint data valueRow.pointTo(valueRowBuffer, (valueSize / 8) * 8) + if (!isValidated) { + StateStoreProvider.validateStateRowFormat( + keyRow, keySchema, valueRow, valueSchema, storeConf) + isValidated = true + } map.put(keyRow, valueRow) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 7d80fd0c591f9..092ca968f59c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -27,9 +27,10 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{SparkContext, SparkEnv} +import org.apache.spark.{SparkContext, SparkEnv, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.util.UnsafeRowUtils import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo import org.apache.spark.sql.types.StructType import org.apache.spark.util.{ThreadUtils, Utils} @@ -143,6 +144,16 @@ case class StateStoreCustomSumMetric(name: String, desc: String) extends StateSt case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric +/** + * An exception thrown when an invalid UnsafeRow is detected in state store. + */ +class InvalidUnsafeRowException + extends RuntimeException("The streaming query failed by state format invalidation. " + + "The following reasons may cause this: 1. An old Spark version wrote the checkpoint that is " + + "incompatible with the current one; 2. Broken checkpoint files; 3. The query is changed " + + "among restart. For the first case, you can try to restart the application without " + + "checkpoint or use the legacy Spark version to process the streaming state.", null) + /** * Trait representing a provider that provide [[StateStore]] instances representing * versions of state data. @@ -230,6 +241,26 @@ object StateStoreProvider { provider.init(stateStoreId, keySchema, valueSchema, indexOrdinal, storeConf, hadoopConf) provider } + + /** + * Use the expected schema to check whether the UnsafeRow is valid. + */ + def validateStateRowFormat( + keyRow: UnsafeRow, + keySchema: StructType, + valueRow: UnsafeRow, + valueSchema: StructType, + conf: StateStoreConf): Unit = { + if (conf.formatValidationEnabled) { + if (!UnsafeRowUtils.validateStructuralIntegrity(keyRow, keySchema)) { + throw new InvalidUnsafeRowException + } + if (conf.formatValidationCheckValue && + !UnsafeRowUtils.validateStructuralIntegrity(valueRow, valueSchema)) { + throw new InvalidUnsafeRowException + } + } + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala index d145082a39b57..84d0b76ac9158 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql.execution.streaming.state import org.apache.spark.sql.internal.SQLConf /** A class that contains configuration parameters for [[StateStore]]s. */ -class StateStoreConf(@transient private val sqlConf: SQLConf) +class StateStoreConf( + @transient private val sqlConf: SQLConf, + extraOptions: Map[String, String] = Map.empty) extends Serializable { def this() = this(new SQLConf) @@ -43,14 +45,25 @@ class StateStoreConf(@transient private val sqlConf: SQLConf) */ val providerClass: String = sqlConf.stateStoreProviderClass + /** Whether validate the underlying format or not. */ + val formatValidationEnabled: Boolean = sqlConf.stateStoreFormatValidationEnabled + + /** Whether validate the value format when the format invalidation enabled. */ + val formatValidationCheckValue: Boolean = + extraOptions.getOrElse(StateStoreConf.FORMAT_VALIDATION_CHECK_VALUE_CONFIG, "true") == "true" + /** * Additional configurations related to state store. This will capture all configs in - * SQLConf that start with `spark.sql.streaming.stateStore.` */ + * SQLConf that start with `spark.sql.streaming.stateStore.` and extraOptions for a specific + * operator. + */ val confs: Map[String, String] = - sqlConf.getAllConfs.filter(_._1.startsWith("spark.sql.streaming.stateStore.")) + sqlConf.getAllConfs.filter(_._1.startsWith("spark.sql.streaming.stateStore.")) ++ extraOptions } object StateStoreConf { + val FORMAT_VALIDATION_CHECK_VALUE_CONFIG = "formatValidationCheckValue" + val empty = new StateStoreConf() def apply(conf: SQLConf): StateStoreConf = new StateStoreConf(conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala index 4a69a48fed75f..0eb3dce1bbd27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala @@ -45,10 +45,11 @@ class StateStoreRDD[T: ClassTag, U: ClassTag]( valueSchema: StructType, indexOrdinal: Option[Int], sessionState: SessionState, - @transient private val storeCoordinator: Option[StateStoreCoordinatorRef]) + @transient private val storeCoordinator: Option[StateStoreCoordinatorRef], + extraOptions: Map[String, String] = Map.empty) extends RDD[U](dataRDD) { - private val storeConf = new StateStoreConf(sessionState.conf) + private val storeConf = new StateStoreConf(sessionState.conf, extraOptions) // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it private val hadoopConfBroadcast = dataRDD.context.broadcast( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala index b6021438e902b..c7a332b6d778e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala @@ -55,7 +55,8 @@ package object state { valueSchema: StructType, indexOrdinal: Option[Int], sessionState: SessionState, - storeCoordinator: Option[StateStoreCoordinatorRef])( + storeCoordinator: Option[StateStoreCoordinatorRef], + extraOptions: Map[String, String] = Map.empty)( storeUpdateFunction: (StateStore, Iterator[T]) => Iterator[U]): StateStoreRDD[T, U] = { val cleanedF = dataRDD.sparkContext.clean(storeUpdateFunction) @@ -78,7 +79,8 @@ package object state { valueSchema, indexOrdinal, sessionState, - storeCoordinator) + storeCoordinator, + extraOptions) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index 74daaf80b10e8..a9c01e69b9b13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.streaming.state._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, StateOperatorProgress} import org.apache.spark.sql.types._ import org.apache.spark.util.{CompletionIterator, NextIterator, Utils} @@ -460,7 +461,10 @@ case class StreamingDeduplicateExec( child.output.toStructType, indexOrdinal = None, sqlContext.sessionState, - Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) => + Some(sqlContext.streams.stateStoreCoordinator), + // We won't check value row in state store since the value StreamingDeduplicateExec.EMPTY_ROW + // is unrelated to the output schema. + Map(StateStoreConf.FORMAT_VALIDATION_CHECK_VALUE_CONFIG -> "false")) { (store, iter) => val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) val numOutputRows = longMetric("numOutputRows") val numTotalStateRows = longMetric("numTotalStateRows") From abc8ccc37be60df3325791a11d6f0c52050f2271 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 19 Jun 2020 05:58:54 +0000 Subject: [PATCH 041/384] [SPARK-31926][SQL][TESTS][FOLLOWUP][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber ### What changes were proposed in this pull request? This PR brings https://github.com/apache/spark/pull/28751 back - It once reverted by 4a25200 because of inevitable maven test failure - See related updates in this followup https://github.com/apache/spark/commit/a0187cd6b59a6b6bb2cadc6711bb663d4d35a844 - And reverted again because of the flakiness of the added unit tests - In this PR, The flakiness reason found is caused by the hive metastore connection that the SparkSQLCLIService trying to create which turns out is unnecessary at all. This metastore client points to a dummy metastore server only. - Also, add some cleanups for SharedThriftServer trait in before and after to prevent its configurations being polluted or polluting others ### Why are the changes needed? fix flaky test ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? passing sbt and maven tests Closes #28835 from yaooqinn/SPARK-31926-F. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- project/SparkBuild.scala | 1 - .../thriftserver/SparkSQLCLIService.scala | 50 ++++++++++++-- .../src/test/resources/log4j.properties | 65 +++++++++++++++++++ .../thriftserver/SharedThriftServer.scala | 60 ++++++++++++----- .../ThriftServerQueryTestSuite.scala | 3 + .../ThriftServerWithSparkContextSuite.scala | 11 +++- .../cli/thrift/ThriftBinaryCLIService.java | 11 +++- .../service/cli/thrift/ThriftCLIService.java | 3 + .../cli/thrift/ThriftHttpCLIService.java | 21 ++++-- .../cli/thrift/ThriftBinaryCLIService.java | 11 +++- .../service/cli/thrift/ThriftCLIService.java | 3 + .../cli/thrift/ThriftHttpCLIService.java | 21 ++++-- 12 files changed, 224 insertions(+), 36 deletions(-) create mode 100644 sql/hive-thriftserver/src/test/resources/log4j.properties diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0035f1d95a90d..04a3fc4b63050 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -480,7 +480,6 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" ) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 1644ecb2453be..984625c76e057 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -22,13 +22,14 @@ import java.util.{List => JList} import javax.security.auth.login.LoginException import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.commons.logging.Log import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.shims.Utils import org.apache.hadoop.security.{SecurityUtil, UserGroupInformation} -import org.apache.hive.service.{AbstractService, Service, ServiceException} +import org.apache.hive.service.{AbstractService, CompositeService, Service, ServiceException} import org.apache.hive.service.Service.STATE import org.apache.hive.service.auth.HiveAuthFactory import org.apache.hive.service.cli._ @@ -94,6 +95,12 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLC initCompositeService(hiveConf) } + /** + * the super class [[CLIService#start]] starts a useless dummy metastore client, skip it and call + * the ancestor [[CompositeService#start]] directly. + */ + override def start(): Unit = startCompositeService() + override def getInfo(sessionHandle: SessionHandle, getInfoType: GetInfoType): GetInfoValue = { getInfoType match { case GetInfoType.CLI_SERVER_NAME => new GetInfoValue("Spark SQL") @@ -105,6 +112,19 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLC } private[thriftserver] trait ReflectedCompositeService { this: AbstractService => + + private val logInfo = (msg: String) => if (HiveUtils.isHive23) { + getAncestorField[Logger](this, 3, "LOG").info(msg) + } else { + getAncestorField[Log](this, 3, "LOG").info(msg) + } + + private val logError = (msg: String, e: Throwable) => if (HiveUtils.isHive23) { + getAncestorField[Logger](this, 3, "LOG").error(msg, e) + } else { + getAncestorField[Log](this, 3, "LOG").error(msg, e) + } + def initCompositeService(hiveConf: HiveConf): Unit = { // Emulating `CompositeService.init(hiveConf)` val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") @@ -114,10 +134,30 @@ private[thriftserver] trait ReflectedCompositeService { this: AbstractService => invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) setAncestorField(this, 3, "hiveConf", hiveConf) invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) - if (HiveUtils.isHive23) { - getAncestorField[Logger](this, 3, "LOG").info(s"Service: $getName is inited.") - } else { - getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") + logInfo(s"Service: $getName is inited.") + } + + def startCompositeService(): Unit = { + // Emulating `CompositeService.start` + val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") + var serviceStartCount = 0 + try { + serviceList.asScala.foreach { service => + service.start() + serviceStartCount += 1 + } + // Emulating `AbstractService.start` + val startTime = new java.lang.Long(System.currentTimeMillis()) + setAncestorField(this, 3, "startTime", startTime) + invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.INITED) + invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.STARTED) + logInfo(s"Service: $getName is started.") + } catch { + case NonFatal(e) => + logError(s"Error starting services $getName", e) + invoke(classOf[CompositeService], this, "stop", + classOf[Int] -> new Integer(serviceStartCount)) + throw new ServiceException("Failed to Start " + getName, e) } } } diff --git a/sql/hive-thriftserver/src/test/resources/log4j.properties b/sql/hive-thriftserver/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..21975ba818142 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/log4j.properties @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file hive-thriftserver/target/unit-tests.log +log4j.rootLogger=DEBUG, CA, FA + +#Console Appender +log4j.appender.CA=org.apache.log4j.ConsoleAppender +log4j.appender.CA.layout=org.apache.log4j.PatternLayout +log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n +log4j.appender.CA.Threshold = WARN + + +#File Appender +log4j.appender.FA=org.apache.log4j.FileAppender +log4j.appender.FA.append=false +log4j.appender.FA.file=target/unit-tests.log +log4j.appender.FA.layout=org.apache.log4j.PatternLayout +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Set the logger level of File Appender to WARN +log4j.appender.FA.Threshold = DEBUG + +# Some packages are noisy for no good reason. +log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false +log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF + +log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false +log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF + +log4j.additivity.hive.log=false +log4j.logger.hive.log=OFF + +log4j.additivity.parquet.hadoop.ParquetRecordReader=false +log4j.logger.parquet.hadoop.ParquetRecordReader=OFF + +log4j.additivity.org.apache.parquet.hadoop.ParquetRecordReader=false +log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF + +log4j.additivity.org.apache.parquet.hadoop.ParquetOutputCommitter=false +log4j.logger.org.apache.parquet.hadoop.ParquetOutputCommitter=OFF + +log4j.additivity.hive.ql.metadata.Hive=false +log4j.logger.hive.ql.metadata.Hive=OFF + +log4j.additivity.org.apache.hadoop.hive.ql.io.RCFile=false +log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR + +# Parquet related logging +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index e002bc0117c8b..df86cdef3a337 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -24,6 +24,8 @@ import scala.concurrent.duration._ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftCLIService import org.apache.spark.sql.test.SharedSparkSession @@ -33,6 +35,8 @@ trait SharedThriftServer extends SharedSparkSession { private var hiveServer2: HiveThriftServer2 = _ private var serverPort: Int = 0 + def mode: ServerMode.Value + override def beforeAll(): Unit = { super.beforeAll() // Retries up to 3 times with different port numbers if the server fails to start @@ -47,17 +51,27 @@ trait SharedThriftServer extends SharedSparkSession { override def afterAll(): Unit = { try { - hiveServer2.stop() + if (hiveServer2 != null) { + hiveServer2.stop() + } } finally { super.afterAll() + SessionState.detachSession() + Hive.closeCurrent() } } + protected def jdbcUri: String = if (mode == ServerMode.http) { + s"jdbc:hive2://localhost:$serverPort/default;transportMode=http;httpPath=cliservice" + } else { + s"jdbc:hive2://localhost:$serverPort/" + } + protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") val connections = - fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } + fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } val statements = connections.map(_.createStatement()) try { @@ -69,23 +83,39 @@ trait SharedThriftServer extends SharedSparkSession { } private def startThriftServer(attempt: Int): Unit = { - logInfo(s"Trying to start HiveThriftServer2:, attempt=$attempt") + logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt") val sqlContext = spark.newSession().sqlContext - // Set the HIVE_SERVER2_THRIFT_PORT to 0, so it could randomly pick any free port to use. + // Set the HIVE_SERVER2_THRIFT_PORT and HIVE_SERVER2_THRIFT_HTTP_PORT to 0, so it could + // randomly pick any free port to use. // It's much more robust than set a random port generated by ourselves ahead sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") - hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) - hiveServer2.getServices.asScala.foreach { - case t: ThriftCLIService if t.getPortNumber != 0 => - serverPort = t.getPortNumber - logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt") - case _ => - } + sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") + sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) + + try { + hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) + hiveServer2.getServices.asScala.foreach { + case t: ThriftCLIService => + serverPort = t.getPortNumber + logInfo(s"Started HiveThriftServer2: mode=$mode, port=$serverPort, attempt=$attempt") + case _ => + } - // Wait for thrift server to be ready to serve the query, via executing simple query - // till the query succeeds. See SPARK-30345 for more details. - eventually(timeout(30.seconds), interval(1.seconds)) { - withJdbcStatement { _.execute("SELECT 1") } + // Wait for thrift server to be ready to serve the query, via executing simple query + // till the query succeeds. See SPARK-30345 for more details. + eventually(timeout(30.seconds), interval(1.seconds)) { + withJdbcStatement { _.execute("SELECT 1") } + } + } catch { + case e: Exception => + logError("Error start hive server with Context ", e) + if (hiveServer2 != null) { + hiveServer2.stop() + hiveServer2 = null + } + SessionState.detachSession() + Hive.closeCurrent() + throw e } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 9b68144b52005..686dc1c9bad6b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -54,6 +54,9 @@ import org.apache.spark.sql.types._ */ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer { + + override def mode: ServerMode.Value = ServerMode.binary + override protected def testFile(fileName: String): String = { val url = Thread.currentThread().getContextClassLoader.getResource(fileName) // Copy to avoid URISyntaxException during accessing the resources in `sql/core` diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 3e1fce78ae71c..d6420dee41adb 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -class ThriftServerWithSparkContextSuite extends SharedThriftServer { +trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("SPARK-29911: Uncache cached tables when session closed") { val cacheManager = spark.sharedState.cacheManager @@ -42,3 +42,12 @@ class ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } + + +class ThriftServerWithSparkContextInBinarySuite extends ThriftServerWithSparkContextSuite { + override def mode: ServerMode.Value = ServerMode.binary +} + +class ThriftServerWithSparkContextInHttpSuite extends ThriftServerWithSparkContextSuite { + override def mode: ServerMode.Value = ServerMode.http +} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index e1ee503b81209..00bdf7e19126e 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -45,7 +46,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - public void run() { + protected void initializeServer() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -100,6 +101,14 @@ public void run() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + @Override + public void run() { + try { server.serve(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 8fce9d9383438..783e5795aca76 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -175,6 +175,7 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { + initializeServer(); new Thread(this).start(); isStarted = true; } @@ -633,6 +634,8 @@ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { return resp; } + protected abstract void initializeServer(); + @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 1099a00b67eb7..bd64c777c1d76 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.cli.thrift.TCLIService.Iface; @@ -53,13 +54,8 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ @Override - public void run() { + protected void initializeServer() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -150,6 +146,19 @@ public void run() { + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { httpServer.join(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index a7de9c0f3d0d2..ce79e3c8228a6 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -46,7 +47,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - public void run() { + protected void initializeServer() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -101,6 +102,14 @@ public void run() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + @Override + public void run() { + try { server.serve(); } catch (Throwable t) { LOG.error( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index d41c3b493bb47..e46799a1c427d 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -176,6 +176,7 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { + initializeServer(); new Thread(this).start(); isStarted = true; } @@ -670,6 +671,8 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) return resp; } + protected abstract void initializeServer(); + @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 73d5f84476af0..ab9ed5b1f371e 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.rpc.thrift.TCLIService; @@ -54,13 +55,8 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ @Override - public void run() { + protected void initializeServer() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -151,6 +147,19 @@ public void run() { + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { httpServer.join(); } catch (Throwable t) { LOG.error( From 6fe3bf66eb03892aeee5d8dcf118bc2e7446c96b Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 19 Jun 2020 06:01:06 +0000 Subject: [PATCH 042/384] [SPARK-31993][SQL] Build arrays for passing variables generated from children for 'concat_ws' with columns having at least one of array type ### What changes were proposed in this pull request? Please refer the next section `Why are the changes needed?` for details how the current implementation of `concat_ws` is broken for some condition. This patch fixes the code generation logic for columns having at least one array types of columns in `concat_ws` to build two arrays for storing isNull and value from children's generated code and pass these arrays to the both varargCounts and varargBuilds. This change guarantees that both varargCounts and varargBuilds can access the relevant local variables the children's generated code makes as array parameters, which is critical to ensure both varargCounts and varargBuilds succeed to compile. Below is the generated code for newly added UT, `SPARK-31993: concat_ws in agg function with plenty of string/array types columns`. > before the patch ``` /* 001 */ public java.lang.Object generate(Object[] references) { /* 002 */ return new SpecificUnsafeProjection(references); /* 003 */ } /* 004 */ /* 005 */ class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection { /* 006 */ /* 007 */ private Object[] references; /* 008 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1]; /* 009 */ /* 010 */ public SpecificUnsafeProjection(Object[] references) { /* 011 */ this.references = references; /* 012 */ mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32); /* 013 */ /* 014 */ } /* 015 */ /* 016 */ public void initialize(int partitionIndex) { /* 017 */ /* 018 */ } /* 019 */ /* 020 */ // Scala.Function1 need this /* 021 */ public java.lang.Object apply(java.lang.Object row) { /* 022 */ return apply((InternalRow) row); /* 023 */ } /* 024 */ /* 025 */ public UnsafeRow apply(InternalRow i) { /* 026 */ mutableStateArray_0[0].reset(); /* 027 */ /* 028 */ /* 029 */ mutableStateArray_0[0].zeroOutNullBytes(); /* 030 */ /* 031 */ apply_0_0(i); /* 032 */ apply_0_1(i); /* 033 */ int varargNum_0 = 30; /* 034 */ int idxInVararg_0 = 0; /* 035 */ /* 036 */ if (!isNull_2) { /* 037 */ varargNum_0 += value_2.numElements(); /* 038 */ } /* 039 */ /* 040 */ if (!isNull_3) { /* 041 */ varargNum_0 += value_3.numElements(); /* 042 */ } /* 043 */ /* 044 */ UTF8String[] array_0 = new UTF8String[varargNum_0]; /* 045 */ idxInVararg_0 = varargBuildsConcatWs_0_0(i, array_0, idxInVararg_0); /* 046 */ idxInVararg_0 = varargBuildsConcatWs_0_1(i, array_0, idxInVararg_0); /* 047 */ idxInVararg_0 = varargBuildsConcatWs_0_2(i, array_0, idxInVararg_0); /* 048 */ UTF8String value_0 = UTF8String.concatWs(((UTF8String) references[0] /* literal */), array_0); /* 049 */ boolean isNull_0 = value_0 == null; /* 050 */ mutableStateArray_0[0].write(0, value_0); /* 051 */ return (mutableStateArray_0[0].getRow()); /* 052 */ } /* 053 */ /* 054 */ /* 055 */ private void apply_0_1(InternalRow i) { /* 056 */ UTF8String value_25 = i.getUTF8String(22);UTF8String value_26 = i.getUTF8String(23);UTF8String value_27 = i.getUTF8String(24);UTF8String value_28 = i.getUTF8String(25);UTF8String value_29 = i.getUTF8String(26);UTF8String value_30 = i.getUTF8String(27);UTF8String value_31 = i.getUTF8String(28);UTF8String value_32 = i.getUTF8String(29);UTF8String value_33 = i.getUTF8String(30); /* 057 */ } /* 058 */ /* 059 */ /* 060 */ private int varargBuildsConcatWs_0_0(InternalRow i, UTF8String [] array_0, int idxInVararg_0) { /* 061 */ /* 062 */ /* 063 */ if (!isNull_2) { /* 064 */ final int n_0 = value_2.numElements(); /* 065 */ for (int j = 0; j < n_0; j ++) { /* 066 */ array_0[idxInVararg_0 ++] = value_2.getUTF8String(j); /* 067 */ } /* 068 */ } /* 069 */ /* 070 */ if (!isNull_3) { /* 071 */ final int n_1 = value_3.numElements(); /* 072 */ for (int j = 0; j < n_1; j ++) { /* 073 */ array_0[idxInVararg_0 ++] = value_3.getUTF8String(j); /* 074 */ } /* 075 */ } /* 076 */ array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_4;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_5;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_6;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_7;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_8;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_9;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_10;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_11; /* 077 */ return idxInVararg_0; /* 078 */ /* 079 */ } /* 080 */ /* 081 */ /* 082 */ private int varargBuildsConcatWs_0_2(InternalRow i, UTF8String [] array_0, int idxInVararg_0) { /* 083 */ /* 084 */ array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_28;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_29;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_30;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_31;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_32;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_33; /* 085 */ return idxInVararg_0; /* 086 */ /* 087 */ } /* 088 */ /* 089 */ /* 090 */ private void apply_0_0(InternalRow i) { /* 091 */ boolean isNull_2 = i.isNullAt(31); /* 092 */ ArrayData value_2 = isNull_2 ? /* 093 */ null : (i.getArray(31));boolean isNull_3 = i.isNullAt(32); /* 094 */ ArrayData value_3 = isNull_3 ? /* 095 */ null : (i.getArray(32));UTF8String value_4 = i.getUTF8String(1);UTF8String value_5 = i.getUTF8String(2);UTF8String value_6 = i.getUTF8String(3);UTF8String value_7 = i.getUTF8String(4);UTF8String value_8 = i.getUTF8String(5);UTF8String value_9 = i.getUTF8String(6);UTF8String value_10 = i.getUTF8String(7);UTF8String value_11 = i.getUTF8String(8);UTF8String value_12 = i.getUTF8String(9);UTF8String value_13 = i.getUTF8String(10);UTF8String value_14 = i.getUTF8String(11);UTF8String value_15 = i.getUTF8String(12);UTF8String value_16 = i.getUTF8String(13);UTF8String value_17 = i.getUTF8String(14);UTF8String value_18 = i.getUTF8String(15);UTF8String value_19 = i.getUTF8String(16);UTF8String value_20 = i.getUTF8String(17);UTF8String value_21 = i.getUTF8String(18);UTF8String value_22 = i.getUTF8String(19);UTF8String value_23 = i.getUTF8String(20);UTF8String value_24 = i.getUTF8String(21); /* 096 */ } /* 097 */ /* 098 */ /* 099 */ private int varargBuildsConcatWs_0_1(InternalRow i, UTF8String [] array_0, int idxInVararg_0) { /* 100 */ /* 101 */ array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_12;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_13;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_14;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_15;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_16;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_17;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_18;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_19;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_20;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_21;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_22;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_23;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_24;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_25;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_26;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_27; /* 102 */ return idxInVararg_0; /* 103 */ /* 104 */ } /* 105 */ /* 106 */ } ``` Compilation of the generated code fails with error message: `org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 36, Column 6: Expression "isNull_2" is not an rvalue` > after the patch ``` /* 001 */ public java.lang.Object generate(Object[] references) { /* 002 */ return new SpecificUnsafeProjection(references); /* 003 */ } /* 004 */ /* 005 */ class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection { /* 006 */ /* 007 */ private Object[] references; /* 008 */ private boolean globalIsNull_0; /* 009 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1]; /* 010 */ /* 011 */ public SpecificUnsafeProjection(Object[] references) { /* 012 */ this.references = references; /* 013 */ /* 014 */ mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32); /* 015 */ /* 016 */ } /* 017 */ /* 018 */ public void initialize(int partitionIndex) { /* 019 */ /* 020 */ } /* 021 */ /* 022 */ // Scala.Function1 need this /* 023 */ public java.lang.Object apply(java.lang.Object row) { /* 024 */ return apply((InternalRow) row); /* 025 */ } /* 026 */ /* 027 */ public UnsafeRow apply(InternalRow i) { /* 028 */ mutableStateArray_0[0].reset(); /* 029 */ /* 030 */ /* 031 */ mutableStateArray_0[0].zeroOutNullBytes(); /* 032 */ /* 033 */ UTF8String value_34 = ConcatWs_0(i); /* 034 */ mutableStateArray_0[0].write(0, value_34); /* 035 */ return (mutableStateArray_0[0].getRow()); /* 036 */ } /* 037 */ /* 038 */ /* 039 */ private void initializeArgsArrays_0_0(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 040 */ /* 041 */ boolean isNull_2 = i.isNullAt(31); /* 042 */ ArrayData value_2 = isNull_2 ? /* 043 */ null : (i.getArray(31)); /* 044 */ isNullArgs_0[0] = isNull_2; /* 045 */ valueArgs_0[0] = value_2; /* 046 */ /* 047 */ boolean isNull_3 = i.isNullAt(32); /* 048 */ ArrayData value_3 = isNull_3 ? /* 049 */ null : (i.getArray(32)); /* 050 */ isNullArgs_0[1] = isNull_3; /* 051 */ valueArgs_0[1] = value_3; /* 052 */ /* 053 */ UTF8String value_4 = i.getUTF8String(1); /* 054 */ isNullArgs_0[2] = false; /* 055 */ valueArgs_0[2] = value_4; /* 056 */ /* 057 */ UTF8String value_5 = i.getUTF8String(2); /* 058 */ isNullArgs_0[3] = false; /* 059 */ valueArgs_0[3] = value_5; /* 060 */ /* 061 */ UTF8String value_6 = i.getUTF8String(3); /* 062 */ isNullArgs_0[4] = false; /* 063 */ valueArgs_0[4] = value_6; /* 064 */ /* 065 */ UTF8String value_7 = i.getUTF8String(4); /* 066 */ isNullArgs_0[5] = false; /* 067 */ valueArgs_0[5] = value_7; /* 068 */ /* 069 */ UTF8String value_8 = i.getUTF8String(5); /* 070 */ isNullArgs_0[6] = false; /* 071 */ valueArgs_0[6] = value_8; /* 072 */ /* 073 */ } /* 074 */ /* 075 */ /* 076 */ private void initializeArgsArrays_0_3(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 077 */ /* 078 */ UTF8String value_25 = i.getUTF8String(22); /* 079 */ isNullArgs_0[23] = false; /* 080 */ valueArgs_0[23] = value_25; /* 081 */ /* 082 */ UTF8String value_26 = i.getUTF8String(23); /* 083 */ isNullArgs_0[24] = false; /* 084 */ valueArgs_0[24] = value_26; /* 085 */ /* 086 */ UTF8String value_27 = i.getUTF8String(24); /* 087 */ isNullArgs_0[25] = false; /* 088 */ valueArgs_0[25] = value_27; /* 089 */ /* 090 */ UTF8String value_28 = i.getUTF8String(25); /* 091 */ isNullArgs_0[26] = false; /* 092 */ valueArgs_0[26] = value_28; /* 093 */ /* 094 */ UTF8String value_29 = i.getUTF8String(26); /* 095 */ isNullArgs_0[27] = false; /* 096 */ valueArgs_0[27] = value_29; /* 097 */ /* 098 */ UTF8String value_30 = i.getUTF8String(27); /* 099 */ isNullArgs_0[28] = false; /* 100 */ valueArgs_0[28] = value_30; /* 101 */ /* 102 */ UTF8String value_31 = i.getUTF8String(28); /* 103 */ isNullArgs_0[29] = false; /* 104 */ valueArgs_0[29] = value_31; /* 105 */ /* 106 */ UTF8String value_32 = i.getUTF8String(29); /* 107 */ isNullArgs_0[30] = false; /* 108 */ valueArgs_0[30] = value_32; /* 109 */ /* 110 */ } /* 111 */ /* 112 */ /* 113 */ private int varargBuildsConcatWs_0_3(InternalRow i, UTF8String [] array_0, int idxInVararg_0, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 114 */ /* 115 */ array_0[idxInVararg_0 ++] = isNullArgs_0[29] ? (UTF8String) null : ((UTF8String) valueArgs_0[29]);array_0[idxInVararg_0 ++] = isNullArgs_0[30] ? (UTF8String) null : ((UTF8String) valueArgs_0[30]);array_0[idxInVararg_0 ++] = isNullArgs_0[31] ? (UTF8String) null : ((UTF8String) valueArgs_0[31]); /* 116 */ return idxInVararg_0; /* 117 */ /* 118 */ } /* 119 */ /* 120 */ /* 121 */ private int varargBuildsConcatWs_0_0(InternalRow i, UTF8String [] array_0, int idxInVararg_0, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 122 */ /* 123 */ /* 124 */ if (!isNullArgs_0[0]) { /* 125 */ final int n_0 = ((ArrayData) valueArgs_0[0]).numElements(); /* 126 */ for (int j = 0; j < n_0; j ++) { /* 127 */ array_0[idxInVararg_0 ++] = ((ArrayData) valueArgs_0[0]).getUTF8String(j); /* 128 */ } /* 129 */ } /* 130 */ /* 131 */ if (!isNullArgs_0[1]) { /* 132 */ final int n_1 = ((ArrayData) valueArgs_0[1]).numElements(); /* 133 */ for (int j = 0; j < n_1; j ++) { /* 134 */ array_0[idxInVararg_0 ++] = ((ArrayData) valueArgs_0[1]).getUTF8String(j); /* 135 */ } /* 136 */ } /* 137 */ array_0[idxInVararg_0 ++] = isNullArgs_0[2] ? (UTF8String) null : ((UTF8String) valueArgs_0[2]);array_0[idxInVararg_0 ++] = isNullArgs_0[3] ? (UTF8String) null : ((UTF8String) valueArgs_0[3]);array_0[idxInVararg_0 ++] = isNullArgs_0[4] ? (UTF8String) null : ((UTF8String) valueArgs_0[4]);array_0[idxInVararg_0 ++] = isNullArgs_0[5] ? (UTF8String) null : ((UTF8String) valueArgs_0[5]);array_0[idxInVararg_0 ++] = isNullArgs_0[6] ? (UTF8String) null : ((UTF8String) valueArgs_0[6]); /* 138 */ return idxInVararg_0; /* 139 */ /* 140 */ } /* 141 */ /* 142 */ /* 143 */ private UTF8String ConcatWs_0(InternalRow i) { /* 144 */ boolean[] isNullArgs_0 = new boolean[32]; /* 145 */ Object[] valueArgs_0 = new Object[32]; /* 146 */ initializeArgsArrays_0_0(i, isNullArgs_0, valueArgs_0); /* 147 */ initializeArgsArrays_0_1(i, isNullArgs_0, valueArgs_0); /* 148 */ initializeArgsArrays_0_2(i, isNullArgs_0, valueArgs_0); /* 149 */ initializeArgsArrays_0_3(i, isNullArgs_0, valueArgs_0); /* 150 */ initializeArgsArrays_0_4(i, isNullArgs_0, valueArgs_0); /* 151 */ int varargNum_0 = 30; /* 152 */ int idxInVararg_0 = 0; /* 153 */ /* 154 */ if (!isNullArgs_0[0]) { /* 155 */ varargNum_0 += ((ArrayData) valueArgs_0[0]).numElements(); /* 156 */ } /* 157 */ /* 158 */ if (!isNullArgs_0[1]) { /* 159 */ varargNum_0 += ((ArrayData) valueArgs_0[1]).numElements(); /* 160 */ } /* 161 */ /* 162 */ UTF8String[] array_0 = new UTF8String[varargNum_0]; /* 163 */ idxInVararg_0 = varargBuildsConcatWs_0_0(i, array_0, idxInVararg_0, isNullArgs_0, valueArgs_0); /* 164 */ idxInVararg_0 = varargBuildsConcatWs_0_1(i, array_0, idxInVararg_0, isNullArgs_0, valueArgs_0); /* 165 */ idxInVararg_0 = varargBuildsConcatWs_0_2(i, array_0, idxInVararg_0, isNullArgs_0, valueArgs_0); /* 166 */ idxInVararg_0 = varargBuildsConcatWs_0_3(i, array_0, idxInVararg_0, isNullArgs_0, valueArgs_0); /* 167 */ UTF8String value_0 = UTF8String.concatWs(((UTF8String) references[0] /* literal */), array_0); /* 168 */ boolean isNull_0 = value_0 == null; /* 169 */ globalIsNull_0 = isNull_0; /* 170 */ return value_0; /* 171 */ } /* 172 */ /* 173 */ /* 174 */ private void initializeArgsArrays_0_2(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 175 */ /* 176 */ UTF8String value_17 = i.getUTF8String(14); /* 177 */ isNullArgs_0[15] = false; /* 178 */ valueArgs_0[15] = value_17; /* 179 */ /* 180 */ UTF8String value_18 = i.getUTF8String(15); /* 181 */ isNullArgs_0[16] = false; /* 182 */ valueArgs_0[16] = value_18; /* 183 */ /* 184 */ UTF8String value_19 = i.getUTF8String(16); /* 185 */ isNullArgs_0[17] = false; /* 186 */ valueArgs_0[17] = value_19; /* 187 */ /* 188 */ UTF8String value_20 = i.getUTF8String(17); /* 189 */ isNullArgs_0[18] = false; /* 190 */ valueArgs_0[18] = value_20; /* 191 */ /* 192 */ UTF8String value_21 = i.getUTF8String(18); /* 193 */ isNullArgs_0[19] = false; /* 194 */ valueArgs_0[19] = value_21; /* 195 */ /* 196 */ UTF8String value_22 = i.getUTF8String(19); /* 197 */ isNullArgs_0[20] = false; /* 198 */ valueArgs_0[20] = value_22; /* 199 */ /* 200 */ UTF8String value_23 = i.getUTF8String(20); /* 201 */ isNullArgs_0[21] = false; /* 202 */ valueArgs_0[21] = value_23; /* 203 */ /* 204 */ UTF8String value_24 = i.getUTF8String(21); /* 205 */ isNullArgs_0[22] = false; /* 206 */ valueArgs_0[22] = value_24; /* 207 */ /* 208 */ } /* 209 */ /* 210 */ /* 211 */ private int varargBuildsConcatWs_0_2(InternalRow i, UTF8String [] array_0, int idxInVararg_0, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 212 */ /* 213 */ array_0[idxInVararg_0 ++] = isNullArgs_0[18] ? (UTF8String) null : ((UTF8String) valueArgs_0[18]);array_0[idxInVararg_0 ++] = isNullArgs_0[19] ? (UTF8String) null : ((UTF8String) valueArgs_0[19]);array_0[idxInVararg_0 ++] = isNullArgs_0[20] ? (UTF8String) null : ((UTF8String) valueArgs_0[20]);array_0[idxInVararg_0 ++] = isNullArgs_0[21] ? (UTF8String) null : ((UTF8String) valueArgs_0[21]);array_0[idxInVararg_0 ++] = isNullArgs_0[22] ? (UTF8String) null : ((UTF8String) valueArgs_0[22]);array_0[idxInVararg_0 ++] = isNullArgs_0[23] ? (UTF8String) null : ((UTF8String) valueArgs_0[23]);array_0[idxInVararg_0 ++] = isNullArgs_0[24] ? (UTF8String) null : ((UTF8String) valueArgs_0[24]);array_0[idxInVararg_0 ++] = isNullArgs_0[25] ? (UTF8String) null : ((UTF8String) valueArgs_0[25]);array_0[idxInVararg_0 ++] = isNullArgs_0[26] ? (UTF8String) null : ((UTF8String) valueArgs_0[26]);array_0[idxInVararg_0 ++] = isNullArgs_0[27] ? (UTF8String) null : ((UTF8String) valueArgs_0[27]);array_0[idxInVararg_0 ++] = isNullArgs_0[28] ? (UTF8String) null : ((UTF8String) valueArgs_0[28]); /* 214 */ return idxInVararg_0; /* 215 */ /* 216 */ } /* 217 */ /* 218 */ /* 219 */ private void initializeArgsArrays_0_4(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 220 */ /* 221 */ UTF8String value_33 = i.getUTF8String(30); /* 222 */ isNullArgs_0[31] = false; /* 223 */ valueArgs_0[31] = value_33; /* 224 */ /* 225 */ } /* 226 */ /* 227 */ /* 228 */ private void initializeArgsArrays_0_1(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 229 */ /* 230 */ UTF8String value_9 = i.getUTF8String(6); /* 231 */ isNullArgs_0[7] = false; /* 232 */ valueArgs_0[7] = value_9; /* 233 */ /* 234 */ UTF8String value_10 = i.getUTF8String(7); /* 235 */ isNullArgs_0[8] = false; /* 236 */ valueArgs_0[8] = value_10; /* 237 */ /* 238 */ UTF8String value_11 = i.getUTF8String(8); /* 239 */ isNullArgs_0[9] = false; /* 240 */ valueArgs_0[9] = value_11; /* 241 */ /* 242 */ UTF8String value_12 = i.getUTF8String(9); /* 243 */ isNullArgs_0[10] = false; /* 244 */ valueArgs_0[10] = value_12; /* 245 */ /* 246 */ UTF8String value_13 = i.getUTF8String(10); /* 247 */ isNullArgs_0[11] = false; /* 248 */ valueArgs_0[11] = value_13; /* 249 */ /* 250 */ UTF8String value_14 = i.getUTF8String(11); /* 251 */ isNullArgs_0[12] = false; /* 252 */ valueArgs_0[12] = value_14; /* 253 */ /* 254 */ UTF8String value_15 = i.getUTF8String(12); /* 255 */ isNullArgs_0[13] = false; /* 256 */ valueArgs_0[13] = value_15; /* 257 */ /* 258 */ UTF8String value_16 = i.getUTF8String(13); /* 259 */ isNullArgs_0[14] = false; /* 260 */ valueArgs_0[14] = value_16; /* 261 */ /* 262 */ } /* 263 */ /* 264 */ /* 265 */ private int varargBuildsConcatWs_0_1(InternalRow i, UTF8String [] array_0, int idxInVararg_0, boolean [] isNullArgs_0, Object [] valueArgs_0) { /* 266 */ /* 267 */ array_0[idxInVararg_0 ++] = isNullArgs_0[7] ? (UTF8String) null : ((UTF8String) valueArgs_0[7]);array_0[idxInVararg_0 ++] = isNullArgs_0[8] ? (UTF8String) null : ((UTF8String) valueArgs_0[8]);array_0[idxInVararg_0 ++] = isNullArgs_0[9] ? (UTF8String) null : ((UTF8String) valueArgs_0[9]);array_0[idxInVararg_0 ++] = isNullArgs_0[10] ? (UTF8String) null : ((UTF8String) valueArgs_0[10]);array_0[idxInVararg_0 ++] = isNullArgs_0[11] ? (UTF8String) null : ((UTF8String) valueArgs_0[11]);array_0[idxInVararg_0 ++] = isNullArgs_0[12] ? (UTF8String) null : ((UTF8String) valueArgs_0[12]);array_0[idxInVararg_0 ++] = isNullArgs_0[13] ? (UTF8String) null : ((UTF8String) valueArgs_0[13]);array_0[idxInVararg_0 ++] = isNullArgs_0[14] ? (UTF8String) null : ((UTF8String) valueArgs_0[14]);array_0[idxInVararg_0 ++] = isNullArgs_0[15] ? (UTF8String) null : ((UTF8String) valueArgs_0[15]);array_0[idxInVararg_0 ++] = isNullArgs_0[16] ? (UTF8String) null : ((UTF8String) valueArgs_0[16]);array_0[idxInVararg_0 ++] = isNullArgs_0[17] ? (UTF8String) null : ((UTF8String) valueArgs_0[17]); /* 268 */ return idxInVararg_0; /* 269 */ /* 270 */ } /* 271 */ /* 272 */ } ``` ### Why are the changes needed? The generated code in `concat_ws` fails to compile when the below conditions are met: * Plenty of columns are provided as input of `concat_ws`. * There's at least one column with array[string] type. (In other words, not all columns are string type.) * Splitting methods is triggered in `splitExpressionsWithCurrentInputs`. * This is a bit tricky, as the method won't split methods under whole stage codegen, as well as it will be simply no-op (inlined) if the number of blocks to convert into methods is 1. https://github.com/apache/spark/blob/a0187cd6b59a6b6bb2cadc6711bb663d4d35a844/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala#L88-L195 There're three parts of generated code in `concat_ws` (`codes`, `varargCounts`, `varargBuilds`) and all parts try to split method by itself, while `varargCounts` and `varargBuilds` refer on the generated code in `codes`, hence the overall generated code fails to compile if any of part succeeds to split. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New UTs added. (One for verification of the patch, another one for regression test) Closes #28831 from HeartSaVioR/SPARK-31993. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Wenchen Fan --- .../expressions/stringExpressions.scala | 63 ++++++++++++++----- .../spark/sql/StringFunctionsSuite.scala | 60 ++++++++++++++++++ 2 files changed, 106 insertions(+), 17 deletions(-) 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 334a079fc1892..4f339986a44e5 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 @@ -118,47 +118,72 @@ case class ConcatWs(children: Seq[Expression]) boolean ${ev.isNull} = ${ev.value} == null; """) } else { + val isNullArgs = ctx.freshName("isNullArgs") + val valueArgs = ctx.freshName("valueArgs") + val array = ctx.freshName("array") val varargNum = ctx.freshName("varargNum") val idxVararg = ctx.freshName("idxInVararg") val evals = children.map(_.genCode(ctx)) - val (varargCount, varargBuild) = children.tail.zip(evals.tail).map { case (child, eval) => - child.dataType match { + val (argBuild, varargCount, varargBuild) = children.tail.zip(evals.tail) + .zipWithIndex.map { case ((child, eval), idx) => + val reprForIsNull = s"$isNullArgs[$idx]" + val reprForValue = s"$valueArgs[$idx]" + + val arg = + s""" + ${eval.code} + $reprForIsNull = ${eval.isNull}; + $reprForValue = ${eval.value}; + """ + + val (varCount, varBuild) = child.dataType match { case StringType => + val reprForValueCast = s"((UTF8String) $reprForValue)" ("", // we count all the StringType arguments num at once below. - if (eval.isNull == TrueLiteral) { - "" - } else { - s"$array[$idxVararg ++] = ${eval.isNull} ? (UTF8String) null : ${eval.value};" - }) + if (eval.isNull == TrueLiteral) { + "" + } else { + s"$array[$idxVararg ++] = $reprForIsNull ? (UTF8String) null : $reprForValueCast;" + }) case _: ArrayType => + val reprForValueCast = s"((ArrayData) $reprForValue)" val size = ctx.freshName("n") if (eval.isNull == TrueLiteral) { ("", "") } else { + // scalastyle:off line.size.limit (s""" - if (!${eval.isNull}) { - $varargNum += ${eval.value}.numElements(); + if (!$reprForIsNull) { + $varargNum += $reprForValueCast.numElements(); } """, - s""" - if (!${eval.isNull}) { - final int $size = ${eval.value}.numElements(); + s""" + if (!$reprForIsNull) { + final int $size = $reprForValueCast.numElements(); for (int j = 0; j < $size; j ++) { - $array[$idxVararg ++] = ${CodeGenerator.getValue(eval.value, StringType, "j")}; + $array[$idxVararg ++] = ${CodeGenerator.getValue(reprForValueCast, StringType, "j")}; } } """) + // scalastyle:on line.size.limit } } - }.unzip - val codes = ctx.splitExpressionsWithCurrentInputs(evals.map(_.code.toString)) + (arg, varCount, varBuild) + }.unzip3 + + val argBuilds = ctx.splitExpressionsWithCurrentInputs( + expressions = argBuild, + funcName = "initializeArgsArrays", + extraArguments = ("boolean []", isNullArgs) :: ("Object []", valueArgs) :: Nil + ) val varargCounts = ctx.splitExpressionsWithCurrentInputs( expressions = varargCount, funcName = "varargCountsConcatWs", + extraArguments = ("boolean []", isNullArgs) :: ("Object []", valueArgs) :: Nil, returnType = "int", makeSplitFunction = body => s""" @@ -171,7 +196,8 @@ case class ConcatWs(children: Seq[Expression]) val varargBuilds = ctx.splitExpressionsWithCurrentInputs( expressions = varargBuild, funcName = "varargBuildsConcatWs", - extraArguments = ("UTF8String []", array) :: ("int", idxVararg) :: Nil, + extraArguments = ("UTF8String []", array) :: ("int", idxVararg) :: + ("boolean []", isNullArgs) :: ("Object []", valueArgs) :: Nil, returnType = "int", makeSplitFunction = body => s""" @@ -182,12 +208,15 @@ case class ConcatWs(children: Seq[Expression]) ev.copy( code""" - $codes + boolean[] $isNullArgs = new boolean[${children.length - 1}]; + Object[] $valueArgs = new Object[${children.length - 1}]; + $argBuilds int $varargNum = ${children.count(_.dataType == StringType) - 1}; int $idxVararg = 0; $varargCounts UTF8String[] $array = new UTF8String[$varargNum]; $varargBuilds + ${evals.head.code} UTF8String ${ev.value} = UTF8String.concatWs(${evals.head.value}, $array); boolean ${ev.isNull} = ${ev.value} == null; """) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index ec698818a0d85..f904b53fe47eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -48,6 +49,65 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { Row("a||b")) } + test("SPARK-31993: concat_ws in agg function with plenty of string/array types columns") { + withSQLConf(SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1024", + SQLConf.CODEGEN_FACTORY_MODE.key -> "CODEGEN_ONLY") { + + val (df, genColNames, genColValues) = prepareTestConcatWsColumns() + val groupedCols = Seq($"a") ++ genColNames.map(col) + val concatCols = Seq(collect_list($"b"), collect_list($"c")) ++ genColNames.map(col) + val df2 = df + .groupBy(groupedCols: _*) + .agg(concat_ws(",", concatCols: _*).as("con")) + .select("con") + + val expected = Seq( + Row((Seq("b1", "b2") ++ genColValues).mkString(",")), + Row((Seq("b3", "b4") ++ genColValues).mkString(",")) + ) + + checkAnswer(df2, expected) + } + } + + // This test doesn't fail without SPARK-31993, but still be useful for regression test. + test("SPARK-31993: concat_ws in agg function with plenty of string types columns") { + withSQLConf(SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1024", + SQLConf.CODEGEN_FACTORY_MODE.key -> "CODEGEN_ONLY") { + + val (df, genColNames, genColValues) = prepareTestConcatWsColumns() + val groupedCols = Seq($"a") ++ genColNames.map(col) + val concatCols = groupedCols + val df2 = df + .groupBy(groupedCols: _*) + .agg(concat_ws(",", concatCols: _*).as("con")) + .select("con") + + val expected = Seq( + Row((Seq("a") ++ genColValues).mkString(",")), + Row((Seq("b") ++ genColValues).mkString(",")) + ) + + checkAnswer(df2, expected) + } + } + + private def prepareTestConcatWsColumns(): (DataFrame, Seq[String], Seq[String]) = { + val genColNames = (1 to 30).map { idx => s"col_$idx" } + val genColValues = (1 to 30).map { _.toString } + val genCols = genColValues.map(lit) + + val df = Seq[(String, String, String)]( + ("a", "b1", null), + ("a", "b2", null), + ("b", "b3", null), + ("b", "b4", null)) + .toDF("a", "b", "c") + .withColumns(genColNames, genCols) + + (df, genColNames, genColValues) + } + test("string elt") { val df = Seq[(String, String, String, Int)](("hello", "world", null, 15)) .toDF("a", "b", "c", "d") From 5ee5cfd9c09fc8a94c5cb499d28ee5b731f709ec Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 19 Jun 2020 12:45:47 +0000 Subject: [PATCH 043/384] [SPARK-31826][SQL] Support composed type of case class for typed Scala UDF ### What changes were proposed in this pull request? This PR adds support for typed Scala UDF to accept composed type of case class, e.g. Seq[T], Array[T], Map[Int, T] (assuming T is case class type), as input parameter type. ### Why are the changes needed? After #27937, typed Scala UDF now has supported case class as its input parameter type. However, it can not accept the composed type of case class, such as Seq[T], Array[T], Map[Int, T] (assuming T is case class type), which causing confuse(e.g. https://github.com/apache/spark/pull/27937#discussion_r422699979) to the user. ### Does this PR introduce _any_ user-facing change? Yes. Run the query: ``` scala> case class Person(name: String, age: Int) scala> Seq((1, Seq(Person("Jack", 5)))).toDF("id", "persons").withColumn("ages", udf{ s: Seq[Person] => s.head.age }.apply(col("persons"))).show ``` Before: ``` org.apache.spark.SparkException: Failed to execute user defined function($read$$Lambda$2861/628175152: (array>) => int) at org.apache.spark.sql.catalyst.expressions.ScalaUDF.eval(ScalaUDF.scala:1129) at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:156) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.apply(InterpretedMutableProjection.scala:83) at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$17.$anonfun$applyOrElse$69(Optimizer.scala:1492) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238) .... Caused by: java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema cannot be cast to Person at $anonfun$res3$1(:30) at $anonfun$res3$1$adapted(:30) at org.apache.spark.sql.catalyst.expressions.ScalaUDF.$anonfun$f$2(ScalaUDF.scala:156) at org.apache.spark.sql.catalyst.expressions.ScalaUDF.eval(ScalaUDF.scala:1126) ... 142 more ``` After: ``` +---+-----------+----+ | id| persons|ages| +---+-----------+----+ | 1|[[Jack, 5]]| [5]| +---+-----------+----+ ``` ### How was this patch tested? Added tests. Closes #28645 from Ngone51/impr-udf. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 42 ++++++++- .../sql/catalyst/expressions/ScalaUDF.scala | 79 +++++++++++----- .../sql/catalyst/analysis/AnalysisSuite.scala | 13 ++- .../catalyst/expressions/ScalaUDFSuite.scala | 18 ++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 90 ++++++++++++++++++- 5 files changed, 210 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9c97e1e9b441b..49303978d1ce8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -256,7 +256,8 @@ class Analyzer( Batch("Nondeterministic", Once, PullOutNondeterministic), Batch("UDF", Once, - HandleNullInputsForUDF), + HandleNullInputsForUDF, + ResolveEncodersInUDF), Batch("UpdateNullability", Once, UpdateAttributeNullability), Batch("Subquery", Once, @@ -2852,6 +2853,45 @@ class Analyzer( } } + /** + * Resolve the encoders for the UDF by explicitly given the attributes. We give the + * attributes explicitly in order to handle the case where the data type of the input + * value is not the same with the internal schema of the encoder, which could cause + * data loss. For example, the encoder should not cast the input value to Decimal(38, 18) + * if the actual data type is Decimal(30, 0). + * + * The resolved encoders then will be used to deserialize the internal row to Scala value. + */ + object ResolveEncodersInUDF extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case p if !p.resolved => p // Skip unresolved nodes. + + case p => p transformExpressionsUp { + + case udf: ScalaUDF if udf.inputEncoders.nonEmpty => + val boundEncoders = udf.inputEncoders.zipWithIndex.map { case (encOpt, i) => + val dataType = udf.children(i).dataType + if (dataType.existsRecursively(_.isInstanceOf[UserDefinedType[_]])) { + // for UDT, we use `CatalystTypeConverters` + None + } else { + encOpt.map { enc => + val attrs = if (enc.isSerializedAsStructForTopLevel) { + dataType.asInstanceOf[StructType].toAttributes + } else { + // the field name doesn't matter here, so we use + // a simple literal to avoid any overhead + new StructType().add("input", dataType).toAttributes + } + enc.resolveAndBind(attrs) + } + } + } + udf.copy(inputEncoders = boundEncoders) + } + } + } + /** * Check and add proper window frames for all window functions. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index e80f03ea84756..1e3e6d90b8501 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -17,14 +17,13 @@ package org.apache.spark.sql.catalyst.expressions -import scala.collection.mutable - import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.CatalystTypeConverters.{createToCatalystConverter, createToScalaConverter => catalystCreateToScalaConverter, isPrimitive} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, DataType} +import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, DataType, UserDefinedType} /** * User-defined function. @@ -103,21 +102,46 @@ case class ScalaUDF( } } - private def createToScalaConverter(i: Int, dataType: DataType): Any => Any = { - if (inputEncoders.isEmpty) { - // for untyped Scala UDF - CatalystTypeConverters.createToScalaConverter(dataType) - } else { - val encoder = inputEncoders(i) - if (encoder.isDefined && encoder.get.isSerializedAsStructForTopLevel) { - val fromRow = encoder.get.resolveAndBind().createDeserializer() + /** + * Create the converter which converts the catalyst data type to the scala data type. + * We use `CatalystTypeConverters` to create the converter for: + * - UDF which doesn't provide inputEncoders, e.g., untyped Scala UDF and Java UDF + * - type which isn't supported by `ExpressionEncoder`, e.g., Any + * - primitive types, in order to use `identity` for better performance + * - UserDefinedType which isn't fully supported by `ExpressionEncoder` + * For other cases like case class, Option[T], we use `ExpressionEncoder` instead since + * `CatalystTypeConverters` doesn't support these data types. + * + * @param i the index of the child + * @param dataType the output data type of the i-th child + * @return the converter and a boolean value to indicate whether the converter is + * created by using `ExpressionEncoder`. + */ + private def scalaConverter(i: Int, dataType: DataType): (Any => Any, Boolean) = { + val useEncoder = + !(inputEncoders.isEmpty || // for untyped Scala UDF and Java UDF + inputEncoders(i).isEmpty || // for types aren't supported by encoder, e.g. Any + inputPrimitives(i) || // for primitive types + dataType.existsRecursively(_.isInstanceOf[UserDefinedType[_]])) + + if (useEncoder) { + val enc = inputEncoders(i).get + val fromRow = enc.createDeserializer() + val converter = if (enc.isSerializedAsStructForTopLevel) { row: Any => fromRow(row.asInstanceOf[InternalRow]) } else { - CatalystTypeConverters.createToScalaConverter(dataType) + val inputRow = new GenericInternalRow(1) + value: Any => inputRow.update(0, value); fromRow(inputRow) } + (converter, true) + } else { // use CatalystTypeConverters + (catalystCreateToScalaConverter(dataType), false) } } + private def createToScalaConverter(i: Int, dataType: DataType): Any => Any = + scalaConverter(i, dataType)._1 + // scalastyle:off line.size.limit /** This method has been generated by this script @@ -1045,10 +1069,11 @@ case class ScalaUDF( ev: ExprCode): ExprCode = { val converterClassName = classOf[Any => Any].getName - // The type converters for inputs and the result. - val converters: Array[Any => Any] = children.zipWithIndex.map { case (c, i) => - createToScalaConverter(i, c.dataType) - }.toArray :+ CatalystTypeConverters.createToCatalystConverter(dataType) + // The type converters for inputs and the result + val (converters, useEncoders): (Array[Any => Any], Array[Boolean]) = + (children.zipWithIndex.map { case (c, i) => + scalaConverter(i, c.dataType) + }.toArray :+ (createToCatalystConverter(dataType), false)).unzip val convertersTerm = ctx.addReferenceObj("converters", converters, s"$converterClassName[]") val errorMsgTerm = ctx.addReferenceObj("errMsg", udfErrorMessage) val resultTerm = ctx.freshName("result") @@ -1064,12 +1089,26 @@ case class ScalaUDF( val (funcArgs, initArgs) = evals.zipWithIndex.zip(children.map(_.dataType)).map { case ((eval, i), dt) => val argTerm = ctx.freshName("arg") - val initArg = if (CatalystTypeConverters.isPrimitive(dt)) { + // Check `inputPrimitives` when it's not empty in order to figure out the Option + // type as non primitive type, e.g., Option[Int]. Fall back to `isPrimitive` when + // `inputPrimitives` is empty for other cases, e.g., Java UDF, untyped Scala UDF + val primitive = (inputPrimitives.isEmpty && isPrimitive(dt)) || + (inputPrimitives.nonEmpty && inputPrimitives(i)) + val initArg = if (primitive) { val convertedTerm = ctx.freshName("conv") s""" |${CodeGenerator.boxedType(dt)} $convertedTerm = ${eval.value}; |Object $argTerm = ${eval.isNull} ? null : $convertedTerm; """.stripMargin + } else if (useEncoders(i)) { + s""" + |Object $argTerm = null; + |if (${eval.isNull}) { + | $argTerm = $convertersTerm[$i].apply(null); + |} else { + | $argTerm = $convertersTerm[$i].apply(${eval.value}); + |} + """.stripMargin } else { s"Object $argTerm = ${eval.isNull} ? null : $convertersTerm[$i].apply(${eval.value});" } @@ -1081,7 +1120,7 @@ case class ScalaUDF( val resultConverter = s"$convertersTerm[${children.length}]" val boxedType = CodeGenerator.boxedType(dataType) - val funcInvokation = if (CatalystTypeConverters.isPrimitive(dataType) + val funcInvokation = if (isPrimitive(dataType) // If the output is nullable, the returned value must be unwrapped from the Option && !nullable) { s"$resultTerm = ($boxedType)$getFuncResult" @@ -1112,7 +1151,7 @@ case class ScalaUDF( """.stripMargin) } - private[this] val resultConverter = CatalystTypeConverters.createToCatalystConverter(dataType) + private[this] val resultConverter = createToCatalystConverter(dataType) lazy val udfErrorMessage = { val funcCls = function.getClass.getSimpleName diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 02472e153b09e..189152374b0d1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import java.util.{Locale, TimeZone} import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag import org.apache.log4j.Level import org.scalatest.Matchers @@ -307,6 +308,10 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-11725: correctly handle null inputs for ScalaUDF") { + def resolvedEncoder[T : TypeTag](): ExpressionEncoder[T] = { + ExpressionEncoder[T]().resolveAndBind() + } + val testRelation = LocalRelation( AttributeReference("a", StringType)(), AttributeReference("b", DoubleType)(), @@ -328,20 +333,20 @@ class AnalysisSuite extends AnalysisTest with Matchers { // non-primitive parameters do not need special null handling val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil, - Option(ExpressionEncoder[String]()) :: Nil) + Option(resolvedEncoder[String]()) :: Nil) val expected1 = udf1 checkUDF(udf1, expected1) // only primitive parameter needs special null handling val udf2 = ScalaUDF((s: String, d: Double) => "x", StringType, string :: double :: Nil, - Option(ExpressionEncoder[String]()) :: Option(ExpressionEncoder[Double]()) :: Nil) + Option(resolvedEncoder[String]()) :: Option(resolvedEncoder[Double]()) :: Nil) val expected2 = If(IsNull(double), nullResult, udf2.copy(children = string :: KnownNotNull(double) :: Nil)) checkUDF(udf2, expected2) // special null handling should apply to all primitive parameters val udf3 = ScalaUDF((s: Short, d: Double) => "x", StringType, short :: double :: Nil, - Option(ExpressionEncoder[Short]()) :: Option(ExpressionEncoder[Double]()) :: Nil) + Option(resolvedEncoder[Short]()) :: Option(resolvedEncoder[Double]()) :: Nil) val expected3 = If( IsNull(short) || IsNull(double), nullResult, @@ -353,7 +358,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { (s: Short, d: Double) => "x", StringType, short :: nonNullableDouble :: Nil, - Option(ExpressionEncoder[Short]()) :: Option(ExpressionEncoder[Double]()) :: Nil) + Option(resolvedEncoder[Short]()) :: Option(resolvedEncoder[Double]()) :: Nil) val expected4 = If( IsNull(short), nullResult, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index 836b2eaa642a3..1b40e02aa8662 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Locale +import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext @@ -27,13 +29,17 @@ import org.apache.spark.sql.types.{DecimalType, IntegerType, StringType} class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { + private def resolvedEncoder[T : TypeTag](): ExpressionEncoder[T] = { + ExpressionEncoder[T]().resolveAndBind() + } + test("basic") { val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil, - Option(ExpressionEncoder[Int]()) :: Nil) + Option(resolvedEncoder[Int]()) :: Nil) checkEvaluation(intUdf, 2) val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, - Option(ExpressionEncoder[String]()) :: Nil) + Option(resolvedEncoder[String]()) :: Nil) checkEvaluation(stringUdf, "ax") } @@ -42,7 +48,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { (s: String) => s.toLowerCase(Locale.ROOT), StringType, Literal.create(null, StringType) :: Nil, - Option(ExpressionEncoder[String]()) :: Nil) + Option(resolvedEncoder[String]()) :: Nil) val e1 = intercept[SparkException](udf.eval()) assert(e1.getMessage.contains("Failed to execute user defined function")) @@ -56,7 +62,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-22695: ScalaUDF should not use global variables") { val ctx = new CodegenContext ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, - Option(ExpressionEncoder[String]()) :: Nil).genCode(ctx) + Option(resolvedEncoder[String]()) :: Nil).genCode(ctx) assert(ctx.inlinedMutableStates.isEmpty) } @@ -66,7 +72,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, Literal(BigDecimal("12345678901234567890.123")) :: Nil, - Option(ExpressionEncoder[java.math.BigDecimal]()) :: Nil) + Option(resolvedEncoder[java.math.BigDecimal]()) :: Nil) val e1 = intercept[ArithmeticException](udf.eval()) assert(e1.getMessage.contains("cannot be represented as Decimal")) val e2 = intercept[SparkException] { @@ -79,7 +85,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, Literal(BigDecimal("12345678901234567890.123")) :: Nil, - Option(ExpressionEncoder[java.math.BigDecimal]()) :: Nil) + Option(resolvedEncoder[java.math.BigDecimal]()) :: Nil) checkEvaluation(udf, null) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index e2747d7db9f3f..5c1fe265c15d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.{QueryExecution, SimpleMode} import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, ExplainCommand} import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand -import org.apache.spark.sql.functions.{lit, udf} +import org.apache.spark.sql.functions.{lit, struct, udf} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ @@ -581,4 +581,92 @@ class UDFSuite extends QueryTest with SharedSparkSession { .toDF("col1", "col2") checkAnswer(df.select(myUdf(Column("col1"), Column("col2"))), Row(2020) :: Nil) } + + test("case class as element type of Seq/Array") { + val f1 = (s: Seq[TestData]) => s.map(d => d.key * d.value.toInt).sum + val myUdf1 = udf(f1) + val df1 = Seq(("data", Seq(TestData(50, "2")))).toDF("col1", "col2") + checkAnswer(df1.select(myUdf1(Column("col2"))), Row(100) :: Nil) + + val f2 = (s: Array[TestData]) => s.map(d => d.key * d.value.toInt).sum + val myUdf2 = udf(f2) + val df2 = Seq(("data", Array(TestData(50, "2")))).toDF("col1", "col2") + checkAnswer(df2.select(myUdf2(Column("col2"))), Row(100) :: Nil) + } + + test("case class as key/value type of Map") { + val f1 = (s: Map[TestData, Int]) => s.keys.head.key * s.keys.head.value.toInt + val myUdf1 = udf(f1) + val df1 = Seq(("data", Map(TestData(50, "2") -> 502))).toDF("col1", "col2") + checkAnswer(df1.select(myUdf1(Column("col2"))), Row(100) :: Nil) + + val f2 = (s: Map[Int, TestData]) => s.values.head.key * s.values.head.value.toInt + val myUdf2 = udf(f2) + val df2 = Seq(("data", Map(502 -> TestData(50, "2")))).toDF("col1", "col2") + checkAnswer(df2.select(myUdf2(Column("col2"))), Row(100) :: Nil) + + val f3 = (s: Map[TestData, TestData]) => s.keys.head.key * s.values.head.value.toInt + val myUdf3 = udf(f3) + val df3 = Seq(("data", Map(TestData(50, "2") -> TestData(50, "2")))).toDF("col1", "col2") + checkAnswer(df3.select(myUdf3(Column("col2"))), Row(100) :: Nil) + } + + test("case class as element of tuple") { + val f = (s: (TestData, Int)) => s._1.key * s._2 + val myUdf = udf(f) + val df = Seq(("data", (TestData(50, "2"), 2))).toDF("col1", "col2") + checkAnswer(df.select(myUdf(Column("col2"))), Row(100) :: Nil) + } + + test("case class as generic type of Option") { + val f = (o: Option[TestData]) => o.map(t => t.key * t.value.toInt) + val myUdf = udf(f) + val df1 = Seq(("data", Some(TestData(50, "2")))).toDF("col1", "col2") + checkAnswer(df1.select(myUdf(Column("col2"))), Row(100) :: Nil) + val df2 = Seq(("data", None: Option[TestData])).toDF("col1", "col2") + checkAnswer(df2.select(myUdf(Column("col2"))), Row(null) :: Nil) + } + + test("more input fields than expect for case class") { + val f = (t: TestData2) => t.a * t.b + val myUdf = udf(f) + val df = spark.range(1) + .select(lit(50).as("a"), lit(2).as("b"), lit(2).as("c")) + .select(struct("a", "b", "c").as("col")) + checkAnswer(df.select(myUdf(Column("col"))), Row(100) :: Nil) + } + + test("less input fields than expect for case class") { + val f = (t: TestData2) => t.a * t.b + val myUdf = udf(f) + val df = spark.range(1) + .select(lit(50).as("a")) + .select(struct("a").as("col")) + val error = intercept[AnalysisException](df.select(myUdf(Column("col")))) + assert(error.getMessage.contains("cannot resolve '`b`' given input columns: [a]")) + } + + test("wrong order of input fields for case class") { + val f = (t: TestData) => t.key * t.value.toInt + val myUdf = udf(f) + val df = spark.range(1) + .select(lit("2").as("value"), lit(50).as("key")) + .select(struct("value", "key").as("col")) + checkAnswer(df.select(myUdf(Column("col"))), Row(100) :: Nil) + } + + test("top level Option primitive type") { + val f = (i: Option[Int]) => i.map(_ * 10) + val myUdf = udf(f) + val df = Seq(Some(10), None).toDF("col") + checkAnswer(df.select(myUdf(Column("col"))), Row(100) :: Row(null) :: Nil) + } + + test("array Option") { + val f = (i: Array[Option[TestData]]) => + i.map(_.map(t => t.key * t.value.toInt).getOrElse(0)).sum + val myUdf = udf(f) + val df = Seq(Array(Some(TestData(50, "2")), None)).toDF("col") + checkAnswer(df.select(myUdf(Column("col"))), Row(100) :: Nil) + } } From 3c34e45df4b3e0610dde5334716025a85cbbc05b Mon Sep 17 00:00:00 2001 From: Shanyu Zhao Date: Fri, 19 Jun 2020 09:59:14 -0500 Subject: [PATCH 044/384] [SPARK-31029] Avoid using global execution context in driver main thread for YarnSchedulerBackend #31029 # What changes were proposed in this pull request? In YarnSchedulerBackend, we should avoid using the global execution context for its Future. Otherwise if user's Spark application also uses global execution context for its Future, the user is facing indeterministic behavior in terms of the thread's context class loader. ### Why are the changes needed? When running tpc-ds test (https://github.com/databricks/spark-sql-perf), occasionally we see error related to class not found: 2020-02-04 20:00:26,673 ERROR yarn.ApplicationMaster: User class threw exception: scala.ScalaReflectionException: class com.databricks.spark.sql.perf.ExperimentRun in JavaMirror with sun.misc.Launcher$AppClassLoader28ba21f3 of type class sun.misc.Launcher$AppClassLoader with classpath [...] and parent being sun.misc.Launcher$ExtClassLoader3ff5d147 of type class sun.misc.Launcher$ExtClassLoader with classpath [...] and parent being primordial classloader with boot classpath [...] not found. This is the root cause for the problem: Spark driver starts ApplicationMaster in the main thread, which starts a user thread and set MutableURLClassLoader to that thread's ContextClassLoader. userClassThread = startUserApplication() The main thread then setup YarnSchedulerBackend RPC endpoints, which handles these calls using scala Future with the default global ExecutionContext: doRequestTotalExecutors doKillExecutors So for the main thread and user thread, whoever starts the future first get a chance to set ContextClassLoader to the default thread pool: - If main thread starts a future to handle doKillExecutors() before user thread does then the default thread pool thread's ContextClassLoader would be the default (AppClassLoader). - If user thread starts a future first then the thread pool thread will have MutableURLClassLoader. Note that only MutableURLClassLoader can load user provided class for the Spark app, you will see errors related to class not found if the ContextClassLoader is AppClassLoader. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing unit tests and manual tests Closes #27843 from shanyu/shanyu-31029. Authored-by: Shanyu Zhao Signed-off-by: Thomas Graves --- .../scheduler/cluster/YarnSchedulerBackend.scala | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index e428bab4f96f3..0475b0aed0ec4 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -21,8 +21,7 @@ import java.util.EnumSet import java.util.concurrent.atomic.{AtomicBoolean} import javax.servlet.DispatcherType -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.Future +import scala.concurrent.{ExecutionContext, Future} import scala.util.{Failure, Success} import scala.util.control.NonFatal @@ -67,6 +66,14 @@ private[spark] abstract class YarnSchedulerBackend( private implicit val askTimeout = RpcUtils.askRpcTimeout(sc.conf) + /** + * Declare implicit single thread execution context for futures doRequestTotalExecutors and + * doKillExecutors below, avoiding using the global execution context that may cause conflict + * with user code's execution of futures. + */ + private implicit val schedulerEndpointEC = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonSingleThreadExecutor("yarn-scheduler-endpoint")) + /** Application ID. */ protected var appId: Option[ApplicationId] = None From a9247c39d2974b29302f13defdf61cb640024e46 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Fri, 19 Jun 2020 14:46:26 -0700 Subject: [PATCH 045/384] [SPARK-32033][SS][DSTEAMS] Use new poll API in Kafka connector executor side to avoid infinite wait ### What changes were proposed in this pull request? Spark uses an old and deprecated API named `KafkaConsumer.poll(long)` which never returns and stays in live lock if metadata is not updated (for instance when broker disappears at consumer creation). Please see [Kafka documentation](https://kafka.apache.org/25/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll-long-) and [standalone test application](https://github.com/gaborgsomogyi/kafka-get-assignment) for further details. In this PR I've applied the new `KafkaConsumer.poll(Duration)` API on executor side. Please note driver side still uses the old API which will be fixed in SPARK-32032. ### Why are the changes needed? Infinite wait in `KafkaConsumer.poll(long)`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #28871 from gaborgsomogyi/SPARK-32033. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- docs/structured-streaming-kafka-integration.md | 5 +++-- .../spark/sql/kafka010/consumer/KafkaDataConsumer.scala | 3 ++- .../apache/spark/streaming/kafka010/KafkaDataConsumer.scala | 3 ++- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 016faa735acd6..58fa01edb80e8 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -440,9 +440,10 @@ The following configurations are optional: kafkaConsumer.pollTimeoutMs long - 512 + 120000 streaming and batch - The timeout in milliseconds to poll data from Kafka in executors. + The timeout in milliseconds to poll data from Kafka in executors. When not defined it falls + back to spark.network.timeout. fetchOffset.numRetries diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala index 5f23029d9fed3..f2bf7cd1360ec 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.kafka010.consumer import java.{util => ju} import java.io.Closeable +import java.time.Duration import java.util.concurrent.TimeoutException import scala.collection.JavaConverters._ @@ -73,7 +74,7 @@ private[kafka010] class InternalKafkaConsumer( // Seek to the offset because we may call seekToBeginning or seekToEnd before this. seek(offset) - val p = consumer.poll(pollTimeoutMs) + val p = consumer.poll(Duration.ofMillis(pollTimeoutMs)) val r = p.records(topicPartition) logDebug(s"Polled $groupId ${p.partitions()} ${r.size}") val offsetAfterPoll = consumer.position(topicPartition) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala index 142e946188ace..09af5a0815147 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.kafka010 import java.{util => ju} +import java.time.Duration import scala.collection.JavaConverters._ @@ -203,7 +204,7 @@ private[kafka010] class InternalKafkaConsumer[K, V]( } private def poll(timeout: Long): Unit = { - val p = consumer.poll(timeout) + val p = consumer.poll(Duration.ofMillis(timeout)) val r = p.records(topicPartition) logDebug(s"Polled ${p.partitions()} ${r.size}") buffer = r.listIterator From 7b8683820bf408a82fcdb8716d963990a6955cee Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 20 Jun 2020 08:20:45 +0900 Subject: [PATCH 046/384] [SPARK-31350][SQL] Coalesce bucketed tables for sort merge join if applicable MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? When two bucketed tables with different number of buckets are joined, it can introduce a full shuffle: ``` spark.conf.set("spark.sql.autoBroadcastJoinThreshold", "0") val df1 = (0 until 20).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") val df2 = (0 until 20).map(i => (i % 7, i % 11, i.toString)).toDF("i", "j", "k") df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") df2.write.format("parquet").bucketBy(4, "i").saveAsTable("t2") val t1 = spark.table("t1") val t2 = spark.table("t2") val joined = t1.join(t2, t1("i") === t2("i")) joined.explain == Physical Plan == *(5) SortMergeJoin [i#44], [i#50], Inner :- *(2) Sort [i#44 ASC NULLS FIRST], false, 0 :  +- Exchange hashpartitioning(i#44, 200), true, [id=#105] :     +- *(1) Project [i#44, j#45, k#46] :        +- *(1) Filter isnotnull(i#44) :           +- *(1) ColumnarToRow :              +- FileScan parquet default.t1[i#44,j#45,k#46] Batched: true, DataFilters: [isnotnull(i#44)], Format: Parquet, Location: InMemoryFileIndex[...], PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct, SelectedBucketsCount: 8 out of 8 +- *(4) Sort [i#50 ASC NULLS FIRST], false, 0    +- Exchange hashpartitioning(i#50, 200), true, [id=#115]       +- *(3) Project [i#50, j#51, k#52]          +- *(3) Filter isnotnull(i#50)             +- *(3) ColumnarToRow                +- FileScan parquet default.t2[i#50,j#51,k#52] Batched: true, DataFilters: [isnotnull(i#50)], Format: Parquet, Location: InMemoryFileIndex[...], PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct, SelectedBucketsCount: 4 out of 4 ``` This PR proposes to introduce coalescing buckets when the following conditions are met to eliminate the full shuffle: - Join is the sort merge one (which is created only for equi-join). - Join keys match with output partition expressions on their respective sides. - The larger bucket number is divisible by the smaller bucket number. - `spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled` is set to `true`. - The ratio of the number of buckets should be less than the value set in `spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio`. ### Why are the changes needed? Eliminating the full shuffle can benefit for scenarios where two large tables are joined. Especially when the tables are already bucketed but differ in the number of buckets, we could take advantage of it. ### Does this PR introduce any user-facing change? If the bucket coalescing conditions explained above are met, a full shuffle can be eliminated (also note that you will see `SelectedBucketsCount: 8 out of 8 (Coalesced to 4)` in the physical plan): ``` spark.conf.set("spark.sql.autoBroadcastJoinThreshold", "0") spark.conf.set("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled", "true") val df1 = (0 until 20).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") val df2 = (0 until 20).map(i => (i % 7, i % 11, i.toString)).toDF("i", "j", "k") df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") df2.write.format("parquet").bucketBy(4, "i").saveAsTable("t2") val t1 = spark.table("t1") val t2 = spark.table("t2") val joined = t1.join(t2, t1("i") === t2("i")) joined.explain == Physical Plan == *(3) SortMergeJoin [i#44], [i#50], Inner :- *(1) Sort [i#44 ASC NULLS FIRST], false, 0 : +- *(1) Project [i#44, j#45, k#46] : +- *(1) Filter isnotnull(i#44) : +- *(1) ColumnarToRow : +- FileScan parquet default.t1[i#44,j#45,k#46] Batched: true, DataFilters: [isnotnull(i#44)], Format: Parquet, Location: InMemoryFileIndex[...], PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct, SelectedBucketsCount: 8 out of 8 (Coalesced to 4) +- *(2) Sort [i#50 ASC NULLS FIRST], false, 0 +- *(2) Project [i#50, j#51, k#52] +- *(2) Filter isnotnull(i#50) +- *(2) ColumnarToRow +- FileScan parquet default.t2[i#50,j#51,k#52] Batched: true, DataFilters: [isnotnull(i#50)], Format: Parquet, Location: InMemoryFileIndex[...], PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct, SelectedBucketsCount: 4 out of 4 ``` ### How was this patch tested? Added unit tests Closes #28123 from imback82/coalescing_bucket. Authored-by: Terry Kim Signed-off-by: Takeshi Yamamuro --- .../apache/spark/sql/internal/SQLConf.scala | 20 ++ .../sql/execution/DataSourceScanExec.scala | 29 ++- .../spark/sql/execution/QueryExecution.scala | 2 + .../CoalesceBucketsInSortMergeJoin.scala | 132 ++++++++++++ .../datasources/FileSourceStrategy.scala | 1 + .../apache/spark/sql/DataFrameJoinSuite.scala | 2 +- .../org/apache/spark/sql/ExplainSuite.scala | 17 ++ .../org/apache/spark/sql/SubquerySuite.scala | 2 +- .../CoalesceBucketsInSortMergeJoinSuite.scala | 194 ++++++++++++++++++ .../spark/sql/sources/BucketedReadSuite.scala | 137 ++++++++++++- 10 files changed, 523 insertions(+), 13 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6bbeb2de7538c..af6e7eeddbb6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2627,6 +2627,26 @@ object SQLConf { .booleanConf .createWithDefault(false) + val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED = + buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled") + .doc("When true, if two bucketed tables with the different number of buckets are joined, " + + "the side with a bigger number of buckets will be coalesced to have the same number " + + "of buckets as the other side. Bucket coalescing is applied only to sort-merge joins " + + "and only when the bigger number of buckets is divisible by the smaller number of buckets.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + + val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO = + buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio") + .doc("The ratio of the number of two buckets being coalesced should be less than or " + + "equal to this value for bucket coalescing to be applied. This configuration only " + + s"has an effect when '${COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key}' is set to true.") + .version("3.1.0") + .intConf + .checkValue(_ > 0, "The difference must be positive.") + .createWithDefault(4) + /** * Holds information about keys that have been deprecated. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 0ae39cf8560e6..458e11b97db6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -155,7 +155,8 @@ case class RowDataSourceScanExec( * @param output Output attributes of the scan, including data attributes and partition attributes. * @param requiredSchema Required schema of the underlying relation, excluding partition columns. * @param partitionFilters Predicates to use for partition pruning. - * @param optionalBucketSet Bucket ids for bucket pruning + * @param optionalBucketSet Bucket ids for bucket pruning. + * @param optionalNumCoalescedBuckets Number of coalesced buckets. * @param dataFilters Filters on non-partition columns. * @param tableIdentifier identifier for the table in the metastore. */ @@ -165,6 +166,7 @@ case class FileSourceScanExec( requiredSchema: StructType, partitionFilters: Seq[Expression], optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], dataFilters: Seq[Expression], tableIdentifier: Option[TableIdentifier]) extends DataSourceScanExec { @@ -291,7 +293,8 @@ case class FileSourceScanExec( // above val spec = relation.bucketSpec.get val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - val partitioning = HashPartitioning(bucketColumns, spec.numBuckets) + val numPartitions = optionalNumCoalescedBuckets.getOrElse(spec.numBuckets) + val partitioning = HashPartitioning(bucketColumns, numPartitions) val sortColumns = spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) val shouldCalculateSortOrder = @@ -311,7 +314,8 @@ case class FileSourceScanExec( files.map(_.getPath.getName).groupBy(file => BucketingUtils.getBucketId(file)) val singleFilePartitions = bucketToFilesGrouping.forall(p => p._2.length <= 1) - if (singleFilePartitions) { + // TODO SPARK-24528 Sort order is currently ignored if buckets are coalesced. + if (singleFilePartitions && optionalNumCoalescedBuckets.isEmpty) { // TODO Currently Spark does not support writing columns sorting in descending order // so using Ascending order. This can be fixed in future sortColumns.map(attribute => SortOrder(attribute, Ascending)) @@ -356,7 +360,8 @@ case class FileSourceScanExec( spec.numBuckets } metadata + ("SelectedBucketsCount" -> - s"$numSelectedBuckets out of ${spec.numBuckets}") + (s"$numSelectedBuckets out of ${spec.numBuckets}" + + optionalNumCoalescedBuckets.map { b => s" (Coalesced to $b)"}.getOrElse(""))) } getOrElse { metadata } @@ -544,8 +549,19 @@ case class FileSourceScanExec( filesGroupedToBuckets } - val filePartitions = Seq.tabulate(bucketSpec.numBuckets) { bucketId => - FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + val filePartitions = optionalNumCoalescedBuckets.map { numCoalescedBuckets => + logInfo(s"Coalescing to ${numCoalescedBuckets} buckets") + val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) + Seq.tabulate(numCoalescedBuckets) { bucketId => + val partitionedFiles = coalescedBuckets.get(bucketId).map { + _.values.flatten.toArray + }.getOrElse(Array.empty) + FilePartition(bucketId, partitionedFiles) + } + }.getOrElse { + Seq.tabulate(bucketSpec.numBuckets) { bucketId => + FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + } } new FileScanRDD(fsRelation.sparkSession, readFile, filePartitions) @@ -599,6 +615,7 @@ case class FileSourceScanExec( requiredSchema, QueryPlan.normalizePredicates(partitionFilters, output), optionalBucketSet, + optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, output), None) } 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 89915d254883d..bf60427e5f3bf 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 @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.adaptive.{AdaptiveExecutionContext, InsertAdaptiveSparkPlan} +import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInSortMergeJoin import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} @@ -331,6 +332,7 @@ object QueryExecution { // as the original plan is hidden behind `AdaptiveSparkPlanExec`. adaptiveExecutionRule.toSeq ++ Seq( + CoalesceBucketsInSortMergeJoin(sparkSession.sessionState.conf), PlanDynamicPruningFilters(sparkSession), PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala new file mode 100644 index 0000000000000..3bb0597ecd87c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.internal.SQLConf + +/** + * This rule coalesces one side of the `SortMergeJoin` if the following conditions are met: + * - Two bucketed tables are joined. + * - Join keys match with output partition expressions on their respective sides. + * - The larger bucket number is divisible by the smaller bucket number. + * - COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED is set to true. + * - The ratio of the number of buckets is less than the value set in + * COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO. + */ +case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] { + private def mayCoalesce(numBuckets1: Int, numBuckets2: Int, conf: SQLConf): Option[Int] = { + assert(numBuckets1 != numBuckets2) + val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) + // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller + // number of buckets because bucket id is calculated by modding the total number of buckets. + if (large % small == 0 && + large / small <= conf.getConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO)) { + Some(small) + } else { + None + } + } + + private def updateNumCoalescedBuckets(plan: SparkPlan, numCoalescedBuckets: Int): SparkPlan = { + plan.transformUp { + case f: FileSourceScanExec => + f.copy(optionalNumCoalescedBuckets = Some(numCoalescedBuckets)) + } + } + + def apply(plan: SparkPlan): SparkPlan = { + if (!conf.getConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED)) { + return plan + } + + plan transform { + case ExtractSortMergeJoinWithBuckets(smj, numLeftBuckets, numRightBuckets) + if numLeftBuckets != numRightBuckets => + mayCoalesce(numLeftBuckets, numRightBuckets, conf).map { numCoalescedBuckets => + if (numCoalescedBuckets != numLeftBuckets) { + smj.copy(left = updateNumCoalescedBuckets(smj.left, numCoalescedBuckets)) + } else { + smj.copy(right = updateNumCoalescedBuckets(smj.right, numCoalescedBuckets)) + } + }.getOrElse(smj) + case other => other + } + } +} + +/** + * An extractor that extracts `SortMergeJoinExec` where both sides of the join have the bucketed + * tables and are consisted of only the scan operation. + */ +object ExtractSortMergeJoinWithBuckets { + private def isScanOperation(plan: SparkPlan): Boolean = plan match { + case f: FilterExec => isScanOperation(f.child) + case p: ProjectExec => isScanOperation(p.child) + case _: FileSourceScanExec => true + case _ => false + } + + private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { + plan.collectFirst { + case f: FileSourceScanExec if f.relation.bucketSpec.nonEmpty && + f.optionalNumCoalescedBuckets.isEmpty => + f.relation.bucketSpec.get + } + } + + /** + * The join keys should match with expressions for output partitioning. Note that + * the ordering does not matter because it will be handled in `EnsureRequirements`. + */ + private def satisfiesOutputPartitioning( + keys: Seq[Expression], + partitioning: Partitioning): Boolean = { + partitioning match { + case HashPartitioning(exprs, _) if exprs.length == keys.length => + exprs.forall(e => keys.exists(_.semanticEquals(e))) + case _ => false + } + } + + private def isApplicable(s: SortMergeJoinExec): Boolean = { + isScanOperation(s.left) && + isScanOperation(s.right) && + satisfiesOutputPartitioning(s.leftKeys, s.left.outputPartitioning) && + satisfiesOutputPartitioning(s.rightKeys, s.right.outputPartitioning) + } + + def unapply(plan: SparkPlan): Option[(SortMergeJoinExec, Int, Int)] = { + plan match { + case s: SortMergeJoinExec if isApplicable(s) => + val leftBucket = getBucketSpec(s.left) + val rightBucket = getBucketSpec(s.right) + if (leftBucket.isDefined && rightBucket.isDefined) { + Some(s, leftBucket.get.numBuckets, rightBucket.get.numBuckets) + } else { + None + } + case _ => None + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 477937d66ad9b..512024dff0051 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -208,6 +208,7 @@ object FileSourceStrategy extends Strategy with Logging { outputSchema, partitionKeyFilters.toSeq, bucketSet, + None, dataFilters, table.map(_.identifier)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 6b772e53ac184..0b4f43b72366b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -345,7 +345,7 @@ class DataFrameJoinSuite extends QueryTest } assert(broadcastExchanges.size == 1) val tables = broadcastExchanges.head.collect { - case FileSourceScanExec(_, _, _, _, _, _, Some(tableIdent)) => tableIdent + case FileSourceScanExec(_, _, _, _, _, _, _, Some(tableIdent)) => tableIdent } assert(tables.size == 1) assert(tables.head === TableIdentifier(table1Name, Some(dbName))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index 5aeecd2df91e9..1ad97185a564a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -343,6 +343,23 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite assert(getNormalizedExplain(df1, FormattedMode) === getNormalizedExplain(df2, FormattedMode)) } } + + test("Coalesced bucket info should be a part of explain string") { + withTable("t1", "t2") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + Seq(1, 2).toDF("i").write.bucketBy(8, "i").saveAsTable("t1") + Seq(2, 3).toDF("i").write.bucketBy(4, "i").saveAsTable("t2") + val df1 = spark.table("t1") + val df2 = spark.table("t2") + val joined = df1.join(df2, df1("i") === df2("i")) + checkKeywordsExistsInExplain( + joined, + SimpleMode, + "SelectedBucketsCount: 8 out of 8 (Coalesced to 4)" :: Nil: _*) + } + } + } } class ExplainSuiteAE extends ExplainSuiteHelper with EnableAdaptiveExecutionSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 8118eb4f79c17..347bc735a8b76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1313,7 +1313,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark // need to execute the query before we can examine fs.inputRDDs() assert(stripAQEPlan(df.queryExecution.executedPlan) match { case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( - fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _)))) => + fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _, _)))) => partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala new file mode 100644 index 0000000000000..6a70045c55e64 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.optimizer.BuildLeft +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.execution.{BinaryExecNode, FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, PartitionSpec} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.types.{IntegerType, StructType} + +class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkSession { + case class RelationSetting( + cols: Seq[Attribute], + numBuckets: Int, + expectedCoalescedNumBuckets: Option[Int]) + + object RelationSetting { + def apply(numBuckets: Int, expectedCoalescedNumBuckets: Option[Int]): RelationSetting = { + val cols = Seq(AttributeReference("i", IntegerType)()) + RelationSetting(cols, numBuckets, expectedCoalescedNumBuckets) + } + } + + case class JoinSetting( + leftKeys: Seq[Attribute], + rightKeys: Seq[Attribute], + leftRelation: RelationSetting, + rightRelation: RelationSetting, + isSortMergeJoin: Boolean) + + object JoinSetting { + def apply(l: RelationSetting, r: RelationSetting, isSortMergeJoin: Boolean): JoinSetting = { + JoinSetting(l.cols, r.cols, l, r, isSortMergeJoin) + } + } + + private def newFileSourceScanExec(setting: RelationSetting): FileSourceScanExec = { + val relation = HadoopFsRelation( + location = new InMemoryFileIndex(spark, Nil, Map.empty, None), + partitionSchema = PartitionSpec.emptySpec.partitionColumns, + dataSchema = StructType.fromAttributes(setting.cols), + bucketSpec = Some(BucketSpec(setting.numBuckets, setting.cols.map(_.name), Nil)), + fileFormat = new ParquetFileFormat(), + options = Map.empty)(spark) + FileSourceScanExec(relation, setting.cols, relation.dataSchema, Nil, None, None, Nil, None) + } + + private def run(setting: JoinSetting): Unit = { + val swappedSetting = setting.copy( + leftKeys = setting.rightKeys, + rightKeys = setting.leftKeys, + leftRelation = setting.rightRelation, + rightRelation = setting.leftRelation) + + Seq(setting, swappedSetting).foreach { case s => + val lScan = newFileSourceScanExec(s.leftRelation) + val rScan = newFileSourceScanExec(s.rightRelation) + val join = if (s.isSortMergeJoin) { + SortMergeJoinExec(s.leftKeys, s.rightKeys, Inner, None, lScan, rScan) + } else { + BroadcastHashJoinExec( + s.leftKeys, s.rightKeys, Inner, BuildLeft, None, lScan, rScan) + } + + val plan = CoalesceBucketsInSortMergeJoin(spark.sessionState.conf)(join) + + def verify(expected: Option[Int], subPlan: SparkPlan): Unit = { + val coalesced = subPlan.collect { + case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.nonEmpty => + f.optionalNumCoalescedBuckets.get + } + if (expected.isDefined) { + assert(coalesced.size == 1 && coalesced(0) == expected.get) + } else { + assert(coalesced.isEmpty) + } + } + + verify(s.leftRelation.expectedCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].left) + verify(s.rightRelation.expectedCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].right) + } + } + + test("bucket coalescing - basic") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, Some(4)), isSortMergeJoin = true)) + } + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { + run(JoinSetting(RelationSetting(4, None), RelationSetting(8, None), isSortMergeJoin = true)) + } + } + + test("bucket coalescing should work only for sort merge join") { + Seq(true, false).foreach { enabled => + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> enabled.toString) { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, None), isSortMergeJoin = false)) + } + } + } + + test("bucket coalescing shouldn't be applied when the number of buckets are the same") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + run(JoinSetting(RelationSetting(8, None), RelationSetting(8, None), isSortMergeJoin = true)) + } + } + + test("number of bucket is not divisible by other number of bucket") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + run(JoinSetting(RelationSetting(3, None), RelationSetting(8, None), isSortMergeJoin = true)) + } + } + + test("the ratio of the number of buckets is greater than max allowed") { + withSQLConf( + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { + run(JoinSetting(RelationSetting(4, None), RelationSetting(16, None), isSortMergeJoin = true)) + } + } + + test("join keys should match with output partitioning") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + val lCols = Seq( + AttributeReference("l1", IntegerType)(), + AttributeReference("l2", IntegerType)()) + val rCols = Seq( + AttributeReference("r1", IntegerType)(), + AttributeReference("r2", IntegerType)()) + + val lRel = RelationSetting(lCols, 4, None) + val rRel = RelationSetting(rCols, 8, None) + + // The following should not be coalesced because join keys do not match with output + // partitioning (missing one expression). + run(JoinSetting( + leftKeys = Seq(lCols.head), + rightKeys = Seq(rCols.head), + leftRelation = lRel, + rightRelation = rRel, + isSortMergeJoin = true)) + + // The following should not be coalesced because join keys do not match with output + // partitioning (more expressions). + run(JoinSetting( + leftKeys = lCols :+ AttributeReference("l3", IntegerType)(), + rightKeys = rCols :+ AttributeReference("r3", IntegerType)(), + leftRelation = lRel, + rightRelation = rRel, + isSortMergeJoin = true)) + + // The following will be coalesced since ordering should not matter because it will be + // adjusted in `EnsureRequirements`. + run(JoinSetting( + leftKeys = lCols.reverse, + rightKeys = rCols.reverse, + leftRelation = lRel, + rightRelation = RelationSetting(rCols, 8, Some(4)), + isSortMergeJoin = true)) + } + } + + test("FileSourceScanExec's metadata should be updated with coalesced info") { + val scan = newFileSourceScanExec(RelationSetting(8, None)) + val value = scan.metadata("SelectedBucketsCount") + assert(value === "8 out of 8") + + val scanWithCoalescing = scan.copy(optionalNumCoalescedBuckets = Some(4)) + val valueWithCoalescing = scanWithCoalescing.metadata("SelectedBucketsCount") + assert(valueWithCoalescing == "8 out of 8 (Coalesced to 4)") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 1d8303b9e7750..b6767eb3132ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -332,7 +332,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { bucketSpec: Option[BucketSpec], numPartitions: Int = 10, expectedShuffle: Boolean = true, - expectedSort: Boolean = true) + expectedSort: Boolean = true, + expectedNumOutputPartitions: Option[Int] = None) /** * A helper method to test the bucket read functionality using join. It will save `df1` and `df2` @@ -345,10 +346,18 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { bucketedTableTestSpecRight: BucketedTableTestSpec, joinType: String = "inner", joinCondition: (DataFrame, DataFrame) => Column): Unit = { - val BucketedTableTestSpec(bucketSpecLeft, numPartitionsLeft, shuffleLeft, sortLeft) = - bucketedTableTestSpecLeft - val BucketedTableTestSpec(bucketSpecRight, numPartitionsRight, shuffleRight, sortRight) = - bucketedTableTestSpecRight + val BucketedTableTestSpec( + bucketSpecLeft, + numPartitionsLeft, + shuffleLeft, + sortLeft, + numOutputPartitionsLeft) = bucketedTableTestSpecLeft + val BucketedTableTestSpec( + bucketSpecRight, + numPartitionsRight, + shuffleRight, + sortRight, + numOutputPartitionsRight) = bucketedTableTestSpecRight withTable("bucketed_table1", "bucketed_table2") { def withBucket( @@ -413,6 +422,16 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { assert( joinOperator.right.find(_.isInstanceOf[SortExec]).isDefined == sortRight, s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") + + // check the output partitioning + if (numOutputPartitionsLeft.isDefined) { + assert(joinOperator.left.outputPartitioning.numPartitions === + numOutputPartitionsLeft.get) + } + if (numOutputPartitionsRight.isDefined) { + assert(joinOperator.right.outputPartitioning.numPartitions === + numOutputPartitionsRight.get) + } } } } @@ -855,4 +874,112 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } } + + test("bucket coalescing eliminates shuffle") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. + // Currently, sort will be introduced for the side that is coalesced. + val testSpec1 = BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = true, + expectedNumOutputPartitions = Some(4)) + val testSpec2 = BucketedTableTestSpec( + Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = false, + expectedNumOutputPartitions = Some(4)) + + Seq((testSpec1, testSpec2), (testSpec2, testSpec1)).foreach { specs => + testBucketing( + bucketedTableTestSpecLeft = specs._1, + bucketedTableTestSpecRight = specs._2, + joinCondition = joinCondition(Seq("i", "j"))) + } + } + } + + test("bucket coalescing is not satisfied") { + def run(testSpec1: BucketedTableTestSpec, testSpec2: BucketedTableTestSpec): Unit = { + Seq((testSpec1, testSpec2), (testSpec2, testSpec1)).foreach { specs => + testBucketing( + bucketedTableTestSpecLeft = specs._1, + bucketedTableTestSpecRight = specs._2, + joinCondition = joinCondition(Seq("i", "j"))) + } + } + + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { + // Coalescing buckets is disabled by a config. + run( + BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = false), + BucketedTableTestSpec( + Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) + } + + withSQLConf( + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { + // Coalescing buckets is not applied because the ratio of the number of buckets (3) + // is greater than max allowed (2). + run( + BucketedTableTestSpec( + Some(BucketSpec(12, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = false), + BucketedTableTestSpec( + Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) + } + + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + run( + // Coalescing buckets is not applied because the bigger number of buckets (8) is not + // divisible by the smaller number of buckets (7). + BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = false), + BucketedTableTestSpec( + Some(BucketSpec(7, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) + } + } + + test("bucket coalescing is applied when join expressions match with partitioning expressions") { + withTable("t1", "t2") { + df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t1") + df2.write.format("parquet").bucketBy(4, "i", "j").saveAsTable("t2") + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + def verify( + query: String, + expectedNumShuffles: Int, + expectedCoalescedNumBuckets: Option[Int]): Unit = { + val plan = sql(query).queryExecution.executedPlan + val shuffles = plan.collect { case s: ShuffleExchangeExec => s } + assert(shuffles.length == expectedNumShuffles) + + val scans = plan.collect { + case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.isDefined => f + } + if (expectedCoalescedNumBuckets.isDefined) { + assert(scans.length == 1) + assert(scans(0).optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + } else { + assert(scans.isEmpty) + } + } + + // Coalescing applied since join expressions match with the bucket columns. + verify("SELECT * FROM t1 JOIN t2 ON t1.i = t2.i AND t1.j = t2.j", 0, Some(4)) + // Coalescing applied when columns are aliased. + verify( + "SELECT * FROM t1 JOIN (SELECT i AS x, j AS y FROM t2) ON t1.i = x AND t1.j = y", + 0, + Some(4)) + // Coalescing is not applied when join expressions do not match with bucket columns. + verify("SELECT * FROM t1 JOIN t2 ON t1.i = t2.i", 2, None) + } + } + } } From 177a380bcf1f56982760442e8418d28415bef8b0 Mon Sep 17 00:00:00 2001 From: TJX2014 Date: Fri, 19 Jun 2020 19:24:34 -0700 Subject: [PATCH 047/384] [SPARK-31980][SQL] Function sequence() fails if start and end of range are equal dates ### What changes were proposed in this pull request? 1. Add judge equal as bigger condition in `org.apache.spark.sql.catalyst.expressions.Sequence.TemporalSequenceImpl#eval` 2. Unit test for interval `day`, `month`, `year` ### Why are the changes needed? Bug exists when sequence input get same equal start and end dates, which will occur `while loop` forever ### Does this PR introduce _any_ user-facing change? Yes, Before this PR, people will get a `java.lang.ArrayIndexOutOfBoundsException`, when eval as below: `sql("select sequence(cast('2011-03-01' as date), cast('2011-03-01' as date), interval 1 year)").show(false) ` ### How was this patch tested? Unit test. Closes #28819 from TJX2014/master-SPARK-31980. Authored-by: TJX2014 Signed-off-by: Dongjoon Hyun --- .../expressions/collectionOperations.scala | 4 ++-- .../CollectionExpressionsSuite.scala | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index b32e9ee05f1ef..026a2a677baec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -2628,7 +2628,7 @@ object Sequence { val maxEstimatedArrayLength = getSequenceLength(startMicros, stopMicros, intervalStepInMicros) - val stepSign = if (stopMicros > startMicros) +1 else -1 + val stepSign = if (stopMicros >= startMicros) +1 else -1 val exclusiveItem = stopMicros + stepSign val arr = new Array[T](maxEstimatedArrayLength) var t = startMicros @@ -2690,7 +2690,7 @@ object Sequence { | | $sequenceLengthCode | - | final int $stepSign = $stopMicros > $startMicros ? +1 : -1; + | final int $stepSign = $stopMicros >= $startMicros ? +1 : -1; | final long $exclusiveItem = $stopMicros + $stepSign; | | $arr = new $elemType[$arrLength]; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index a4baad0c72444..3a0c02b29d92c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -1836,4 +1836,22 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(ArrayIntersect(empty, oneNull), Seq.empty) checkEvaluation(ArrayIntersect(oneNull, empty), Seq.empty) } + + test("SPARK-31980: Start and end equal in month range") { + checkEvaluation(new Sequence( + Literal(Date.valueOf("2018-01-01")), + Literal(Date.valueOf("2018-01-01")), + Literal(stringToInterval("interval 1 day"))), + Seq(Date.valueOf("2018-01-01"))) + checkEvaluation(new Sequence( + Literal(Date.valueOf("2018-01-01")), + Literal(Date.valueOf("2018-01-01")), + Literal(stringToInterval("interval 1 month"))), + Seq(Date.valueOf("2018-01-01"))) + checkEvaluation(new Sequence( + Literal(Date.valueOf("2018-01-01")), + Literal(Date.valueOf("2018-01-01")), + Literal(stringToInterval("interval 1 year"))), + Seq(Date.valueOf("2018-01-01"))) + } } From 66ba35666a49023cd8da30cf84aff5c6fb1d8799 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 19 Jun 2020 19:33:13 -0700 Subject: [PATCH 048/384] [SPARK-32021][SQL] Increase precision of seconds and fractions of `make_interval` ### What changes were proposed in this pull request? Change precision of seconds and its fraction from 8 to 18 to be able to construct intervals of max allowed microseconds value (long). ### Why are the changes needed? To improve UX of Spark SQL. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? - Add tests to IntervalExpressionsSuite - Add an example to the `MakeInterval` expression - Add tests to `interval.sql` Closes #28873 from MaxGekk/make_interval-sec-precision. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- .../expressions/intervalExpressions.scala | 8 ++++--- .../IntervalExpressionsSuite.scala | 9 +++++++- .../sql-functions/sql-expression-schema.md | 2 +- .../resources/sql-tests/inputs/interval.sql | 2 ++ .../sql-tests/results/ansi/interval.sql.out | 21 +++++++++++++++++-- .../sql-tests/results/interval.sql.out | 20 ++++++++++++++++-- 6 files changed, 53 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index baab224691bc1..db3b2a38fece0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -149,6 +149,8 @@ case class DivideInterval( 100 years 11 months 8 days 12 hours 30 minutes 1.001001 seconds > SELECT _FUNC_(100, null, 3); NULL + > SELECT _FUNC_(0, 1, 0, 1, 0, 0, 100.000001); + 1 months 1 days 1 minutes 40.000001 seconds """, since = "3.0.0") // scalastyle:on line.size.limit @@ -169,7 +171,7 @@ case class MakeInterval( days: Expression, hours: Expression, mins: Expression) = { - this(years, months, weeks, days, hours, mins, Literal(Decimal(0, 8, 6))) + this(years, months, weeks, days, hours, mins, Literal(Decimal(0, Decimal.MAX_LONG_DIGITS, 6))) } def this( years: Expression, @@ -191,7 +193,7 @@ case class MakeInterval( // Accept `secs` as DecimalType to avoid loosing precision of microseconds while converting // them to the fractional part of `secs`. override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType, IntegerType, IntegerType, - IntegerType, IntegerType, IntegerType, DecimalType(8, 6)) + IntegerType, IntegerType, IntegerType, DecimalType(Decimal.MAX_LONG_DIGITS, 6)) override def dataType: DataType = CalendarIntervalType override def nullable: Boolean = true @@ -211,7 +213,7 @@ case class MakeInterval( day.asInstanceOf[Int], hour.asInstanceOf[Int], min.asInstanceOf[Int], - sec.map(_.asInstanceOf[Decimal]).getOrElse(Decimal(0, 8, 6))) + sec.map(_.asInstanceOf[Decimal]).getOrElse(Decimal(0, Decimal.MAX_LONG_DIGITS, 6))) } catch { case _: ArithmeticException => null } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index 8c972a9917043..6b7be4f1609a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -190,7 +190,8 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { micros: Int = 0): Unit = { val secFrac = DateTimeTestUtils.secFrac(seconds, millis, micros) val intervalExpr = MakeInterval(Literal(years), Literal(months), Literal(weeks), - Literal(days), Literal(hours), Literal(minutes), Literal(Decimal(secFrac, 8, 6))) + Literal(days), Literal(hours), Literal(minutes), + Literal(Decimal(secFrac, Decimal.MAX_LONG_DIGITS, 6))) val totalMonths = years * MONTHS_PER_YEAR + months val totalDays = weeks * DAYS_PER_WEEK + days val totalMicros = secFrac + minutes * MICROS_PER_MINUTE + hours * MICROS_PER_HOUR @@ -206,5 +207,11 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { check(years = 10000, micros = -1) check(-9999, -11, 0, -31, -23, -59, -59, -999, -999) check(years = -10000, micros = 1) + check( + hours = Int.MaxValue, + minutes = Int.MaxValue, + seconds = Int.MaxValue, + millis = Int.MaxValue, + micros = Int.MaxValue) } } 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 1a9d5bb775a0a..8898a11ec08fb 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 @@ -166,7 +166,7 @@ | 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 | +| org.apache.spark.sql.catalyst.expressions.MakeInterval | make_interval | SELECT make_interval(100, 11, 1, 1, 12, 30, 01.001001) | struct | | org.apache.spark.sql.catalyst.expressions.MakeTimestamp | make_timestamp | SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887) | struct | | org.apache.spark.sql.catalyst.expressions.MapConcat | map_concat | SELECT map_concat(map(1, 'a', 2, 'b'), map(3, 'c')) | struct> | | org.apache.spark.sql.catalyst.expressions.MapEntries | map_entries | SELECT map_entries(map(1, 'a', 2, 'b')) | struct>> | diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index 76d4a8c87eef5..e881250ed7bee 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -29,6 +29,8 @@ select make_interval(1, 2, 3, 4); select make_interval(1, 2, 3, 4, 5); select make_interval(1, 2, 3, 4, 5, 6); select make_interval(1, 2, 3, 4, 5, 6, 7.008009); +select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456); +select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789); -- cast string to intervals select cast('1 second' as interval); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 39b230fd19f3d..8644c668782ae 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 91 +-- Number of queries: 93 -- !query @@ -178,11 +178,28 @@ struct -- !query select make_interval(1, 2, 3, 4, 5, 6, 7.008009) -- !query schema -struct +struct -- !query output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds +-- !query +select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 34293552 hours 30 minutes 12.123456 seconds + + +-- !query +select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Decimal(expanded,1234567890123456789,20,0}) cannot be represented as Decimal(18, 6). + + -- !query select cast('1 second' as interval) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 01db43ce9e8bc..438a2766061b4 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 91 +-- Number of queries: 93 -- !query @@ -173,11 +173,27 @@ struct -- !query select make_interval(1, 2, 3, 4, 5, 6, 7.008009) -- !query schema -struct +struct -- !query output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds +-- !query +select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 34293552 hours 30 minutes 12.123456 seconds + + +-- !query +select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) +-- !query schema +struct +-- !query output +NULL + + -- !query select cast('1 second' as interval) -- !query schema From 93529a8536a1b67b590101674d3cc58f908daf05 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 19 Jun 2020 19:54:46 -0700 Subject: [PATCH 049/384] [SPARK-31957][SQL] Cleanup hive scratch dir for the developer api startWithContext ### What changes were proposed in this pull request? Comparing to the long-running ThriftServer via start-script, we are more likely to hit the issue https://issues.apache.org/jira/browse/HIVE-10415 / https://issues.apache.org/jira/browse/SPARK-31626 in the developer API `startWithContext` This PR apply SPARK-31626 to the developer API `startWithContext` ### Why are the changes needed? Fix the issue described in SPARK-31626 ### Does this PR introduce _any_ user-facing change? Yes, the hive scratch dir will be deleted if cleanup is enabled for calling `startWithContext` ### How was this patch tested? new test Closes #28784 from yaooqinn/SPARK-31957. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../hive/thriftserver/HiveThriftServer2.scala | 19 ++++++------------- .../thriftserver/SharedThriftServer.scala | 15 +++++++++++++++ .../ThriftServerWithSparkContextSuite.scala | 4 ++++ 3 files changed, 25 insertions(+), 13 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index f9f2ceeed8a75..4e6729faced43 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -52,14 +52,17 @@ object HiveThriftServer2 extends Logging { */ @DeveloperApi def startWithContext(sqlContext: SQLContext): HiveThriftServer2 = { - val server = new HiveThriftServer2(sqlContext) - val executionHive = HiveUtils.newClientForExecution( sqlContext.sparkContext.conf, sqlContext.sessionState.newHadoopConf()) + // Cleanup the scratch dir before starting + ServerUtils.cleanUpScratchDir(executionHive.conf) + val server = new HiveThriftServer2(sqlContext) + server.init(executionHive.conf) server.start() + logInfo("HiveThriftServer2 started") createListenerAndUI(server, sqlContext.sparkContext) server } @@ -97,18 +100,8 @@ object HiveThriftServer2 extends Logging { uiTab.foreach(_.detach()) } - val executionHive = HiveUtils.newClientForExecution( - SparkSQLEnv.sqlContext.sparkContext.conf, - SparkSQLEnv.sqlContext.sessionState.newHadoopConf()) - try { - // Cleanup the scratch dir before starting - ServerUtils.cleanUpScratchDir(executionHive.conf) - val server = new HiveThriftServer2(SparkSQLEnv.sqlContext) - server.init(executionHive.conf) - server.start() - logInfo("HiveThriftServer2 started") - createListenerAndUI(server, SparkSQLEnv.sparkContext) + startWithContext(SparkSQLEnv.sqlContext) // If application was killed before HiveThriftServer2 start successfully then SparkSubmit // process can not exit, so check whether if SparkContext was stopped. if (SparkSQLEnv.sparkContext.stopped.get()) { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index df86cdef3a337..3d7933fba17d8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive.thriftserver +import java.io.File import java.sql.{DriverManager, Statement} import scala.collection.JavaConverters._ @@ -29,12 +30,20 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftCLIService import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.util.Utils trait SharedThriftServer extends SharedSparkSession { private var hiveServer2: HiveThriftServer2 = _ private var serverPort: Int = 0 + protected val tempScratchDir: File = { + val dir = Utils.createTempDir() + dir.setWritable(true, false) + Utils.createTempDir(dir.getAbsolutePath) + dir + } + def mode: ServerMode.Value override def beforeAll(): Unit = { @@ -91,6 +100,8 @@ trait SharedThriftServer extends SharedSparkSession { sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) + sqlContext.setConf(ConfVars.SCRATCHDIR.varname, tempScratchDir.getAbsolutePath) + sqlContext.setConf(ConfVars.HIVE_START_CLEANUP_SCRATCHDIR.varname, "true") try { hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) @@ -101,6 +112,10 @@ trait SharedThriftServer extends SharedSparkSession { case _ => } + // the scratch dir will be recreated after the probe sql `SELECT 1` executed, so we + // check it here first. + assert(!tempScratchDir.exists()) + // Wait for thrift server to be ready to serve the query, via executing simple query // till the query succeeds. See SPARK-30345 for more details. eventually(timeout(30.seconds), interval(1.seconds)) { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index d6420dee41adb..1382eb2d79f38 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -19,6 +19,10 @@ package org.apache.spark.sql.hive.thriftserver trait ThriftServerWithSparkContextSuite extends SharedThriftServer { + test("the scratch dir will be deleted during server start but recreated with new operation") { + assert(tempScratchDir.exists()) + } + test("SPARK-29911: Uncache cached tables when session closed") { val cacheManager = spark.sharedState.cacheManager val globalTempDB = spark.sharedState.globalTempViewManager.database From 297016e34e29858f6f0d003ce3d1ee85b1599f9b Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 20 Jun 2020 08:43:28 -0500 Subject: [PATCH 050/384] [SPARK-31893][ML] Add a generic ClassificationSummary trait ### What changes were proposed in this pull request? Add a generic ClassificationSummary trait ### Why are the changes needed? Add a generic ClassificationSummary trait so all the classification models can use it to implement summary. Currently in classification, we only have summary implemented in ```LogisticRegression```. There are requests to implement summary for ```LinearSVCModel``` in https://issues.apache.org/jira/browse/SPARK-20249 and to implement summary for ```RandomForestClassificationModel``` in https://issues.apache.org/jira/browse/SPARK-23631. If we add a generic ClassificationSummary trait and put all the common code there, we can easily add summary to ```LinearSVCModel``` and ```RandomForestClassificationModel```, and also add summary to all the other classification models. We can use the same approach to add a generic RegressionSummary trait to regression package and implement summary for all the regression models. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? existing tests Closes #28710 from huaxingao/summary_trait. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../ClassificationSummary.scala | 246 +++++++++ .../classification/LogisticRegression.scala | 225 +------- .../LogisticRegressionSuite.scala | 4 +- project/MimaExcludes.scala | 29 +- python/pyspark/ml/classification.py | 505 +++++++++--------- 5 files changed, 543 insertions(+), 466 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala new file mode 100644 index 0000000000000..e9ea38161d3c0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +import org.apache.spark.annotation.Since +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.mllib.evaluation.{BinaryClassificationMetrics, MulticlassMetrics} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.types.DoubleType + + +/** + * Abstraction for multiclass classification results for a given model. + */ +private[classification] trait ClassificationSummary extends Serializable { + + /** + * Dataframe output by the model's `transform` method. + */ + @Since("3.1.0") + def predictions: DataFrame + + /** Field in "predictions" which gives the prediction of each class. */ + @Since("3.1.0") + def predictionCol: String + + /** Field in "predictions" which gives the true label of each instance (if available). */ + @Since("3.1.0") + def labelCol: String + + /** Field in "predictions" which gives the weight of each instance as a vector. */ + @Since("3.1.0") + def weightCol: String + + @transient private val multiclassMetrics = { + val weightColumn = if (predictions.schema.fieldNames.contains(weightCol)) { + col(weightCol).cast(DoubleType) + } else { + lit(1.0) + } + new MulticlassMetrics( + predictions.select(col(predictionCol), col(labelCol).cast(DoubleType), weightColumn) + .rdd.map { + case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight) + }) + } + + /** + * Returns the sequence of labels in ascending order. This order matches the order used + * in metrics which are specified as arrays over labels, e.g., truePositiveRateByLabel. + * + * Note: In most cases, it will be values {0.0, 1.0, ..., numClasses-1}, However, if the + * training set is missing a label, then all of the arrays over labels + * (e.g., from truePositiveRateByLabel) will be of length numClasses-1 instead of the + * expected numClasses. + */ + @Since("3.1.0") + def labels: Array[Double] = multiclassMetrics.labels + + /** Returns true positive rate for each label (category). */ + @Since("3.1.0") + def truePositiveRateByLabel: Array[Double] = recallByLabel + + /** Returns false positive rate for each label (category). */ + @Since("3.1.0") + def falsePositiveRateByLabel: Array[Double] = { + multiclassMetrics.labels.map(label => multiclassMetrics.falsePositiveRate(label)) + } + + /** Returns precision for each label (category). */ + @Since("3.1.0") + def precisionByLabel: Array[Double] = { + multiclassMetrics.labels.map(label => multiclassMetrics.precision(label)) + } + + /** Returns recall for each label (category). */ + @Since("3.1.0") + def recallByLabel: Array[Double] = { + multiclassMetrics.labels.map(label => multiclassMetrics.recall(label)) + } + + /** Returns f-measure for each label (category). */ + @Since("3.1.0") + def fMeasureByLabel(beta: Double): Array[Double] = { + multiclassMetrics.labels.map(label => multiclassMetrics.fMeasure(label, beta)) + } + + /** Returns f1-measure for each label (category). */ + @Since("3.1.0") + def fMeasureByLabel: Array[Double] = fMeasureByLabel(1.0) + + /** + * Returns accuracy. + * (equals to the total number of correctly classified instances + * out of the total number of instances.) + */ + @Since("3.1.0") + def accuracy: Double = multiclassMetrics.accuracy + + /** + * Returns weighted true positive rate. + * (equals to precision, recall and f-measure) + */ + @Since("3.1.0") + def weightedTruePositiveRate: Double = weightedRecall + + /** Returns weighted false positive rate. */ + @Since("3.1.0") + def weightedFalsePositiveRate: Double = multiclassMetrics.weightedFalsePositiveRate + + /** + * Returns weighted averaged recall. + * (equals to precision, recall and f-measure) + */ + @Since("3.1.0") + def weightedRecall: Double = multiclassMetrics.weightedRecall + + /** Returns weighted averaged precision. */ + @Since("3.1.0") + def weightedPrecision: Double = multiclassMetrics.weightedPrecision + + /** Returns weighted averaged f-measure. */ + @Since("3.1.0") + def weightedFMeasure(beta: Double): Double = multiclassMetrics.weightedFMeasure(beta) + + /** Returns weighted averaged f1-measure. */ + @Since("3.1.0") + def weightedFMeasure: Double = multiclassMetrics.weightedFMeasure(1.0) +} + +/** + * Abstraction for training results. + */ +private[classification] trait TrainingSummary { + + /** + * objective function (scaled loss + regularization) at each iteration. + * It contains one more element, the initial state, than number of iterations. + */ + @Since("3.1.0") + def objectiveHistory: Array[Double] + + /** Number of training iterations. */ + @Since("3.1.0") + def totalIterations: Int = { + assert(objectiveHistory.length > 0, "objectiveHistory length should be greater than 0.") + objectiveHistory.length - 1 + } +} + +/** + * Abstraction for binary classification results for a given model. + */ +private[classification] trait BinaryClassificationSummary extends ClassificationSummary { + + private val sparkSession = predictions.sparkSession + import sparkSession.implicits._ + + /** + * Field in "predictions" which gives the probability or rawPrediction of each class as a + * vector. + */ + def scoreCol: String = null + + @transient private val binaryMetrics = { + val weightColumn = if (predictions.schema.fieldNames.contains(weightCol)) { + col(weightCol).cast(DoubleType) + } else { + lit(1.0) + } + + // TODO: Allow the user to vary the number of bins using a setBins method in + // BinaryClassificationMetrics. For now the default is set to 1000. + new BinaryClassificationMetrics( + predictions.select(col(scoreCol), col(labelCol).cast(DoubleType), weightColumn).rdd.map { + case Row(score: Vector, label: Double, weight: Double) => (score(1), label, weight) + }, 1000 + ) + } + + /** + * Returns the receiver operating characteristic (ROC) curve, + * which is a Dataframe having two fields (FPR, TPR) + * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. + * See http://en.wikipedia.org/wiki/Receiver_operating_characteristic + */ + @Since("3.1.0") + @transient lazy val roc: DataFrame = binaryMetrics.roc().toDF("FPR", "TPR") + + /** + * Computes the area under the receiver operating characteristic (ROC) curve. + */ + @Since("3.1.0") + lazy val areaUnderROC: Double = binaryMetrics.areaUnderROC() + + /** + * Returns the precision-recall curve, which is a Dataframe containing + * two fields recall, precision with (0.0, 1.0) prepended to it. + */ + @Since("3.1.0") + @transient lazy val pr: DataFrame = binaryMetrics.pr().toDF("recall", "precision") + + /** + * Returns a dataframe with two fields (threshold, F-Measure) curve with beta = 1.0. + */ + @Since("3.1.0") + @transient lazy val fMeasureByThreshold: DataFrame = { + binaryMetrics.fMeasureByThreshold().toDF("threshold", "F-Measure") + } + + /** + * Returns a dataframe with two fields (threshold, precision) curve. + * Every possible probability obtained in transforming the dataset are used + * as thresholds used in calculating the precision. + */ + @Since("3.1.0") + @transient lazy val precisionByThreshold: DataFrame = { + binaryMetrics.precisionByThreshold().toDF("threshold", "precision") + } + + /** + * Returns a dataframe with two fields (threshold, recall) curve. + * Every possible probability obtained in transforming the dataset are used + * as thresholds used in calculating the recall. + */ + @Since("3.1.0") + @transient lazy val recallByThreshold: DataFrame = { + binaryMetrics.recallByThreshold().toDF("threshold", "recall") + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 1f5976c59235b..20d619334f7b9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -29,7 +29,6 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.feature._ -import org.apache.spark.ml.functions.checkNonNegativeWeight import org.apache.spark.ml.linalg._ import org.apache.spark.ml.optim.aggregator._ import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} @@ -38,12 +37,10 @@ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.stat._ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.mllib.evaluation.{BinaryClassificationMetrics, MulticlassMetrics} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} -import org.apache.spark.sql.functions.{col, lit} -import org.apache.spark.sql.types.{DataType, DoubleType, StructType} +import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.VersionUtils @@ -1396,136 +1393,16 @@ object LogisticRegressionModel extends MLReadable[LogisticRegressionModel] { /** * Abstraction for logistic regression results for a given model. */ -sealed trait LogisticRegressionSummary extends Serializable { - - /** - * Dataframe output by the model's `transform` method. - */ - @Since("1.5.0") - def predictions: DataFrame +sealed trait LogisticRegressionSummary extends ClassificationSummary { /** Field in "predictions" which gives the probability of each class as a vector. */ @Since("1.5.0") def probabilityCol: String - /** Field in "predictions" which gives the prediction of each class. */ - @Since("2.3.0") - def predictionCol: String - - /** Field in "predictions" which gives the true label of each instance (if available). */ - @Since("1.5.0") - def labelCol: String - /** Field in "predictions" which gives the features of each instance as a vector. */ @Since("1.6.0") def featuresCol: String - /** Field in "predictions" which gives the weight of each instance as a vector. */ - @Since("3.1.0") - def weightCol: String - - @transient private val multiclassMetrics = { - if (predictions.schema.fieldNames.contains(weightCol)) { - new MulticlassMetrics( - predictions.select( - col(predictionCol), - col(labelCol).cast(DoubleType), - checkNonNegativeWeight(col(weightCol).cast(DoubleType))).rdd.map { - case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight) - }) - } else { - new MulticlassMetrics( - predictions.select( - col(predictionCol), - col(labelCol).cast(DoubleType), - lit(1.0)).rdd.map { - case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight) - }) - } - } - - /** - * Returns the sequence of labels in ascending order. This order matches the order used - * in metrics which are specified as arrays over labels, e.g., truePositiveRateByLabel. - * - * Note: In most cases, it will be values {0.0, 1.0, ..., numClasses-1}, However, if the - * training set is missing a label, then all of the arrays over labels - * (e.g., from truePositiveRateByLabel) will be of length numClasses-1 instead of the - * expected numClasses. - */ - @Since("2.3.0") - def labels: Array[Double] = multiclassMetrics.labels - - /** Returns true positive rate for each label (category). */ - @Since("2.3.0") - def truePositiveRateByLabel: Array[Double] = recallByLabel - - /** Returns false positive rate for each label (category). */ - @Since("2.3.0") - def falsePositiveRateByLabel: Array[Double] = { - multiclassMetrics.labels.map(label => multiclassMetrics.falsePositiveRate(label)) - } - - /** Returns precision for each label (category). */ - @Since("2.3.0") - def precisionByLabel: Array[Double] = { - multiclassMetrics.labels.map(label => multiclassMetrics.precision(label)) - } - - /** Returns recall for each label (category). */ - @Since("2.3.0") - def recallByLabel: Array[Double] = { - multiclassMetrics.labels.map(label => multiclassMetrics.recall(label)) - } - - /** Returns f-measure for each label (category). */ - @Since("2.3.0") - def fMeasureByLabel(beta: Double): Array[Double] = { - multiclassMetrics.labels.map(label => multiclassMetrics.fMeasure(label, beta)) - } - - /** Returns f1-measure for each label (category). */ - @Since("2.3.0") - def fMeasureByLabel: Array[Double] = fMeasureByLabel(1.0) - - /** - * Returns accuracy. - * (equals to the total number of correctly classified instances - * out of the total number of instances.) - */ - @Since("2.3.0") - def accuracy: Double = multiclassMetrics.accuracy - - /** - * Returns weighted true positive rate. - * (equals to precision, recall and f-measure) - */ - @Since("2.3.0") - def weightedTruePositiveRate: Double = weightedRecall - - /** Returns weighted false positive rate. */ - @Since("2.3.0") - def weightedFalsePositiveRate: Double = multiclassMetrics.weightedFalsePositiveRate - - /** - * Returns weighted averaged recall. - * (equals to precision, recall and f-measure) - */ - @Since("2.3.0") - def weightedRecall: Double = multiclassMetrics.weightedRecall - - /** Returns weighted averaged precision. */ - @Since("2.3.0") - def weightedPrecision: Double = multiclassMetrics.weightedPrecision - - /** Returns weighted averaged f-measure. */ - @Since("2.3.0") - def weightedFMeasure(beta: Double): Double = multiclassMetrics.weightedFMeasure(beta) - - /** Returns weighted averaged f1-measure. */ - @Since("2.3.0") - def weightedFMeasure: Double = multiclassMetrics.weightedFMeasure(1.0) - /** * Convenient method for casting to binary logistic regression summary. * This method will throw an Exception if the summary is not a binary summary. @@ -1540,101 +1417,21 @@ sealed trait LogisticRegressionSummary extends Serializable { /** * Abstraction for multiclass logistic regression training results. - * Currently, the training summary ignores the training weights except - * for the objective trace. */ -sealed trait LogisticRegressionTrainingSummary extends LogisticRegressionSummary { - - /** - * objective function (scaled loss + regularization) at each iteration. - * It contains one more element, the initial state, than number of iterations. - */ - @Since("1.5.0") - def objectiveHistory: Array[Double] - - /** Number of training iterations. */ - @Since("1.5.0") - def totalIterations: Int = { - assert(objectiveHistory.length > 0, s"objectiveHistory length should be greater than 1.") - objectiveHistory.length - 1 - } - +sealed trait LogisticRegressionTrainingSummary extends LogisticRegressionSummary + with TrainingSummary { } /** * Abstraction for binary logistic regression results for a given model. */ -sealed trait BinaryLogisticRegressionSummary extends LogisticRegressionSummary { - - private val sparkSession = predictions.sparkSession - import sparkSession.implicits._ - - // TODO: Allow the user to vary the number of bins using a setBins method in - // BinaryClassificationMetrics. For now the default is set to 100. - @transient private val binaryMetrics = if (predictions.schema.fieldNames.contains(weightCol)) { - new BinaryClassificationMetrics( - predictions.select(col(probabilityCol), col(labelCol).cast(DoubleType), - checkNonNegativeWeight(col(weightCol).cast(DoubleType))).rdd.map { - case Row(score: Vector, label: Double, weight: Double) => (score(1), label, weight) - }, 100 - ) - } else { - new BinaryClassificationMetrics( - predictions.select(col(probabilityCol), col(labelCol).cast(DoubleType), - lit(1.0)).rdd.map { - case Row(score: Vector, label: Double, weight: Double) => (score(1), label, weight) - }, 100 - ) - } +sealed trait BinaryLogisticRegressionSummary extends LogisticRegressionSummary + with BinaryClassificationSummary { - /** - * Returns the receiver operating characteristic (ROC) curve, - * which is a Dataframe having two fields (FPR, TPR) - * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. - * See http://en.wikipedia.org/wiki/Receiver_operating_characteristic - */ - @Since("1.5.0") - @transient lazy val roc: DataFrame = binaryMetrics.roc().toDF("FPR", "TPR") - - /** - * Computes the area under the receiver operating characteristic (ROC) curve. - */ - @Since("1.5.0") - lazy val areaUnderROC: Double = binaryMetrics.areaUnderROC() - - /** - * Returns the precision-recall curve, which is a Dataframe containing - * two fields recall, precision with (0.0, 1.0) prepended to it. - */ - @Since("1.5.0") - @transient lazy val pr: DataFrame = binaryMetrics.pr().toDF("recall", "precision") - - /** - * Returns a dataframe with two fields (threshold, F-Measure) curve with beta = 1.0. - */ - @Since("1.5.0") - @transient lazy val fMeasureByThreshold: DataFrame = { - binaryMetrics.fMeasureByThreshold().toDF("threshold", "F-Measure") - } - - /** - * Returns a dataframe with two fields (threshold, precision) curve. - * Every possible probability obtained in transforming the dataset are used - * as thresholds used in calculating the precision. - */ - @Since("1.5.0") - @transient lazy val precisionByThreshold: DataFrame = { - binaryMetrics.precisionByThreshold().toDF("threshold", "precision") - } - - /** - * Returns a dataframe with two fields (threshold, recall) curve. - * Every possible probability obtained in transforming the dataset are used - * as thresholds used in calculating the recall. - */ - @Since("1.5.0") - @transient lazy val recallByThreshold: DataFrame = { - binaryMetrics.recallByThreshold().toDF("threshold", "recall") + override def scoreCol: String = if (probabilityCol.nonEmpty) { + probabilityCol + } else { + throw new SparkException("probabilityCol is required for BinaryLogisticRegressionSummary.") } } @@ -1674,7 +1471,7 @@ private class LogisticRegressionTrainingSummaryImpl( * * @param predictions dataframe output by the model's `transform` method. * @param probabilityCol field in "predictions" which gives the probability of - * each class as a vector. + * each class as a vector. * @param predictionCol field in "predictions" which gives the prediction for a data instance as a * double. * @param labelCol field in "predictions" which gives the true label of each instance. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 30c21d8b06670..ecee531c88a8f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -313,12 +313,12 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(mlorModel2.summary.isInstanceOf[LogisticRegressionTrainingSummary]) withClue("cannot get binary summary for multiclass model") { intercept[RuntimeException] { - mlorModel.binarySummary + mlorModel2.binarySummary } } withClue("cannot cast summary to binary summary multiclass model") { intercept[RuntimeException] { - mlorModel.summary.asBinary + mlorModel2.summary.asBinary } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index addb2d8152189..0be7b4c1003a7 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -39,6 +39,7 @@ object MimaExcludes { // [SPARK-31077] Remove ChiSqSelector dependency on mllib.ChiSqSelectorModel // private constructor ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.this"), + // [SPARK-31127] Implement abstract Selector // org.apache.spark.ml.feature.ChiSqSelectorModel type hierarchy change // before: class ChiSqSelector extends Estimator with ChiSqSelectorParams @@ -46,11 +47,31 @@ object MimaExcludes { // false positive, no binary incompatibility ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel"), ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.ChiSqSelector"), - //[SPARK-31840] Add instance weight support in LogisticRegressionSummary - // weightCol in org.apache.spark.ml.classification.LogisticRegressionSummary is present only in current version - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightCol"), + // [SPARK-24634] Add a new metric regarding number of inputs later than watermark plus allowed delay - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StateOperatorProgress.$default$4") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StateOperatorProgress.$default$4"), + + //[SPARK-31893] Add a generic ClassificationSummary trait + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionTrainingSummary.org$apache$spark$ml$classification$ClassificationSummary$_setter_$org$apache$spark$ml$classification$ClassificationSummary$$multiclassMetrics_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionTrainingSummary.org$apache$spark$ml$classification$ClassificationSummary$$multiclassMetrics"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionTrainingSummary.weightCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.org$apache$spark$ml$classification$BinaryClassificationSummary$_setter_$org$apache$spark$ml$classification$BinaryClassificationSummary$$sparkSession_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.org$apache$spark$ml$classification$BinaryClassificationSummary$_setter_$org$apache$spark$ml$classification$BinaryClassificationSummary$$binaryMetrics_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.org$apache$spark$ml$classification$BinaryClassificationSummary$$binaryMetrics"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.org$apache$spark$ml$classification$BinaryClassificationSummary$$sparkSession"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.org$apache$spark$ml$classification$ClassificationSummary$_setter_$org$apache$spark$ml$classification$ClassificationSummary$$multiclassMetrics_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.org$apache$spark$ml$classification$ClassificationSummary$$multiclassMetrics"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.weightCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.org$apache$spark$ml$classification$ClassificationSummary$_setter_$org$apache$spark$ml$classification$ClassificationSummary$$multiclassMetrics_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.org$apache$spark$ml$classification$ClassificationSummary$$multiclassMetrics"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.org$apache$spark$ml$classification$BinaryClassificationSummary$_setter_$org$apache$spark$ml$classification$BinaryClassificationSummary$$sparkSession_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.org$apache$spark$ml$classification$BinaryClassificationSummary$_setter_$org$apache$spark$ml$classification$BinaryClassificationSummary$$binaryMetrics_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.org$apache$spark$ml$classification$BinaryClassificationSummary$$binaryMetrics"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.org$apache$spark$ml$classification$BinaryClassificationSummary$$sparkSession"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.org$apache$spark$ml$classification$ClassificationSummary$_setter_$org$apache$spark$ml$classification$ClassificationSummary$$multiclassMetrics_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.org$apache$spark$ml$classification$ClassificationSummary$$multiclassMetrics"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.weightCol") ) // Exclude rules for 3.0.x diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 3f3699ce53b51..ff506066519cd 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -240,6 +240,259 @@ def predictProbability(self, value): return self._call_java("predictProbability", value) +@inherit_doc +class _ClassificationSummary(JavaWrapper): + """ + Abstraction for multiclass classification results for a given model. + + .. versionadded:: 3.1.0 + """ + + @property + @since("3.1.0") + def predictions(self): + """ + Dataframe outputted by the model's `transform` method. + """ + return self._call_java("predictions") + + @property + @since("3.1.0") + def predictionCol(self): + """ + Field in "predictions" which gives the prediction of each class. + """ + return self._call_java("predictionCol") + + @property + @since("3.1.0") + def labelCol(self): + """ + Field in "predictions" which gives the true label of each + instance. + """ + return self._call_java("labelCol") + + @property + @since("3.1.0") + def weightCol(self): + """ + Field in "predictions" which gives the weight of each instance + as a vector. + """ + return self._call_java("weightCol") + + @property + @since("3.1.0") + def labels(self): + """ + Returns the sequence of labels in ascending order. This order matches the order used + in metrics which are specified as arrays over labels, e.g., truePositiveRateByLabel. + + Note: In most cases, it will be values {0.0, 1.0, ..., numClasses-1}, However, if the + training set is missing a label, then all of the arrays over labels + (e.g., from truePositiveRateByLabel) will be of length numClasses-1 instead of the + expected numClasses. + """ + return self._call_java("labels") + + @property + @since("3.1.0") + def truePositiveRateByLabel(self): + """ + Returns true positive rate for each label (category). + """ + return self._call_java("truePositiveRateByLabel") + + @property + @since("3.1.0") + def falsePositiveRateByLabel(self): + """ + Returns false positive rate for each label (category). + """ + return self._call_java("falsePositiveRateByLabel") + + @property + @since("3.1.0") + def precisionByLabel(self): + """ + Returns precision for each label (category). + """ + return self._call_java("precisionByLabel") + + @property + @since("3.1.0") + def recallByLabel(self): + """ + Returns recall for each label (category). + """ + return self._call_java("recallByLabel") + + @since("3.1.0") + def fMeasureByLabel(self, beta=1.0): + """ + Returns f-measure for each label (category). + """ + return self._call_java("fMeasureByLabel", beta) + + @property + @since("3.1.0") + def accuracy(self): + """ + Returns accuracy. + (equals to the total number of correctly classified instances + out of the total number of instances.) + """ + return self._call_java("accuracy") + + @property + @since("3.1.0") + def weightedTruePositiveRate(self): + """ + Returns weighted true positive rate. + (equals to precision, recall and f-measure) + """ + return self._call_java("weightedTruePositiveRate") + + @property + @since("3.1.0") + def weightedFalsePositiveRate(self): + """ + Returns weighted false positive rate. + """ + return self._call_java("weightedFalsePositiveRate") + + @property + @since("3.1.0") + def weightedRecall(self): + """ + Returns weighted averaged recall. + (equals to precision, recall and f-measure) + """ + return self._call_java("weightedRecall") + + @property + @since("3.1.0") + def weightedPrecision(self): + """ + Returns weighted averaged precision. + """ + return self._call_java("weightedPrecision") + + @since("3.1.0") + def weightedFMeasure(self, beta=1.0): + """ + Returns weighted averaged f-measure. + """ + return self._call_java("weightedFMeasure", beta) + + +@inherit_doc +class _TrainingSummary(JavaWrapper): + """ + Abstraction for Training results. + + .. versionadded:: 3.1.0 + """ + + @property + @since("3.1.0") + def objectiveHistory(self): + """ + Objective function (scaled loss + regularization) at each + iteration. It contains one more element, the initial state, + than number of iterations. + """ + return self._call_java("objectiveHistory") + + @property + @since("3.1.0") + def totalIterations(self): + """ + Number of training iterations until termination. + """ + return self._call_java("totalIterations") + + +@inherit_doc +class _BinaryClassificationSummary(_ClassificationSummary): + """ + Binary classification results for a given model. + + .. versionadded:: 3.1.0 + """ + + @property + @since("3.1.0") + def scoreCol(self): + """ + Field in "predictions" which gives the probability or raw prediction + of each class as a vector. + """ + return self._call_java("scoreCol") + + @property + @since("3.1.0") + def roc(self): + """ + Returns the receiver operating characteristic (ROC) curve, + which is a Dataframe having two fields (FPR, TPR) with + (0.0, 0.0) prepended and (1.0, 1.0) appended to it. + + .. seealso:: `Wikipedia reference + `_ + """ + return self._call_java("roc") + + @property + @since("3.1.0") + def areaUnderROC(self): + """ + Computes the area under the receiver operating characteristic + (ROC) curve. + """ + return self._call_java("areaUnderROC") + + @property + @since("3.1.0") + def pr(self): + """ + Returns the precision-recall curve, which is a Dataframe + containing two fields recall, precision with (0.0, 1.0) prepended + to it. + """ + return self._call_java("pr") + + @property + @since("3.1.0") + def fMeasureByThreshold(self): + """ + Returns a dataframe with two fields (threshold, F-Measure) curve + with beta = 1.0. + """ + return self._call_java("fMeasureByThreshold") + + @property + @since("3.1.0") + def precisionByThreshold(self): + """ + Returns a dataframe with two fields (threshold, precision) curve. + Every possible probability obtained in transforming the dataset + are used as thresholds used in calculating the precision. + """ + return self._call_java("precisionByThreshold") + + @property + @since("3.1.0") + def recallByThreshold(self): + """ + Returns a dataframe with two fields (threshold, recall) curve. + Every possible probability obtained in transforming the dataset + are used as thresholds used in calculating the recall. + """ + return self._call_java("recallByThreshold") + + class _LinearSVCParams(_ClassifierParams, HasRegParam, HasMaxIter, HasFitIntercept, HasTol, HasStandardization, HasWeightCol, HasAggregationDepth, HasThreshold, HasBlockSize): @@ -940,21 +1193,13 @@ def evaluate(self, dataset): return LogisticRegressionSummary(java_blr_summary) -class LogisticRegressionSummary(JavaWrapper): +class LogisticRegressionSummary(_ClassificationSummary): """ Abstraction for Logistic Regression Results for a given model. .. versionadded:: 2.0.0 """ - @property - @since("2.0.0") - def predictions(self): - """ - Dataframe outputted by the model's `transform` method. - """ - return self._call_java("predictions") - @property @since("2.0.0") def probabilityCol(self): @@ -964,23 +1209,6 @@ def probabilityCol(self): """ return self._call_java("probabilityCol") - @property - @since("2.3.0") - def predictionCol(self): - """ - Field in "predictions" which gives the prediction of each class. - """ - return self._call_java("predictionCol") - - @property - @since("2.0.0") - def labelCol(self): - """ - Field in "predictions" which gives the true label of each - instance. - """ - return self._call_java("labelCol") - @property @since("2.0.0") def featuresCol(self): @@ -990,241 +1218,26 @@ def featuresCol(self): """ return self._call_java("featuresCol") - @property - @since("3.1.0") - def weightCol(self): - """ - Field in "predictions" which gives the weight of each instance - as a vector. - """ - return self._call_java("weightCol") - - @property - @since("2.3.0") - def labels(self): - """ - Returns the sequence of labels in ascending order. This order matches the order used - in metrics which are specified as arrays over labels, e.g., truePositiveRateByLabel. - - Note: In most cases, it will be values {0.0, 1.0, ..., numClasses-1}, However, if the - training set is missing a label, then all of the arrays over labels - (e.g., from truePositiveRateByLabel) will be of length numClasses-1 instead of the - expected numClasses. - """ - return self._call_java("labels") - - @property - @since("2.3.0") - def truePositiveRateByLabel(self): - """ - Returns true positive rate for each label (category). - """ - return self._call_java("truePositiveRateByLabel") - - @property - @since("2.3.0") - def falsePositiveRateByLabel(self): - """ - Returns false positive rate for each label (category). - """ - return self._call_java("falsePositiveRateByLabel") - - @property - @since("2.3.0") - def precisionByLabel(self): - """ - Returns precision for each label (category). - """ - return self._call_java("precisionByLabel") - - @property - @since("2.3.0") - def recallByLabel(self): - """ - Returns recall for each label (category). - """ - return self._call_java("recallByLabel") - - @since("2.3.0") - def fMeasureByLabel(self, beta=1.0): - """ - Returns f-measure for each label (category). - """ - return self._call_java("fMeasureByLabel", beta) - - @property - @since("2.3.0") - def accuracy(self): - """ - Returns accuracy. - (equals to the total number of correctly classified instances - out of the total number of instances.) - """ - return self._call_java("accuracy") - - @property - @since("2.3.0") - def weightedTruePositiveRate(self): - """ - Returns weighted true positive rate. - (equals to precision, recall and f-measure) - """ - return self._call_java("weightedTruePositiveRate") - - @property - @since("2.3.0") - def weightedFalsePositiveRate(self): - """ - Returns weighted false positive rate. - """ - return self._call_java("weightedFalsePositiveRate") - - @property - @since("2.3.0") - def weightedRecall(self): - """ - Returns weighted averaged recall. - (equals to precision, recall and f-measure) - """ - return self._call_java("weightedRecall") - - @property - @since("2.3.0") - def weightedPrecision(self): - """ - Returns weighted averaged precision. - """ - return self._call_java("weightedPrecision") - - @since("2.3.0") - def weightedFMeasure(self, beta=1.0): - """ - Returns weighted averaged f-measure. - """ - return self._call_java("weightedFMeasure", beta) - @inherit_doc -class LogisticRegressionTrainingSummary(LogisticRegressionSummary): +class LogisticRegressionTrainingSummary(LogisticRegressionSummary, _TrainingSummary): """ Abstraction for multinomial Logistic Regression Training results. - Currently, the training summary ignores the training weights except - for the objective trace. .. versionadded:: 2.0.0 """ - - @property - @since("2.0.0") - def objectiveHistory(self): - """ - Objective function (scaled loss + regularization) at each - iteration. It contains one more element, the initial state, - than number of iterations. - """ - return self._call_java("objectiveHistory") - - @property - @since("2.0.0") - def totalIterations(self): - """ - Number of training iterations until termination. - """ - return self._call_java("totalIterations") + pass @inherit_doc -class BinaryLogisticRegressionSummary(LogisticRegressionSummary): +class BinaryLogisticRegressionSummary(_BinaryClassificationSummary, + LogisticRegressionSummary): """ Binary Logistic regression results for a given model. .. versionadded:: 2.0.0 """ - - @property - @since("2.0.0") - def roc(self): - """ - Returns the receiver operating characteristic (ROC) curve, - which is a Dataframe having two fields (FPR, TPR) with - (0.0, 0.0) prepended and (1.0, 1.0) appended to it. - - .. seealso:: `Wikipedia reference - `_ - - .. note:: This ignores instance weights (setting all to 1.0) from - `LogisticRegression.weightCol`. This will change in later Spark - versions. - """ - return self._call_java("roc") - - @property - @since("2.0.0") - def areaUnderROC(self): - """ - Computes the area under the receiver operating characteristic - (ROC) curve. - - .. note:: This ignores instance weights (setting all to 1.0) from - `LogisticRegression.weightCol`. This will change in later Spark - versions. - """ - return self._call_java("areaUnderROC") - - @property - @since("2.0.0") - def pr(self): - """ - Returns the precision-recall curve, which is a Dataframe - containing two fields recall, precision with (0.0, 1.0) prepended - to it. - - .. note:: This ignores instance weights (setting all to 1.0) from - `LogisticRegression.weightCol`. This will change in later Spark - versions. - """ - return self._call_java("pr") - - @property - @since("2.0.0") - def fMeasureByThreshold(self): - """ - Returns a dataframe with two fields (threshold, F-Measure) curve - with beta = 1.0. - - .. note:: This ignores instance weights (setting all to 1.0) from - `LogisticRegression.weightCol`. This will change in later Spark - versions. - """ - return self._call_java("fMeasureByThreshold") - - @property - @since("2.0.0") - def precisionByThreshold(self): - """ - Returns a dataframe with two fields (threshold, precision) curve. - Every possible probability obtained in transforming the dataset - are used as thresholds used in calculating the precision. - - .. note:: This ignores instance weights (setting all to 1.0) from - `LogisticRegression.weightCol`. This will change in later Spark - versions. - """ - return self._call_java("precisionByThreshold") - - @property - @since("2.0.0") - def recallByThreshold(self): - """ - Returns a dataframe with two fields (threshold, recall) curve. - Every possible probability obtained in transforming the dataset - are used as thresholds used in calculating the recall. - - .. note:: This ignores instance weights (setting all to 1.0) from - `LogisticRegression.weightCol`. This will change in later Spark - versions. - """ - return self._call_java("recallByThreshold") + pass @inherit_doc From 978493467c7423647b8905a1c56081707ef303e2 Mon Sep 17 00:00:00 2001 From: ulysses Date: Sat, 20 Jun 2020 18:38:44 -0700 Subject: [PATCH 051/384] [SPARK-32019][SQL] Add spark.sql.files.minPartitionNum config ### What changes were proposed in this pull request? Add a new config `spark.sql.files.minPartitionNum` to control file split partition in local session. ### Why are the changes needed? Aims to control file split partitions in session level. More details see discuss in [PR-28778](https://github.com/apache/spark/pull/28778). ### Does this PR introduce _any_ user-facing change? Yes, new config. ### How was this patch tested? Add UT. Closes #28853 from ulysses-you/SPARK-32019. Authored-by: ulysses Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/internal/SQLConf.scala | 11 ++++++ .../execution/datasources/FilePartition.scala | 5 +-- .../datasources/FileSourceStrategySuite.scala | 35 +++++++++++++++++++ 3 files changed, 49 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index af6e7eeddbb6e..3149d14c1ddcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1176,6 +1176,15 @@ object SQLConf { .longConf .createWithDefault(4 * 1024 * 1024) + val FILES_MIN_PARTITION_NUM = buildConf("spark.sql.files.minPartitionNum") + .doc("The suggested (not guaranteed) minimum number of split file partitions. " + + "If not set, the default value is `spark.default.parallelism`. This configuration is " + + "effective only when using file-based sources such as Parquet, JSON and ORC.") + .version("3.1.0") + .intConf + .checkValue(v => v > 0, "The min partition number must be a positive integer.") + .createOptional + val IGNORE_CORRUPT_FILES = buildConf("spark.sql.files.ignoreCorruptFiles") .doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " + "encountering corrupted files and the contents that have been read will still be returned. " + @@ -2814,6 +2823,8 @@ class SQLConf extends Serializable with Logging { def filesOpenCostInBytes: Long = getConf(FILES_OPEN_COST_IN_BYTES) + def filesMinPartitionNum: Option[Int] = getConf(FILES_MIN_PARTITION_NUM) + def ignoreCorruptFiles: Boolean = getConf(IGNORE_CORRUPT_FILES) def ignoreMissingFiles: Boolean = getConf(IGNORE_MISSING_FILES) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala index b4fc94e097aa8..095940772ae78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala @@ -88,9 +88,10 @@ object FilePartition extends Logging { selectedPartitions: Seq[PartitionDirectory]): Long = { val defaultMaxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes - val defaultParallelism = sparkSession.sparkContext.defaultParallelism + val minPartitionNum = sparkSession.sessionState.conf.filesMinPartitionNum + .getOrElse(sparkSession.sparkContext.defaultParallelism) val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum - val bytesPerCore = totalBytes / defaultParallelism + val bytesPerCore = totalBytes / minPartitionNum Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 812305ba24403..8a6e6b5ee801d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -528,6 +528,41 @@ class FileSourceStrategySuite extends QueryTest with SharedSparkSession with Pre } } + test("SPARK-32019: Add spark.sql.files.minPartitionNum config") { + withSQLConf(SQLConf.FILES_MIN_PARTITION_NUM.key -> "1") { + val table = + createTable(files = Seq( + "file1" -> 1, + "file2" -> 1, + "file3" -> 1 + )) + assert(table.rdd.partitions.length == 1) + } + + withSQLConf(SQLConf.FILES_MIN_PARTITION_NUM.key -> "10") { + val table = + createTable(files = Seq( + "file1" -> 1, + "file2" -> 1, + "file3" -> 1 + )) + assert(table.rdd.partitions.length == 3) + } + + withSQLConf(SQLConf.FILES_MIN_PARTITION_NUM.key -> "16") { + val partitions = (1 to 100).map(i => s"file$i" -> 128 * 1024 * 1024) + val table = createTable(files = partitions) + // partition is limited by filesMaxPartitionBytes(128MB) + assert(table.rdd.partitions.length == 100) + } + + withSQLConf(SQLConf.FILES_MIN_PARTITION_NUM.key -> "32") { + val partitions = (1 to 800).map(i => s"file$i" -> 4 * 1024 * 1024) + val table = createTable(files = partitions) + assert(table.rdd.partitions.length == 50) + } + } + // Helpers for checking the arguments passed to the FileFormat. protected val checkPartitionSchema = From d2a656c81ef784657a02e7347bfe87e4331fd2c9 Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Sat, 20 Jun 2020 19:20:54 -0700 Subject: [PATCH 052/384] [SPARK-27702][K8S] Allow using some alternatives for service accounts ## What changes were proposed in this pull request? To allow alternatives to serviceaccounts ### Why are the changes needed? Although we provide some authentication configuration, such as spark.kubernetes.authenticate.driver.mounted.oauthTokenFile, spark.kubernetes.authenticate.driver.mounted.caCertFile, etc. But there is a bug as we forced the service account so when we use one of them, driver still use the KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH file, and the error look like bellow: the KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH serviceAccount not exists ### Does this PR introduce any user-facing change? Yes user can now use `spark.kubernetes.authenticate.driver.mounted.caCertFile` or token file by `spark.kubernetes.authenticate.driver.mounted.oauthTokenFile` ## How was this patch tested? Manually passed the certificates using `spark.kubernetes.authenticate.driver.mounted.caCertFile` or token file by `spark.kubernetes.authenticate.driver.mounted.oauthTokenFile` if there is no default service account available. Closes #24601 from Udbhav30/serviceaccount. Authored-by: Udbhav30 Signed-off-by: Dongjoon Hyun --- .../scheduler/cluster/k8s/KubernetesClusterManager.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index b9d7a7083f41a..fcaa3687b14b4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -50,10 +50,14 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit require(sc.conf.get(KUBERNETES_DRIVER_POD_NAME).isDefined, "If the application is deployed using spark-submit in cluster mode, the driver pod name " + "must be provided.") + val serviceAccountToken = + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)).filter(_.exists) + val serviceAccountCaCrt = + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)).filter(_.exists) (KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX, sc.conf.get(KUBERNETES_DRIVER_MASTER_URL), - Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), - Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) + serviceAccountToken, + serviceAccountCaCrt) } else { (KUBERNETES_AUTH_CLIENT_MODE_PREFIX, KubernetesUtils.parseMasterUrl(masterURL), From 9f8e15bb2e2189812ee34e3e64baede0d799ba76 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sun, 21 Jun 2020 16:28:00 -0700 Subject: [PATCH 053/384] [SPARK-32034][SQL] Port HIVE-14817: Shutdown the SessionManager timeoutChecker thread properly upon shutdown ### What changes were proposed in this pull request? This PR port https://issues.apache.org/jira/browse/HIVE-14817 for spark thrift server. ### Why are the changes needed? When stopping the HiveServer2, the non-daemon thread stops the server from terminating ```sql "HiveServer2-Background-Pool: Thread-79" #79 prio=5 os_prio=31 tid=0x00007fde26138800 nid=0x13713 waiting on condition [0x0000700010c32000] java.lang.Thread.State: TIMED_WAITING (sleeping) at java.lang.Thread.sleep(Native Method) at org.apache.hive.service.cli.session.SessionManager$1.sleepInterval(SessionManager.java:178) at org.apache.hive.service.cli.session.SessionManager$1.run(SessionManager.java:156) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) ``` Here is an example to reproduce: https://github.com/yaooqinn/kyuubi/blob/master/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/spark/SparkSQLEngineApp.scala Also, it causes issues as HIVE-14817 described which ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? Passing Jenkins Closes #28870 from yaooqinn/SPARK-32034. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../service/cli/session/SessionManager.java | 32 ++++++++++++++----- .../service/cli/session/SessionManager.java | 32 ++++++++++++++----- 2 files changed, 48 insertions(+), 16 deletions(-) diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index 859f9c8b449e5..ad6fb3ba37a0e 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -148,14 +148,20 @@ public synchronized void start() { } } + private final Object timeoutCheckerLock = new Object(); + private void startTimeoutChecker() { final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds - Runnable timeoutChecker = new Runnable() { + final Runnable timeoutChecker = new Runnable() { @Override public void run() { - for (sleepInterval(interval); !shutdown; sleepInterval(interval)) { + sleepFor(interval); + while (!shutdown) { long current = System.currentTimeMillis(); for (HiveSession session : new ArrayList(handleToSession.values())) { + if (shutdown) { + break; + } if (sessionTimeout > 0 && session.getLastAccessTime() + sessionTimeout <= current && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) { SessionHandle handle = session.getSessionHandle(); @@ -170,24 +176,34 @@ public void run() { session.closeExpiredOperations(); } } + sleepFor(interval); } } - private void sleepInterval(long interval) { - try { - Thread.sleep(interval); - } catch (InterruptedException e) { - // ignore + private void sleepFor(long interval) { + synchronized (timeoutCheckerLock) { + try { + timeoutCheckerLock.wait(interval); + } catch (InterruptedException e) { + // Ignore, and break. + } } } }; backgroundOperationPool.execute(timeoutChecker); } + private void shutdownTimeoutChecker() { + shutdown = true; + synchronized (timeoutCheckerLock) { + timeoutCheckerLock.notify(); + } + } + @Override public synchronized void stop() { super.stop(); - shutdown = true; + shutdownTimeoutChecker(); if (backgroundOperationPool != null) { backgroundOperationPool.shutdown(); long timeout = hiveConf.getTimeVar( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index 49221b13bb892..5a381d170b4f9 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -148,14 +148,20 @@ public synchronized void start() { } } + private final Object timeoutCheckerLock = new Object(); + private void startTimeoutChecker() { final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds - Runnable timeoutChecker = new Runnable() { + final Runnable timeoutChecker = new Runnable() { @Override public void run() { - for (sleepInterval(interval); !shutdown; sleepInterval(interval)) { + sleepFor(interval); + while (!shutdown) { long current = System.currentTimeMillis(); for (HiveSession session : new ArrayList(handleToSession.values())) { + if (shutdown) { + break; + } if (sessionTimeout > 0 && session.getLastAccessTime() + sessionTimeout <= current && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) { SessionHandle handle = session.getSessionHandle(); @@ -170,24 +176,34 @@ public void run() { session.closeExpiredOperations(); } } + sleepFor(interval); } } - private void sleepInterval(long interval) { - try { - Thread.sleep(interval); - } catch (InterruptedException e) { - // ignore + private void sleepFor(long interval) { + synchronized (timeoutCheckerLock) { + try { + timeoutCheckerLock.wait(interval); + } catch (InterruptedException e) { + // Ignore, and break. + } } } }; backgroundOperationPool.execute(timeoutChecker); } + private void shutdownTimeoutChecker() { + shutdown = true; + synchronized (timeoutCheckerLock) { + timeoutCheckerLock.notify(); + } + } + @Override public synchronized void stop() { super.stop(); - shutdown = true; + shutdownTimeoutChecker(); if (backgroundOperationPool != null) { backgroundOperationPool.shutdown(); long timeout = hiveConf.getTimeVar( From aa4c10025a02466e0cb9a79bc1bd82879a321c66 Mon Sep 17 00:00:00 2001 From: mcheah Date: Sun, 21 Jun 2020 19:47:24 -0700 Subject: [PATCH 054/384] [SPARK-31798][SHUFFLE][API] Shuffle Writer API changes to return custom map output metadata Introduces the concept of a `MapOutputMetadata` opaque object that can be returned from map output writers. Note that this PR only proposes the API changes on the shuffle writer side. Following patches will be proposed for actually accepting the metadata on the driver and persisting it in the driver's shuffle metadata storage plugin. ### Why are the changes needed? For a more complete design discussion on this subject as a whole, refer to [this design document](https://docs.google.com/document/d/1Aj6IyMsbS2sdIfHxLvIbHUNjHIWHTabfknIPoxOrTjk/edit#). ### Does this PR introduce any user-facing change? Enables additional APIs for the shuffle storage plugin tree. Usage will become more apparent as the API evolves. ### How was this patch tested? No tests here, since this is only an API-side change that is not consumed by core Spark itself. Closes #28616 from mccheah/return-map-output-metadata. Authored-by: mcheah Signed-off-by: Dongjoon Hyun --- .../shuffle/api/ShuffleMapOutputWriter.java | 12 +++- .../api/metadata/MapOutputCommitMessage.java | 64 +++++++++++++++++++ .../api/metadata/MapOutputMetadata.java | 31 +++++++++ .../sort/BypassMergeSortShuffleWriter.java | 4 +- .../shuffle/sort/UnsafeShuffleWriter.java | 4 +- .../io/LocalDiskShuffleMapOutputWriter.java | 5 +- .../shuffle/sort/SortShuffleWriter.scala | 2 +- ...LocalDiskShuffleMapOutputWriterSuite.scala | 2 +- 8 files changed, 113 insertions(+), 11 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/shuffle/api/metadata/MapOutputCommitMessage.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/api/metadata/MapOutputMetadata.java diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 21abe9a57cd25..0167002ceedb8 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.spark.annotation.Private; +import org.apache.spark.shuffle.api.metadata.MapOutputCommitMessage; /** * :: Private :: @@ -60,10 +61,15 @@ public interface ShuffleMapOutputWriter { *

* This can also close any resources and clean up temporary state if necessary. *

- * The returned array should contain, for each partition from (0) to (numPartitions - 1), the - * number of bytes written by the partition writer for that partition id. + * The returned commit message is a structure with two components: + *

+ * 1) An array of longs, which should contain, for each partition from (0) to + * (numPartitions - 1), the number of bytes written by the partition writer + * for that partition id. + *

+ * 2) An optional metadata blob that can be used by shuffle readers. */ - long[] commitAllPartitions() throws IOException; + MapOutputCommitMessage commitAllPartitions() throws IOException; /** * Abort all of the writes done by any writers returned by {@link #getPartitionWriter(int)}. diff --git a/core/src/main/java/org/apache/spark/shuffle/api/metadata/MapOutputCommitMessage.java b/core/src/main/java/org/apache/spark/shuffle/api/metadata/MapOutputCommitMessage.java new file mode 100644 index 0000000000000..7050690aaddf2 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/api/metadata/MapOutputCommitMessage.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.api.metadata; + +import java.util.Optional; + +import org.apache.spark.annotation.Private; + +/** + * :: Private :: + * + * Represents the result of writing map outputs for a shuffle map task. + *

+ * Partition lengths represents the length of each block written in the map task. This can + * be used for downstream readers to allocate resources, such as in-memory buffers. + *

+ * Map output writers can choose to attach arbitrary metadata tags to register with a + * shuffle output tracker (a module that is currently yet to be built in a future + * iteration of the shuffle storage APIs). + */ +@Private +public final class MapOutputCommitMessage { + + private final long[] partitionLengths; + private final Optional mapOutputMetadata; + + private MapOutputCommitMessage( + long[] partitionLengths, Optional mapOutputMetadata) { + this.partitionLengths = partitionLengths; + this.mapOutputMetadata = mapOutputMetadata; + } + + public static MapOutputCommitMessage of(long[] partitionLengths) { + return new MapOutputCommitMessage(partitionLengths, Optional.empty()); + } + + public static MapOutputCommitMessage of( + long[] partitionLengths, MapOutputMetadata mapOutputMetadata) { + return new MapOutputCommitMessage(partitionLengths, Optional.of(mapOutputMetadata)); + } + + public long[] getPartitionLengths() { + return partitionLengths; + } + + public Optional getMapOutputMetadata() { + return mapOutputMetadata; + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/api/metadata/MapOutputMetadata.java b/core/src/main/java/org/apache/spark/shuffle/api/metadata/MapOutputMetadata.java new file mode 100644 index 0000000000000..6f0e5da4ffeb1 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/api/metadata/MapOutputMetadata.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.api.metadata; + +import java.io.Serializable; + +/** + * :: Private :: + * + * An opaque metadata tag for registering the result of committing the output of a + * shuffle map task. + *

+ * All implementations must be serializable since this is sent from the executors to + * the driver. + */ +public interface MapOutputMetadata extends Serializable {} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index dc157eaa3b253..256789b8c7827 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -130,7 +130,7 @@ public void write(Iterator> records) throws IOException { .createMapOutputWriter(shuffleId, mapId, numPartitions); try { if (!records.hasNext()) { - partitionLengths = mapOutputWriter.commitAllPartitions(); + partitionLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths(); mapStatus = MapStatus$.MODULE$.apply( blockManager.shuffleServerId(), partitionLengths, mapId); return; @@ -219,7 +219,7 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro } partitionWriters = null; } - return mapOutputWriter.commitAllPartitions(); + return mapOutputWriter.commitAllPartitions().getPartitionLengths(); } private void writePartitionedDataWithChannel( diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index d09282e61a9c7..5515a85295d78 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -266,7 +266,7 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { if (spills.length == 0) { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); - return mapWriter.commitAllPartitions(); + return mapWriter.commitAllPartitions().getPartitionLengths(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapId); @@ -327,7 +327,7 @@ private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOExcep // to be counted as shuffle write, but this will lead to double-counting of the final // SpillInfo's bytes. writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); - partitionLengths = mapWriter.commitAllPartitions(); + partitionLengths = mapWriter.commitAllPartitions().getPartitionLengths(); } catch (Exception e) { try { mapWriter.abort(e); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index a6529fd76188a..eea6c762f5c63 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -35,6 +35,7 @@ import org.apache.spark.shuffle.api.WritableByteChannelWrapper; import org.apache.spark.internal.config.package$; import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.shuffle.api.metadata.MapOutputCommitMessage; import org.apache.spark.util.Utils; /** @@ -97,7 +98,7 @@ public ShufflePartitionWriter getPartitionWriter(int reducePartitionId) throws I } @Override - public long[] commitAllPartitions() throws IOException { + public MapOutputCommitMessage commitAllPartitions() throws IOException { // Check the position after transferTo loop to see if it is in the right position and raise a // exception if it is incorrect. The position will not be increased to the expected length // after calling transferTo in kernel version 2.6.32. This issue is described at @@ -113,7 +114,7 @@ public long[] commitAllPartitions() throws IOException { cleanUp(); File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); - return partitionLengths; + return MapOutputCommitMessage.of(partitionLengths); } @Override diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index a391bdf2db44e..83ebe3e12946c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -67,7 +67,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( dep.shuffleId, mapId, dep.partitioner.numPartitions) sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) - val partitionLengths = mapOutputWriter.commitAllPartitions() + val partitionLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index f92455912f510..d2c7d195e06fe 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -136,7 +136,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA } private def verifyWrittenRecords(): Unit = { - val committedLengths = mapOutputWriter.commitAllPartitions() + val committedLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths assert(partitionSizesInMergedFile === partitionLengths) assert(committedLengths === partitionLengths) assert(mergedOutputFile.length() === partitionLengths.sum) From 6fdea63b15ed30ee9e90f075669e4739a2c92339 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 22 Jun 2020 07:56:59 +0000 Subject: [PATCH 055/384] [SPARK-31905][SS] Add compatibility tests for streaming state store format ### What changes were proposed in this pull request? Add compatibility tests for streaming state store format. ### Why are the changes needed? After SPARK-31894, we have a validation checking for the streaming state store. It's better to add integrated tests in the PR builder as soon as the breaking changes introduced. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Test only. Closes #28725 from xuanyuanking/compatibility_check. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../commits/0 | 2 + .../metadata | 1 + .../offsets/0 | 3 + .../state/0/0/1.delta | Bin 0 -> 46 bytes .../state/0/1/1.delta | Bin 0 -> 351 bytes .../state/0/2/1.delta | Bin 0 -> 218 bytes .../state/0/3/1.delta | Bin 0 -> 227 bytes .../state/0/4/1.delta | Bin 0 -> 224 bytes .../commits/0 | 2 + .../metadata | 1 + .../offsets/0 | 3 + .../state/0/0/1.delta | Bin 0 -> 46 bytes .../state/0/1/1.delta | Bin 0 -> 127 bytes .../state/0/2/1.delta | Bin 0 -> 46 bytes .../state/0/3/1.delta | Bin 0 -> 126 bytes .../state/0/4/1.delta | Bin 0 -> 94 bytes .../commits/0 | 2 + .../metadata | 1 + .../offsets/0 | 3 + .../state/0/0/1.delta | Bin 0 -> 320 bytes .../state/0/1/1.delta | Bin 0 -> 46 bytes .../state/0/2/1.delta | Bin 0 -> 209 bytes .../state/0/3/1.delta | Bin 0 -> 198 bytes .../state/0/4/1.delta | Bin 0 -> 205 bytes .../commits/0 | 2 + .../metadata | 1 + .../offsets/0 | 3 + .../state/0/0/1.delta | Bin 0 -> 46 bytes .../state/0/1/1.delta | Bin 0 -> 117 bytes .../state/0/2/1.delta | Bin 0 -> 46 bytes .../state/0/3/1.delta | Bin 0 -> 118 bytes .../state/0/4/1.delta | Bin 0 -> 46 bytes ...ngStateStoreFormatCompatibilitySuite.scala | 249 ++++++++++++++++++ 33 files changed, 273 insertions(+) create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/commits/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/metadata create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/offsets/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/0/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/1/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/2/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/3/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/4/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/commits/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/metadata create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/offsets/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/0/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/1/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/2/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/3/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/4/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/commits/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/metadata create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/offsets/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/0/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/1/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/2/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/3/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/4/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/commits/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/metadata create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/offsets/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/0/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/1/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/2/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/3/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/4/1.delta create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingStateStoreFormatCompatibilitySuite.scala diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/metadata new file mode 100644 index 0000000000000..26a0d5d707d44 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/metadata @@ -0,0 +1 @@ +{"id":"5f1362da-99ce-4ba2-97e1-d992c295e563"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/offsets/0 new file mode 100644 index 0000000000000..43cde63b8f68d --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1591241624051,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/0/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..6352978051846970ca41a0ca97fd79952105726d GIT binary patch literal 46 icmeZ?GI7euPtF!)VPIeY;oA+q9RGp92POd&g989JFAHe^ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/1/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..e7de33ff90ca7ced9d998eae695c92a5bd25ccde GIT binary patch literal 351 zcmY+Au}T9$5Qe|G-HVracza3Ch&Cyb<4}u$22MzE*r<@g!a_tLpq+}XWja${=^NPl z4uYkRVVO73Sx~`&UHHG5`S)k17yE~I57)QDvB1E<+so_P72puI<*~y-w+d1cN3T^f z0ttE2eqYbKBMsMSI4^sjiX3?)1$IObFz8kh^Fjv;HIL|?-eZnQH(I{i9<2}tu;Z8Q zIoWuVR)8&tE11BE#7j_e=I^ASX_Ouz0jX(t7T2tE1c66zqR|`?xv9+nVqgC#T~A)A$`MCe{rBnkrLwpTosHjfu755Q521b19eQ1()1&z^As#KGr+9ymE!;nfu9ze zDU-Ibt-ma7PlfpTE=JRgT$WqpLIFc*@F>l8B*s*#XZ&6eXVZx<&6!|WM`^r=u;d(+ cV4}=B4nk@dpeckTWt=38k7)XPqAdQeFRWi1`Tzg` literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/3/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..82386b020a045d3272790e9453561825943db24b GIT binary patch literal 227 zcmX|+u}T9$5Qe{9_wE*=+=e?jku46vun>d z)_IDQd4o6$A%8J5|3ClCFx}JjZ29)Gxcmja6|6s2#l6%HMWrke)O${@nfRzrC>`f! zih0IZ9$Rt#Ir{mW+(ApT5y>&ID&UGMGR=jLJ$`%);tg`aQs2FDbVGG!LF2|5g@$38 qH(xXppJX8^iY0Iyp@v&!wippdn1|FY1DT{x*~(7P?Y}t{e!xG0N*iwg literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-common-functions/state/0/4/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..ac70b46fc576dcce44561c48dfdcf908d9646bee GIT binary patch literal 224 zcmX|+v1$TA5Qe|qTQ3_7IHI`=Xb}z~3la+vw9Aqzfhz>9Y%MjUO?h2x13o~WB6Uh1 zCC^}EMiB=F=KuMAW~N+>p4aco)x!_4HSo0`Br}}_L(6(1n2(gy(ko&YEDa%d6P!oP z6Q9c7Ywv{SRYodDng6LJ^t=hk^&>S)QN%jmMGVRY-Q#Ib-$-7_?>01tnzYbAhTVp2 l+`(A`kXv3><0jH4Q=BGOXH7Y8#IB1pUS8i2b>K{IF91Z{g literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/metadata new file mode 100644 index 0000000000000..0b78699f07bad --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/metadata @@ -0,0 +1 @@ +{"id":"26fe8d3d-d101-44b0-b9c1-a2f9f09cea69"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/offsets/0 new file mode 100644 index 0000000000000..3c0c901ee0c38 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1592306585407,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/0/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..6352978051846970ca41a0ca97fd79952105726d GIT binary patch literal 46 icmeZ?GI7euPtF!)VPIeY;oA+q9RGp92POd&g989JFAHe^ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/1/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..71ab60304105ab5efe66ce6125cca48d8247294d GIT binary patch literal 127 zcmeZ?GI7euPtH~iWnf@f0>t69@!Y{c+Jb?RK{%a*L6{eaIXD<5GXez-fS6f;LD-ms z!2!q=kYV6vVqgtm5YCWf5at6aoM8TRDS)v;_krgK!E5gD@8mvvM#5GXez-co-O%7}$9j z1O#LlxS1GO0~mx;B^iXdfoj+|7#K_#7+e?_fD9%E0Yg>>CME`+0EPgD|3KgawMZ0b H3>*LeWO)%| literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-deduplicate/state/0/4/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..9a014b2029cecf66612c684e44140678ded59bae GIT binary patch literal 94 zcmeZ?GI7euPtH~~VPIeg0pgd6G7P~$+Jb?RK{${GXez-fS5^uLD-0c a!2!q=kYQkFVhCXP4+K6?gG7PiZ~y?m$_}jn literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/metadata new file mode 100644 index 0000000000000..196f2e0e5c9f3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/metadata @@ -0,0 +1 @@ +{"id":"8fc17276-e48b-4e8a-a9c9-31f0045ed860"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/offsets/0 new file mode 100644 index 0000000000000..622db1a5f4f5b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1591253127196,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/0/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..31a67ed57d7d139cfc461989aa94d05176b49de4 GIT binary patch literal 320 zcmX|7F-yZx7`*rLQs1+UUr5qe1bIO$s2~bT*9378DN+y|1R*E|Q9*|e>EHYinn}s9Nc$Y^LTsb>iX=m9x`C;0pB0Jxf1{`7}izlgE-1w zqTve(e^tWbSiV1WVM)*mgs~@%-i%nNhieFpY-`?yOC!t|UZkxpmQ#!WXz|PL=!Fe5 zYa7B*rO;$>+)yHIs$q9&6p!HtDSSwXp`-|lc1>Xq_9-j%EXq{fU1&k3;2ch-rUkxA zU1oA-=~)cSZRp#5Gz@v5IuAA=-woq<(v?@l!^yxP-pRuf3}g#1Ffs@;b1*nCFmNWYG4L}nNC-LTFbIor zFbHdMFfcPQsB{DiFaTBXYcQ|?rT_EF8z?Y}$sf>S5Cd|HSeuwYqQde_YK;GdS(q63 zW0-P87&|~l8S(ouSOaMj`34UL6Ch&~e--N-AkE3f&jF;$`5ib|`6U=!ls?%1W)K!h gU=U{DU|>1G!LPs|%PPSc;P4*^e4uU@1v(cF0E|x|bpQYW literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/3/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..e0a21939d2645a3ffe69f5a08a6217b0f8a21a5a GIT binary patch literal 198 zcmeZ?GI7euPtI1I!@$5$!^yy)ang=C7|0f2U}O+x;$U!KVBkz(W8i0EkPvdvVGtJL zU=Y^iU|?oqQ0WL3U|{B8;MZVa0ZKoWXZgusD}Nx6!4}99msikc5C<~8FpDq)sZRdI z3}Nz&Y>b^i&i@Mb2P~}n*nrHN{1cds_+1%{fb1CNEeyO49Q+atE=oV_A25ip6f$vV aFz_od$g)UqCOokJ4+K6?2a5vT2nPV3ek2zF literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-statistical-functions/state/0/4/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..91e1e2dbf560be75c5c4b493560cf5e51049058c GIT binary patch literal 205 zcmeZ?GI7euPtI0d%)r1~qiufQP7BEgyP!2UlF_(0t)3Un$Q006^PC6oXF literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/metadata new file mode 100644 index 0000000000000..1f8077da27c29 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/metadata @@ -0,0 +1 @@ +{"id":"ac9f2680-3a39-4afd-824b-7beefdf7d7a7"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/offsets/0 new file mode 100644 index 0000000000000..b0b5ea1df545a --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1591234028611,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/0/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..6352978051846970ca41a0ca97fd79952105726d GIT binary patch literal 46 icmeZ?GI7euPtF!)VPIeY;oA+q9RGp92POd&g989JFAHe^ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/1/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..99110e438ff2f1042d6d3dd47d581943946f6c70 GIT binary patch literal 117 zcmeZ?GI7euPtI2LW?*2b0ph#6@AAe0X#oaC29X3|hA&KvK%N8;TL7^J5Q|7Ma5FJ5 o$S^Q?GDx!mWtbQ~Z~_GwSb&%fh&g~51pWho57c~7h!BJX08XS4RR910 literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/2/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..6352978051846970ca41a0ca97fd79952105726d GIT binary patch literal 46 icmeZ?GI7euPtF!)VPIeY;oA+q9RGp92POd&g989JFAHe^ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.5-for-compatibility-test-sum-decimal/state/0/3/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..cec2e6be7c9f750d302244f5613728329ca456c9 GIT binary patch literal 118 zcmeZ?GI7euPtI2LVPIgW0b*w*U*=#SEx^FYAk4_Y;0R { + // Check the exception message to make sure the state store format changing. + assert(e.getCause.getCause.getMessage.contains( + "The streaming query failed by state format invalidation.")) + }) + ) + } +} From 2e4557f45ce65ad0cf501c1734f2d4a50b00af54 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 22 Jun 2020 04:58:22 -0700 Subject: [PATCH 056/384] [SPARK-32038][SQL] NormalizeFloatingNumbers should also work on distinct aggregate ### What changes were proposed in this pull request? This patch applies `NormalizeFloatingNumbers` to distinct aggregate to fix a regression of distinct aggregate on NaNs. ### Why are the changes needed? We added `NormalizeFloatingNumbers` optimization rule in 3.0.0 to normalize special floating numbers (NaN and -0.0). But it is missing in distinct aggregate so causes a regression. We need to apply this rule on distinct aggregate to fix it. ### Does this PR introduce _any_ user-facing change? Yes, fixing a regression of distinct aggregate on NaNs. ### How was this patch tested? Added unit test. Closes #28876 from viirya/SPARK-32038. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/SparkStrategies.scala | 18 ++++++++++++++++++ .../sql/execution/aggregate/AggUtils.scala | 16 ++++------------ .../spark/sql/DataFrameAggregateSuite.scala | 16 ++++++++++++++++ 3 files changed, 38 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 302aae08d588b..4d23e5e8a65b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -448,10 +448,28 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { resultExpressions, planLater(child)) } else { + // functionsWithDistinct is guaranteed to be non-empty. Even though it may contain + // more than one DISTINCT aggregate function, all of those functions will have the + // same column expressions. For example, it would be valid for functionsWithDistinct + // to be [COUNT(DISTINCT foo), MAX(DISTINCT foo)], but + // [COUNT(DISTINCT bar), COUNT(DISTINCT foo)] is disallowed because those two distinct + // aggregates have different column expressions. + val distinctExpressions = functionsWithDistinct.head.aggregateFunction.children + val normalizedNamedDistinctExpressions = distinctExpressions.map { e => + // Ideally this should be done in `NormalizeFloatingNumbers`, but we do it here + // because `distinctExpressions` is not extracted during logical phase. + NormalizeFloatingNumbers.normalize(e) match { + case ne: NamedExpression => ne + case other => Alias(other, other.toString)() + } + } + AggUtils.planAggregateWithOneDistinct( normalizedGroupingExpressions, functionsWithDistinct, functionsWithoutDistinct, + distinctExpressions, + normalizedNamedDistinctExpressions, resultExpressions, planLater(child)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index 56a287d4d0279..761ac20e84744 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -135,20 +135,12 @@ object AggUtils { groupingExpressions: Seq[NamedExpression], functionsWithDistinct: Seq[AggregateExpression], functionsWithoutDistinct: Seq[AggregateExpression], + distinctExpressions: Seq[Expression], + normalizedNamedDistinctExpressions: Seq[NamedExpression], resultExpressions: Seq[NamedExpression], child: SparkPlan): Seq[SparkPlan] = { - // functionsWithDistinct is guaranteed to be non-empty. Even though it may contain more than one - // DISTINCT aggregate function, all of those functions will have the same column expressions. - // For example, it would be valid for functionsWithDistinct to be - // [COUNT(DISTINCT foo), MAX(DISTINCT foo)], but [COUNT(DISTINCT bar), COUNT(DISTINCT foo)] is - // disallowed because those two distinct aggregates have different column expressions. - val distinctExpressions = functionsWithDistinct.head.aggregateFunction.children - val namedDistinctExpressions = distinctExpressions.map { - case ne: NamedExpression => ne - case other => Alias(other, other.toString)() - } - val distinctAttributes = namedDistinctExpressions.map(_.toAttribute) + val distinctAttributes = normalizedNamedDistinctExpressions.map(_.toAttribute) val groupingAttributes = groupingExpressions.map(_.toAttribute) // 1. Create an Aggregate Operator for partial aggregations. @@ -159,7 +151,7 @@ object AggUtils { // DISTINCT column. For example, for AVG(DISTINCT value) GROUP BY key, the grouping // expressions will be [key, value]. createAggregate( - groupingExpressions = groupingExpressions ++ namedDistinctExpressions, + groupingExpressions = groupingExpressions ++ normalizedNamedDistinctExpressions, aggregateExpressions = aggregateExpressions, aggregateAttributes = aggregateAttributes, resultExpressions = groupingAttributes ++ distinctAttributes ++ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 2293d4ae61aff..f7438f3ffec04 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -1012,4 +1012,20 @@ class DataFrameAggregateSuite extends QueryTest } } } + + test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") { + withTempView("view") { + val nan1 = java.lang.Float.intBitsToFloat(0x7f800001) + val nan2 = java.lang.Float.intBitsToFloat(0x7fffffff) + + Seq(("mithunr", Float.NaN), + ("mithunr", nan1), + ("mithunr", nan2), + ("abellina", 1.0f), + ("abellina", 2.0f)).toDF("uid", "score").createOrReplaceTempView("view") + + val df = spark.sql("select uid, count(distinct score) from view group by 1 order by 1 asc") + checkAnswer(df, Row("abellina", 2) :: Row("mithunr", 1) :: Nil) + } + } } From 6293c38cff66c503370e970b7e8e0af9f993a3c2 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Mon, 22 Jun 2020 21:38:19 +0900 Subject: [PATCH 057/384] [MINOR][SQL] Add `IS [NOT] NULL` examples to ArrayFilter/ArrayExists ### What changes were proposed in this pull request? A minor PR that adds a couple of usage examples for ArrayFilter and ArrayExists that shows how to deal with NULL data. ### Why are the changes needed? Enhances the examples that shows how to filter out null values from an array and also to test if null value exists in an array. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Tested manually. Closes #28890 from dilipbiswal/array_func_description. Authored-by: Dilip Biswal Signed-off-by: HyukjinKwon --- .../sql/catalyst/expressions/higherOrderFunctions.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index e91bd0c7015ee..e5cf8c0a023d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -454,6 +454,8 @@ case class MapFilter( [1,3] > SELECT _FUNC_(array(0, 2, 3), (x, i) -> x > i); [2,3] + > SELECT _FUNC_(array(0, null, 2, 3, null), x -> x IS NOT NULL); + [0,2,3] """, since = "2.4.0", note = """ @@ -518,6 +520,10 @@ case class ArrayFilter( false > SELECT _FUNC_(array(1, null, 3), x -> x % 2 == 0); NULL + > SELECT _FUNC_(array(0, null, 2, 3, null), x -> x IS NULL); + true + > SELECT _FUNC_(array(1, 2, 3), x -> x IS NULL); + false """, since = "2.4.0") case class ArrayExists( From 338efee509661961fca862a0d556674edd8d4bc6 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 22 Jun 2020 13:59:46 +0000 Subject: [PATCH 058/384] [SPARK-32031][SQL] Fix the wrong references of the PartialMerge/Final AggregateExpression ### What changes were proposed in this pull request? This PR changes the references of the `PartialMerge`/`Final` `AggregateExpression` from `aggBufferAttributes` to `inputAggBufferAttributes`. After this change, the tests of `SPARK-31620` can fail on the assertion of `QueryTest.assertEmptyMissingInput`. So, this PR also fixes it by overriding the `inputAggBufferAttributes` of the Aggregate operators. ### Why are the changes needed? With my understanding of Aggregate framework, especially, according to the logic of `AggUtils.planAggXXX`, I think for the `PartialMerge`/`Final` `AggregateExpression` the right references should be `inputAggBufferAttributes`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Before this patch, for an Aggregate operator, its input attributes will always be equal to or more than(because it refers to its own attributes while it should refer to the attributes from the child) its reference attributes. Therefore, its missing inputs must always be empty and break nothing. Thus, it's impossible to add a UT for this patch. However, after correcting the right references in this PR, the problem is then exposed by `QueryTest.assertEmptyMissingInput` in the UT of SPARK-31620, since missing inputs are no longer always empty. This PR can fix the problem. Closes #28869 from Ngone51/fix-agg-reference. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../expressions/aggregate/interfaces.scala | 2 +- .../aggregate/BaseAggregateExec.scala | 31 ++++++++++++++----- .../aggregate/HashAggregateExec.scala | 9 ------ .../aggregate/ObjectHashAggregateExec.scala | 9 ------ .../aggregate/SortAggregateExec.scala | 9 ------ 5 files changed, 25 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index 222ad6fab19e0..8e8862edb6dd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -140,7 +140,7 @@ case class AggregateExpression( override lazy val references: AttributeSet = { val aggAttributes = mode match { case Partial | Complete => aggregateFunction.references - case PartialMerge | Final => AttributeSet(aggregateFunction.aggBufferAttributes) + case PartialMerge | Final => AttributeSet(aggregateFunction.inputAggBufferAttributes) } aggAttributes ++ filterAttributes } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala index f1e053f7fb2a5..f6d04601fc7a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.aggregate -import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, NamedExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Final, PartialMerge} import org.apache.spark.sql.execution.{ExplainUtils, UnaryExecNode} @@ -53,15 +53,32 @@ trait BaseAggregateExec extends UnaryExecNode { // can't bind the `mergeExpressions` with the output of the partial aggregate, as they use // the `inputAggBufferAttributes` of the original `DeclarativeAggregate` before copy. Instead, // we shall use `inputAggBufferAttributes` after copy to match the new `mergeExpressions`. - val aggAttrs = aggregateExpressions - // there're exactly four cases needs `inputAggBufferAttributes` from child according to the - // agg planning in `AggUtils`: Partial -> Final, PartialMerge -> Final, - // Partial -> PartialMerge, PartialMerge -> PartialMerge. - .filter(a => a.mode == Final || a.mode == PartialMerge).map(_.aggregateFunction) - .flatMap(_.inputAggBufferAttributes) + val aggAttrs = inputAggBufferAttributes child.output.dropRight(aggAttrs.length) ++ aggAttrs } else { child.output } } + + private val inputAggBufferAttributes: Seq[Attribute] = { + aggregateExpressions + // there're exactly four cases needs `inputAggBufferAttributes` from child according to the + // agg planning in `AggUtils`: Partial -> Final, PartialMerge -> Final, + // Partial -> PartialMerge, PartialMerge -> PartialMerge. + .filter(a => a.mode == Final || a.mode == PartialMerge) + .flatMap(_.aggregateFunction.inputAggBufferAttributes) + } + + protected val aggregateBufferAttributes: Seq[AttributeReference] = { + aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) + } + + override def producedAttributes: AttributeSet = + AttributeSet(aggregateAttributes) ++ + AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++ + AttributeSet(aggregateBufferAttributes) ++ + // it's not empty when the inputAggBufferAttributes is not equal to the aggregate buffer + // attributes of the child Aggregate, when the child Aggregate contains the subquery in + // AggregateFunction. See SPARK-31620 for more details. + AttributeSet(inputAggBufferAttributes.filterNot(child.output.contains)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 9c07ea10a87e7..40b95df44a83a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -57,10 +57,6 @@ case class HashAggregateExec( with BlockingOperatorWithCodegen with AliasAwareOutputPartitioning { - private[this] val aggregateBufferAttributes = { - aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) - } - require(HashAggregateExec.supportsAggregate(aggregateBufferAttributes)) override lazy val allAttributes: AttributeSeq = @@ -79,11 +75,6 @@ case class HashAggregateExec( override protected def outputExpressions: Seq[NamedExpression] = resultExpressions - override def producedAttributes: AttributeSet = - AttributeSet(aggregateAttributes) ++ - AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++ - AttributeSet(aggregateBufferAttributes) - override def requiredChildDistribution: List[Distribution] = { requiredChildDistributionExpressions match { case Some(exprs) if exprs.isEmpty => AllTuples :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index f1c0719ff8948..231adbeb244fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -69,10 +69,6 @@ case class ObjectHashAggregateExec( child: SparkPlan) extends BaseAggregateExec with AliasAwareOutputPartitioning { - private[this] val aggregateBufferAttributes = { - aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) - } - override lazy val allAttributes: AttributeSeq = child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ aggregateExpressions.flatMap(_.aggregateFunction.inputAggBufferAttributes) @@ -84,11 +80,6 @@ case class ObjectHashAggregateExec( override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) - override def producedAttributes: AttributeSet = - AttributeSet(aggregateAttributes) ++ - AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++ - AttributeSet(aggregateBufferAttributes) - override def requiredChildDistribution: List[Distribution] = { requiredChildDistributionExpressions match { case Some(exprs) if exprs.isEmpty => AllTuples :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index ba0c3517a1a14..48763686e4b14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -42,15 +42,6 @@ case class SortAggregateExec( with AliasAwareOutputPartitioning with AliasAwareOutputOrdering { - private[this] val aggregateBufferAttributes = { - aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) - } - - override def producedAttributes: AttributeSet = - AttributeSet(aggregateAttributes) ++ - AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++ - AttributeSet(aggregateBufferAttributes) - override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) From 979a8eb04ae1dec2928cc49028b3df03b4ba78ce Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 23 Jun 2020 05:53:04 +0000 Subject: [PATCH 059/384] [MINOR][SQL] Simplify DateTimeUtils.cleanLegacyTimestampStr ### What changes were proposed in this pull request? Call the `replace()` method from `UTF8String` to remove the `GMT` string from the input of `DateTimeUtils.cleanLegacyTimestampStr`. It removes all `GMT` substrings. ### Why are the changes needed? Simpler code improves maintainability ### Does this PR introduce _any_ user-facing change? Should not ### How was this patch tested? By existing test suites `JsonSuite` and `UnivocityParserSuite`. Closes #28892 from MaxGekk/simplify-cleanLegacyTimestampStr. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/csv/UnivocityParser.scala | 4 ++-- .../spark/sql/catalyst/json/JacksonParser.scala | 4 ++-- .../spark/sql/catalyst/util/DateTimeUtils.scala | 16 +++------------- 3 files changed, 7 insertions(+), 17 deletions(-) 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 3898eca79478e..898b963fd0ab5 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 @@ -186,7 +186,7 @@ class UnivocityParser( case NonFatal(e) => // If fails to parse, then tries the way used in 2.0 and 1.x for backwards // compatibility. - val str = UTF8String.fromString(DateTimeUtils.cleanLegacyTimestampStr(datum)) + val str = DateTimeUtils.cleanLegacyTimestampStr(UTF8String.fromString(datum)) DateTimeUtils.stringToTimestamp(str, options.zoneId).getOrElse(throw e) } } @@ -199,7 +199,7 @@ class UnivocityParser( case NonFatal(e) => // If fails to parse, then tries the way used in 2.0 and 1.x for backwards // compatibility. - val str = UTF8String.fromString(DateTimeUtils.cleanLegacyTimestampStr(datum)) + val str = DateTimeUtils.cleanLegacyTimestampStr(UTF8String.fromString(datum)) DateTimeUtils.stringToDate(str, options.zoneId).getOrElse(throw e) } } 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 e038f777c7a41..7f69180e87e7e 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 @@ -238,7 +238,7 @@ class JacksonParser( case NonFatal(e) => // If fails to parse, then tries the way used in 2.0 and 1.x for backwards // compatibility. - val str = UTF8String.fromString(DateTimeUtils.cleanLegacyTimestampStr(parser.getText)) + val str = DateTimeUtils.cleanLegacyTimestampStr(UTF8String.fromString(parser.getText)) DateTimeUtils.stringToTimestamp(str, options.zoneId).getOrElse(throw e) } @@ -255,7 +255,7 @@ class JacksonParser( case NonFatal(e) => // If fails to parse, then tries the way used in 2.0 and 1.x for backwards // compatibility. - val str = UTF8String.fromString(DateTimeUtils.cleanLegacyTimestampStr(parser.getText)) + val str = DateTimeUtils.cleanLegacyTimestampStr(UTF8String.fromString(parser.getText)) DateTimeUtils.stringToDate(str, options.zoneId).getOrElse { // In Spark 1.5.0, we store the data as number of days since epoch in string. // So, we just convert it to Int. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index c466a60259c7f..5c3d91c105094 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -203,20 +203,10 @@ object DateTimeUtils { Math.multiplyExact(millis, MICROS_PER_MILLIS) } + private final val gmtUtf8 = UTF8String.fromString("GMT") // The method is called by JSON/CSV parser to clean up the legacy timestamp string by removing - // the "GMT" string. - def cleanLegacyTimestampStr(s: String): String = { - val indexOfGMT = s.indexOf("GMT") - if (indexOfGMT != -1) { - // ISO8601 with a weird time zone specifier (2000-01-01T00:00GMT+01:00) - val s0 = s.substring(0, indexOfGMT) - val s1 = s.substring(indexOfGMT + 3) - // Mapped to 2000-01-01T00:00+01:00 - s0 + s1 - } else { - s - } - } + // the "GMT" string. For example, it returns 2000-01-01T00:00+01:00 for 2000-01-01T00:00GMT+01:00. + def cleanLegacyTimestampStr(s: UTF8String): UTF8String = s.replace(gmtUtf8, UTF8String.EMPTY_UTF8) /** * Trims and parses a given UTF8 timestamp string to the corresponding a corresponding [[Long]] From fcf976809860eb6e4f05fa4d282e0bf774c6faec Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 23 Jun 2020 06:13:55 +0000 Subject: [PATCH 060/384] [SPARK-32052][SQL] Extract common code from date-time field expressions ### What changes were proposed in this pull request? Extract common code from the expressions that get date or time fields from input dates/timestamps to new expressions `GetDateField` and `GetTimeField`, and re-use the common traits from the affected classes. ### Why are the changes needed? Code deduplication improves maintainability. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By `DateExpressionsSuite` Closes #28894 from MaxGekk/get-date-time-field-expr. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../expressions/datetimeExpressions.scala | 290 +++++------------- .../sql/catalyst/util/DateTimeUtils.scala | 20 ++ 2 files changed, 93 insertions(+), 217 deletions(-) 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 b9ba32b8ee337..5d86c8def0d95 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 @@ -274,39 +274,43 @@ case class DateSub(startDate: Expression, days: Expression) override def prettyName: String = "date_sub" } -@ExpressionDescription( - usage = "_FUNC_(timestamp) - Returns the hour component of the string/timestamp.", - examples = """ - Examples: - > SELECT _FUNC_('2009-07-30 12:58:59'); - 12 - """, - group = "datetime_funcs", - since = "1.5.0") -case class Hour(child: Expression, timeZoneId: Option[String] = None) - extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes - with NullIntolerant { +trait GetTimeField extends UnaryExpression + with TimeZoneAwareExpression with ImplicitCastInputTypes with NullIntolerant { - def this(child: Expression) = this(child, None) + val func: (Long, ZoneId) => Any + val funcName: String override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) override def dataType: DataType = IntegerType - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) - override protected def nullSafeEval(timestamp: Any): Any = { - DateTimeUtils.getHours(timestamp.asInstanceOf[Long], zoneId) + func(timestamp.asInstanceOf[Long], zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getHours($c, $zid)") + defineCodeGen(ctx, ev, c => s"$dtu.$funcName($c, $zid)") } } +@ExpressionDescription( + usage = "_FUNC_(timestamp) - Returns the hour component of the string/timestamp.", + examples = """ + Examples: + > SELECT _FUNC_('2009-07-30 12:58:59'); + 12 + """, + group = "datetime_funcs", + since = "1.5.0") +case class Hour(child: Expression, timeZoneId: Option[String] = None) extends GetTimeField { + def this(child: Expression) = this(child, None) + override def withTimeZone(timeZoneId: String): Hour = copy(timeZoneId = Option(timeZoneId)) + override val func = DateTimeUtils.getHours + override val funcName = "getHours" +} + @ExpressionDescription( usage = "_FUNC_(timestamp) - Returns the minute component of the string/timestamp.", examples = """ @@ -316,28 +320,11 @@ case class Hour(child: Expression, timeZoneId: Option[String] = None) """, group = "datetime_funcs", since = "1.5.0") -case class Minute(child: Expression, timeZoneId: Option[String] = None) - extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes - with NullIntolerant { - +case class Minute(child: Expression, timeZoneId: Option[String] = None) extends GetTimeField { def this(child: Expression) = this(child, None) - - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) - - override def dataType: DataType = IntegerType - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) - - override protected def nullSafeEval(timestamp: Any): Any = { - DateTimeUtils.getMinutes(timestamp.asInstanceOf[Long], zoneId) - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getMinutes($c, $zid)") - } + override def withTimeZone(timeZoneId: String): Minute = copy(timeZoneId = Option(timeZoneId)) + override val func = DateTimeUtils.getMinutes + override val funcName = "getMinutes" } @ExpressionDescription( @@ -349,52 +336,39 @@ case class Minute(child: Expression, timeZoneId: Option[String] = None) """, group = "datetime_funcs", since = "1.5.0") -case class Second(child: Expression, timeZoneId: Option[String] = None) - extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes - with NullIntolerant { - +case class Second(child: Expression, timeZoneId: Option[String] = None) extends GetTimeField { def this(child: Expression) = this(child, None) - - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) - - override def dataType: DataType = IntegerType - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) - - override protected def nullSafeEval(timestamp: Any): Any = { - DateTimeUtils.getSeconds(timestamp.asInstanceOf[Long], zoneId) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getSeconds($c, $zid)") - } + override def withTimeZone(timeZoneId: String): Second = copy(timeZoneId = Option(timeZoneId)) + override val func = DateTimeUtils.getSeconds + override val funcName = "getSeconds" } case class SecondWithFraction(child: Expression, timeZoneId: Option[String] = None) - extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes - with NullIntolerant { - + extends GetTimeField { def this(child: Expression) = this(child, None) - - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) - // 2 digits for seconds, and 6 digits for the fractional part with microsecond precision. override def dataType: DataType = DecimalType(8, 6) - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + override def withTimeZone(timeZoneId: String): SecondWithFraction = copy(timeZoneId = Option(timeZoneId)) + override val func = DateTimeUtils.getSecondsWithFraction + override val funcName = "getSecondsWithFraction" +} - override protected def nullSafeEval(timestamp: Any): Any = { - DateTimeUtils.getSecondsWithFraction(timestamp.asInstanceOf[Long], zoneId) +trait GetDateField extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { + val func: Int => Any + val funcName: String + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + func(date.asInstanceOf[Int]) } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getSecondsWithFraction($c, $zid)") + defineCodeGen(ctx, ev, c => s"$dtu.$funcName($c)") } } @@ -407,21 +381,9 @@ case class SecondWithFraction(child: Expression, timeZoneId: Option[String] = No """, group = "datetime_funcs", since = "1.5.0") -case class DayOfYear(child: Expression) - extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = IntegerType - - override protected def nullSafeEval(date: Any): Any = { - DateTimeUtils.getDayInYear(date.asInstanceOf[Int]) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getDayInYear($c)") - } +case class DayOfYear(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getDayInYear + override val funcName = "getDayInYear" } abstract class NumberToTimestampBase extends UnaryExpression @@ -510,38 +472,14 @@ case class MicrosToTimestamp(child: Expression) """, group = "datetime_funcs", since = "1.5.0") -case class Year(child: Expression) - extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = IntegerType - - override protected def nullSafeEval(date: Any): Any = { - DateTimeUtils.getYear(date.asInstanceOf[Int]) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getYear($c)") - } +case class Year(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getYear + override val funcName = "getYear" } -case class YearOfWeek(child: Expression) - extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = IntegerType - - override protected def nullSafeEval(date: Any): Any = { - DateTimeUtils.getWeekBasedYear(date.asInstanceOf[Int]) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getWeekBasedYear($c)") - } +case class YearOfWeek(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getWeekBasedYear + override val funcName = "getWeekBasedYear" } @ExpressionDescription( @@ -553,21 +491,9 @@ case class YearOfWeek(child: Expression) """, group = "datetime_funcs", since = "1.5.0") -case class Quarter(child: Expression) - extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = IntegerType - - override protected def nullSafeEval(date: Any): Any = { - DateTimeUtils.getQuarter(date.asInstanceOf[Int]) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getQuarter($c)") - } +case class Quarter(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getQuarter + override val funcName = "getQuarter" } @ExpressionDescription( @@ -579,21 +505,9 @@ case class Quarter(child: Expression) """, group = "datetime_funcs", since = "1.5.0") -case class Month(child: Expression) - extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = IntegerType - - override protected def nullSafeEval(date: Any): Any = { - DateTimeUtils.getMonth(date.asInstanceOf[Int]) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getMonth($c)") - } +case class Month(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getMonth + override val funcName = "getMonth" } @ExpressionDescription( @@ -604,21 +518,9 @@ case class Month(child: Expression) 30 """, since = "1.5.0") -case class DayOfMonth(child: Expression) - extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = IntegerType - - override protected def nullSafeEval(date: Any): Any = { - DateTimeUtils.getDayOfMonth(date.asInstanceOf[Int]) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getDayOfMonth($c)") - } +case class DayOfMonth(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getDayOfMonth + override val funcName = "getDayOfMonth" } // scalastyle:off line.size.limit @@ -632,20 +534,9 @@ case class DayOfMonth(child: Expression) group = "datetime_funcs", since = "2.3.0") // scalastyle:on line.size.limit -case class DayOfWeek(child: Expression) extends DayWeek { - - override protected def nullSafeEval(date: Any): Any = { - val localDate = LocalDate.ofEpochDay(date.asInstanceOf[Int]) - localDate.getDayOfWeek.plus(1).getValue - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, days => { - s""" - ${ev.value} = java.time.LocalDate.ofEpochDay($days).getDayOfWeek().plus(1).getValue(); - """ - }) - } +case class DayOfWeek(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getDayOfWeek + override val funcName = "getDayOfWeek" } // scalastyle:off line.size.limit @@ -659,27 +550,9 @@ case class DayOfWeek(child: Expression) extends DayWeek { group = "datetime_funcs", since = "2.4.0") // scalastyle:on line.size.limit -case class WeekDay(child: Expression) extends DayWeek { - - override protected def nullSafeEval(date: Any): Any = { - val localDate = LocalDate.ofEpochDay(date.asInstanceOf[Int]) - localDate.getDayOfWeek.ordinal() - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, days => { - s""" - ${ev.value} = java.time.LocalDate.ofEpochDay($days).getDayOfWeek().ordinal(); - """ - }) - } -} - -abstract class DayWeek extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = IntegerType +case class WeekDay(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getWeekDay + override val funcName = "getWeekDay" } // scalastyle:off line.size.limit @@ -693,26 +566,9 @@ abstract class DayWeek extends UnaryExpression with ImplicitCastInputTypes with group = "datetime_funcs", since = "1.5.0") // scalastyle:on line.size.limit -case class WeekOfYear(child: Expression) - extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = IntegerType - - override protected def nullSafeEval(date: Any): Any = { - val localDate = LocalDate.ofEpochDay(date.asInstanceOf[Int]) - localDate.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR) - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, days => { - s""" - |${ev.value} = java.time.LocalDate.ofEpochDay($days).get( - | java.time.temporal.IsoFields.WEEK_OF_WEEK_BASED_YEAR); - """.stripMargin - }) - } +case class WeekOfYear(child: Expression) extends GetDateField { + override val func = DateTimeUtils.getWeekOfYear + override val funcName = "getWeekOfYear" } // scalastyle:off line.size.limit diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 5c3d91c105094..f0dab8095fc75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -535,6 +535,26 @@ object DateTimeUtils { */ def getDayOfMonth(days: Int): Int = daysToLocalDate(days).getDayOfMonth + /** + * Returns the day of the week for the given number of days since 1970-01-01 + * (1 = Sunday, 2 = Monday, ..., 7 = Saturday). + */ + def getDayOfWeek(days: Int): Int = LocalDate.ofEpochDay(days).getDayOfWeek.plus(1).getValue + + /** + * Returns the day of the week for the given number of days since 1970-01-01 + * (0 = Monday, 1 = Tuesday, ..., 6 = Sunday). + */ + def getWeekDay(days: Int): Int = LocalDate.ofEpochDay(days).getDayOfWeek.ordinal() + + /** + * Returns the week of the year of the given date expressed as the number of days from 1970-01-01. + * A week is considered to start on a Monday and week 1 is the first week with > 3 days. + */ + def getWeekOfYear(days: Int): Int = { + LocalDate.ofEpochDay(days).get(IsoFields.WEEK_OF_WEEK_BASED_YEAR) + } + /** * Adds an year-month interval to a date represented as days since 1970-01-01. * @return a date value, expressed in days since 1970-01-01. From 2bcbe3dd9a265a785cc9d5bdf6f683cbf014c80c Mon Sep 17 00:00:00 2001 From: William Hyun Date: Tue, 23 Jun 2020 16:59:55 +0900 Subject: [PATCH 061/384] [SPARK-32045][BUILD] Upgrade to Apache Commons Lang 3.10 ### What changes were proposed in this pull request? This PR aims to upgrade to Apache Commons Lang 3.10. ### Why are the changes needed? This will bring the latest bug fixes like [LANG-1453](https://issues.apache.org/jira/browse/LANG-1453). https://commons.apache.org/proper/commons-lang/release-notes/RELEASE-NOTES-3.10.txt ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #28889 from williamhyun/commons-lang-3.10. Authored-by: William Hyun Signed-off-by: HyukjinKwon --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 2 +- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 0fd8005582738..f8a43488d0f7f 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -44,7 +44,7 @@ commons-digester/1.8//commons-digester-1.8.jar commons-httpclient/3.1//commons-httpclient-3.1.jar commons-io/2.4//commons-io-2.4.jar commons-lang/2.6//commons-lang-2.6.jar -commons-lang3/3.9//commons-lang3-3.9.jar +commons-lang3/3.10//commons-lang3-3.10.jar commons-logging/1.1.3//commons-logging-1.1.3.jar commons-math3/3.4.1//commons-math3-3.4.1.jar commons-net/3.1//commons-net-3.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index e4df088e08b65..a34970b3c9d1d 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -42,7 +42,7 @@ commons-digester/1.8//commons-digester-1.8.jar commons-httpclient/3.1//commons-httpclient-3.1.jar commons-io/2.4//commons-io-2.4.jar commons-lang/2.6//commons-lang-2.6.jar -commons-lang3/3.9//commons-lang3-3.9.jar +commons-lang3/3.10//commons-lang3-3.10.jar commons-logging/1.1.3//commons-logging-1.1.3.jar commons-math3/3.4.1//commons-math3-3.4.1.jar commons-net/3.1//commons-net-3.1.jar 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 7f3f74e3e039d..c8fade45739c0 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -39,7 +39,7 @@ commons-dbcp/1.4//commons-dbcp-1.4.jar commons-httpclient/3.1//commons-httpclient-3.1.jar commons-io/2.5//commons-io-2.5.jar commons-lang/2.6//commons-lang-2.6.jar -commons-lang3/3.9//commons-lang3-3.9.jar +commons-lang3/3.10//commons-lang3-3.10.jar commons-logging/1.1.3//commons-logging-1.1.3.jar commons-math3/3.4.1//commons-math3-3.4.1.jar commons-net/3.1//commons-net-3.1.jar diff --git a/pom.xml b/pom.xml index 3abbf871ad2de..7b6846565f96b 100644 --- a/pom.xml +++ b/pom.xml @@ -174,7 +174,7 @@ 2.6 - 3.9 + 3.10 2.6.2 4.1.17 From 2dbfae8775e00da521f59c6328428ae541396802 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 23 Jun 2020 03:58:40 -0700 Subject: [PATCH 062/384] [SPARK-32049][SQL][TESTS] Upgrade Oracle JDBC Driver 8 ### What changes were proposed in this pull request? `OracleIntegrationSuite` is not using the latest oracle JDBC driver. In this PR I've upgraded the driver to the latest which supports JDK8, JDK9, and JDK11. ### Why are the changes needed? Old JDBC driver. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Existing integration tests (especially `OracleIntegrationSuite`) Closes #28893 from gaborgsomogyi/SPARK-32049. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- external/docker-integration-tests/pom.xml | 12 ++----- .../sql/jdbc/DockerJDBCIntegrationSuite.scala | 3 +- .../sql/jdbc/OracleIntegrationSuite.scala | 36 +++++++++++-------- pom.xml | 6 ++++ 4 files changed, 32 insertions(+), 25 deletions(-) diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 298e3d36c145a..b240dd281823a 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -130,15 +130,9 @@ postgresql test - - - com.oracle - ojdbc6 - 11.2.0.1.0 + + com.oracle.database.jdbc + ojdbc8 test diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index d15b366bfc9b0..6d1a22dd22b65 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -95,6 +95,7 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu protected val dockerIp = DockerUtils.getDockerIp() val db: DatabaseOnDocker + val connectionTimeout = timeout(2.minutes) private var docker: DockerClient = _ protected var externalPort: Int = _ @@ -155,7 +156,7 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu docker.startContainer(containerId) jdbcUrl = db.getJdbcUrl(dockerIp, externalPort) var conn: Connection = null - eventually(timeout(2.minutes), interval(1.second)) { + eventually(connectionTimeout, interval(1.second)) { conn = getConnection() } // Run any setup queries: diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 24c3adb9c0153..9c59023cd8766 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -21,6 +21,8 @@ import java.math.BigDecimal import java.sql.{Connection, Date, Timestamp} import java.util.{Properties, TimeZone} +import org.scalatest.time.SpanSugar._ + import org.apache.spark.sql.{Row, SaveMode} import org.apache.spark.sql.execution.{RowDataSourceScanExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -31,27 +33,27 @@ import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * This patch was tested using the Oracle docker. Created this integration suite for the same. - * The ojdbc6-11.2.0.2.0.jar was to be downloaded from the maven repository. Since there was - * no jdbc jar available in the maven repository, the jar was downloaded from oracle site - * manually and installed in the local; thus tested. So, for SparkQA test case run, the - * ojdbc jar might be manually placed in the local maven repository(com/oracle/ojdbc6/11.2.0.2.0) - * while Spark QA test run. - * * The following would be the steps to test this * 1. Build Oracle database in Docker, please refer below link about how to. * https://github.com/oracle/docker-images/blob/master/OracleDatabase/SingleInstance/README.md * 2. export ORACLE_DOCKER_IMAGE_NAME=$ORACLE_DOCKER_IMAGE_NAME * Pull oracle $ORACLE_DOCKER_IMAGE_NAME image - docker pull $ORACLE_DOCKER_IMAGE_NAME * 3. Start docker - sudo service docker start - * 4. Download oracle 11g driver jar and put it in maven local repo: - * (com/oracle/ojdbc6/11.2.0.2.0/ojdbc6-11.2.0.2.0.jar) - * 5. The timeout and interval parameter to be increased from 60,1 to a high value for oracle test - * in DockerJDBCIntegrationSuite.scala (Locally tested with 200,200 and executed successfully). - * 6. Run spark test - ./build/sbt "test-only org.apache.spark.sql.jdbc.OracleIntegrationSuite" + * 4. Run spark test - ./build/sbt -Pdocker-integration-tests + * "test-only org.apache.spark.sql.jdbc.OracleIntegrationSuite" + * + * An actual sequence of commands to run the test is as follows * - * All tests in this suite are ignored because of the dependency with the oracle jar from maven - * repository. + * $ git clone https://github.com/oracle/docker-images.git + * // Head SHA: 3e352a22618070595f823977a0fd1a3a8071a83c + * $ cd docker-images/OracleDatabase/SingleInstance/dockerfiles + * $ ./buildDockerImage.sh -v 18.4.0 -x + * $ export ORACLE_DOCKER_IMAGE_NAME=oracle/database:18.4.0-xe + * $ cd $SPARK_HOME + * $ ./build/sbt -Pdocker-integration-tests + * "test-only org.apache.spark.sql.jdbc.OracleIntegrationSuite" + * + * It has been validated with 18.4.0 Express Edition. */ @DockerTest class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSparkSession { @@ -60,7 +62,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSpark override val db = new DatabaseOnDocker { override val imageName = sys.env("ORACLE_DOCKER_IMAGE_NAME") override val env = Map( - "ORACLE_ROOT_PASSWORD" -> "oracle" + "ORACLE_PWD" -> "oracle" ) override val usesIpc = false override val jdbcPort: Int = 1521 @@ -68,7 +70,11 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSpark s"jdbc:oracle:thin:system/oracle@//$ip:$port/xe" } + override val connectionTimeout = timeout(7.minutes) + override def dataPreparation(conn: Connection): Unit = { + // In 18.4.0 Express Edition auto commit is enabled by default. + conn.setAutoCommit(false) conn.prepareStatement("CREATE TABLE datetime (id NUMBER(10), d DATE, t TIMESTAMP)") .executeUpdate() conn.prepareStatement( diff --git a/pom.xml b/pom.xml index 7b6846565f96b..82c12ae3dcb80 100644 --- a/pom.xml +++ b/pom.xml @@ -984,6 +984,12 @@ 8.2.2.jre8 test + + com.oracle.database.jdbc + ojdbc8 + 19.6.0.0 + test + org.apache.curator curator-recipes From e00f43cb86a6c76720b45176e9f9a7fba1dc3a35 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 23 Jun 2020 11:45:12 +0000 Subject: [PATCH 063/384] [SPARK-32043][SQL] Replace Decimal by Int op in `make_interval` and `make_timestamp` ### What changes were proposed in this pull request? Replace Decimal by Int op in the `MakeInterval` & `MakeTimestamp` expression. For instance, `(secs * Decimal(MICROS_PER_SECOND)).toLong` can be replaced by the unscaled long because the former one already contains microseconds. ### Why are the changes needed? To improve performance. Before: ``` make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ... make_timestamp(2019, 1, 2, 3, 4, 50.123456) 94 99 4 10.7 93.8 38.8X ``` After: ``` make_timestamp(2019, 1, 2, 3, 4, 50.123456) 76 92 15 13.1 76.5 48.1X ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - By existing test suites `IntervalExpressionsSuite`, `DateExpressionsSuite` and etc. - Re-generate results of `MakeDateTimeBenchmark` 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 | Closes #28886 from MaxGekk/make_interval-opt-decimal. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../expressions/datetimeExpressions.scala | 14 ++++++---- .../sql/catalyst/util/IntervalUtils.scala | 3 +- .../MakeDateTimeBenchmark-jdk11-results.txt | 28 +++++++++---------- .../MakeDateTimeBenchmark-results.txt | 28 +++++++++---------- 4 files changed, 39 insertions(+), 34 deletions(-) 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 5d86c8def0d95..0e4ff4f9f2cb4 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 @@ -1850,13 +1850,17 @@ case class MakeTimestamp( day: Int, hour: Int, min: Int, - secAndNanos: Decimal, + secAndMicros: Decimal, zoneId: ZoneId): Any = { try { - val secFloor = secAndNanos.floor - val nanosPerSec = Decimal(NANOS_PER_SECOND, 10, 0) - val nanos = ((secAndNanos - secFloor) * nanosPerSec).toInt - val seconds = secFloor.toInt + assert(secAndMicros.scale == 6, + s"Seconds fraction must have 6 digits for microseconds but got ${secAndMicros.scale}") + val unscaledSecFrac = secAndMicros.toUnscaledLong + assert(secAndMicros.precision <= 8, + s"Seconds and fraction cannot have more than 8 digits but got ${secAndMicros.precision}") + val totalMicros = unscaledSecFrac.toInt // 8 digits cannot overflow Int + val seconds = Math.floorDiv(totalMicros, MICROS_PER_SECOND.toInt) + val nanos = Math.floorMod(totalMicros, MICROS_PER_SECOND.toInt) * NANOS_PER_MICROS.toInt val ldt = if (seconds == 60) { if (nanos == 0) { // This case of sec = 60 and nanos = 0 is supported for compatibility with PostgreSQL diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 7e5a71e0cb3f7..fe7f6522826ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -751,7 +751,8 @@ object IntervalUtils { secs: Decimal): CalendarInterval = { val totalMonths = Math.addExact(months, Math.multiplyExact(years, MONTHS_PER_YEAR)) val totalDays = Math.addExact(days, Math.multiplyExact(weeks, DAYS_PER_WEEK)) - var micros = (secs * Decimal(MICROS_PER_SECOND)).toLong + assert(secs.scale == 6, "Seconds fractional must have 6 digits for microseconds") + var micros = secs.toUnscaledLong micros = Math.addExact(micros, Math.multiplyExact(hours, MICROS_PER_HOUR)) micros = Math.addExact(micros, Math.multiplyExact(mins, MICROS_PER_MINUTE)) diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt index 65faa752b94cb..9690ebb02dd5d 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt @@ -1,22 +1,22 @@ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 3204 3323 139 31.2 32.0 1.0X -make_date(2019, 9, 16) 2529 2604 126 39.5 25.3 1.3X -make_date(*, *, *) 5102 5113 10 19.6 51.0 0.6X +prepare make_date() 3170 3321 230 31.5 31.7 1.0X +make_date(2019, 9, 16) 2359 2566 343 42.4 23.6 1.3X +make_date(*, *, *) 4398 4455 53 22.7 44.0 0.7X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_timestamp() 3484 3513 28 0.3 3484.3 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 112 131 17 9.0 111.5 31.2X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 93 102 10 10.8 92.8 37.6X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 85 88 4 11.8 84.8 41.1X -make_timestamp(*, *, *, 3, 4, 50.123456) 303 308 8 3.3 302.8 11.5X -make_timestamp(*, *, *, *, *, 0) 303 307 3 3.3 302.8 11.5X -make_timestamp(*, *, *, *, *, 60.0) 289 297 8 3.5 289.1 12.1X -make_timestamp(2019, 1, 2, *, *, *) 3576 3585 11 0.3 3576.4 1.0X -make_timestamp(*, *, *, *, *, *) 3610 3618 12 0.3 3610.0 1.0X +prepare make_timestamp() 3492 3536 70 0.3 3491.7 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 94 110 16 10.7 93.6 37.3X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 78 83 8 12.8 78.0 44.7X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 66 72 6 15.1 66.2 52.7X +make_timestamp(*, *, *, 3, 4, 50.123456) 306 313 12 3.3 305.9 11.4X +make_timestamp(*, *, *, *, *, 0) 282 284 2 3.5 281.9 12.4X +make_timestamp(*, *, *, *, *, 60.0) 281 288 6 3.6 281.0 12.4X +make_timestamp(2019, 1, 2, *, *, *) 3637 3641 3 0.3 3636.7 1.0X +make_timestamp(*, *, *, *, *, *) 3635 3648 12 0.3 3634.9 1.0X diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt index 92bcc4444e60a..157bce3a38dbf 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt @@ -1,22 +1,22 @@ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 2920 3239 278 34.3 29.2 1.0X -make_date(2019, 9, 16) 2322 2371 61 43.1 23.2 1.3X -make_date(*, *, *) 4553 4560 6 22.0 45.5 0.6X +prepare make_date() 2951 3182 355 33.9 29.5 1.0X +make_date(2019, 9, 16) 2325 2415 101 43.0 23.2 1.3X +make_date(*, *, *) 4556 4573 17 21.9 45.6 0.6X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_timestamp() 3636 3673 38 0.3 3635.7 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 94 99 4 10.7 93.8 38.8X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 68 80 13 14.6 68.3 53.2X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 65 79 19 15.3 65.3 55.7X -make_timestamp(*, *, *, 3, 4, 50.123456) 271 280 14 3.7 270.7 13.4X -make_timestamp(*, *, *, *, *, 0) 255 263 11 3.9 255.5 14.2X -make_timestamp(*, *, *, *, *, 60.0) 254 258 4 3.9 254.2 14.3X -make_timestamp(2019, 1, 2, *, *, *) 3714 3722 8 0.3 3713.9 1.0X -make_timestamp(*, *, *, *, *, *) 3736 3741 6 0.3 3736.3 1.0X +prepare make_timestamp() 3677 3730 87 0.3 3677.0 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 76 92 15 13.1 76.5 48.1X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 70 72 3 14.4 69.7 52.8X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 68 75 12 14.7 68.0 54.1X +make_timestamp(*, *, *, 3, 4, 50.123456) 266 277 12 3.8 265.8 13.8X +make_timestamp(*, *, *, *, *, 0) 258 286 46 3.9 258.3 14.2X +make_timestamp(*, *, *, *, *, 60.0) 261 269 11 3.8 260.7 14.1X +make_timestamp(2019, 1, 2, *, *, *) 3748 3762 22 0.3 3748.3 1.0X +make_timestamp(*, *, *, *, *, *) 3761 3762 2 0.3 3760.9 1.0X From 11d2b07b74c73ce6d59ac4f7446f1eb8bc6bbb4b Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 24 Jun 2020 11:03:05 +0900 Subject: [PATCH 064/384] [SPARK-31918][R] Ignore S4 generic methods under SparkR namespace in closure cleaning to support R 4.0.0+ MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR proposes to ignore S4 generic methods under SparkR namespace in closure cleaning to support R 4.0.0+. Currently, when you run the codes that runs R native codes, it fails as below with R 4.0.0: ```r df <- createDataFrame(lapply(seq(100), function (e) list(value=e))) count(dapply(df, function(x) as.data.frame(x[x$value < 50,]), schema(df))) ``` ``` org.apache.spark.SparkException: R unexpectedly exited. R worker produced errors: Error in lapply(part, FUN) : attempt to bind a variable to R_UnboundValue ``` The root cause seems to be related to when an S4 generic method is manually included into the closure's environment via `SparkR:::cleanClosure`. For example, when an RRDD is created via `createDataFrame` with calling `lapply` to convert, `lapply` itself: https://github.com/apache/spark/blob/f53d8c63e80172295e2fbc805c0c391bdececcaa/R/pkg/R/RDD.R#L484 is added into the environment of the cleaned closure - because this is not an exposed namespace; however, this is broken in R 4.0.0+ for an unknown reason with an error message such as "attempt to bind a variable to R_UnboundValue". Actually, we don't need to add the `lapply` into the environment of the closure because it is not supposed to be called in worker side. In fact, there is no private generic methods supposed to be called in worker side in SparkR at all from my understanding. Therefore, this PR takes a simpler path to work around just by explicitly excluding the S4 generic methods under SparkR namespace to support R 4.0.0. in SparkR. ### Why are the changes needed? To support R 4.0.0+ with SparkR, and unblock the releases on CRAN. CRAN requires the tests pass with the latest R. ### Does this PR introduce _any_ user-facing change? Yes, it will support R 4.0.0 to end-users. ### How was this patch tested? Manually tested. Both CRAN and tests with R 4.0.1: ``` ══ testthat results ═══════════════════════════════════════════════════════════ [ OK: 13 | SKIPPED: 0 | WARNINGS: 0 | FAILED: 0 ] ✔ | OK F W S | Context ✔ | 11 | binary functions [2.5 s] ✔ | 4 | functions on binary files [2.1 s] ✔ | 2 | broadcast variables [0.5 s] ✔ | 5 | functions in client.R ✔ | 46 | test functions in sparkR.R [6.3 s] ✔ | 2 | include R packages [0.3 s] ✔ | 2 | JVM API [0.2 s] ✔ | 75 | MLlib classification algorithms, except for tree-based algorithms [86.3 s] ✔ | 70 | MLlib clustering algorithms [44.5 s] ✔ | 6 | MLlib frequent pattern mining [3.0 s] ✔ | 8 | MLlib recommendation algorithms [9.6 s] ✔ | 136 | MLlib regression algorithms, except for tree-based algorithms [76.0 s] ✔ | 8 | MLlib statistics algorithms [0.6 s] ✔ | 94 | MLlib tree-based algorithms [85.2 s] ✔ | 29 | parallelize() and collect() [0.5 s] ✔ | 428 | basic RDD functions [25.3 s] ✔ | 39 | SerDe functionality [2.2 s] ✔ | 20 | partitionBy, groupByKey, reduceByKey etc. [3.9 s] ✔ | 4 | functions in sparkR.R ✔ | 16 | SparkSQL Arrow optimization [19.2 s] ✔ | 6 | test show SparkDataFrame when eager execution is enabled. [1.1 s] ✔ | 1175 | SparkSQL functions [134.8 s] ✔ | 42 | Structured Streaming [478.2 s] ✔ | 16 | tests RDD function take() [1.1 s] ✔ | 14 | the textFile() function [2.9 s] ✔ | 46 | functions in utils.R [0.7 s] ✔ | 0 1 | Windows-specific tests ──────────────────────────────────────────────────────────────────────────────── test_Windows.R:22: skip: sparkJars tag in SparkContext Reason: This test is only for Windows, skipped ──────────────────────────────────────────────────────────────────────────────── ══ Results ═════════════════════════════════════════════════════════════════════ Duration: 987.3 s OK: 2304 Failed: 0 Warnings: 0 Skipped: 1 ... Status: OK + popd Tests passed. ``` Note that I tested to build SparkR in R 4.0.0, and run the tests with R 3.6.3. It all passed. See also [the comment in the JIRA](https://issues.apache.org/jira/browse/SPARK-31918?focusedCommentId=17142837&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17142837). Closes #28907 from HyukjinKwon/SPARK-31918. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/pkg/R/utils.R | 5 ++++- R/pkg/tests/fulltests/test_context.R | 4 +++- .../fulltests/test_mllib_classification.R | 18 +++++++++--------- R/pkg/tests/fulltests/test_mllib_clustering.R | 2 +- R/pkg/tests/fulltests/test_mllib_regression.R | 2 +- 5 files changed, 18 insertions(+), 13 deletions(-) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 65db9c21d9dbb..cef2fa9b47440 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -529,7 +529,10 @@ processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { # Namespaces other than "SparkR" will not be searched. if (!isNamespace(func.env) || (getNamespaceName(func.env) == "SparkR" && - !(nodeChar %in% getNamespaceExports("SparkR")))) { + !(nodeChar %in% getNamespaceExports("SparkR")) && + # Note that generic S4 methods should not be set to the environment of + # cleaned closure. It does not work with R 4.0.0+. See also SPARK-31918. + nodeChar != "" && !methods::isGeneric(nodeChar, func.env))) { # Only include SparkR internals. # Set parameter 'inherits' to FALSE since we do not need to search in diff --git a/R/pkg/tests/fulltests/test_context.R b/R/pkg/tests/fulltests/test_context.R index 6be04b321e985..f86872d727a1d 100644 --- a/R/pkg/tests/fulltests/test_context.R +++ b/R/pkg/tests/fulltests/test_context.R @@ -26,7 +26,9 @@ test_that("Check masked functions", { "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", "summary", "transform", "drop", "window", "as.data.frame", "union", "not") version <- packageVersion("base") - if (as.numeric(version$major) >= 3 && as.numeric(version$minor) >= 3) { + is33Above <- as.numeric(version$major) >= 3 && as.numeric(version$minor) >= 3 + is40Above <- as.numeric(version$major) >= 4 + if (is33Above || is40Above) { namesOfMasked <- c("endsWith", "startsWith", namesOfMasked) } masked <- conflicts(detail = TRUE)$`package:SparkR` diff --git a/R/pkg/tests/fulltests/test_mllib_classification.R b/R/pkg/tests/fulltests/test_mllib_classification.R index 9dd275a173089..5b49a013959df 100644 --- a/R/pkg/tests/fulltests/test_mllib_classification.R +++ b/R/pkg/tests/fulltests/test_mllib_classification.R @@ -34,7 +34,7 @@ test_that("spark.svmLinear", { summary <- summary(model) # test summary coefficients return matrix type - expect_true(class(summary$coefficients) == "matrix") + expect_true(any(class(summary$coefficients) == "matrix")) expect_true(class(summary$coefficients[, 1]) == "numeric") coefs <- summary$coefficients[, "Estimate"] @@ -130,7 +130,7 @@ test_that("spark.logit", { summary <- summary(model) # test summary coefficients return matrix type - expect_true(class(summary$coefficients) == "matrix") + expect_true(any(class(summary$coefficients) == "matrix")) expect_true(class(summary$coefficients[, 1]) == "numeric") versicolorCoefsR <- c(1.52, 0.03, -0.53, 0.04, 0.00) @@ -242,8 +242,8 @@ test_that("spark.logit", { # Test binomial logistic regression against two classes with upperBoundsOnCoefficients # and upperBoundsOnIntercepts u <- matrix(c(1.0, 0.0, 1.0, 0.0), nrow = 1, ncol = 4) - model <- spark.logit(training, Species ~ ., upperBoundsOnCoefficients = u, - upperBoundsOnIntercepts = 1.0) + model <- suppressWarnings(spark.logit(training, Species ~ ., upperBoundsOnCoefficients = u, + upperBoundsOnIntercepts = 1.0)) summary <- summary(model) coefsR <- c(-11.13331, 1.00000, 0.00000, 1.00000, 0.00000) coefs <- summary$coefficients[, "Estimate"] @@ -255,8 +255,8 @@ test_that("spark.logit", { # Test binomial logistic regression against two classes with lowerBoundsOnCoefficients # and lowerBoundsOnIntercepts l <- matrix(c(0.0, -1.0, 0.0, -1.0), nrow = 1, ncol = 4) - model <- spark.logit(training, Species ~ ., lowerBoundsOnCoefficients = l, - lowerBoundsOnIntercepts = 0.0) + model <- suppressWarnings(spark.logit(training, Species ~ ., lowerBoundsOnCoefficients = l, + lowerBoundsOnIntercepts = 0.0)) summary <- summary(model) coefsR <- c(0, 0, -1, 0, 1.902192) coefs <- summary$coefficients[, "Estimate"] @@ -268,9 +268,9 @@ test_that("spark.logit", { # Test multinomial logistic regression with lowerBoundsOnCoefficients # and lowerBoundsOnIntercepts l <- matrix(c(0.0, -1.0, 0.0, -1.0, 0.0, -1.0, 0.0, -1.0), nrow = 2, ncol = 4) - model <- spark.logit(training, Species ~ ., family = "multinomial", - lowerBoundsOnCoefficients = l, - lowerBoundsOnIntercepts = as.array(c(0.0, 0.0))) + model <- suppressWarnings(spark.logit(training, Species ~ ., family = "multinomial", + lowerBoundsOnCoefficients = l, + lowerBoundsOnIntercepts = as.array(c(0.0, 0.0)))) summary <- summary(model) versicolorCoefsR <- c(42.639465, 7.258104, 14.330814, 16.298243, 11.716429) virginicaCoefsR <- c(0.0002970796, 4.79274, 7.65047, 25.72793, 30.0021) diff --git a/R/pkg/tests/fulltests/test_mllib_clustering.R b/R/pkg/tests/fulltests/test_mllib_clustering.R index 028ad574b8134..f180aeea28150 100644 --- a/R/pkg/tests/fulltests/test_mllib_clustering.R +++ b/R/pkg/tests/fulltests/test_mllib_clustering.R @@ -171,7 +171,7 @@ test_that("spark.kmeans", { expect_equal(sort(collect(distinct(select(cluster, "prediction")))$prediction), c(0, 1)) # test summary coefficients return matrix type - expect_true(class(summary.model$coefficients) == "matrix") + expect_true(any(class(summary.model$coefficients) == "matrix")) expect_true(class(summary.model$coefficients[1, ]) == "numeric") # Test model save/load diff --git a/R/pkg/tests/fulltests/test_mllib_regression.R b/R/pkg/tests/fulltests/test_mllib_regression.R index 0f2a62b7229ca..b281cd6235ef0 100644 --- a/R/pkg/tests/fulltests/test_mllib_regression.R +++ b/R/pkg/tests/fulltests/test_mllib_regression.R @@ -116,7 +116,7 @@ test_that("spark.glm summary", { rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = dataset)) # test summary coefficients return matrix type - expect_true(class(stats$coefficients) == "matrix") + expect_true(any(class(stats$coefficients) == "matrix")) expect_true(class(stats$coefficients[, 1]) == "numeric") coefs <- stats$coefficients From b62e2536db9def0d11605ceac8990f72a515e9a0 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 24 Jun 2020 11:05:27 +0900 Subject: [PATCH 065/384] [SPARK-32073][R] Drop R < 3.5 support ### What changes were proposed in this pull request? Spark 3.0 accidentally dropped R < 3.5. It is built by R 3.6.3 which not support R < 3.5: ``` Error in readRDS(pfile) : cannot read workspace version 3 written by R 3.6.3; need R 3.5.0 or newer version. ``` In fact, with SPARK-31918, we will have to drop R < 3.5 entirely to support R 4.0.0. This is inevitable to release on CRAN because they require to make the tests pass with the latest R. ### Why are the changes needed? To show the supported versions correctly, and support R 4.0.0 to unblock the releases. ### Does this PR introduce _any_ user-facing change? In fact, no because Spark 3.0.0 already does not work with R < 3.5. Compared to Spark 2.4, yes. R < 3.5 would not work. ### How was this patch tested? Jenkins should test it out. Closes #28908 from HyukjinKwon/SPARK-32073. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/WINDOWS.md | 4 ++-- R/pkg/DESCRIPTION | 2 +- R/pkg/inst/profile/general.R | 4 ---- R/pkg/inst/profile/shell.R | 4 ---- docs/index.md | 3 +-- 5 files changed, 4 insertions(+), 13 deletions(-) diff --git a/R/WINDOWS.md b/R/WINDOWS.md index dbc27178bdb8c..9fe4a22bf22b2 100644 --- a/R/WINDOWS.md +++ b/R/WINDOWS.md @@ -22,8 +22,8 @@ To build SparkR on Windows, the following steps are required 1. Make sure `bash` is available and in `PATH` if you already have a built-in `bash` on Windows. If you do not have, install [Cygwin](https://www.cygwin.com/). -2. Install R (>= 3.1) and [Rtools](https://cloud.r-project.org/bin/windows/Rtools/). Make sure to -include Rtools and R in `PATH`. Note that support for R prior to version 3.4 is deprecated as of Spark 3.0.0. +2. Install R (>= 3.5) and [Rtools](https://cloud.r-project.org/bin/windows/Rtools/). Make sure to +include Rtools and R in `PATH`. 3. Install JDK that SparkR supports (see `R/pkg/DESCRIPTION`), and set `JAVA_HOME` in the system environment variables. diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 52d7e1f4daa53..c5c08bd7a0636 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -15,7 +15,7 @@ URL: https://www.apache.org/ https://spark.apache.org/ BugReports: https://spark.apache.org/contributing.html SystemRequirements: Java (>= 8, < 12) Depends: - R (>= 3.1), + R (>= 3.5), methods Suggests: knitr, diff --git a/R/pkg/inst/profile/general.R b/R/pkg/inst/profile/general.R index 3efb460846fc2..8c75c19ca7ac3 100644 --- a/R/pkg/inst/profile/general.R +++ b/R/pkg/inst/profile/general.R @@ -16,10 +16,6 @@ # .First <- function() { - if (utils::compareVersion(paste0(R.version$major, ".", R.version$minor), "3.4.0") == -1) { - warning("Support for R prior to version 3.4 is deprecated since Spark 3.0.0") - } - packageDir <- Sys.getenv("SPARKR_PACKAGE_DIR") dirs <- strsplit(packageDir, ",")[[1]] .libPaths(c(dirs, .libPaths())) diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R index e4e0d032997de..f6c20e1a5ebc3 100644 --- a/R/pkg/inst/profile/shell.R +++ b/R/pkg/inst/profile/shell.R @@ -16,10 +16,6 @@ # .First <- function() { - if (utils::compareVersion(paste0(R.version$major, ".", R.version$minor), "3.4.0") == -1) { - warning("Support for R prior to version 3.4 is deprecated since Spark 3.0.0") - } - home <- Sys.getenv("SPARK_HOME") .libPaths(c(file.path(home, "R", "lib"), .libPaths())) Sys.setenv(NOAWT = 1) diff --git a/docs/index.md b/docs/index.md index 38f12dd4db77b..c0771ca170af5 100644 --- a/docs/index.md +++ b/docs/index.md @@ -44,10 +44,9 @@ source, visit [Building Spark](building-spark.html). Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS), and it should run on any platform that runs a supported version of Java. This should include JVMs on x86_64 and ARM64. It's easy to run locally on one machine --- all you need is to have `java` installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. -Spark runs on Java 8/11, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. +Spark runs on Java 8/11, Scala 2.12, Python 2.7+/3.4+ and R 3.5+. Java 8 prior to version 8u92 support is deprecated as of Spark 3.0.0. Python 2 and Python 3 prior to version 3.6 support is deprecated as of Spark 3.0.0. -R prior to version 3.4 support is deprecated as of Spark 3.0.0. For the Scala API, Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version ({{site.SCALA_BINARY_VERSION}}.x). From eedc6cc37df9b32995f41bd0e1779101ba1df1b8 Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Tue, 23 Jun 2020 21:43:02 -0500 Subject: [PATCH 066/384] [SPARK-32028][WEBUI] fix app id link for multi attempts app in history summary page ### What changes were proposed in this pull request? Fix app id link for multi attempts application in history summary page If attempt id is available (yarn), app id link url will contain correct attempt id, like `/history/application_1561589317410_0002/1/jobs/`. If attempt id is not available (standalone), app id link url will not contain fake attempt id, like `/history/app-20190404053606-0000/jobs/`. ### Why are the changes needed? This PR is for fixing [32028](https://issues.apache.org/jira/browse/SPARK-32028). App id link use application attempt count as attempt id. this would cause link url wrong for below cases: 1. there are multi attempts, all links point to last attempt ![multi_same](https://user-images.githubusercontent.com/10524738/85098505-c45c5500-b1af-11ea-8912-fa5fd72ce064.JPG) 2. if there is one attempt, but attempt id is not 1 (before attempt maybe crash or fail to gerenerate event file). link url points to worng attempt (1) here. ![wrong_attemptJPG](https://user-images.githubusercontent.com/10524738/85098513-c9b99f80-b1af-11ea-8cbc-fd7f745c1080.JPG) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tested this manually. Closes #28867 from zhli1142015/fix-appid-link-in-history-page. Authored-by: Zhen Li Signed-off-by: Sean Owen --- .../org/apache/spark/ui/static/historypage-template.html | 6 +++--- .../resources/org/apache/spark/ui/static/historypage.js | 5 +++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html index 33eb7bfa671bc..7e9927d40c191 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html @@ -77,12 +77,12 @@ {{#applications}} + {{#attempts}} {{version}} - {{id}} + {{id}} {{name}} - {{#attempts}} {{#hasMultipleAttempts}} - {{attemptId}} + {{attemptId}} {{/hasMultipleAttempts}} {{startTime}} {{#showCompletedColumns}} diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index 4df5f07f077d7..3a4c815029f6d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -130,7 +130,7 @@ $(document).ready(function() { if (app["attempts"].length > 1) { hasMultipleAttempts = true; } - var num = app["attempts"].length; + for (var j in app["attempts"]) { var attempt = app["attempts"][j]; attempt["startTime"] = formatTimeMillis(attempt["startTimeEpoch"]); @@ -140,7 +140,8 @@ $(document).ready(function() { (attempt.hasOwnProperty("attemptId") ? attempt["attemptId"] + "/" : "") + "logs"; attempt["durationMillisec"] = attempt["duration"]; attempt["duration"] = formatDuration(attempt["duration"]); - var app_clone = {"id" : id, "name" : name, "version": version, "num" : num, "attempts" : [attempt]}; + var hasAttemptId = attempt.hasOwnProperty("attemptId"); + var app_clone = {"id" : id, "name" : name, "version": version, "hasAttemptId" : hasAttemptId, "attempts" : [attempt]}; array.push(app_clone); } } From 986fa01747db4b52bb8ca1165e759ca2d46d26ff Mon Sep 17 00:00:00 2001 From: sidedoorleftroad Date: Wed, 24 Jun 2020 13:39:55 +0900 Subject: [PATCH 067/384] [SPARK-32075][DOCS] Fix a few issues in parameters table ### What changes were proposed in this pull request? Fix a few issues in parameters table in structured-streaming-kafka-integration doc. ### Why are the changes needed? Make the title of the table consistent with the data. ### Does this PR introduce _any_ user-facing change? Yes. Before: ![image](https://user-images.githubusercontent.com/67275816/85414316-8475e300-b59e-11ea-84ec-fa78ecc980b3.png) After: ![image](https://user-images.githubusercontent.com/67275816/85414562-d61e6d80-b59e-11ea-9fe6-247e0ad4d9ee.png) Before: ![image](https://user-images.githubusercontent.com/67275816/85414467-b8510880-b59e-11ea-92a0-7205542fe28b.png) After: ![image](https://user-images.githubusercontent.com/67275816/85414589-de76a880-b59e-11ea-91f2-5073eaf3444b.png) Before: ![image](https://user-images.githubusercontent.com/67275816/85414502-c69f2480-b59e-11ea-837f-1201f10a56b6.png) After: ![image](https://user-images.githubusercontent.com/67275816/85414615-e9313d80-b59e-11ea-9b1a-fc11da0b6bc5.png) ### How was this patch tested? Manually build and check. Closes #28910 from sidedoorleftroad/SPARK-32075. Authored-by: sidedoorleftroad Signed-off-by: HyukjinKwon --- docs/structured-streaming-kafka-integration.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 58fa01edb80e8..0e4d167b58d6b 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -529,28 +529,28 @@ The following properties are available to configure the consumer pool: Property NameDefaultMeaningSince Version spark.kafka.consumer.cache.capacity - The maximum number of consumers cached. Please note that it's a soft limit. 64 + The maximum number of consumers cached. Please note that it's a soft limit. 3.0.0 spark.kafka.consumer.cache.timeout - The minimum amount of time a consumer may sit idle in the pool before it is eligible for eviction by the evictor. 5m (5 minutes) + The minimum amount of time a consumer may sit idle in the pool before it is eligible for eviction by the evictor. 3.0.0 spark.kafka.consumer.cache.evictorThreadRunInterval - The interval of time between runs of the idle evictor thread for consumer pool. When non-positive, no idle evictor thread will be run. 1m (1 minute) + The interval of time between runs of the idle evictor thread for consumer pool. When non-positive, no idle evictor thread will be run. 3.0.0 spark.kafka.consumer.cache.jmx.enable + false Enable or disable JMX for pools created with this configuration instance. Statistics of the pool are available via JMX instance. The prefix of JMX name is set to "kafka010-cached-simple-kafka-consumer-pool". - false 3.0.0 @@ -579,14 +579,14 @@ The following properties are available to configure the fetched data pool: Property NameDefaultMeaningSince Version spark.kafka.consumer.fetchedData.cache.timeout - The minimum amount of time a fetched data may sit idle in the pool before it is eligible for eviction by the evictor. 5m (5 minutes) + The minimum amount of time a fetched data may sit idle in the pool before it is eligible for eviction by the evictor. 3.0.0 spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval - The interval of time between runs of the idle evictor thread for fetched data pool. When non-positive, no idle evictor thread will be run. 1m (1 minute) + The interval of time between runs of the idle evictor thread for fetched data pool. When non-positive, no idle evictor thread will be run. 3.0.0 @@ -826,14 +826,14 @@ The following properties are available to configure the producer pool: Property NameDefaultMeaningSince Version spark.kafka.producer.cache.timeout - The minimum amount of time a producer may sit idle in the pool before it is eligible for eviction by the evictor. 10m (10 minutes) + The minimum amount of time a producer may sit idle in the pool before it is eligible for eviction by the evictor. 2.2.1 spark.kafka.producer.cache.evictorThreadRunInterval - The interval of time between runs of the idle evictor thread for producer pool. When non-positive, no idle evictor thread will be run. 1m (1 minute) + The interval of time between runs of the idle evictor thread for producer pool. When non-positive, no idle evictor thread will be run. 3.0.0 From 045106e29d6b3cbb7be61b46604b85297c405aa3 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 24 Jun 2020 04:43:53 +0000 Subject: [PATCH 068/384] [SPARK-32072][CORE][TESTS] Fix table formatting with benchmark results ### What changes were proposed in this pull request? Set column width w/ benchmark names to maximum of either 1. 40 (before this PR) or 2. The length of benchmark name or 3. Maximum length of cases names ### Why are the changes needed? To improve readability of benchmark results. For example, `MakeDateTimeBenchmark`. Before: ``` make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ prepare make_timestamp() 3636 3673 38 0.3 3635.7 1.0X make_timestamp(2019, 1, 2, 3, 4, 50.123456) 94 99 4 10.7 93.8 38.8X make_timestamp(2019, 1, 2, 3, 4, 60.000000) 68 80 13 14.6 68.3 53.2X make_timestamp(2019, 12, 31, 23, 59, 60.00) 65 79 19 15.3 65.3 55.7X make_timestamp(*, *, *, 3, 4, 50.123456) 271 280 14 3.7 270.7 13.4X ``` After: ``` make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- prepare make_timestamp() 3694 3745 82 0.3 3694.0 1.0X make_timestamp(2019, 1, 2, 3, 4, 50.123456) 82 90 9 12.2 82.3 44.9X make_timestamp(2019, 1, 2, 3, 4, 60.000000) 72 77 5 13.9 71.9 51.4X make_timestamp(2019, 12, 31, 23, 59, 60.00) 67 71 5 15.0 66.8 55.3X make_timestamp(*, *, *, 3, 4, 50.123456) 273 289 14 3.7 273.2 13.5X ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By re-generating benchmark results for `MakeDateTimeBenchmark`: ``` $ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.MakeDateTimeBenchmark" ``` 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 | Closes #28906 from MaxGekk/benchmark-table-formatting. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../apache/spark/benchmark/Benchmark.scala | 9 +++--- .../MakeDateTimeBenchmark-jdk11-results.txt | 28 +++++++++---------- .../MakeDateTimeBenchmark-results.txt | 28 +++++++++---------- 3 files changed, 33 insertions(+), 32 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index 9629f5ab1a3dd..72c05a92848ff 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -112,11 +112,12 @@ private[spark] class Benchmark( // The results are going to be processor specific so it is useful to include that. out.println(Benchmark.getJVMOSInfo()) out.println(Benchmark.getProcessorName()) - out.printf("%-40s %14s %14s %11s %12s %13s %10s\n", name + ":", "Best Time(ms)", "Avg Time(ms)", "Stdev(ms)", "Rate(M/s)", - "Per Row(ns)", "Relative") - out.println("-" * 120) + val nameLen = Math.max(40, Math.max(name.length, benchmarks.map(_.name.length).max)) + out.printf(s"%-${nameLen}s %14s %14s %11s %12s %13s %10s\n", + name + ":", "Best Time(ms)", "Avg Time(ms)", "Stdev(ms)", "Rate(M/s)", "Per Row(ns)", "Relative") + out.println("-" * (nameLen + 80)) results.zip(benchmarks).foreach { case (result, benchmark) => - out.printf("%-40s %14s %14s %11s %12s %13s %10s\n", + out.printf(s"%-${nameLen}s %14s %14s %11s %12s %13s %10s\n", benchmark.name, "%5.0f" format result.bestMs, "%4.0f" format result.avgMs, diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt index 9690ebb02dd5d..bd2dff0271c8e 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt @@ -2,21 +2,21 @@ 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 make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 3170 3321 230 31.5 31.7 1.0X -make_date(2019, 9, 16) 2359 2566 343 42.4 23.6 1.3X -make_date(*, *, *) 4398 4455 53 22.7 44.0 0.7X +prepare make_date() 3214 3344 209 31.1 32.1 1.0X +make_date(2019, 9, 16) 2342 2348 6 42.7 23.4 1.4X +make_date(*, *, *) 4485 4533 56 22.3 44.8 0.7X 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 -make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -prepare make_timestamp() 3492 3536 70 0.3 3491.7 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 94 110 16 10.7 93.6 37.3X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 78 83 8 12.8 78.0 44.7X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 66 72 6 15.1 66.2 52.7X -make_timestamp(*, *, *, 3, 4, 50.123456) 306 313 12 3.3 305.9 11.4X -make_timestamp(*, *, *, *, *, 0) 282 284 2 3.5 281.9 12.4X -make_timestamp(*, *, *, *, *, 60.0) 281 288 6 3.6 281.0 12.4X -make_timestamp(2019, 1, 2, *, *, *) 3637 3641 3 0.3 3636.7 1.0X -make_timestamp(*, *, *, *, *, *) 3635 3648 12 0.3 3634.9 1.0X +make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +--------------------------------------------------------------------------------------------------------------------------- +prepare make_timestamp() 3744 3775 35 0.3 3744.1 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 82 91 9 12.2 82.3 45.5X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 81 89 7 12.4 81.0 46.2X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 70 80 9 14.3 69.9 53.5X +make_timestamp(*, *, *, 3, 4, 50.123456) 308 314 7 3.2 308.1 12.2X +make_timestamp(*, *, *, *, *, 0) 302 316 14 3.3 301.9 12.4X +make_timestamp(*, *, *, *, *, 60.0) 290 296 6 3.4 290.4 12.9X +make_timestamp(2019, 1, 2, *, *, *) 3888 3902 15 0.3 3888.1 1.0X +make_timestamp(*, *, *, *, *, *) 3902 3908 8 0.3 3901.6 1.0X diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt index 157bce3a38dbf..2191be264a51c 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt @@ -2,21 +2,21 @@ 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 make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 2951 3182 355 33.9 29.5 1.0X -make_date(2019, 9, 16) 2325 2415 101 43.0 23.2 1.3X -make_date(*, *, *) 4556 4573 17 21.9 45.6 0.6X +prepare make_date() 3309 3429 110 30.2 33.1 1.0X +make_date(2019, 9, 16) 2336 2359 23 42.8 23.4 1.4X +make_date(*, *, *) 4588 4618 27 21.8 45.9 0.7X 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 -make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -prepare make_timestamp() 3677 3730 87 0.3 3677.0 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 76 92 15 13.1 76.5 48.1X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 70 72 3 14.4 69.7 52.8X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 68 75 12 14.7 68.0 54.1X -make_timestamp(*, *, *, 3, 4, 50.123456) 266 277 12 3.8 265.8 13.8X -make_timestamp(*, *, *, *, *, 0) 258 286 46 3.9 258.3 14.2X -make_timestamp(*, *, *, *, *, 60.0) 261 269 11 3.8 260.7 14.1X -make_timestamp(2019, 1, 2, *, *, *) 3748 3762 22 0.3 3748.3 1.0X -make_timestamp(*, *, *, *, *, *) 3761 3762 2 0.3 3760.9 1.0X +make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +--------------------------------------------------------------------------------------------------------------------------- +prepare make_timestamp() 3651 3697 58 0.3 3651.4 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 89 99 10 11.3 88.6 41.2X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 72 73 1 13.9 72.1 50.6X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 66 68 3 15.2 65.8 55.5X +make_timestamp(*, *, *, 3, 4, 50.123456) 265 272 6 3.8 265.1 13.8X +make_timestamp(*, *, *, *, *, 0) 259 266 6 3.9 259.1 14.1X +make_timestamp(*, *, *, *, *, 60.0) 271 278 9 3.7 271.2 13.5X +make_timestamp(2019, 1, 2, *, *, *) 3838 3850 12 0.3 3837.7 1.0X +make_timestamp(*, *, *, *, *, *) 3854 3877 20 0.3 3853.8 0.9X From 9f540fac2e50bbcc214351f0c80690eae7be6b98 Mon Sep 17 00:00:00 2001 From: ulysses Date: Wed, 24 Jun 2020 04:50:46 +0000 Subject: [PATCH 069/384] [SPARK-32062][SQL] Reset listenerRegistered in SparkSession ### What changes were proposed in this pull request? Reset listenerRegistered when application end. ### Why are the changes needed? Within a jvm, stop and create `SparkContext` multi times will cause the bug. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Add UT. Closes #28899 from ulysses-you/SPARK-32062. Authored-by: ulysses Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/SparkSession.scala | 1 + .../spark/sql/SparkSessionBuilderSuite.scala | 17 +++++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 60a60377d8a3f..ea1a9f12cd24b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -1063,6 +1063,7 @@ object SparkSession extends Logging { sparkContext.addSparkListener(new SparkListener { override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { defaultSession.set(null) + listenerRegistered.set(false) } }) listenerRegistered.set(true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 0a522fdbdeed8..e914d8398e921 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -240,4 +240,21 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { assert(session.conf.get(GLOBAL_TEMP_DATABASE) === "globaltempdb-spark-31532-2") assert(session.conf.get(WAREHOUSE_PATH) === "SPARK-31532-db-2") } + + test("SPARK-32062: reset listenerRegistered in SparkSession") { + (1 to 2).foreach { i => + val conf = new SparkConf() + .setMaster("local") + .setAppName(s"test-SPARK-32062-$i") + val context = new SparkContext(conf) + val beforeListenerSize = context.listenerBus.listeners.size() + SparkSession + .builder() + .sparkContext(context) + .getOrCreate() + val afterListenerSize = context.listenerBus.listeners.size() + assert(beforeListenerSize + 1 == afterListenerSize) + context.stop() + } + } } From e29ec428796eac4ebdb9c853131465d7570dc2f1 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 24 Jun 2020 15:37:41 +0900 Subject: [PATCH 070/384] [SPARK-32074][BUILD][R] Update AppVeyor R version to 4.0.2 ### What changes were proposed in this pull request? R version 4.0.2 was released, see https://cran.r-project.org/doc/manuals/r-release/NEWS.html. This PR targets to upgrade R version in AppVeyor CI environment. ### Why are the changes needed? To test the latest R versions before the release, and see if there are any regressions. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? AppVeyor will test. Closes #28909 from HyukjinKwon/SPARK-32074. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/install-dev.bat | 8 +++++++- dev/appveyor-install-dependencies.ps1 | 4 ++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/R/install-dev.bat b/R/install-dev.bat index ed1c91ae3a0ff..c570d93049a14 100644 --- a/R/install-dev.bat +++ b/R/install-dev.bat @@ -24,7 +24,13 @@ set SPARK_HOME=%~dp0.. MKDIR %SPARK_HOME%\R\lib -R.exe CMD INSTALL --library="%SPARK_HOME%\R\lib" %SPARK_HOME%\R\pkg\ +rem When you pass the package path directly as an argument to R CMD INSTALL, +rem it takes the path as 'C:\projects\spark\R\..\R\pkg"' as an example at +rem R 4.0. To work around this, directly go to the directoy and install it. +rem See also SPARK-32074 +pushd %SPARK_HOME%\R\pkg\ +R.exe CMD INSTALL --library="%SPARK_HOME%\R\lib" . +popd rem Zip the SparkR package so that it can be distributed to worker nodes on YARN pushd %SPARK_HOME%\R\lib diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index 4be420b4e8594..87eb82935e4e0 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -115,8 +115,8 @@ $env:Path += ";$env:HADOOP_HOME\bin" Pop-Location # ========================== R -$rVer = "3.6.2" -$rToolsVer = "4.0.0" +$rVer = "4.0.2" +$rToolsVer = "4.0.2" InstallR InstallRtools From df04107934241965199bd5454c62e1016bb3bdd9 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Wed, 24 Jun 2020 22:13:54 +0900 Subject: [PATCH 071/384] [SPARK-32080][SPARK-31998][SQL] Simplify ArrowColumnVector ListArray accessor ### What changes were proposed in this pull request? This change simplifies the ArrowColumnVector ListArray accessor to use provided Arrow APIs available in v0.15.0 to calculate element indices. ### Why are the changes needed? This simplifies the code by avoiding manual calculations on the Arrow offset buffer and makes use of more stable APIs. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests Closes #28915 from BryanCutler/arrow-simplify-ArrowColumnVector-ListArray-SPARK-32080. Authored-by: Bryan Cutler Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/vectorized/ArrowColumnVector.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index d2220dc2668f9..72fccd451ca12 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -17,7 +17,6 @@ package org.apache.spark.sql.vectorized; -import io.netty.buffer.ArrowBuf; import org.apache.arrow.vector.*; import org.apache.arrow.vector.complex.*; import org.apache.arrow.vector.holders.NullableVarCharHolder; @@ -458,10 +457,8 @@ final boolean isNullAt(int rowId) { @Override final ColumnarArray getArray(int rowId) { - ArrowBuf offsets = accessor.getOffsetBuffer(); - int index = rowId * ListVector.OFFSET_WIDTH; - int start = offsets.getInt(index); - int end = offsets.getInt(index + ListVector.OFFSET_WIDTH); + int start = accessor.getElementStartIndex(rowId); + int end = accessor.getElementEndIndex(rowId); return new ColumnarArray(arrayData, start, end - start); } } From 47fb9d60549da02b869a3f0aad2ccb34d455c963 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 24 Jun 2020 14:50:45 +0000 Subject: [PATCH 072/384] [SPARK-32087][SQL] Allow UserDefinedType to use encoder to deserialize rows in ScalaUDF as well ### What changes were proposed in this pull request? This PR tries to address the comment: https://github.com/apache/spark/pull/28645#discussion_r442183888 It changes `canUpCast/canCast` to allow cast from sub UDT to base UDT, in order to achieve the goal to allow UserDefinedType to use `ExpressionEncoder` to deserialize rows in ScalaUDF as well. One thing that needs to mention is, even we allow cast from sub UDT to base UDT, it doesn't really do the cast in `Cast`. Because, yet, sub UDT and base UDT are considered as the same type(because of #16660), see: https://github.com/apache/spark/blob/5264164a67df498b73facae207eda12ee133be7d/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala#L81-L86 https://github.com/apache/spark/blob/5264164a67df498b73facae207eda12ee133be7d/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala#L92-L95 Therefore, the optimize rule `SimplifyCast` will eliminate the cast at the end. ### Why are the changes needed? Reduce the special case caused by `UserDefinedType` in `ResolveEncodersInUDF` and `ScalaUDF`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? It should be covered by the test of `SPARK-19311`, which is also updated a little in this PR. Closes #28920 from Ngone51/fix-udf-udt. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 21 +++++++------------ .../spark/sql/catalyst/expressions/Cast.scala | 8 +++---- .../sql/catalyst/expressions/ScalaUDF.scala | 4 +--- .../spark/sql/UserDefinedTypeSuite.scala | 4 ++-- 4 files changed, 15 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 49303978d1ce8..d08a6382f738b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2871,20 +2871,15 @@ class Analyzer( case udf: ScalaUDF if udf.inputEncoders.nonEmpty => val boundEncoders = udf.inputEncoders.zipWithIndex.map { case (encOpt, i) => val dataType = udf.children(i).dataType - if (dataType.existsRecursively(_.isInstanceOf[UserDefinedType[_]])) { - // for UDT, we use `CatalystTypeConverters` - None - } else { - encOpt.map { enc => - val attrs = if (enc.isSerializedAsStructForTopLevel) { - dataType.asInstanceOf[StructType].toAttributes - } else { - // the field name doesn't matter here, so we use - // a simple literal to avoid any overhead - new StructType().add("input", dataType).toAttributes - } - enc.resolveAndBind(attrs) + encOpt.map { enc => + val attrs = if (enc.isSerializedAsStructForTopLevel) { + dataType.asInstanceOf[StructType].toAttributes + } else { + // the field name doesn't matter here, so we use + // a simple literal to avoid any overhead + new StructType().add("input", dataType).toAttributes } + enc.resolveAndBind(attrs) } } udf.copy(inputEncoders = boundEncoders) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 3ce284d5518a8..e27c021556377 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -93,8 +93,7 @@ object Cast { toField.nullable) } - case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt1.userClass == udt2.userClass => - true + case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true case _ => false } @@ -157,6 +156,8 @@ object Cast { resolvableNullability(f1.nullable, f2.nullable) && canUpCast(f1.dataType, f2.dataType) } + case (from: UserDefinedType[_], to: UserDefinedType[_]) if to.acceptsType(from) => true + case _ => false } @@ -810,8 +811,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit castArray(from.asInstanceOf[ArrayType].elementType, array.elementType) case map: MapType => castMap(from.asInstanceOf[MapType], map) case struct: StructType => castStruct(from.asInstanceOf[StructType], struct) - case udt: UserDefinedType[_] - if udt.userClass == from.asInstanceOf[UserDefinedType[_]].userClass => + case udt: UserDefinedType[_] if udt.acceptsType(from) => identity[Any] case _: UserDefinedType[_] => throw new SparkException(s"Cannot cast $from to $to.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 1e3e6d90b8501..3d10b084a8db1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -108,7 +108,6 @@ case class ScalaUDF( * - UDF which doesn't provide inputEncoders, e.g., untyped Scala UDF and Java UDF * - type which isn't supported by `ExpressionEncoder`, e.g., Any * - primitive types, in order to use `identity` for better performance - * - UserDefinedType which isn't fully supported by `ExpressionEncoder` * For other cases like case class, Option[T], we use `ExpressionEncoder` instead since * `CatalystTypeConverters` doesn't support these data types. * @@ -121,8 +120,7 @@ case class ScalaUDF( val useEncoder = !(inputEncoders.isEmpty || // for untyped Scala UDF and Java UDF inputEncoders(i).isEmpty || // for types aren't supported by encoder, e.g. Any - inputPrimitives(i) || // for primitive types - dataType.existsRecursively(_.isInstanceOf[UserDefinedType[_]])) + inputPrimitives(i)) // for primitive types if (useEncoder) { val enc = inputEncoders(i).get diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index ed8ab1cb3a603..3fd5cc72cb95e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -275,11 +275,11 @@ class UserDefinedTypeSuite extends QueryTest with SharedSparkSession with Parque // this worked already before the fix SPARK-19311: // return type of doUDF equals parameter type of doOtherUDF - sql("SELECT doOtherUDF(doUDF(41))") + checkAnswer(sql("SELECT doOtherUDF(doUDF(41))"), Row(41) :: Nil) // this one passes only with the fix SPARK-19311: // return type of doSubUDF is a subtype of the parameter type of doOtherUDF - sql("SELECT doOtherUDF(doSubTypeUDF(42))") + checkAnswer(sql("SELECT doOtherUDF(doSubTypeUDF(42))"), Row(42) :: Nil) } test("except on UDT") { From 71b6d462fbeebf5e7e9a95896f0dca8297d0b8dd Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 24 Jun 2020 14:56:55 +0000 Subject: [PATCH 073/384] [SPARK-32089][R][BUILD] Upgrade R version to 4.0.2 in the release DockerFiile ### What changes were proposed in this pull request? This PR proposes to upgrade R version to 4.0.2 in the release docker image. As of SPARK-31918, we should make a release with R 4.0.0+ which works with R 3.5+ too. ### Why are the changes needed? To unblock releases on CRAN. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually tested via scripts under `dev/create-release`, manually attaching to the container and checking the R version. Closes #28922 from HyukjinKwon/SPARK-32089. Authored-by: HyukjinKwon Signed-off-by: Wenchen Fan --- dev/create-release/spark-rm/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 540dc90f42817..44d602415b262 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -21,7 +21,7 @@ # * Java 8 # * Ivy # * Python (2.7.15/3.6.7) -# * R-base/R-base-dev (3.6.1) +# * R-base/R-base-dev (4.0.2) # * Ruby 2.3 build utilities FROM ubuntu:18.04 @@ -43,7 +43,7 @@ ARG GEM_PKGS="jekyll:4.0.0 jekyll-redirect-from:0.16.0 rouge:3.15.0" # This is all in a single "RUN" command so that if anything changes, "apt update" is run to fetch # the most current package versions (instead of potentially using old versions cached by docker). RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ - echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' >> /etc/apt/sources.list && \ + echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list && \ gpg --keyserver keyserver.ubuntu.com --recv-key E298A3A825C0D65DFD57CBB651716619E084DAB9 && \ gpg -a --export E084DAB9 | apt-key add - && \ apt-get clean && \ From d06604f60a8a2ba0877616370a20aa18be15f8c4 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 24 Jun 2020 11:00:20 -0700 Subject: [PATCH 074/384] [SPARK-32078][DOC] Add a redirect to sql-ref from sql-reference MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR is to add a redirect to sql-ref.html. ### Why are the changes needed? Before Spark 3.0 release, we are using sql-reference.md, which was replaced by sql-ref.md instead. A number of Google searches I’ve done today have turned up https://spark.apache.org/docs/latest/sql-reference.html, which does not exist any more. Thus, we should add a redirect to sql-ref.html. ### Does this PR introduce _any_ user-facing change? https://spark.apache.org/docs/latest/sql-reference.html will be redirected to https://spark.apache.org/docs/latest/sql-ref.html ### How was this patch tested? Build it in my local environment. It works well. The sql-reference.html file was generated. The contents are like: ``` Redirecting…

Redirecting…

Click here if you are not redirected. ``` Closes #28914 from gatorsmile/addRedirectSQLRef. Authored-by: gatorsmile Signed-off-by: Dongjoon Hyun --- docs/_config.yml | 3 +++ docs/sql-ref.md | 1 + 2 files changed, 4 insertions(+) diff --git a/docs/_config.yml b/docs/_config.yml index f82394ed63694..93e1566a8e638 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -8,6 +8,9 @@ gems: kramdown: entity_output: numeric +plugins: + - jekyll-redirect-from + include: - _static - _modules diff --git a/docs/sql-ref.md b/docs/sql-ref.md index f88026b7abf02..8d0c6734c4bd6 100644 --- a/docs/sql-ref.md +++ b/docs/sql-ref.md @@ -1,6 +1,7 @@ --- layout: global title: SQL Reference +redirect_from: /sql-reference displayTitle: SQL Reference license: | Licensed to the Apache Software Foundation (ASF) under one or more From 1af19a7b6836f87a3b34189a8a13b6d21d3a37d8 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 25 Jun 2020 11:04:47 -0700 Subject: [PATCH 075/384] [SPARK-32098][PYTHON] Use iloc for positional slicing instead of direct slicing in createDataFrame with Arrow ### What changes were proposed in this pull request? When you use floats are index of pandas, it creates a Spark DataFrame with a wrong results as below when Arrow is enabled: ```bash ./bin/pyspark --conf spark.sql.execution.arrow.pyspark.enabled=true ``` ```python >>> import pandas as pd >>> spark.createDataFrame(pd.DataFrame({'a': [1,2,3]}, index=[2., 3., 4.])).show() +---+ | a| +---+ | 1| | 1| | 2| +---+ ``` This is because direct slicing uses the value as index when the index contains floats: ```python >>> pd.DataFrame({'a': [1,2,3]}, index=[2., 3., 4.])[2:] a 2.0 1 3.0 2 4.0 3 >>> pd.DataFrame({'a': [1,2,3]}, index=[2., 3., 4.]).iloc[2:] a 4.0 3 >>> pd.DataFrame({'a': [1,2,3]}, index=[2, 3, 4])[2:] a 4 3 ``` This PR proposes to explicitly use `iloc` to positionally slide when we create a DataFrame from a pandas DataFrame with Arrow enabled. FWIW, I was trying to investigate why direct slicing refers the index value or the positional index sometimes but I stopped investigating further after reading this https://pandas.pydata.org/pandas-docs/stable/getting_started/10min.html#selection > While standard Python / Numpy expressions for selecting and setting are intuitive and come in handy for interactive work, for production code, we recommend the optimized pandas data access methods, `.at`, `.iat`, `.loc` and `.iloc`. ### Why are the changes needed? To create the correct Spark DataFrame from a pandas DataFrame without a data loss. ### Does this PR introduce _any_ user-facing change? Yes, it is a bug fix. ```bash ./bin/pyspark --conf spark.sql.execution.arrow.pyspark.enabled=true ``` ```python import pandas as pd spark.createDataFrame(pd.DataFrame({'a': [1,2,3]}, index=[2., 3., 4.])).show() ``` Before: ``` +---+ | a| +---+ | 1| | 1| | 2| +---+ ``` After: ``` +---+ | a| +---+ | 1| | 2| | 3| +---+ ``` ### How was this patch tested? Manually tested and unittest were added. Closes #28928 from HyukjinKwon/SPARK-32098. Authored-by: HyukjinKwon Signed-off-by: Bryan Cutler --- python/pyspark/sql/pandas/conversion.py | 2 +- python/pyspark/sql/tests/test_arrow.py | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/pandas/conversion.py b/python/pyspark/sql/pandas/conversion.py index 251625ae412cf..e6d8e9f24a557 100644 --- a/python/pyspark/sql/pandas/conversion.py +++ b/python/pyspark/sql/pandas/conversion.py @@ -413,7 +413,7 @@ def _create_from_pandas_with_arrow(self, pdf, schema, timezone): # Slice the DataFrame to be batched step = -(-len(pdf) // self.sparkContext.defaultParallelism) # round int up - pdf_slices = (pdf[start:start + step] for start in xrange(0, len(pdf), step)) + pdf_slices = (pdf.iloc[start:start + step] for start in xrange(0, len(pdf), step)) # Create list of Arrow (columns, type) for serializer dump_stream arrow_data = [[(c, t) for (_, c), t in zip(pdf_slice.iteritems(), arrow_types)] diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index c59765dd79eb9..913b43b6ddb5a 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -442,6 +442,12 @@ def test_createDateFrame_with_category_type(self): self.assertIsInstance(arrow_first_category_element, str) self.assertIsInstance(spark_first_category_element, str) + def test_createDataFrame_with_float_index(self): + # SPARK-32098: float index should not produce duplicated or truncated Spark DataFrame + self.assertEqual( + self.spark.createDataFrame( + pd.DataFrame({'a': [1, 2, 3]}, index=[2., 3., 4.])).distinct().count(), 3) + @unittest.skipIf( not have_pandas or not have_pyarrow, From 594cb560750db1a9844386178027f9b28069525a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 25 Jun 2020 16:21:14 -0700 Subject: [PATCH 076/384] [SPARK-32100][CORE][TESTS] Add WorkerDecommissionExtendedSuite ### What changes were proposed in this pull request? This PR aims to add `WorkerDecomissionExtendedSuite` for various worker decommission combinations. ### Why are the changes needed? This will improve the test coverage. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #28929 from dongjoon-hyun/SPARK-WD-TEST. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../WorkerDecommissionExtendedSuite.scala | 73 +++++++++++++++++++ 1 file changed, 73 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala new file mode 100644 index 0000000000000..02c72fa349a79 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.concurrent.duration._ + +import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} + +import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite, TestUtils} +import org.apache.spark.LocalSparkContext.withSpark +import org.apache.spark.internal.config.{DYN_ALLOCATION_ENABLED, DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT, DYN_ALLOCATION_INITIAL_EXECUTORS, DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED} +import org.apache.spark.internal.config.Worker.WORKER_DECOMMISSION_ENABLED +import org.apache.spark.launcher.SparkLauncher.{EXECUTOR_MEMORY, SPARK_MASTER} +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend + +/** This test suite aims to test worker decommission with various configurations. */ +class WorkerDecommissionExtendedSuite extends SparkFunSuite with LocalSparkContext { + private val conf = new org.apache.spark.SparkConf() + .setAppName(getClass.getName) + .set(SPARK_MASTER, "local-cluster[20,1,512]") + .set(EXECUTOR_MEMORY, "512m") + .set(DYN_ALLOCATION_ENABLED, true) + .set(DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED, true) + .set(DYN_ALLOCATION_INITIAL_EXECUTORS, 20) + .set(WORKER_DECOMMISSION_ENABLED, true) + + test("Worker decommission and executor idle timeout") { + sc = new SparkContext(conf.set(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "10s")) + withSpark(sc) { sc => + TestUtils.waitUntilExecutorsUp(sc, 20, 60000) + val rdd1 = sc.parallelize(1 to 10, 2) + val rdd2 = rdd1.map(x => (1, x)) + val rdd3 = rdd2.reduceByKey(_ + _) + val rdd4 = rdd3.sortByKey() + assert(rdd4.count() === 1) + eventually(timeout(20.seconds), interval(1.seconds)) { + assert(sc.getExecutorIds().length < 5) + } + } + } + + test("Decommission 19 executors from 20 executors in total") { + sc = new SparkContext(conf) + withSpark(sc) { sc => + TestUtils.waitUntilExecutorsUp(sc, 20, 60000) + val rdd1 = sc.parallelize(1 to 100000, 200) + val rdd2 = rdd1.map(x => (x % 100, x)) + val rdd3 = rdd2.reduceByKey(_ + _) + assert(rdd3.count() === 100) + + val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] + sc.getExecutorIds().tail.foreach { id => + sched.decommissionExecutor(id) + assert(rdd3.sortByKey().collect().length === 100) + } + } + } +} From bbb2cba6158455c9e7744b1dbc778eb5d0a8860f Mon Sep 17 00:00:00 2001 From: Pablo Langa Date: Fri, 26 Jun 2020 10:41:27 +0900 Subject: [PATCH 077/384] [SPARK-32025][SQL] Csv schema inference problems with different types in the same column ### What changes were proposed in this pull request? This pull request fixes a bug present in the csv type inference. We have problems when we have different types in the same column. **Previously:** ``` $ cat /example/f1.csv col1 43200000 true spark.read.csv(path="file:///example/*.csv", header=True, inferSchema=True).show() +----+ |col1| +----+ |null| |true| +----+ root |-- col1: boolean (nullable = true) ``` **Now** ``` spark.read.csv(path="file:///example/*.csv", header=True, inferSchema=True).show() +-------------+ |col1 | +-------------+ |43200000 | |true | +-------------+ root |-- col1: string (nullable = true) ``` Previously the hierarchy of type inference is the following: > IntegerType > > LongType > > > DecimalType > > > > DoubleType > > > > > TimestampType > > > > > > BooleanType > > > > > > > StringType So, when, for example, we have integers in one column, and the last element is a boolean, all the column is inferred as a boolean column incorrectly and all the number are shown as null when you see the data We need the following hierarchy. When we have different numeric types in the column it will be resolved correctly. And when we have other different types it will be resolved as a String type column > IntegerType > > LongType > > > DecimalType > > > > DoubleType > > > > > StringType > TimestampType > > StringType > BooleanType > > StringType > StringType ### Why are the changes needed? Fix the bug explained ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test and manual tests Closes #28896 from planga82/feature/SPARK-32025_csv_inference. Authored-by: Pablo Langa Signed-off-by: HyukjinKwon --- .../spark/sql/catalyst/csv/CSVInferSchema.scala | 7 +++---- .../spark/sql/catalyst/csv/CSVInferSchemaSuite.scala | 10 +++++----- .../sql/execution/datasources/csv/CSVSuite.scala | 12 ++++++++++++ 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala index f0df18da8eed6..56677d7d97af2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala @@ -102,13 +102,11 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { if (field == null || field.isEmpty || field == options.nullValue) { typeSoFar } else { - typeSoFar match { + val typeElemInfer = typeSoFar match { case NullType => tryParseInteger(field) case IntegerType => tryParseInteger(field) case LongType => tryParseLong(field) - case _: DecimalType => - // DecimalTypes have different precisions and scales, so we try to find the common type. - compatibleType(typeSoFar, tryParseDecimal(field)).getOrElse(StringType) + case _: DecimalType => tryParseDecimal(field) case DoubleType => tryParseDouble(field) case TimestampType => tryParseTimestamp(field) case BooleanType => tryParseBoolean(field) @@ -116,6 +114,7 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { case other: DataType => throw new UnsupportedOperationException(s"Unexpected data type $other") } + compatibleType(typeSoFar, typeElemInfer).getOrElse(StringType) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala index b014eb92fae50..d268f8c2e7210 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala @@ -56,11 +56,11 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { assert(inferSchema.inferField(IntegerType, "1.0") == DoubleType) assert(inferSchema.inferField(DoubleType, null) == DoubleType) assert(inferSchema.inferField(DoubleType, "test") == StringType) - assert(inferSchema.inferField(LongType, "2015-08-20 14:57:00") == TimestampType) - assert(inferSchema.inferField(DoubleType, "2015-08-20 15:57:00") == TimestampType) - assert(inferSchema.inferField(LongType, "True") == BooleanType) - assert(inferSchema.inferField(IntegerType, "FALSE") == BooleanType) - assert(inferSchema.inferField(TimestampType, "FALSE") == BooleanType) + assert(inferSchema.inferField(LongType, "2015-08-20 14:57:00") == StringType) + assert(inferSchema.inferField(DoubleType, "2015-08-20 15:57:00") == StringType) + assert(inferSchema.inferField(LongType, "True") == StringType) + assert(inferSchema.inferField(IntegerType, "FALSE") == StringType) + assert(inferSchema.inferField(TimestampType, "FALSE") == StringType) val textValueOne = Long.MaxValue.toString + "0" val decimalValueOne = new java.math.BigDecimal(textValueOne) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 366cf11871fa0..fcb7bdc25f08f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2341,6 +2341,18 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa checkAnswer(csv, Row(null)) } } + + test("SPARK-32025: infer the schema from mixed-type values") { + withTempPath { path => + Seq("col_mixed_types", "2012", "1997", "True").toDS.write.text(path.getCanonicalPath) + val df = spark.read.format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load(path.getCanonicalPath) + + assert(df.schema.last == StructField("col_mixed_types", StringType, true)) + } + } } class CSVv1Suite extends CSVSuite { From d1255297b85d9b39376bb479821cfb603bc7b47b Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 26 Jun 2020 12:55:38 -0500 Subject: [PATCH 078/384] [SPARK-19939][ML] Add support for association rules in ML ### What changes were proposed in this pull request? Adding support to Association Rules in Spark ml.fpm. ### Why are the changes needed? Support is an indication of how frequently the itemset of an association rule appears in the database and suggests if the rules are generally applicable to the dateset. Refer to [wiki](https://en.wikipedia.org/wiki/Association_rule_learning#Support) for more details. ### Does this PR introduce _any_ user-facing change? Yes. Associate Rules now have support measure ### How was this patch tested? existing and new unit test Closes #28903 from huaxingao/fpm. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- R/pkg/R/mllib_fpm.R | 5 +-- R/pkg/tests/fulltests/test_mllib_fpm.R | 3 +- .../org/apache/spark/ml/fpm/FPGrowth.scala | 20 +++++++----- .../spark/mllib/fpm/AssociationRules.scala | 2 +- .../apache/spark/ml/fpm/FPGrowthSuite.scala | 31 +++++++++++++++++-- python/pyspark/ml/fpm.py | 18 +++++------ python/pyspark/ml/tests/test_algorithms.py | 4 +-- 7 files changed, 57 insertions(+), 26 deletions(-) diff --git a/R/pkg/R/mllib_fpm.R b/R/pkg/R/mllib_fpm.R index 0cc7a16c302dc..30bc51b932041 100644 --- a/R/pkg/R/mllib_fpm.R +++ b/R/pkg/R/mllib_fpm.R @@ -122,11 +122,12 @@ setMethod("spark.freqItemsets", signature(object = "FPGrowthModel"), # Get association rules. #' @return A \code{SparkDataFrame} with association rules. -#' The \code{SparkDataFrame} contains four columns: +#' The \code{SparkDataFrame} contains five columns: #' \code{antecedent} (an array of the same type as the input column), #' \code{consequent} (an array of the same type as the input column), #' \code{condfidence} (confidence for the rule) -#' and \code{lift} (lift for the rule) +#' \code{lift} (lift for the rule) +#' and \code{support} (support for the rule) #' @rdname spark.fpGrowth #' @aliases associationRules,FPGrowthModel-method #' @note spark.associationRules(FPGrowthModel) since 2.2.0 diff --git a/R/pkg/tests/fulltests/test_mllib_fpm.R b/R/pkg/tests/fulltests/test_mllib_fpm.R index bc1e17538d41a..78d26d3324473 100644 --- a/R/pkg/tests/fulltests/test_mllib_fpm.R +++ b/R/pkg/tests/fulltests/test_mllib_fpm.R @@ -45,7 +45,8 @@ test_that("spark.fpGrowth", { antecedent = I(list(list("2"), list("3"))), consequent = I(list(list("1"), list("1"))), confidence = c(1, 1), - lift = c(1, 1) + lift = c(1, 1), + support = c(0.75, 0.5) ) expect_equivalent(expected_association_rules, collect(spark.associationRules(model))) diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala index e50d4255b1f37..f1a68edaed950 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -244,9 +244,9 @@ class FPGrowthModel private[ml] ( @transient private var _cachedRules: DataFrame = _ /** - * Get association rules fitted using the minConfidence. Returns a dataframe with four fields, - * "antecedent", "consequent", "confidence" and "lift", where "antecedent" and "consequent" are - * Array[T], whereas "confidence" and "lift" are Double. + * Get association rules fitted using the minConfidence. Returns a dataframe with five fields, + * "antecedent", "consequent", "confidence", "lift" and "support", where "antecedent" and + * "consequent" are Array[T], whereas "confidence", "lift" and "support" are Double. */ @Since("2.2.0") @transient def associationRules: DataFrame = { @@ -254,7 +254,8 @@ class FPGrowthModel private[ml] ( _cachedRules } else { _cachedRules = AssociationRules - .getAssociationRulesFromFP(freqItemsets, "items", "freq", $(minConfidence), itemSupport) + .getAssociationRulesFromFP(freqItemsets, "items", "freq", $(minConfidence), itemSupport, + numTrainingRecords) _cachedMinConf = $(minConfidence) _cachedRules } @@ -385,6 +386,7 @@ private[fpm] object AssociationRules { * @param freqCol column name for appearance count of the frequent itemsets * @param minConfidence minimum confidence for generating the association rules * @param itemSupport map containing an item and its support + * @param numTrainingRecords count of training Dataset * @return a DataFrame("antecedent"[Array], "consequent"[Array], "confidence"[Double], * "lift" [Double]) containing the association rules. */ @@ -393,21 +395,23 @@ private[fpm] object AssociationRules { itemsCol: String, freqCol: String, minConfidence: Double, - itemSupport: scala.collection.Map[T, Double]): DataFrame = { - + itemSupport: scala.collection.Map[T, Double], + numTrainingRecords: Long): DataFrame = { val freqItemSetRdd = dataset.select(itemsCol, freqCol).rdd .map(row => new FreqItemset(row.getSeq[T](0).toArray, row.getLong(1))) val rows = new MLlibAssociationRules() .setMinConfidence(minConfidence) .run(freqItemSetRdd, itemSupport) - .map(r => Row(r.antecedent, r.consequent, r.confidence, r.lift.orNull)) + .map(r => Row(r.antecedent, r.consequent, r.confidence, r.lift.orNull, + r.freqUnion / numTrainingRecords)) val dt = dataset.schema(itemsCol).dataType val schema = StructType(Seq( StructField("antecedent", dt, nullable = false), StructField("consequent", dt, nullable = false), StructField("confidence", DoubleType, nullable = false), - StructField("lift", DoubleType))) + StructField("lift", DoubleType), + StructField("support", DoubleType, nullable = false))) val rules = dataset.sparkSession.createDataFrame(rows, schema) rules } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 43d256bbc46c3..601c7da30ffed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -124,7 +124,7 @@ object AssociationRules { class Rule[Item] private[fpm] ( @Since("1.5.0") val antecedent: Array[Item], @Since("1.5.0") val consequent: Array[Item], - freqUnion: Double, + private[spark] val freqUnion: Double, freqAntecedent: Double, freqConsequent: Option[Double]) extends Serializable { diff --git a/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala index b75526a48371a..d42ced0f8f91b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala @@ -39,9 +39,9 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul val model = new FPGrowth().setMinSupport(0.5).fit(data) val generatedRules = model.setMinConfidence(0.5).associationRules val expectedRules = spark.createDataFrame(Seq( - (Array("2"), Array("1"), 1.0, 1.0), - (Array("1"), Array("2"), 0.75, 1.0) - )).toDF("antecedent", "consequent", "confidence", "lift") + (Array("2"), Array("1"), 1.0, 1.0, 0.75), + (Array("1"), Array("2"), 0.75, 1.0, 0.75) + )).toDF("antecedent", "consequent", "confidence", "lift", "support") .withColumn("antecedent", col("antecedent").cast(ArrayType(dt))) .withColumn("consequent", col("consequent").cast(ArrayType(dt))) assert(expectedRules.sort("antecedent").rdd.collect().sameElements( @@ -61,6 +61,31 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul } } + test("FPGrowth associationRules") { + val dataset = spark.createDataFrame(Seq( + (1, Array("1", "2")), + (2, Array("3")), + (3, Array("4", "5")), + (4, Array("1", "2", "3")), + (5, Array("2")) + )).toDF("id", "items") + val model = new FPGrowth().setMinSupport(0.1).setMinConfidence(0.1).fit(dataset) + val expectedRules = spark.createDataFrame(Seq( + (Array("2"), Array("1"), 0.6666666666666666, 1.6666666666666665, 0.4), + (Array("2"), Array("3"), 0.3333333333333333, 0.8333333333333333, 0.2), + (Array("3"), Array("1"), 0.5, 1.25, 0.2), + (Array("3"), Array("2"), 0.5, 0.8333333333333334, 0.2), + (Array("1", "3"), Array("2"), 1.0, 1.6666666666666667, 0.2), + (Array("1", "2"), Array("3"), 0.5, 1.25, 0.2), + (Array("4"), Array("5"), 1.0, 5.0, 0.2), + (Array("5"), Array("4"), 1.0, 5.0, 0.2), + (Array("1"), Array("3"), 0.5, 1.25, 0.2), + (Array("1"), Array("2"), 1.0, 1.6666666666666667, 0.4), + (Array("3", "2"), Array("1"), 1.0, 2.5, 0.2) + )).toDF("antecedent", "consequent", "confidence", "lift", "support") + assert(expectedRules.collect().toSet.equals(model.associationRules.collect().toSet)) + } + test("FPGrowth getFreqItems") { val model = new FPGrowth().setMinSupport(0.7).fit(dataset) val expectedFreq = spark.createDataFrame(Seq( diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index 7d933daf9e032..7a5591f3fbf76 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -180,15 +180,15 @@ class FPGrowth(JavaEstimator, _FPGrowthParams, JavaMLWritable, JavaMLReadable): only showing top 5 rows ... >>> fpm.associationRules.show(5) - +----------+----------+----------+----+ - |antecedent|consequent|confidence|lift| - +----------+----------+----------+----+ - | [t, s]| [y]| 1.0| 2.0| - | [t, s]| [x]| 1.0| 1.5| - | [t, s]| [z]| 1.0| 1.2| - | [p]| [r]| 1.0| 2.0| - | [p]| [z]| 1.0| 1.2| - +----------+----------+----------+----+ + +----------+----------+----------+----+------------------+ + |antecedent|consequent|confidence|lift| support| + +----------+----------+----------+----+------------------+ + | [t, s]| [y]| 1.0| 2.0|0.3333333333333333| + | [t, s]| [x]| 1.0| 1.5|0.3333333333333333| + | [t, s]| [z]| 1.0| 1.2|0.3333333333333333| + | [p]| [r]| 1.0| 2.0|0.3333333333333333| + | [p]| [z]| 1.0| 1.2|0.3333333333333333| + +----------+----------+----------+----+------------------+ only showing top 5 rows ... >>> new_data = spark.createDataFrame([(["t", "s"], )], ["items"]) diff --git a/python/pyspark/ml/tests/test_algorithms.py b/python/pyspark/ml/tests/test_algorithms.py index 2faf2d98f0271..c948bd0c646de 100644 --- a/python/pyspark/ml/tests/test_algorithms.py +++ b/python/pyspark/ml/tests/test_algorithms.py @@ -226,8 +226,8 @@ def test_association_rules(self): fpm = fp.fit(self.data) expected_association_rules = self.spark.createDataFrame( - [([3], [1], 1.0, 1.0), ([2], [1], 1.0, 1.0)], - ["antecedent", "consequent", "confidence", "lift"] + [([3], [1], 1.0, 1.0, 0.5), ([2], [1], 1.0, 1.0, 0.75)], + ["antecedent", "consequent", "confidence", "lift", "support"] ) actual_association_rules = fpm.associationRules From 879513370767f647765ff5b96adb08f5b8c46489 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 26 Jun 2020 12:57:30 -0500 Subject: [PATCH 079/384] [SPARK-20249][ML][PYSPARK] Add training summary for LinearSVCModel ### What changes were proposed in this pull request? Add training summary for LinearSVCModel...... ### Why are the changes needed? so that user can get the training process status, such as loss value of each iteration and total iteration number. ### Does this PR introduce _any_ user-facing change? Yes ```LinearSVCModel.summary``` ```LinearSVCModel.evaluate``` ### How was this patch tested? new tests Closes #28884 from huaxingao/svc_summary. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../spark/ml/classification/LinearSVC.scala | 117 +++++++++++++++++- .../ml/classification/LinearSVCSuite.scala | 53 +++++++- python/pyspark/ml/classification.py | 48 ++++++- .../pyspark/ml/tests/test_training_summary.py | 46 ++++++- 4 files changed, 257 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 217398c51b393..1659bbb1d34b3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -36,7 +36,7 @@ import org.apache.spark.ml.stat._ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql._ import org.apache.spark.storage.StorageLevel /** Params for linear SVM Classifier. */ @@ -267,7 +267,26 @@ class LinearSVC @Since("2.2.0") ( if (featuresStd(i) != 0.0) rawCoefficients(i) / featuresStd(i) else 0.0 } val intercept = if ($(fitIntercept)) rawCoefficients.last else 0.0 - copyValues(new LinearSVCModel(uid, Vectors.dense(coefficientArray), intercept)) + createModel(dataset, Vectors.dense(coefficientArray), intercept, objectiveHistory) + } + + private def createModel( + dataset: Dataset[_], + coefficients: Vector, + intercept: Double, + objectiveHistory: Array[Double]): LinearSVCModel = { + val model = copyValues(new LinearSVCModel(uid, coefficients, intercept)) + val weightColName = if (!isDefined(weightCol)) "weightCol" else $(weightCol) + + val (summaryModel, rawPredictionColName, predictionColName) = model.findSummaryModel() + val summary = new LinearSVCTrainingSummaryImpl( + summaryModel.transform(dataset), + rawPredictionColName, + predictionColName, + $(labelCol), + weightColName, + objectiveHistory) + model.setSummary(Some(summary)) } private def trainOnRows( @@ -352,7 +371,7 @@ class LinearSVCModel private[classification] ( @Since("2.2.0") val coefficients: Vector, @Since("2.2.0") val intercept: Double) extends ClassificationModel[Vector, LinearSVCModel] - with LinearSVCParams with MLWritable { + with LinearSVCParams with MLWritable with HasTrainingSummary[LinearSVCTrainingSummary] { @Since("2.2.0") override val numClasses: Int = 2 @@ -368,6 +387,48 @@ class LinearSVCModel private[classification] ( BLAS.dot(features, coefficients) + intercept } + /** + * Gets summary of model on training set. An exception is thrown + * if `hasSummary` is false. + */ + @Since("3.1.0") + override def summary: LinearSVCTrainingSummary = super.summary + + /** + * If the rawPrediction and prediction columns are set, this method returns the current model, + * otherwise it generates new columns for them and sets them as columns on a new copy of + * the current model + */ + private[classification] def findSummaryModel(): (LinearSVCModel, String, String) = { + val model = if ($(rawPredictionCol).isEmpty && $(predictionCol).isEmpty) { + copy(ParamMap.empty) + .setRawPredictionCol("rawPrediction_" + java.util.UUID.randomUUID.toString) + .setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) + } else if ($(rawPredictionCol).isEmpty) { + copy(ParamMap.empty).setRawPredictionCol("rawPrediction_" + + java.util.UUID.randomUUID.toString) + } else if ($(predictionCol).isEmpty) { + copy(ParamMap.empty).setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) + } else { + this + } + (model, model.getRawPredictionCol, model.getPredictionCol) + } + + /** + * Evaluates the model on a test dataset. + * + * @param dataset Test dataset to evaluate model on. + */ + @Since("3.1.0") + def evaluate(dataset: Dataset[_]): LinearSVCSummary = { + val weightColName = if (!isDefined(weightCol)) "weightCol" else $(weightCol) + // Handle possible missing or invalid rawPrediction or prediction columns + val (summaryModel, rawPrediction, predictionColName) = findSummaryModel() + new LinearSVCSummaryImpl(summaryModel.transform(dataset), + rawPrediction, predictionColName, $(labelCol), weightColName) + } + override def predict(features: Vector): Double = { if (margin(features) > $(threshold)) 1.0 else 0.0 } @@ -439,3 +500,53 @@ object LinearSVCModel extends MLReadable[LinearSVCModel] { } } } + +/** + * Abstraction for LinearSVC results for a given model. + */ +sealed trait LinearSVCSummary extends BinaryClassificationSummary + +/** + * Abstraction for LinearSVC training results. + */ +sealed trait LinearSVCTrainingSummary extends LinearSVCSummary with TrainingSummary + +/** + * LinearSVC results for a given model. + * + * @param predictions dataframe output by the model's `transform` method. + * @param scoreCol field in "predictions" which gives the rawPrediction of each instance. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + */ +private class LinearSVCSummaryImpl( + @transient override val predictions: DataFrame, + override val scoreCol: String, + override val predictionCol: String, + override val labelCol: String, + override val weightCol: String) + extends LinearSVCSummary + +/** + * LinearSVC training results. + * + * @param predictions dataframe output by the model's `transform` method. + * @param scoreCol field in "predictions" which gives the rawPrediction of each instance. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. + */ +private class LinearSVCTrainingSummaryImpl( + predictions: DataFrame, + scoreCol: String, + predictionCol: String, + labelCol: String, + weightCol: String, + override val objectiveHistory: Array[Double]) + extends LinearSVCSummaryImpl( + predictions, scoreCol, predictionCol, labelCol, weightCol) + with LinearSVCTrainingSummary diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala index 579d6b12ab99f..a66397324c1a6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.udf +import org.apache.spark.sql.functions._ class LinearSVCSuite extends MLTest with DefaultReadWriteTest { @@ -284,6 +284,57 @@ class LinearSVCSuite extends MLTest with DefaultReadWriteTest { assert(model1.coefficients ~== coefficientsSK relTol 4E-3) } + test("summary and training summary") { + val lsvc = new LinearSVC() + val model = lsvc.setMaxIter(5).fit(smallBinaryDataset) + + val summary = model.evaluate(smallBinaryDataset) + + assert(model.summary.accuracy === summary.accuracy) + assert(model.summary.weightedPrecision === summary.weightedPrecision) + assert(model.summary.weightedRecall === summary.weightedRecall) + assert(model.summary.pr.collect() === summary.pr.collect()) + assert(model.summary.roc.collect() === summary.roc.collect()) + assert(model.summary.areaUnderROC === summary.areaUnderROC) + + // verify instance weight works + val lsvc2 = new LinearSVC() + .setMaxIter(5) + .setWeightCol("weight") + + val smallBinaryDatasetWithWeight = + smallBinaryDataset.select(col("label"), col("features"), lit(2.5).as("weight")) + + val summary2 = model.evaluate(smallBinaryDatasetWithWeight) + + val model2 = lsvc2.fit(smallBinaryDatasetWithWeight) + assert(model2.summary.accuracy === summary2.accuracy) + assert(model2.summary.weightedPrecision ~== summary2.weightedPrecision relTol 1e-6) + assert(model2.summary.weightedRecall === summary2.weightedRecall) + assert(model2.summary.pr.collect() === summary2.pr.collect()) + assert(model2.summary.roc.collect() === summary2.roc.collect()) + assert(model2.summary.areaUnderROC === summary2.areaUnderROC) + + assert(model2.summary.accuracy === model.summary.accuracy) + assert(model2.summary.weightedPrecision ~== model.summary.weightedPrecision relTol 1e-6) + assert(model2.summary.weightedRecall === model.summary.weightedRecall) + assert(model2.summary.pr.collect() === model.summary.pr.collect()) + assert(model2.summary.roc.collect() === model.summary.roc.collect()) + assert(model2.summary.areaUnderROC === model.summary.areaUnderROC) + } + + test("linearSVC training summary totalIterations") { + Seq(1, 5, 10, 20, 100).foreach { maxIter => + val trainer = new LinearSVC().setMaxIter(maxIter) + val model = trainer.fit(smallBinaryDataset) + if (maxIter == 1) { + assert(model.summary.totalIterations === maxIter) + } else { + assert(model.summary.totalIterations <= maxIter) + } + } + } + test("read/write: SVM") { def checkModelData(model: LinearSVCModel, model2: LinearSVCModel): Unit = { assert(model.intercept === model2.intercept) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index ff506066519cd..bdd37c99df0a8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -39,6 +39,7 @@ from pyspark.storagelevel import StorageLevel __all__ = ['LinearSVC', 'LinearSVCModel', + 'LinearSVCSummary', 'LinearSVCTrainingSummary', 'LogisticRegression', 'LogisticRegressionModel', 'LogisticRegressionSummary', 'LogisticRegressionTrainingSummary', 'BinaryLogisticRegressionSummary', 'BinaryLogisticRegressionTrainingSummary', @@ -683,7 +684,8 @@ def setBlockSize(self, value): return self._set(blockSize=value) -class LinearSVCModel(_JavaClassificationModel, _LinearSVCParams, JavaMLWritable, JavaMLReadable): +class LinearSVCModel(_JavaClassificationModel, _LinearSVCParams, JavaMLWritable, JavaMLReadable, + HasTrainingSummary): """ Model fitted by LinearSVC. @@ -713,6 +715,50 @@ def intercept(self): """ return self._call_java("intercept") + @since("3.1.0") + def summary(self): + """ + Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + trained on the training set. An exception is thrown if `trainingSummary is None`. + """ + if self.hasSummary: + return LinearSVCTrainingSummary(super(LinearSVCModel, self).summary) + else: + raise RuntimeError("No training summary available for this %s" % + self.__class__.__name__) + + @since("3.1.0") + def evaluate(self, dataset): + """ + Evaluates the model on a test dataset. + + :param dataset: + Test dataset to evaluate model on, where dataset is an + instance of :py:class:`pyspark.sql.DataFrame` + """ + if not isinstance(dataset, DataFrame): + raise ValueError("dataset must be a DataFrame but got %s." % type(dataset)) + java_lsvc_summary = self._call_java("evaluate", dataset) + return LinearSVCSummary(java_lsvc_summary) + + +class LinearSVCSummary(_BinaryClassificationSummary): + """ + Abstraction for LinearSVC Results for a given model. + .. versionadded:: 3.1.0 + """ + pass + + +@inherit_doc +class LinearSVCTrainingSummary(LinearSVCSummary, _TrainingSummary): + """ + Abstraction for LinearSVC Training results. + + .. versionadded:: 3.1.0 + """ + pass + class _LogisticRegressionParams(_ProbabilisticClassifierParams, HasRegParam, HasElasticNetParam, HasMaxIter, HasFitIntercept, HasTol, diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index ac944d8397a86..19acd194f4ddf 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -21,8 +21,8 @@ if sys.version > '3': basestring = str -from pyspark.ml.classification import BinaryLogisticRegressionSummary, LogisticRegression, \ - LogisticRegressionSummary +from pyspark.ml.classification import BinaryLogisticRegressionSummary, LinearSVC, \ + LinearSVCSummary, LogisticRegression, LogisticRegressionSummary from pyspark.ml.clustering import BisectingKMeans, GaussianMixture, KMeans from pyspark.ml.linalg import Vectors from pyspark.ml.regression import GeneralizedLinearRegression, LinearRegression @@ -193,6 +193,48 @@ def test_multiclass_logistic_regression_summary(self): self.assertFalse(isinstance(sameSummary, BinaryLogisticRegressionSummary)) self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) + def test_linear_svc_summary(self): + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0, 1.0, 1.0)), + (0.0, 2.0, Vectors.dense(1.0, 2.0, 3.0))], + ["label", "weight", "features"]) + svc = LinearSVC(maxIter=5, weightCol="weight") + model = svc.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary() + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.scoreCol, "rawPrediction") + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.predictionCol, "prediction") + objHist = s.objectiveHistory + self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) + self.assertGreater(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertTrue(isinstance(s.roc, DataFrame)) + self.assertAlmostEqual(s.areaUnderROC, 1.0, 2) + self.assertTrue(isinstance(s.pr, DataFrame)) + self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame)) + self.assertTrue(isinstance(s.precisionByThreshold, DataFrame)) + self.assertTrue(isinstance(s.recallByThreshold, DataFrame)) + print(s.weightedTruePositiveRate) + self.assertAlmostEqual(s.weightedTruePositiveRate, 0.5, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.5, 2) + self.assertAlmostEqual(s.weightedRecall, 0.5, 2) + self.assertAlmostEqual(s.weightedPrecision, 0.25, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 0.3333333333333333, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 0.3333333333333333, 2) + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, LinearSVCSummary)) + self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) + def test_gaussian_mixture_summary(self): data = [(Vectors.dense(1.0),), (Vectors.dense(5.0),), (Vectors.dense(10.0),), (Vectors.sparse(1, [], []),)] From ac3a0551d82c8e808d01aecbd1f6918cfe331ec4 Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Fri, 26 Jun 2020 19:06:31 -0700 Subject: [PATCH 080/384] [SPARK-32088][PYTHON] Pin the timezone in timestamp_seconds doctest ### What changes were proposed in this pull request? Add American timezone during timestamp_seconds doctest ### Why are the changes needed? `timestamp_seconds` doctest in `functions.py` used default timezone to get expected result For example: ```python >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() [Row(ts=datetime.datetime(2008, 12, 25, 7, 30))] ``` But when we have a non-american timezone, the test case will get different test result. For example, when we set current timezone as `Asia/Shanghai`, the test result will be ``` [Row(ts=datetime.datetime(2008, 12, 25, 23, 30))] ``` So no matter where we run the test case ,we will always get the expected permanent result if we set the timezone on one specific area. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test Closes #28932 from GuoPhilipse/SPARK-32088-fix-timezone-issue. Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Co-authored-by: GuoPhilipse Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/functions.py | 2 ++ sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 0c8c34dd87996..b0498d0298785 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1431,9 +1431,11 @@ def to_utc_timestamp(timestamp, tz): def timestamp_seconds(col): """ >>> from pyspark.sql.functions import timestamp_seconds + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() [Row(ts=datetime.datetime(2008, 12, 25, 7, 30))] + >>> spark.conf.unset("spark.sql.session.timeZone") """ sc = SparkContext._active_spark_context 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 62ad5ea9b5935..239b705a473d0 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 @@ -3360,8 +3360,8 @@ object functions { /** * Creates timestamp from the number of seconds since UTC epoch. - * @group = datetime_funcs - * @since = 3.1.0 + * @group datetime_funcs + * @since 3.1.0 */ def timestamp_seconds(e: Column): Column = withExpr { SecondsToTimestamp(e.expr) From 44aecaa9124fb2158f009771022c64ede4b582dc Mon Sep 17 00:00:00 2001 From: Guy Khazma Date: Fri, 26 Jun 2020 19:12:42 -0700 Subject: [PATCH 081/384] [SPARK-32099][DOCS] Remove broken link in cloud integration documentation ### What changes were proposed in this pull request? The 3rd link in `IBM Cloud Object Storage connector for Apache Spark` is broken. The PR removes this link. ### Why are the changes needed? broken link ### Does this PR introduce _any_ user-facing change? yes, the broken link is removed from the doc. ### How was this patch tested? doc generation passes successfully as before Closes #28927 from guykhazma/spark32099. Authored-by: Guy Khazma Signed-off-by: Dongjoon Hyun --- docs/cloud-integration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index b2a3e77f1ee9d..01f437f38ef17 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -257,5 +257,5 @@ Here is the documentation on the standard connectors both from Apache and the cl * [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon * [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google * [The Azure Blob Filesystem driver (ABFS)](https://docs.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-abfs-driver) -* IBM Cloud Object Storage connector for Apache Spark: [Stocator](https://github.com/CODAIT/stocator), [IBM Object Storage](https://www.ibm.com/cloud/object-storage), [how-to-use-connector](https://developer.ibm.com/code/2018/08/16/installing-running-stocator-apache-spark-ibm-cloud-object-storage). From IBM +* IBM Cloud Object Storage connector for Apache Spark: [Stocator](https://github.com/CODAIT/stocator), [IBM Object Storage](https://www.ibm.com/cloud/object-storage). From IBM From 7445c7534ba11bcbdf2e05259cd4f5cde13fe5fb Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Fri, 26 Jun 2020 19:36:06 -0700 Subject: [PATCH 082/384] [SPARK-31845][CORE][TESTS] Refactor DAGSchedulerSuite by introducing completeAndCheckAnswer and using completeNextStageWithFetchFailure ### What changes were proposed in this pull request? **First** `DAGSchedulerSuite` provides `completeNextStageWithFetchFailure` to make all tasks in non first stage occur `FetchFailed`. But many test case uses complete directly as follows: ```scala complete(taskSets(1), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDep1.shuffleId, 0L, 0, 0, "ignored"), null))) ``` We need to reuse `completeNextStageWithFetchFailure`. **Second** `DAGSchedulerSuite` also check the results show below: ```scala complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) ``` We can extract it as a generic method of `checkAnswer`. ### Why are the changes needed? Reuse `completeNextStageWithFetchFailure` ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28866 from beliefer/reuse-completeNextStageWithFetchFailure. Authored-by: gengjiaan Signed-off-by: Dongjoon Hyun --- .../spark/scheduler/DAGSchedulerSuite.scala | 51 +++++++++---------- 1 file changed, 23 insertions(+), 28 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9d412f2dba3ce..51d20d3428915 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -349,9 +349,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi it.next.asInstanceOf[Tuple2[_, _]]._1 /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ - private def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]): Unit = { - assert(taskSet.tasks.size >= results.size) - for ((result, i) <- results.zipWithIndex) { + private def complete(taskSet: TaskSet, taskEndInfos: Seq[(TaskEndReason, Any)]): Unit = { + assert(taskSet.tasks.size >= taskEndInfos.size) + for ((result, i) <- taskEndInfos.zipWithIndex) { if (i < taskSet.tasks.size) { runEvent(makeCompletionEvent(taskSet.tasks(i), result._1, result._2)) } @@ -405,6 +405,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(JobCancelled(jobId, None)) } + /** Make some tasks in task set success and check results. */ + private def completeAndCheckAnswer( + taskSet: TaskSet, + taskEndInfos: Seq[(TaskEndReason, Any)], + expected: Map[Int, Any]): Unit = { + complete(taskSet, taskEndInfos) + assert(this.results === expected) + } + test("[SPARK-3353] parent stage should have lower stage id") { sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count() val stageByOrderOfExecution = sparkListener.stageByOrderOfExecution @@ -461,8 +470,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi completeShuffleMapStageSuccessfully(0, 0, 1) completeShuffleMapStageSuccessfully(1, 0, 1) completeShuffleMapStageSuccessfully(2, 0, 1) - complete(taskSets(3), Seq((Success, 42))) - assert(results === Map(0 -> 42)) + completeAndCheckAnswer(taskSets(3), Seq((Success, 42)), Map(0 -> 42)) assertDataStructuresEmpty() } @@ -558,8 +566,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi test("run trivial job") { submit(new MyRDD(sc, 1, Nil), Array(0)) - complete(taskSets(0), List((Success, 42))) - assert(results === Map(0 -> 42)) + completeAndCheckAnswer(taskSets(0), Seq((Success, 42)), Map(0 -> 42)) assertDataStructuresEmpty() } @@ -567,8 +574,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val baseRdd = new MyRDD(sc, 1, Nil) val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) submit(finalRdd, Array(0)) - complete(taskSets(0), Seq((Success, 42))) - assert(results === Map(0 -> 42)) + completeAndCheckAnswer(taskSets(0), Seq((Success, 42)), Map(0 -> 42)) assertDataStructuresEmpty() } @@ -592,8 +598,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(finalRdd, Array(0)) val taskSet = taskSets(0) assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) - complete(taskSet, Seq((Success, 42))) - assert(results === Map(0 -> 42)) + completeAndCheckAnswer(taskSet, Seq((Success, 42)), Map(0 -> 42)) assertDataStructuresEmpty() } @@ -729,8 +734,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(failure === null) // When the task set completes normally, state should be correctly updated. - complete(taskSets(0), Seq((Success, 42))) - assert(results === Map(0 -> 42)) + completeAndCheckAnswer(taskSets(0), Seq((Success, 42)), Map(0 -> 42)) assertDataStructuresEmpty() assert(sparkListener.failedStages.isEmpty) @@ -746,8 +750,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi completeShuffleMapStageSuccessfully(0, 0, 1) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) - complete(taskSets(1), Seq((Success, 42))) - assert(results === Map(0 -> 42)) + completeAndCheckAnswer(taskSets(1), Seq((Success, 42)), Map(0 -> 42)) assertDataStructuresEmpty() } @@ -771,8 +774,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // we can see both result blocks now assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === HashSet("hostA", "hostB")) - complete(taskSets(3), Seq((Success, 43))) - assert(results === Map(0 -> 42, 1 -> 43)) + completeAndCheckAnswer(taskSets(3), Seq((Success, 43)), Map(0 -> 42, 1 -> 43)) assertDataStructuresEmpty() } @@ -1454,8 +1456,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) // finish the next stage normally, which completes the job - complete(taskSets(1), Seq((Success, 42), (Success, 43))) - assert(results === Map(0 -> 42, 1 -> 43)) + completeAndCheckAnswer(taskSets(1), Seq((Success, 42), (Success, 43)), Map(0 -> 42, 1 -> 43)) assertDataStructuresEmpty() } @@ -1796,9 +1797,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // lets say there is a fetch failure in this task set, which makes us go back and // run stage 0, attempt 1 - complete(taskSets(1), Seq( - (FetchFailed(makeBlockManagerId("hostA"), - shuffleDep1.shuffleId, 0L, 0, 0, "ignored"), null))) + completeNextStageWithFetchFailure(1, 0, shuffleDep1) scheduler.resubmitFailedStages() // stage 0, attempt 1 should have the properties of job2 @@ -1872,9 +1871,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // have the second stage complete normally completeShuffleMapStageSuccessfully(1, 0, 1, Seq("hostA", "hostC")) // fail the third stage because hostA went down - complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), - shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) + completeNextStageWithFetchFailure(2, 0, shuffleDepTwo) // TODO assert this: // blockManagerMaster.removeExecutor("hostA-exec") // have DAGScheduler try again @@ -1900,9 +1897,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // complete stage 1 completeShuffleMapStageSuccessfully(1, 0, 1) // pretend stage 2 failed because hostA went down - complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), - shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) + completeNextStageWithFetchFailure(2, 0, shuffleDepTwo) // TODO assert this: // blockManagerMaster.removeExecutor("hostA-exec") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. From 9c134b57bff5b7e7f9c85aeed2e9539117a5b57d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 26 Jun 2020 19:43:29 -0700 Subject: [PATCH 083/384] [SPARK-32058][BUILD] Use Apache Hadoop 3.2.0 dependency by default ### What changes were proposed in this pull request? According to the dev mailing list discussion, this PR aims to switch the default Apache Hadoop dependency from 2.7.4 to 3.2.0 for Apache Spark 3.1.0 on December 2020. | Item | Default Hadoop Dependency | |------|-----------------------------| | Apache Spark Website | 3.2.0 | | Apache Download Site | 3.2.0 | | Apache Snapshot | 3.2.0 | | Maven Central | 3.2.0 | | PyPI | 2.7.4 (We will switch later) | | CRAN | 2.7.4 (We will switch later) | | Homebrew | 3.2.0 (already) | In Apache Spark 3.0.0 release, we focused on the other features. This PR targets for [Apache Spark 3.1.0 scheduled on December 2020](https://spark.apache.org/versioning-policy.html). ### Why are the changes needed? Apache Hadoop 3.2 has many fixes and new cloud-friendly features. **Reference** - 2017-08-04: https://hadoop.apache.org/release/2.7.4.html - 2019-01-16: https://hadoop.apache.org/release/3.2.0.html ### Does this PR introduce _any_ user-facing change? Since the default Hadoop dependency changes, the users will get a better support in a cloud environment. ### How was this patch tested? Pass the Jenkins. Closes #28897 from dongjoon-hyun/SPARK-32058. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 4 ++-- dev/run-tests.py | 2 +- pom.xml | 18 +++++++++--------- .../kubernetes/integration-tests/pom.xml | 6 +++--- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index eb972589a995e..31633456a6590 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -276,14 +276,14 @@ if [[ "$1" == "package" ]]; then # list of packages to be built, so it's ok for things to be missing in BINARY_PKGS_EXTRA. declare -A BINARY_PKGS_ARGS - BINARY_PKGS_ARGS["hadoop2.7"]="-Phadoop-2.7 $HIVE_PROFILES" + BINARY_PKGS_ARGS["hadoop3.2"]="-Phadoop-3.2 $HIVE_PROFILES" if ! is_dry_run; then BINARY_PKGS_ARGS["without-hadoop"]="-Phadoop-provided" if [[ $SPARK_VERSION < "3.0." ]]; then BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" else BINARY_PKGS_ARGS["hadoop2.7-hive1.2"]="-Phadoop-2.7 -Phive-1.2 $HIVE_PROFILES" - BINARY_PKGS_ARGS["hadoop3.2"]="-Phadoop-3.2 $HIVE_PROFILES" + BINARY_PKGS_ARGS["hadoop2.7"]="-Phadoop-2.7 $HIVE_PROFILES" fi fi diff --git a/dev/run-tests.py b/dev/run-tests.py index 5255a77ec2081..ec04c37857d96 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -574,7 +574,7 @@ def main(): # if we're on the Amplab Jenkins build servers setup variables # to reflect the environment settings build_tool = os.environ.get("AMPLAB_JENKINS_BUILD_TOOL", "sbt") - hadoop_version = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE", "hadoop2.7") + hadoop_version = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE", "hadoop3.2") hive_version = os.environ.get("AMPLAB_JENKINS_BUILD_HIVE_PROFILE", "hive2.3") test_env = "amplab_jenkins" # add path for Python3 in Jenkins if we're calling from a Jenkins machine diff --git a/pom.xml b/pom.xml index 82c12ae3dcb80..08ca13bfe9d37 100644 --- a/pom.xml +++ b/pom.xml @@ -119,11 +119,11 @@ spark 1.7.30 1.2.17 - 2.7.4 + 3.2.0 2.5.0 ${hadoop.version} 3.4.14 - 2.7.1 + 2.13.0 org.apache.hive core @@ -170,7 +170,7 @@ 1.1.7.5 1.1.2 1.10 - 2.4 + 2.5 2.6 @@ -3054,16 +3054,16 @@ hadoop-2.7 - + + 2.7.4 + 2.7.1 + 2.4 + hadoop-3.2 - - 3.2.0 - 2.13.0 - 2.5 - + diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 503540403f5ec..d1e00cc0b5b10 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -186,9 +186,6 @@ hadoop-2.7 - - true - com.amazonaws @@ -200,6 +197,9 @@ hadoop-3.2 + + true + com.amazonaws From 8c44d744631516a5cdaf63406e69a9dd11e5b878 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sat, 27 Jun 2020 17:54:06 -0700 Subject: [PATCH 084/384] [SPARK-32071][SQL][TESTS] Add `make_interval` benchmark ### What changes were proposed in this pull request? Add benchmarks for interval constructor `make_interval` and measure perf of 4 cases: 1. Constant (year, month) 2. Constant (week, day) 3. Constant (hour, minute, second, second fraction) 4. All fields are NOT constant. The benchmark results are generated 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 | ### Why are the changes needed? To have a base line for future perf improvements of `make_interval`, and to prevent perf regressions in the future. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running `IntervalBenchmark` via: ``` $ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.IntervalBenchmark" ``` Closes #28905 from MaxGekk/benchmark-make_interval. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- .../IntervalBenchmark-jdk11-results.txt | 63 +++++++++------ .../benchmarks/IntervalBenchmark-results.txt | 63 +++++++++------ .../benchmark/IntervalBenchmark.scala | 81 ++++++++++++++++--- 3 files changed, 146 insertions(+), 61 deletions(-) diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt index 8958d7c53413f..70a64931049c0 100644 --- a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt @@ -1,29 +1,40 @@ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -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 cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare string w/ interval 448 469 20 2.2 447.6 1.0X -prepare string w/o interval 405 409 4 2.5 404.6 1.1X -1 units w/ interval 321 328 6 3.1 321.4 1.4X -1 units w/o interval 303 307 4 3.3 303.1 1.5X -2 units w/ interval 445 458 12 2.2 444.6 1.0X -2 units w/o interval 416 424 10 2.4 416.2 1.1X -3 units w/ interval 1006 1012 8 1.0 1006.4 0.4X -3 units w/o interval 1240 1249 8 0.8 1239.6 0.4X -4 units w/ interval 1295 1418 106 0.8 1295.4 0.3X -4 units w/o interval 1172 1188 15 0.9 1171.6 0.4X -5 units w/ interval 1326 1335 11 0.8 1325.6 0.3X -5 units w/o interval 1309 1336 44 0.8 1308.7 0.3X -6 units w/ interval 1441 1464 29 0.7 1441.0 0.3X -6 units w/o interval 1350 1369 17 0.7 1350.1 0.3X -7 units w/ interval 1606 1669 99 0.6 1605.6 0.3X -7 units w/o interval 1546 1557 12 0.6 1546.3 0.3X -8 units w/ interval 1771 1875 120 0.6 1770.6 0.3X -8 units w/o interval 1775 1789 13 0.6 1775.2 0.3X -9 units w/ interval 2126 2757 849 0.5 2126.4 0.2X -9 units w/o interval 2053 2070 21 0.5 2053.3 0.2X -10 units w/ interval 2209 2243 30 0.5 2209.1 0.2X -10 units w/o interval 2400 2702 365 0.4 2400.2 0.2X -11 units w/ interval 2616 2699 72 0.4 2616.5 0.2X -11 units w/o interval 3218 3380 195 0.3 3218.4 0.1X +prepare string w/ interval 708 829 110 1.4 708.0 1.0X +prepare string w/o interval 660 672 14 1.5 660.3 1.1X +1 units w/ interval 514 543 33 1.9 514.2 1.4X +1 units w/o interval 476 492 20 2.1 475.9 1.5X +2 units w/ interval 751 767 14 1.3 751.0 0.9X +2 units w/o interval 709 716 11 1.4 709.0 1.0X +3 units w/ interval 1541 1551 15 0.6 1540.9 0.5X +3 units w/o interval 1531 1532 1 0.7 1531.5 0.5X +4 units w/ interval 1764 1768 5 0.6 1763.5 0.4X +4 units w/o interval 1737 1745 8 0.6 1736.6 0.4X +5 units w/ interval 1920 1930 10 0.5 1919.7 0.4X +5 units w/o interval 1928 1936 11 0.5 1927.9 0.4X +6 units w/ interval 2124 2127 4 0.5 2124.2 0.3X +6 units w/o interval 2124 2125 1 0.5 2123.7 0.3X +7 units w/ interval 2525 2541 15 0.4 2525.5 0.3X +7 units w/o interval 2512 2518 11 0.4 2511.5 0.3X +8 units w/ interval 2578 2597 19 0.4 2578.1 0.3X +8 units w/o interval 2558 2562 6 0.4 2558.1 0.3X +9 units w/ interval 2742 2750 9 0.4 2741.8 0.3X +9 units w/o interval 2752 2762 11 0.4 2751.8 0.3X +10 units w/ interval 3112 3123 10 0.3 3111.9 0.2X +10 units w/o interval 3116 3130 14 0.3 3115.7 0.2X +11 units w/ interval 3255 3273 20 0.3 3255.3 0.2X +11 units w/o interval 3294 3305 14 0.3 3293.6 0.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 +make_interval(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +-------------------------------------------------------------------------------------------------------------------------- +prepare make_interval() 3395 3410 16 0.3 3395.0 1.0X +make_interval(0, 1, 2, 3, 4, 5, 50.123456) 94 102 9 10.7 93.8 36.2X +make_interval(*, *, 2, 3, 4, 5, 50.123456) 136 139 4 7.3 136.5 24.9X +make_interval(0, 1, *, *, 4, 5, 50.123456) 115 119 4 8.7 114.8 29.6X +make_interval(0, 1, 2, 3, *, *, *) 3359 3382 37 0.3 3358.7 1.0X +make_interval(*, *, *, *, *, *, *) 3382 3388 9 0.3 3382.3 1.0X diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt index 48af333b78ba4..98b9f55c2e15e 100644 --- a/sql/core/benchmarks/IntervalBenchmark-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -1,29 +1,40 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -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 cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare string w/ interval 389 410 21 2.6 388.7 1.0X -prepare string w/o interval 340 360 18 2.9 340.5 1.1X -1 units w/ interval 378 389 16 2.6 377.8 1.0X -1 units w/o interval 346 350 5 2.9 346.2 1.1X -2 units w/ interval 444 457 11 2.3 444.2 0.9X -2 units w/o interval 455 464 12 2.2 455.1 0.9X -3 units w/ interval 942 964 20 1.1 941.5 0.4X -3 units w/o interval 927 1020 93 1.1 927.3 0.4X -4 units w/ interval 1114 1127 17 0.9 1113.9 0.3X -4 units w/o interval 1100 1105 4 0.9 1100.3 0.4X -5 units w/ interval 1180 1244 57 0.8 1180.1 0.3X -5 units w/o interval 1135 1141 6 0.9 1135.2 0.3X -6 units w/ interval 1284 1316 48 0.8 1284.0 0.3X -6 units w/o interval 1276 1357 122 0.8 1276.1 0.3X -7 units w/ interval 1609 1636 32 0.6 1609.1 0.2X -7 units w/o interval 1551 1578 36 0.6 1550.9 0.3X -8 units w/ interval 1787 1874 129 0.6 1787.1 0.2X -8 units w/o interval 1751 1767 15 0.6 1750.6 0.2X -9 units w/ interval 1960 2065 141 0.5 1959.7 0.2X -9 units w/o interval 1885 1908 39 0.5 1885.1 0.2X -10 units w/ interval 2178 2185 11 0.5 2177.9 0.2X -10 units w/o interval 2150 2255 164 0.5 2150.1 0.2X -11 units w/ interval 2457 2542 139 0.4 2456.7 0.2X -11 units w/o interval 2557 2770 188 0.4 2556.7 0.2X +prepare string w/ interval 677 718 40 1.5 677.2 1.0X +prepare string w/o interval 602 624 19 1.7 602.2 1.1X +1 units w/ interval 582 598 20 1.7 581.8 1.2X +1 units w/o interval 549 591 64 1.8 549.1 1.2X +2 units w/ interval 758 773 14 1.3 758.2 0.9X +2 units w/o interval 723 738 14 1.4 722.6 0.9X +3 units w/ interval 1442 1450 11 0.7 1441.8 0.5X +3 units w/o interval 1426 1429 3 0.7 1426.4 0.5X +4 units w/ interval 1645 1652 11 0.6 1645.1 0.4X +4 units w/o interval 1618 1626 10 0.6 1617.6 0.4X +5 units w/ interval 1794 1803 13 0.6 1794.4 0.4X +5 units w/o interval 1783 1793 9 0.6 1783.2 0.4X +6 units w/ interval 1976 1984 11 0.5 1976.2 0.3X +6 units w/o interval 1948 1959 10 0.5 1947.9 0.3X +7 units w/ interval 2394 2408 18 0.4 2393.7 0.3X +7 units w/o interval 2387 2392 8 0.4 2386.8 0.3X +8 units w/ interval 2578 2588 15 0.4 2577.5 0.3X +8 units w/o interval 2572 2578 5 0.4 2571.8 0.3X +9 units w/ interval 2812 2829 19 0.4 2811.7 0.2X +9 units w/o interval 2811 2816 4 0.4 2810.7 0.2X +10 units w/ interval 3108 3116 10 0.3 3107.8 0.2X +10 units w/o interval 3107 3109 3 0.3 3106.8 0.2X +11 units w/ interval 3386 3392 8 0.3 3386.3 0.2X +11 units w/o interval 3374 3377 4 0.3 3374.0 0.2X + +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 +make_interval(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +-------------------------------------------------------------------------------------------------------------------------- +prepare make_interval() 3634 3684 47 0.3 3634.1 1.0X +make_interval(0, 1, 2, 3, 4, 5, 50.123456) 90 100 12 11.1 90.0 40.4X +make_interval(*, *, 2, 3, 4, 5, 50.123456) 114 119 5 8.8 114.3 31.8X +make_interval(0, 1, *, *, 4, 5, 50.123456) 121 138 21 8.3 120.7 30.1X +make_interval(0, 1, 2, 3, *, *, *) 3615 3621 9 0.3 3614.7 1.0X +make_interval(*, *, *, *, *, *, *) 3638 3657 21 0.3 3637.7 1.0X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala index 907e3f40c1911..96ad453aeb2d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala @@ -39,11 +39,11 @@ import org.apache.spark.sql.internal.SQLConf object IntervalBenchmark extends SqlBasedBenchmark { import spark.implicits._ - private def doBenchmark(cardinality: Long, exprs: Column*): Unit = { + private def doBenchmark(cardinality: Long, columns: Column*): Unit = { withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { spark .range(0, cardinality, 1, 1) - .select(exprs: _*) + .select(columns: _*) .queryExecution .toRdd .foreach(_ => ()) @@ -60,6 +60,26 @@ object IntervalBenchmark extends SqlBasedBenchmark { } } + private def doBenchmarkExpr(cardinality: Long, exprs: String*): Unit = { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + spark + .range(0, cardinality, 1, 1) + .selectExpr(exprs: _*) + .queryExecution + .toRdd + .foreach(_ => ()) + } + } + + private def addCaseExpr( + benchmark: Benchmark, + cardinality: Long, + name: String, + exprs: String*): Unit = { + benchmark.addCase(name, numIters = 3) { _ => doBenchmarkExpr(cardinality, exprs: _*) } + } + + private def buildString(withPrefix: Boolean, units: Seq[String] = Seq.empty): Column = { val init = lit(if (withPrefix) "interval" else "") :: ($"id" % 10000).cast("string") :: @@ -78,25 +98,68 @@ object IntervalBenchmark extends SqlBasedBenchmark { } } - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - val N = 1000000 + private def benchmarkIntervalStringParsing(cardinality: Long): Unit = { val timeUnits = Seq( "13 months", " 1 months", "100 weeks", "9 days", "12 hours", "- 3 hours", "5 minutes", "45 seconds", "123 milliseconds", "567 microseconds") val intervalToTest = ListBuffer[String]() - val benchmark = new Benchmark("cast strings to intervals", N, output = output) + val benchmark = new Benchmark("cast strings to intervals", cardinality, output = output) // The first 2 cases are used to show the overhead of preparing the interval string. - addCase(benchmark, N, "prepare string w/ interval", buildString(true, timeUnits)) - addCase(benchmark, N, "prepare string w/o interval", buildString(false, timeUnits)) - addCase(benchmark, N, intervalToTest) // Only years + addCase(benchmark, cardinality, "prepare string w/ interval", buildString(true, timeUnits)) + addCase(benchmark, cardinality, "prepare string w/o interval", buildString(false, timeUnits)) + addCase(benchmark, cardinality, intervalToTest) // Only years for (unit <- timeUnits) { intervalToTest.append(unit) - addCase(benchmark, N, intervalToTest) + addCase(benchmark, cardinality, intervalToTest) } benchmark.run() } + + private def benchmarkMakeInterval(cardinality: Long): Unit = { + val benchmark = new Benchmark("make_interval()", cardinality, output = output) + val hmExprs = Seq("id % 24", "id % 60") + val hmsExprs = hmExprs ++ Seq("cast((id % 500000000) / 1000000.0 as decimal(18, 6))") + val ymExprs = Seq("(2000 + (id % 30))", "((id % 12) + 1)") + val wdExpr = Seq("((id % 54) + 1)", "((id % 1000) + 1)") + val args = ymExprs ++ wdExpr ++ hmsExprs + + addCaseExpr( + benchmark, + cardinality, + "prepare make_interval()", + args: _*) + val foldableExpr = "make_interval(0, 1, 2, 3, 4, 5, 50.123456)" + addCaseExpr(benchmark, cardinality, foldableExpr, foldableExpr) + addCaseExpr( + benchmark, + cardinality, + "make_interval(*, *, 2, 3, 4, 5, 50.123456)", + s"make_interval(${ymExprs.mkString(",")}, 2, 3, 4, 5, 50.123456)") + addCaseExpr( + benchmark, + cardinality, + "make_interval(0, 1, *, *, 4, 5, 50.123456)", + s"make_interval(0, 1, ${wdExpr.mkString(",")}, 4, 5, 50.123456)") + addCaseExpr( + benchmark, + cardinality, + "make_interval(0, 1, 2, 3, *, *, *)", + s"make_interval(0, 1, 2, 3, ${hmsExprs.mkString(",")})") + addCaseExpr( + benchmark, + cardinality, + "make_interval(*, *, *, *, *, *, *)", + s"make_interval(${args.mkString(",")})") + + benchmark.run() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + benchmarkIntervalStringParsing(1000000) + benchmarkMakeInterval(1000000) + } } From 6484c14c57434dd6961cf9e9e73bbe8aa04cda15 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sun, 28 Jun 2020 12:22:44 -0700 Subject: [PATCH 085/384] [SPARK-32115][SQL] Fix SUBSTRING to handle integer overflows ### What changes were proposed in this pull request? Bug fix for overflow case in `UTF8String.substringSQL`. ### Why are the changes needed? SQL query `SELECT SUBSTRING("abc", -1207959552, -1207959552)` incorrectly returns` "abc"` against expected output of `""`. For query `SUBSTRING("abc", -100, -100)`, we'll get the right output of `""`. ### Does this PR introduce _any_ user-facing change? Yes, bug fix for the overflow case. ### How was this patch tested? New UT. Closes #28937 from xuanyuanking/SPARK-32115. Authored-by: Yuanjian Li Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/unsafe/types/UTF8String.java | 11 ++++++++++- .../apache/spark/unsafe/types/UTF8StringSuite.java | 4 ++++ .../catalyst/expressions/StringExpressionsSuite.scala | 4 ++++ 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 186597fa64780..7205293aa48c5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -341,8 +341,17 @@ public UTF8String substringSQL(int pos, int length) { // to the -ith element before the end of the sequence. If a start index i is 0, it // refers to the first element. int len = numChars(); + // `len + pos` does not overflow as `len >= 0`. int start = (pos > 0) ? pos -1 : ((pos < 0) ? len + pos : 0); - int end = (length == Integer.MAX_VALUE) ? len : start + length; + + int end; + if ((long) start + length > Integer.MAX_VALUE) { + end = Integer.MAX_VALUE; + } else if ((long) start + length < Integer.MIN_VALUE) { + end = Integer.MIN_VALUE; + } else { + end = start + length; + } return substring(start, end); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 8f933877f82e6..70e276f7e5a8b 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -390,6 +390,10 @@ public void substringSQL() { assertEquals(fromString("example"), e.substringSQL(0, Integer.MAX_VALUE)); assertEquals(fromString("example"), e.substringSQL(1, Integer.MAX_VALUE)); assertEquals(fromString("xample"), e.substringSQL(2, Integer.MAX_VALUE)); + assertEquals(EMPTY_UTF8, e.substringSQL(-100, -100)); + assertEquals(EMPTY_UTF8, e.substringSQL(-1207959552, -1207959552)); + assertEquals(fromString("pl"), e.substringSQL(-3, 2)); + assertEquals(EMPTY_UTF8, e.substringSQL(Integer.MIN_VALUE, 6)); } @Test diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index f18364d844ce1..967ccc42c632d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -236,6 +236,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { "xample", row) + // Substring with from negative position with negative length + checkEvaluation(Substring(s, Literal.create(-1207959552, IntegerType), + Literal.create(-1207959552, IntegerType)), "", row) + val s_notNull = 'a.string.notNull.at(0) assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable) From 197ac3b13239d50a1f34a5860940e353ca6b99d5 Mon Sep 17 00:00:00 2001 From: Warren Zhu Date: Sun, 28 Jun 2020 21:06:45 -0700 Subject: [PATCH 086/384] [SPARK-32124][CORE][SHS] Fix taskEndReasonFromJson to handle event logs from old Spark versions ### What changes were proposed in this pull request? Fix bug of exception when parse event log of fetch failed task end reason without `Map Index` ### Why are the changes needed? When Spark history server read event log produced by older version of spark 2.4 (which don't have `Map Index` field), parsing of TaskEndReason will fail. This will cause TaskEnd event being ignored. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? JsonProtocolSuite.test("FetchFailed Map Index backwards compatibility") Closes #28941 from warrenzhu25/shs-task. Authored-by: Warren Zhu Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/util/JsonProtocol.scala | 5 ++++- .../org/apache/spark/util/JsonProtocolSuite.scala | 11 +++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 1c788a30022d0..ced3f9d15720d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -1078,7 +1078,10 @@ private[spark] object JsonProtocol { val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") val shuffleId = (json \ "Shuffle ID").extract[Int] val mapId = (json \ "Map ID").extract[Long] - val mapIndex = (json \ "Map Index").extract[Int] + val mapIndex = (json \ "Map Index") match { + case JNothing => 0 + case x => x.extract[Int] + } val reduceId = (json \ "Reduce ID").extract[Int] val message = jsonOption(json \ "Message").map(_.extract[String]) new FetchFailed(blockManagerAddress, shuffleId, mapId, mapIndex, reduceId, diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 5a4073baa19d4..955589fc5b47b 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -327,6 +327,17 @@ class JsonProtocolSuite extends SparkFunSuite { assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } + test("SPARK-32124: FetchFailed Map Index backwards compatibility") { + // FetchFailed in Spark 2.4.0 does not have "Map Index" property. + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, 18, 19, + "ignored") + val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) + .removeField({ _._1 == "Map Index" }) + val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, + 0, 19, "ignored") + assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) + } + test("ShuffleReadMetrics: Local bytes read backwards compatibility") { // Metrics about local shuffle bytes read were added in 1.3.1. val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, From f944603872284c03c557474bb9e816f20094a630 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sun, 28 Jun 2020 21:35:59 -0700 Subject: [PATCH 087/384] [SPARK-32126][SS] Scope Session.active in IncrementalExecution ### What changes were proposed in this pull request? The `optimizedPlan` in IncrementalExecution should also be scoped in `withActive`. ### Why are the changes needed? Follow-up of SPARK-30798 for the Streaming side. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing UT. Closes #28936 from xuanyuanking/SPARK-30798-follow. Authored-by: Yuanjian Li Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/execution/QueryExecution.scala | 2 +- .../spark/sql/execution/streaming/IncrementalExecution.scala | 2 +- 2 files changed, 2 insertions(+), 2 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 bf60427e5f3bf..791e432269632 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 @@ -131,7 +131,7 @@ class QueryExecution( Option(InsertAdaptiveSparkPlan(AdaptiveExecutionContext(sparkSession, this)))) } - private def executePhase[T](phase: String)(block: => T): T = sparkSession.withActive { + protected def executePhase[T](phase: String)(block: => T): T = sparkSession.withActive { tracker.measurePhase(phase)(block) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 09ae7692ec518..7773ac71c4954 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -76,7 +76,7 @@ class IncrementalExecution( * with the desired literal */ override - lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { + lazy val optimizedPlan: LogicalPlan = executePhase(QueryPlanningTracker.OPTIMIZATION) { sparkSession.sessionState.optimizer.executeAndTrack(withCachedData, tracker) transformAllExpressions { case ts @ CurrentBatchTimestamp(timestamp, _, _) => From 0ec17c989debddb56a8048e4bd29ae666e4d9c56 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Sun, 28 Jun 2020 21:49:10 -0700 Subject: [PATCH 088/384] [SPARK-32090][SQL] Improve UserDefinedType.equal() to make it be symmetrical ### What changes were proposed in this pull request? This PR fix `UserDefinedType.equal()` by comparing the UDT class instead of checking `acceptsType()`. ### Why are the changes needed? It's weird that equality comparison between two UDT types can have different result by switching the order: ```scala // ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass val udt1 = new ExampleBaseTypeUDT val udt2 = new ExampleSubTypeUDT println(udt1 == udt2) // true println(udt2 == udt1) // false ``` ### Does this PR introduce _any_ user-facing change? Yes. Before: ```scala // ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass val udt1 = new ExampleBaseTypeUDT val udt2 = new ExampleSubTypeUDT println(udt1 == udt2) // true println(udt2 == udt1) // false ``` After: ```scala // ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass val udt1 = new ExampleBaseTypeUDT val udt2 = new ExampleSubTypeUDT println(udt1 == udt2) // false println(udt2 == udt1) // false ``` ### How was this patch tested? Added a unit test. Closes #28923 from Ngone51/fix-udt-equal. Authored-by: yi.wu Signed-off-by: Dongjoon Hyun --- .../spark/sql/types/UserDefinedType.scala | 2 +- .../spark/sql/UserDefinedTypeSuite.scala | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala index 6af16e2dba105..592ce03606d4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala @@ -90,7 +90,7 @@ abstract class UserDefinedType[UserType >: Null] extends DataType with Serializa override def hashCode(): Int = getClass.hashCode() override def equals(other: Any): Boolean = other match { - case that: UserDefinedType[_] => this.acceptsType(that) + case that: UserDefinedType[_] => this.getClass == that.getClass case _ => false } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 3fd5cc72cb95e..9acb00b7b6d0b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -134,6 +134,24 @@ class UserDefinedTypeSuite extends QueryTest with SharedSparkSession with Parque MyLabeledPoint(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), MyLabeledPoint(0.0, new TestUDT.MyDenseVector(Array(0.3, 3.0)))).toDF() + + test("SPARK-32090: equal") { + val udt1 = new ExampleBaseTypeUDT + val udt2 = new ExampleSubTypeUDT + val udt3 = new ExampleSubTypeUDT + assert(udt1 !== udt2) + assert(udt2 !== udt1) + assert(udt2 === udt3) + assert(udt3 === udt2) + } + + test("SPARK-32090: acceptsType") { + val udt1 = new ExampleBaseTypeUDT + val udt2 = new ExampleSubTypeUDT + assert(udt1.acceptsType(udt2)) + assert(!udt2.acceptsType(udt1)) + } + test("register user type: MyDenseVector for MyLabeledPoint") { val labels: RDD[Double] = pointsRDD.select('label).rdd.map { case Row(v: Double) => v } val labelsArrays: Array[Double] = labels.collect() From 835ef425d03f30984e885448fe785905ed1ee9a7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 28 Jun 2020 21:55:19 -0700 Subject: [PATCH 089/384] [SPARK-32038][SQL][FOLLOWUP] Make the alias name pretty after float/double normalization ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/28876/files This PR proposes to use the name of the original expression, as the alias name of the normalization expression. ### Why are the changes needed? make the query plan looks pretty when EXPLAIN. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? manually explain the query Closes #28919 from cloud-fan/follow. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/execution/SparkStrategies.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 4d23e5e8a65b5..56d421cdcd702 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -436,6 +436,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { val normalizedGroupingExpressions = groupingExpressions.map { e => NormalizeFloatingNumbers.normalize(e) match { case n: NamedExpression => n + // Keep the name of the original expression. case other => Alias(other, e.name)(exprId = e.exprId) } } @@ -460,7 +461,13 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // because `distinctExpressions` is not extracted during logical phase. NormalizeFloatingNumbers.normalize(e) match { case ne: NamedExpression => ne - case other => Alias(other, other.toString)() + case other => + // Keep the name of the original expression. + val name = e match { + case ne: NamedExpression => ne.name + case _ => e.toString + } + Alias(other, name)() } } From 4204a63d4ff628a38107543742753667330d1112 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 29 Jun 2020 11:33:40 +0000 Subject: [PATCH 090/384] [SPARK-32056][SQL] Coalesce partitions for repartition by expressions when AQE is enabled ### What changes were proposed in this pull request? This patch proposes to coalesce partitions for repartition by expressions without specifying number of partitions, when AQE is enabled. ### Why are the changes needed? When repartition by some partition expressions, users can specify number of partitions or not. If the number of partitions is specified, we should not coalesce partitions because it breaks user expectation. But if without specifying number of partitions, AQE should be able to coalesce partitions as other shuffling. ### Does this PR introduce _any_ user-facing change? Yes. After this change, if users don't specify the number of partitions when repartitioning data by expressions, AQE will coalesce partitions. ### How was this patch tested? Added unit test. Closes #28900 from viirya/SPARK-32056. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../plans/logical/basicLogicalOperators.scala | 18 ++++- .../scala/org/apache/spark/sql/Dataset.scala | 54 ++++++++----- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../adaptive/AdaptiveQueryExecSuite.scala | 77 +++++++++++++++++-- 4 files changed, 120 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 79a8380826ab3..039fd9382000a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler @@ -953,16 +954,18 @@ case class Repartition(numPartitions: Int, shuffle: Boolean, child: LogicalPlan) } /** - * This method repartitions data using [[Expression]]s into `numPartitions`, and receives + * This method repartitions data using [[Expression]]s into `optNumPartitions`, and receives * information about the number of partitions during execution. Used when a specific ordering or * distribution is expected by the consumer of the query result. Use [[Repartition]] for RDD-like - * `coalesce` and `repartition`. + * `coalesce` and `repartition`. If no `optNumPartitions` is given, by default it partitions data + * into `numShufflePartitions` defined in `SQLConf`, and could be coalesced by AQE. */ case class RepartitionByExpression( partitionExpressions: Seq[Expression], child: LogicalPlan, - numPartitions: Int) extends RepartitionOperation { + optNumPartitions: Option[Int]) extends RepartitionOperation { + val numPartitions = optNumPartitions.getOrElse(SQLConf.get.numShufflePartitions) require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") val partitioning: Partitioning = { @@ -990,6 +993,15 @@ case class RepartitionByExpression( override def shuffle: Boolean = true } +object RepartitionByExpression { + def apply( + partitionExpressions: Seq[Expression], + child: LogicalPlan, + numPartitions: Int): RepartitionByExpression = { + RepartitionByExpression(partitionExpressions, child, Some(numPartitions)) + } +} + /** * A relation with one row. This is used in "SELECT ..." without a from clause. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 524e231eb7eb9..6f97121d88ede 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2991,17 +2991,9 @@ class Dataset[T] private[sql]( Repartition(numPartitions, shuffle = true, logicalPlan) } - /** - * Returns a new Dataset partitioned by the given partitioning expressions into - * `numPartitions`. The resulting Dataset is hash partitioned. - * - * This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL). - * - * @group typedrel - * @since 2.0.0 - */ - @scala.annotation.varargs - def repartition(numPartitions: Int, partitionExprs: Column*): Dataset[T] = { + private def repartitionByExpression( + numPartitions: Option[Int], + partitionExprs: Seq[Column]): Dataset[T] = { // The underlying `LogicalPlan` operator special-cases all-`SortOrder` arguments. // However, we don't want to complicate the semantics of this API method. // Instead, let's give users a friendly error message, pointing them to the new method. @@ -3015,6 +3007,20 @@ class Dataset[T] private[sql]( } } + /** + * Returns a new Dataset partitioned by the given partitioning expressions into + * `numPartitions`. The resulting Dataset is hash partitioned. + * + * This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL). + * + * @group typedrel + * @since 2.0.0 + */ + @scala.annotation.varargs + def repartition(numPartitions: Int, partitionExprs: Column*): Dataset[T] = { + repartitionByExpression(Some(numPartitions), partitionExprs) + } + /** * Returns a new Dataset partitioned by the given partitioning expressions, using * `spark.sql.shuffle.partitions` as number of partitions. @@ -3027,7 +3033,20 @@ class Dataset[T] private[sql]( */ @scala.annotation.varargs def repartition(partitionExprs: Column*): Dataset[T] = { - repartition(sparkSession.sessionState.conf.numShufflePartitions, partitionExprs: _*) + repartitionByExpression(None, partitionExprs) + } + + private def repartitionByRange( + numPartitions: Option[Int], + partitionExprs: Seq[Column]): Dataset[T] = { + require(partitionExprs.nonEmpty, "At least one partition-by expression must be specified.") + val sortOrder: Seq[SortOrder] = partitionExprs.map(_.expr match { + case expr: SortOrder => expr + case expr: Expression => SortOrder(expr, Ascending) + }) + withTypedPlan { + RepartitionByExpression(sortOrder, logicalPlan, numPartitions) + } } /** @@ -3049,14 +3068,7 @@ class Dataset[T] private[sql]( */ @scala.annotation.varargs def repartitionByRange(numPartitions: Int, partitionExprs: Column*): Dataset[T] = { - require(partitionExprs.nonEmpty, "At least one partition-by expression must be specified.") - val sortOrder: Seq[SortOrder] = partitionExprs.map(_.expr match { - case expr: SortOrder => expr - case expr: Expression => SortOrder(expr, Ascending) - }) - withTypedPlan { - RepartitionByExpression(sortOrder, logicalPlan, numPartitions) - } + repartitionByRange(Some(numPartitions), partitionExprs) } /** @@ -3078,7 +3090,7 @@ class Dataset[T] private[sql]( */ @scala.annotation.varargs def repartitionByRange(partitionExprs: Column*): Dataset[T] = { - repartitionByRange(sparkSession.sessionState.conf.numShufflePartitions, partitionExprs: _*) + repartitionByRange(None, partitionExprs) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 56d421cdcd702..3f339347ab4db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -692,8 +692,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: logical.Range => execution.RangeExec(r) :: Nil case r: logical.RepartitionByExpression => + val canChangeNumParts = r.optNumPartitions.isEmpty exchange.ShuffleExchangeExec( - r.partitioning, planLater(r.child), canChangeNumPartitions = false) :: Nil + r.partitioning, planLater(r.child), canChangeNumParts) :: Nil case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 9fa97bffa8910..27d9748476c98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} import org.apache.spark.sql.execution.{PartialReducerPartitionSpec, ReusedSubqueryExec, ShuffledRowRDD, SparkPlan} import org.apache.spark.sql.execution.command.DataWritingCommandExec -import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.functions._ @@ -1022,18 +1022,81 @@ class AdaptiveQueryExecSuite } } - test("SPARK-31220 repartition obeys initialPartitionNum when adaptiveExecutionEnabled") { + test("SPARK-31220, SPARK-32056: repartition by expression with AQE") { Seq(true, false).foreach { enableAQE => withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, - SQLConf.SHUFFLE_PARTITIONS.key -> "6", - SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { - val partitionsNum = spark.range(10).repartition($"id").rdd.collectPartitions().length + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10", + SQLConf.SHUFFLE_PARTITIONS.key -> "10") { + + val df1 = spark.range(10).repartition($"id") + val df2 = spark.range(10).repartition($"id" + 1) + + val partitionsNum1 = df1.rdd.collectPartitions().length + val partitionsNum2 = df2.rdd.collectPartitions().length + if (enableAQE) { - assert(partitionsNum === 7) + assert(partitionsNum1 < 10) + assert(partitionsNum2 < 10) + + // repartition obeys initialPartitionNum when adaptiveExecutionEnabled + val plan = df1.queryExecution.executedPlan + assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffle.size == 1) + assert(shuffle(0).outputPartitioning.numPartitions == 10) } else { - assert(partitionsNum === 6) + assert(partitionsNum1 === 10) + assert(partitionsNum2 === 10) } + + + // Don't coalesce partitions if the number of partitions is specified. + val df3 = spark.range(10).repartition(10, $"id") + val df4 = spark.range(10).repartition(10) + assert(df3.rdd.collectPartitions().length == 10) + assert(df4.rdd.collectPartitions().length == 10) + } + } + } + + test("SPARK-31220, SPARK-32056: repartition by range with AQE") { + Seq(true, false).foreach { enableAQE => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10", + SQLConf.SHUFFLE_PARTITIONS.key -> "10") { + + val df1 = spark.range(10).toDF.repartitionByRange($"id".asc) + val df2 = spark.range(10).toDF.repartitionByRange(($"id" + 1).asc) + + val partitionsNum1 = df1.rdd.collectPartitions().length + val partitionsNum2 = df2.rdd.collectPartitions().length + + if (enableAQE) { + assert(partitionsNum1 < 10) + assert(partitionsNum2 < 10) + + // repartition obeys initialPartitionNum when adaptiveExecutionEnabled + val plan = df1.queryExecution.executedPlan + assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffle.size == 1) + assert(shuffle(0).outputPartitioning.numPartitions == 10) + } else { + assert(partitionsNum1 === 10) + assert(partitionsNum2 === 10) + } + + // Don't coalesce partitions if the number of partitions is specified. + val df3 = spark.range(10).repartitionByRange(10, $"id".asc) + assert(df3.rdd.collectPartitions().length == 10) } } } From 6fcb70e0cadd8a543cd9be5f606c5dbeec0ae181 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 29 Jun 2020 11:37:03 +0000 Subject: [PATCH 091/384] [SPARK-32055][CORE][SQL] Unify getReader and getReaderForRange in ShuffleManager ### What changes were proposed in this pull request? This PR tries to unify the method `getReader` and `getReaderForRange` in `ShuffleManager`. ### Why are the changes needed? Reduce the duplicate codes, simplify the implementation, and for better maintenance. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Covered by existing tests. Closes #28895 from Ngone51/unify-getreader. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../org/apache/spark/MapOutputTracker.scala | 77 ++++--------------- .../apache/spark/shuffle/ShuffleManager.scala | 18 +++-- .../shuffle/sort/SortShuffleManager.scala | 22 ++---- .../apache/spark/MapOutputTrackerSuite.scala | 2 +- .../BlockStoreShuffleReaderSuite.scala | 4 +- .../spark/sql/execution/ShuffledRowRDD.scala | 4 +- 6 files changed, 40 insertions(+), 87 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ec8621bc55cf3..18cd5de4cfada 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -322,36 +322,22 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // For testing def getMapSizesByExecutorId(shuffleId: Int, reduceId: Int) : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1) + getMapSizesByExecutorId(shuffleId, 0, Int.MaxValue, reduceId, reduceId + 1) } /** * Called from executors to get the server URIs and output sizes for each shuffle block that * needs to be read from a given range of map output partitions (startPartition is included but - * endPartition is excluded from the range). + * endPartition is excluded from the range) within a range of mappers (startMapIndex is included + * but endMapIndex is excluded). If endMapIndex=Int.MaxValue, the actual endMapIndex will be + * changed to the length of total map outputs. * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. + * Note that zero-sized blocks are excluded in the result. */ def getMapSizesByExecutorId( - shuffleId: Int, - startPartition: Int, - endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] - - /** - * Called from executors to get the server URIs and output sizes for each shuffle block that - * needs to be read from a given range of map output partitions (startPartition is included but - * endPartition is excluded from the range) and is produced by - * a range of mappers (startMapIndex, endMapIndex, startMapIndex is included and - * the endMapIndex is excluded). - * - * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block id, shuffle block size, map index) - * tuples describing the shuffle blocks that are stored at that block manager. - */ - def getMapSizesByRange( shuffleId: Int, startMapIndex: Int, endMapIndex: Int, @@ -734,38 +720,22 @@ private[spark] class MapOutputTrackerMaster( } } - // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. // This method is only called in local-mode. def getMapSizesByExecutorId( - shuffleId: Int, - startPartition: Int, - endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") - shuffleStatuses.get(shuffleId) match { - case Some (shuffleStatus) => - shuffleStatus.withMapStatuses { statuses => - MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses, 0, shuffleStatus.mapStatuses.length) - } - case None => - Iterator.empty - } - } - - override def getMapSizesByRange( shuffleId: Int, startMapIndex: Int, endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex-$endMapIndex" + - s"partitions $startPartition-$endPartition") + logDebug(s"Fetching outputs for shuffle $shuffleId") shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => shuffleStatus.withMapStatuses { statuses => + val actualEndMapIndex = if (endMapIndex == Int.MaxValue) statuses.length else endMapIndex + logDebug(s"Convert map statuses for shuffle $shuffleId, " + + s"mappers $startMapIndex-$actualEndMapIndex, partitions $startPartition-$endPartition") MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses, startMapIndex, endMapIndex) + shuffleId, startPartition, endPartition, statuses, startMapIndex, actualEndMapIndex) } case None => Iterator.empty @@ -798,37 +768,20 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr */ private val fetchingLock = new KeyLock[Int] - // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. override def getMapSizesByExecutorId( - shuffleId: Int, - startPartition: Int, - endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") - val statuses = getStatuses(shuffleId, conf) - try { - MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses, 0, statuses.length) - } catch { - case e: MetadataFetchFailedException => - // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: - mapStatuses.clear() - throw e - } - } - - override def getMapSizesByRange( shuffleId: Int, startMapIndex: Int, endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex-$endMapIndex" + - s"partitions $startPartition-$endPartition") + logDebug(s"Fetching outputs for shuffle $shuffleId") val statuses = getStatuses(shuffleId, conf) try { + val actualEndMapIndex = if (endMapIndex == Int.MaxValue) statuses.length else endMapIndex + logDebug(s"Convert map statuses for shuffle $shuffleId, " + + s"mappers $startMapIndex-$actualEndMapIndex, partitions $startPartition-$endPartition") MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses, startMapIndex, endMapIndex) + shuffleId, startPartition, endPartition, statuses, startMapIndex, actualEndMapIndex) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 057b0d6e0b0a7..400c4526f0114 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -43,23 +43,31 @@ private[spark] trait ShuffleManager { context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] + /** - * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to + * read from all map outputs of the shuffle. + * * Called on executors by reduce tasks. */ - def getReader[K, C]( + final def getReader[K, C]( handle: ShuffleHandle, startPartition: Int, endPartition: Int, context: TaskContext, - metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + getReader(handle, 0, Int.MaxValue, startPartition, endPartition, context, metrics) + } /** * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to - * read from map output (startMapIndex to endMapIndex - 1, inclusive). + * read from a range of map outputs(startMapIndex to endMapIndex-1, inclusive). + * If endMapIndex=Int.MaxValue, the actual endMapIndex will be changed to the length of total map + * outputs of the shuffle in `getMapSizesByExecutorId`. + * * Called on executors by reduce tasks. */ - def getReaderForRange[K, C]( + def getReader[K, C]( handle: ShuffleHandle, startMapIndex: Int, endMapIndex: Int, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index aefcb59b8bb87..72460180f5908 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark._ import org.apache.spark.internal.{config, Logging} +import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle._ import org.apache.spark.shuffle.api.{ShuffleDataIO, ShuffleExecutorComponents} import org.apache.spark.util.Utils @@ -115,23 +116,14 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager } /** - * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to + * read from a range of map outputs(startMapIndex to endMapIndex-1, inclusive). + * If endMapIndex=Int.MaxValue, the actual endMapIndex will be changed to the length of total map + * outputs of the shuffle in `getMapSizesByExecutorId`. + * * Called on executors by reduce tasks. */ override def getReader[K, C]( - handle: ShuffleHandle, - startPartition: Int, - endPartition: Int, - context: TaskContext, - metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( - handle.shuffleId, startPartition, endPartition) - new BlockStoreShuffleReader( - handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, - shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) - } - - override def getReaderForRange[K, C]( handle: ShuffleHandle, startMapIndex: Int, endMapIndex: Int, @@ -139,7 +131,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByRange( + val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) new BlockStoreShuffleReader( handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d5ee19bde8edf..630ffd9baa06e 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -317,7 +317,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(size10000, size0, size1000, size0), 6)) assert(tracker.containsShuffle(10)) - assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq === + assert(tracker.getMapSizesByExecutorId(10, 0, 2, 0, 4).toSeq === Seq( (BlockManagerId("a", "hostA", 1000), Seq((ShuffleBlockId(10, 5, 1), size1000, 0), diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index a82f86a11c77e..d964b28df2983 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -104,7 +104,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // shuffle data to read. val mapOutputTracker = mock(classOf[MapOutputTracker]) when(mapOutputTracker.getMapSizesByExecutorId( - shuffleId, reduceId, reduceId + 1)).thenReturn { + shuffleId, 0, numMaps, reduceId, reduceId + 1)).thenReturn { // Test a scenario where all data is local, to avoid creating a bunch of additional mocks // for the code to read data over the network. val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => @@ -132,7 +132,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext val taskContext = TaskContext.empty() val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() val blocksByAddress = mapOutputTracker.getMapSizesByExecutorId( - shuffleId, reduceId, reduceId + 1) + shuffleId, 0, numMaps, reduceId, reduceId + 1) val shuffleReader = new BlockStoreShuffleReader( shuffleHandle, blocksByAddress, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index 5936492dd819c..b5e9655a776b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -191,7 +191,7 @@ class ShuffledRowRDD( sqlMetricsReporter) case PartialReducerPartitionSpec(reducerIndex, startMapIndex, endMapIndex, _) => - SparkEnv.get.shuffleManager.getReaderForRange( + SparkEnv.get.shuffleManager.getReader( dependency.shuffleHandle, startMapIndex, endMapIndex, @@ -201,7 +201,7 @@ class ShuffledRowRDD( sqlMetricsReporter) case PartialMapperPartitionSpec(mapIndex, startReducerIndex, endReducerIndex) => - SparkEnv.get.shuffleManager.getReaderForRange( + SparkEnv.get.shuffleManager.getReader( dependency.shuffleHandle, mapIndex, mapIndex + 1, From a6b6a1fd611edb0b0c11e2a5399cee66a7e74de6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 29 Jun 2020 11:53:56 +0000 Subject: [PATCH 092/384] [MINOR] update dev/create-release/known_translations This is a followup of https://github.com/apache/spark/pull/28861 : 1. sort the names by Github ID, suggested by https://github.com/apache/spark/pull/28861#pullrequestreview-433369190 2. add more full names collected in https://github.com/apache/spark/pull/28861 3. The duplicated entry of `linbojin` is removed. 4. Name format is normalized to First Last name style. Closes #28891 from cloud-fan/update. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- dev/create-release/known_translations | 541 +++++++++++++------------- 1 file changed, 270 insertions(+), 271 deletions(-) diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations index 376398bc3788a..ff41cccde0140 100644 --- a/dev/create-release/known_translations +++ b/dev/create-release/known_translations @@ -1,411 +1,410 @@ # This is a mapping of names to be translated through translate-contributors.py -# The format expected on each line should be: - -CodingCat - Nan Zhu -CrazyJvm - Chao Chen -EugenCepoi - Eugen Cepoi -GraceH - Jie Huang -JerryLead - Lijie Xu -Leolh - Liu Hao -Lewuathe - Kai Sasaki -RongGu - Rong Gu -Shiti - Shiti Saxena -Victsm - Min Shen -WangTaoTheTonic - Wang Tao -XuTingjun - Tingjun Xu -YanTangZhai - Yantang Zhai -alexdebrie - Alex DeBrie -alokito - Alok Saldanha -anantasty - Anant Asthana -andrewor14 - Andrew Or -aniketbhatnagar - Aniket Bhatnagar -arahuja - Arun Ahuja -brkyvz - Burak Yavuz -chesterxgchen - Chester Chen -chiragaggarwal - Chirag Aggarwal -chouqin - Qiping Li -cocoatomo - Tomohiko K. -coderfi - Fairiz Azizi -coderxiang - Shuo Xiang -davies - Davies Liu -epahomov - Egor Pahomov -falaki - Hossein Falaki -freeman-lab - Jeremy Freeman -industrial-sloth - Jascha Swisher -jackylk - Jacky Li -jayunit100 - Jay Vyas -jerryshao - Saisai Shao -jkbradley - Joseph Bradley -lianhuiwang - Lianhui Wang -lirui-intel - Rui Li -luluorta - Lu Lu -luogankun - Gankun Luo -maji2014 - Derek Ma -mccheah - Matthew Cheah -mengxr - Xiangrui Meng -nartz - Nathan Artz -odedz - Oded Zimerman -ravipesala - Ravindra Pesala -roxchkplusony - Victor Tso -scwf - Wang Fei -shimingfei - Shiming Fei -surq - Surong Quan -suyanNone - Su Yan -tedyu - Ted Yu -tigerquoll - Dale Richardson -wangxiaojing - Xiaojing Wang -watermen - Yadong Qi -witgo - Guoqiang Li -xinyunh - Xinyun Huang -zsxwing - Shixiong Zhu -Bilna - Bilna P -DoingDone9 - Doing Done -Earne - Ernest -FlytxtRnD - Meethu Mathew -GenTang - Gen TANG -JoshRosen - Josh Rosen -MechCoder - Manoj Kumar -OopsOutOfMemory - Sheng Li -Peishen-Jia - Peishen Jia -SaintBacchus - Huang Zhaowei -azagrebin - Andrey Zagrebin -bzz - Alexander Bezzubov -fjiang6 - Fan Jiang -gasparms - Gaspar Munoz -guowei2 - Guo Wei -hhbyyh - Yuhao Yang -hseagle - Peng Xu -javadba - Stephen Boesch -jbencook - Ben Cook -kul - Kuldeep -ligangty - Gang Li -marsishandsome - Liangliang Gu -medale - Markus Dale -nemccarthy - Nathan McCarthy -nxwhite-str - Nate Crosswhite -seayi - Xiaohua Yi -tianyi - Yi Tian -uncleGen - Uncle Gen -viper-kun - Xu Kun -x1- - Yuri Saito -zapletal-martin - Martin Zapletal -zuxqoj - Shekhar Bansal -mingyukim - Mingyu Kim -sigmoidanalytics - Mayur Rustagi -AiHe - Ai He -BenFradet - Ben Fradet -FavioVazquez - Favio Vazquez -JaysonSunshine - Jayson Sunshine -Liuchang0812 - Liu Chang -Sephiroth-Lin - Sephiroth Lin -dobashim - Masaru Dobashi -ehnalis - Zoltan Zvara -emres - Emre Sevinc -gchen - Guancheng Chen -haiyangsea - Haiyang Sea -hlin09 - Hao Lin -hqzizania - Qian Huang -jeanlyn - Jean Lyn -jerluc - Jeremy A. Lucas -jrabary - Jaonary Rabarisoa -judynash - Judy Nash -kaka1992 - Chen Song -ksonj - Kalle Jepsen -kuromatsu-nobuyuki - Nobuyuki Kuromatsu -lazyman500 - Dong Xu -leahmcguire - Leah McGuire -mbittmann - Mark Bittmann -mbonaci - Marko Bonaci -meawoppl - Matthew Goodman -nyaapa - Arsenii Krasikov -phatak-dev - Madhukara Phatak -prabeesh - Prabeesh K -rakeshchalasani - Rakesh Chalasani -rekhajoshm - Rekha Joshi -sisihj - June He -szheng79 - Shuai Zheng -texasmichelle - Michelle Casbon -vinodkc - Vinod KC -yongtang - Yong Tang -ypcat - Pei-Lun Lee -zhichao-li - Zhichao Li -zzcclp - Zhichao Zhang -979969786 - Yuming Wang -Rosstin - Rosstin Murphy -ameyc - Amey Chaugule -animeshbaranawal - Animesh Baranawal -cafreeman - Chris Freeman -lee19 - Lee -lockwobr - Brian Lockwood -navis - Navis Ryu -pparkkin - Paavo Parkkinen -HyukjinKwon - Hyukjin Kwon -JDrit - Joseph Batchik -JuhongPark - Juhong Park -KaiXinXiaoLei - KaiXinXIaoLei -NamelessAnalyst - NamelessAnalyst -alyaxey - Alex Slusarenko -baishuo - Shuo Bai -fe2s - Oleksiy Dyagilev -felixcheung - Felix Cheung -feynmanliang - Feynman Liang -josepablocam - Jose Cambronero -kai-zeng - Kai Zeng -mosessky - mosessky -msannell - Michael Sannella -nishkamravi2 - Nishkam Ravi -noel-smith - Noel Smith -petz2000 - Patrick Baier -qiansl127 - Shilei Qian -rahulpalamuttam - Rahul Palamuttam -rowan000 - Rowan Chattaway -sarutak - Kousuke Saruta -sethah - Seth Hendrickson -small-wang - Wang Wei -stanzhai - Stan Zhai -tien-dungle - Tien-Dung Le -xuchenCN - Xu Chen -zhangjiajin - Zhang JiaJin -ClassNotFoundExp - Fu Xing -KevinGrealish - Kevin Grealish -MasterDDT - Mitesh Patel -VinceShieh - Vincent Xie -WeichenXu123 - Weichen Xu -Yunni - Yun Ni -actuaryzhang - Wayne Zhang -alicegugu - Gu Huiqin Alice -anabranch - Bill Chambers -ashangit - Nicolas Fraison -avulanov - Alexander Ulanov -biglobster - Liang Ke -cenyuhai - Yuhai Cen -codlife - Jianfei Wang -david-weiluo-ren - Weiluo (David) Ren -dding3 - Ding Ding -fidato13 - Tarun Kumar -frreiss - Fred Reiss -gatorsmile - Xiao Li -hayashidac - Chie Hayashida -invkrh - Hao Ren -jagadeesanas2 - Jagadeesan A S -jiangxb1987 - Jiang Xingbo -jisookim0513 - Jisoo Kim -junyangq - Junyang Qian -krishnakalyan3 - Krishna Kalyan -linbojin - Linbo Jin -mpjlu - Peng Meng -neggert - Nic Eggert -petermaxlee - Peter Lee -phalodi - Sandeep Purohit -pkch - pkch -priyankagargnitk - Priyanka Garg -sharkdtu - Xiaogang Tu -shenh062326 - Shen Hong -aokolnychyi - Anton Okolnychyi -linbojin - Linbo Jin -lw-lin - Liwei Lin +# The format expected on each line should be: - +012huang - Weiyi Huang +07ARB - Ankit Raj Boudh 10110346 - Xian Liu +979969786 - Yuming Wang Achuth17 - Achuth Narayan Rajagopal Adamyuanyuan - Adam Wang -DylanGuedes - Dylan Guedes -JiahuiJiang - Jiahui Jiang -KevinZwx - Kevin Zhang -LantaoJin - Lantao Jin -Lemonjing - Rann Tao -LucaCanali - Luca Canali -XD-DENG - Xiaodong Deng -aai95 - Aleksei Izmalkin -akonopko - Alexander Konopko -ankuriitg - Ankur Gupta -arucard21 - Riaas Mokiem -attilapiros - Attila Zsolt Piros -bravo-zhang - Bravo Zhang -caneGuy - Kang Zhou -chaoslawful - Xiaozhe Wang -cluo512 - Chuan Luo -codeatri - Neha Patil -crafty-coder - Carlos Pena -debugger87 - Chaozhong Yang -e-dorigatti - Emilio Dorigatti -eric-maynard - Eric Maynard -felixalbani - Felix Albani -fjh100456 - Jinhua Fu -guoxiaolongzte - Xiaolong Guo -heary-cao - Xuewen Cao -huangweizhe123 - Weizhe Huang -ivoson - Tengfei Huang -jinxing64 - Jin Xing -liu-zhaokun - Zhaokun Liu -liutang123 - Lijia Liu -maropu - Takeshi Yamamuro -maryannxue - Maryann Xue -mcteo - Thomas Dunne -mn-mikke - Marek Novotny -myroslavlisniak - Myroslav Lisniak -npoggi - Nicolas Poggi -pgandhi999 - Parth Gandhi -rimolive - Ricardo Martinelli De Oliveira -sadhen - Darcy Shen -sandeep-katta - Sandeep Katta -seancxmao - Chenxiao Mao -sel - Steve Larkin -shimamoto - Takako Shimamoto -shivusondur - Shivakumar Sondur -skonto - Stavros Kontopoulos -trystanleftwich - Trystan Leftwich -ueshin - Takuya Ueshin -uzmijnlm - Weizhe Huang -xuanyuanking - Yuanjian Li -xubo245 - Bo Xu -xueyumusic - Xue Yu -yanlin-Lynn - Yanlin Wang -yucai - Yucai Yu -zhengruifeng - Ruifeng Zheng -zuotingbing - Tingbing Zuo -012huang - Weiyi Huang -07ARB - Ankit Raj Boudh +AiHe - Ai He Andrew-Crosby - Andrew Crosby AngersZhuuuu - Yi Zhu +BenFradet - Ben Fradet +Bilna - Bilna P +ClassNotFoundExp - Fu Xing +CodingCat - Nan Zhu +CrazyJvm - Chao Chen Deegue - Yizhong Zhang +DoingDone9 - Doing Done +DylanGuedes - Dylan Guedes +Earne - Ernest +EugenCepoi - Eugen Cepoi +FavioVazquez - Favio Vazquez +FlytxtRnD - Meethu Mathew +GenTang - Gen TANG +GraceH - Jie Huang Gschiavon - German Schiavon Matteo GuoPhilipse - Philipse Guo Hellsen83 - Erik Christiansen +HyukjinKwon - Hyukjin Kwon Icysandwich - Icysandwich +JDrit - Joseph Batchik JasonWayne - Wenjie Wu +JaysonSunshine - Jayson Sunshine +JerryLead - Lijie Xu +JiahuiJiang - Jiahui Jiang JkSelf - Ke Jia JoanFM - Joan Fontanals +JoshRosen - Josh Rosen +JuhongPark - Juhong Park JulienPeloton - Julien Peloton +KaiXinXiaoLei - KaiXinXIaoLei +KevinGrealish - Kevin Grealish +KevinZwx - Kevin Zhang Koraseg - Artem Kupchinskiy KyleLi1985 - Liang Li +LantaoJin - Lantao Jin +Lemonjing - Rann Tao +Leolh - Liu Hao +Lewuathe - Kai Sasaki LiShuMing - Shuming Li -LinhongLiu - Liu, Linhong +LinhongLiu - Linhong Liu +Liuchang0812 - Liu Chang +LucaCanali - Luca Canali LuciferYang - Yang Jie +MasterDDT - Mitesh Patel MaxGekk - Maxim Gekk +MechCoder - Manoj Kumar +NamelessAnalyst - NamelessAnalyst Ngone51 - Yi Wu +OopsOutOfMemory - Sheng Li PavithraRamachandran - Pavithra Ramachandran +Peishen-Jia - Peishen Jia +RongGu - Rong Gu +Rosstin - Rosstin Murphy +SaintBacchus - Huang Zhaowei +Sephiroth-Lin - Sephiroth Lin +Shiti - Shiti Saxena SongYadong - Yadong Song TigerYang414 - David Yang TomokoKomiyama - Tomoko Komiyama TopGunViper - TopGunViper Udbhav30 - Udbhav Agrawal +Victsm - Min Shen +VinceShieh - Vincent Xie WangGuangxin - Guangxin Wang +WangTaoTheTonic - Wang Tao +WeichenXu123 - Weichen Xu William1104 - William Wong +XD-DENG - Xiaodong Deng +XuTingjun - Tingjun Xu +YanTangZhai - Yantang Zhai YongjinZhou - Yongjin Zhou +Yunni - Yun Ni +aai95 - Aleksei Izmalkin aaruna - Aaruna Godthi +actuaryzhang - Wayne Zhang adrian555 - Weiqiang Zhuang ajithme - Ajith S +akonopko - Alexander Konopko +alexdebrie - Alex DeBrie +alicegugu - Gu Huiqin Alice +alokito - Alok Saldanha +alyaxey - Alex Slusarenko amanomer - Aman Omer +ameyc - Amey Chaugule +anabranch - Bill Chambers +anantasty - Anant Asthana ancasarb - Anca Sarb +andrewor14 - Andrew Or +aniketbhatnagar - Aniket Bhatnagar +animeshbaranawal - Animesh Baranawal +ankuriitg - Ankur Gupta +aokolnychyi - Anton Okolnychyi +arahuja - Arun Ahuja +arucard21 - Riaas Mokiem +ashangit - Nicolas Fraison +attilapiros - Attila Zsolt Piros avkgh - Aleksandr Kashkirov +avulanov - Alexander Ulanov ayudovin - Artsiom Yudovin +azagrebin - Andrey Zagrebin +baishuo - Shuo Bai bartosz25 - Bartosz Konieczny beliefer - Jiaan Geng bettermouse - Chen Hao +biglobster - Liang Ke +bravo-zhang - Bravo Zhang +brkyvz - Burak Yavuz bscan - Brian Scannell +bzz - Alexander Bezzubov +cafreeman - Chris Freeman +caneGuy - Kang Zhou cchung100m - Neo Chien cclauss - Christian Clauss +cenyuhai - Yuhai Cen chakravarthiT - Chakravarthi chandulal - Chandu Kavar +chaoslawful - Xiaozhe Wang +chesterxgchen - Chester Chen +chiragaggarwal - Chirag Aggarwal chitralverma - Chitral Verma -cjn082030 - Jenny +chouqin - Qiping Li +cjn082030 - Juanni Chen cloud-fan - Wenchen Fan +cluo512 - Chuan Luo +cocoatomo - Tomohiko K. +codeatri - Neha Patil codeborui - codeborui +coderfi - Fairiz Azizi +coderxiang - Shuo Xiang +codlife - Jianfei Wang colinmjj - Colin Ma -cxzl25 - cxzl25 +crafty-coder - Carlos Pena +cxzl25 - Shaoyun Chen cyq89051127 - Yongqiang Chai darrentirto - Darren Tirto +david-weiluo-ren - Weiluo (David) Ren daviddingly - Xiaoyuan Ding davidvrba - David Vrba +davies - Davies Liu +dding3 - Ding Ding +debugger87 - Chaozhong Yang deepyaman - Deepyaman Datta denglingang - Lingang Deng dengziming - dengziming deshanxiao - deshanxiao dima-asana - Dima Kamalov dlindelof - David Lindelof +dobashim - Masaru Dobashi dongjoon-hyun - Dongjoon Hyun -eatoncys - eatoncys +e-dorigatti - Emilio Dorigatti +eatoncys - Yanshan Chen +ehnalis - Zoltan Zvara +emres - Emre Sevinc +epahomov - Egor Pahomov +eric-maynard - Eric Maynard +falaki - Hossein Falaki fan31415 - Yijie Fan +fe2s - Oleksiy Dyagilev +felixalbani - Felix Albani +felixcheung - Felix Cheung +feynmanliang - Feynman Liang +fidato13 - Tarun Kumar fitermay - Yuli Fiterman +fjh100456 - Jinhua Fu +fjiang6 - Fan Jiang francis0407 - Mingcong Han +freeman-lab - Jeremy Freeman +frreiss - Fred Reiss fuwhu - Fuwang Hu +gasparms - Gaspar Munoz +gatorsmile - Xiao Li +gchen - Guancheng Chen gss2002 - Greg Senia +guowei2 - Guo Wei +guoxiaolongzte - Xiaolong Guo +haiyangsea - Haiyang Sea +hayashidac - Chie Hayashida hddong - Dongdong Hong +heary-cao - Xuewen Cao hehuiyuan - hehuiyuan helenyugithub - Helen Yu +hhbyyh - Yuhao Yang highmoutain - highmoutain +hlin09 - Hao Lin +hqzizania - Qian Huang +hseagle - Peng Xu httfighter - Tiantian Han huangtianhua - huangtianhua +huangweizhe123 - Weizhe Huang hvanhovell - Herman Van Hovell iRakson - Rakesh Raushan igorcalabria - Igor Calabria imback82 - Terry Kim +industrial-sloth - Jascha Swisher +invkrh - Hao Ren +ivoson - Tengfei Huang +jackylk - Jacky Li +jagadeesanas2 - Jagadeesan A S +javadba - Stephen Boesch javierivanov - Javier Fuentes +jayunit100 - Jay Vyas +jbencook - Ben Cook +jeanlyn - Jean Lyn +jerluc - Jeremy A. Lucas +jerryshao - Saisai Shao +jiangxb1987 - Jiang Xingbo +jinxing64 - Jin Xing +jisookim0513 - Jisoo Kim +jkbradley - Joseph Bradley joelgenter - Joel Genter +josepablocam - Jose Cambronero +jrabary - Jaonary Rabarisoa +judynash - Judy Nash +junyangq - Junyang Qian +kai-zeng - Kai Zeng +kaka1992 - Chen Song ketank-new - Ketan Kunde +krishnakalyan3 - Krishna Kalyan +ksonj - Kalle Jepsen +kul - Kuldeep +kuromatsu-nobuyuki - Nobuyuki Kuromatsu laskfla - Keith Sun +lazyman500 - Dong Xu lcqzte10192193 - Chaoqun Li +leahmcguire - Leah McGuire +lee19 - Lee leoluan2009 - Xuedong Luan liangxs - Xuesen Liang +lianhuiwang - Lianhui Wang lidinghao - Li Hao +ligangty - Gang Li +linbojin - Linbo Jin linehrr - Ryne Yang linzebing - Zebing Lin lipzhu - Lipeng Zhu +lirui-intel - Rui Li +liu-zhaokun - Zhaokun Liu liucht-inspur - liucht-inspur liupc - Pengcheng Liu +liutang123 - Lijia Liu liwensun - Liwen Sun +lockwobr - Brian Lockwood +luluorta - Lu Lu +luogankun - Gankun Luo +lw-lin - Liwei Lin +maji2014 - Derek Ma manuzhang - Manu Zhang mareksimunek - Marek Simunek +maropu - Takeshi Yamamuro +marsishandsome - Liangliang Gu +maryannxue - Maryann Xue masa3141 - Masahiro Kazama +mbittmann - Mark Bittmann +mbonaci - Marko Bonaci +mccheah - Matthew Cheah +mcteo - Thomas Dunne mdianjun - Dianjun Ma +meawoppl - Matthew Goodman +medale - Markus Dale +mengxr - Xiangrui Meng merrily01 - Ruilei Ma +mingyukim - Mingyu Kim +mn-mikke - Marek Novotny mob-ai - mob-ai +mosessky - mosessky +mpjlu - Peng Meng +msannell - Michael Sannella mu5358271 - Shuheng Dai mwlon - Martin Loncaric +myroslavlisniak - Myroslav Lisniak nandorKollar - Nandor Kollar +nartz - Nathan Artz +navis - Navis Ryu +neggert - Nic Eggert +nemccarthy - Nathan McCarthy +nishkamravi2 - Nishkam Ravi +noel-smith - Noel Smith nooberfsh - nooberfsh +npoggi - Nicolas Poggi +nxwhite-str - Nate Crosswhite +nyaapa - Arsenii Krasikov +odedz - Oded Zimerman oleg-smith - Oleg Kuznetsov ozancicek - Ozan Cicekci pengbo - Peng Bo +petermaxlee - Peter Lee +petz2000 - Patrick Baier +pgandhi999 - Parth Gandhi +phalodi - Sandeep Purohit +phatak-dev - Madhukara Phatak +pkch - pkch planga82 - Pablo Langa Blanco +pparkkin - Paavo Parkkinen +prabeesh - Prabeesh K praneetsharma - Praneet Sharma +priyankagargnitk - Priyanka Garg ptkool - Michael Styles qb-tarushg - Tarush Grover +qiansl127 - Shilei Qian +rahulpalamuttam - Rahul Palamuttam +rakeshchalasani - Rakesh Chalasani +ravipesala - Ravindra Pesala redsanket - Sanket Reddy redsk - Nicola Bova -roland1982 - roland1982 +rekhajoshm - Rekha Joshi +rimolive - Ricardo Martinelli De Oliveira +roland1982 - Roland Pogonyi rongma1997 - Rong Ma +rowan000 - Rowan Chattaway +roxchkplusony - Victor Tso rrusso2007 - Rob Russo +sadhen - Darcy Shen samsetegne - Samuel L. Setegne +sandeep-katta - Sandeep Katta sangramga - Sangram Gaikwad sarthfrey - Sarth Frey +sarutak - Kousuke Saruta +scwf - Wang Fei +seancxmao - Chenxiao Mao +seayi - Xiaohua Yi seayoun - Haiyang Yu +sel - Steve Larkin +sethah - Seth Hendrickson sev7e0 - Jiaqi Li -shahidki31 - Shahid +shahidki31 - Shahid K I sharangk - Sharanabasappa G Keriwaddi +sharkdtu - Xiaogang Tu sheepstop - Ting Yang +shenh062326 - Shen Hong +shimamoto - Takako Shimamoto +shimingfei - Shiming Fei shivsood - Shiv Prashant Sood +shivusondur - Shivakumar Sondur +sigmoidanalytics - Mayur Rustagi +sisihj - June He sitegui - Guilherme Souza +skonto - Stavros Kontopoulos slamke - Sun Ke +small-wang - Wang Wei southernriver - Liang Chen squito - Imran Rashid +stanzhai - Stan Zhai stczwd - Jackey Lee sujith71955 - Sujith Chacko +surq - Surong Quan suxingfate - Xinglong Wang -teeyog - teeyog +suyanNone - Su Yan +szheng79 - Shuai Zheng +tedyu - Ted Yu +teeyog - Yong Tian +texasmichelle - Michelle Casbon +tianyi - Yi Tian +tien-dungle - Tien-Dung Le +tigerquoll - Dale Richardson tinhto-000 - Tin Hang To tools4origins - tools4origins triplesheep - triplesheep +trystanleftwich - Trystan Leftwich turboFei - Fei Wang -ulysses-you - ulysses-you +ueshin - Takuya Ueshin +ulysses-you - Xiduo You +uncleGen - Uncle Gen uzadude - Ohad Raviv -wackxu - wackxu -wangjiaochun - wangjiaochun +uzmijnlm - Weizhe Huang +vinodkc - Vinod KC +viper-kun - Xu Kun +wackxu - Shiwei Xu +wangjiaochun - Jiaochun Wang wangshisan - wangshisan +wangxiaojing - Xiaojing Wang +watermen - Yadong Qi weixiuli - XiuLi Wei wenfang6 - wenfang6 wenxuanguan - wenxuanguan windpiger - Song Jun +witgo - Guoqiang Li woudygao - Woudy Gao +x1- - Yuri Saito xianyinxin - Xianyin Xin +xinyunh - Xinyun Huang +xuanyuanking - Yuanjian Li +xubo245 - Bo Xu +xuchenCN - Xu Chen +xueyumusic - Xue Yu +yanlin-Lynn - Yanlin Wang +yongtang - Yong Tang +ypcat - Pei-Lun Lee +yucai - Yucai Yu yunzoud - Yun Zou +zapletal-martin - Martin Zapletal zero323 - Maciej Szymkiewicz +zhangjiajin - Zhang JiaJin +zhengruifeng - Ruifeng Zheng +zhichao-li - Zhichao Li zjf2012 - Jiafu Zhang +zsxwing - Shixiong Zhu +zuotingbing - Tingbing Zuo +zuxqoj - Shekhar Bansal +zzcclp - Zhichao Zhang From 20cd47e82d7d84516455df960ededbd647694aa5 Mon Sep 17 00:00:00 2001 From: xy_xin Date: Mon, 29 Jun 2020 13:13:42 +0000 Subject: [PATCH 093/384] [SPARK-32030][SQL] Support unlimited MATCHED and NOT MATCHED clauses in MERGE INTO ### What changes were proposed in this pull request? This PR add unlimited MATCHED and NOT MATCHED clauses in MERGE INTO statement. ### Why are the changes needed? Now the MERGE INTO syntax is, ``` MERGE INTO [db_name.]target_table [AS target_alias] USING [db_name.]source_table [] [AS source_alias] ON [ WHEN MATCHED [ AND ] THEN ] [ WHEN MATCHED [ AND ] THEN ] [ WHEN NOT MATCHED [ AND ] THEN ] ``` It would be nice if we support unlimited MATCHED and NOT MATCHED clauses in MERGE INTO statement, because users may want to deal with different "AND "s, the result of which just like a series of "CASE WHEN"s. The expected syntax looks like ``` MERGE INTO [db_name.]target_table [AS target_alias] USING [db_name.]source_table [] [AS source_alias] ON [when_matched_clause [, ...]] [when_not_matched_clause [, ...]] ``` where when_matched_clause is ``` WHEN MATCHED [ AND ] THEN ``` and when_not_matched_clause is ``` WHEN NOT MATCHED [ AND ] THEN ``` matched_action can be one of ``` DELETE UPDATE SET * or UPDATE SET col1 = value1 [, col2 = value2, ...] ``` and not_matched_action can be one of ``` INSERT * INSERT (col1 [, col2, ...]) VALUES (value1 [, value2, ...]) ``` ### Does this PR introduce _any_ user-facing change? Yes. The SQL command changes, but it is backward compatible. ### How was this patch tested? New tests added. Closes #28875 from xianyinxin/SPARK-32030. Authored-by: xy_xin Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/AstBuilder.scala | 30 +++---- .../catalyst/plans/logical/v2Commands.scala | 10 +-- .../sql/catalyst/parser/DDLParserSuite.scala | 90 +++++++++---------- 3 files changed, 59 insertions(+), 71 deletions(-) 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 03571a740df3e..d08bcb1420176 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 @@ -411,12 +411,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val mergeCondition = expression(ctx.mergeCondition) - val matchedClauses = ctx.matchedClause() - if (matchedClauses.size() > 2) { - throw new ParseException("There should be at most 2 'WHEN MATCHED' clauses.", - matchedClauses.get(2)) - } - val matchedActions = matchedClauses.asScala.map { + val matchedActions = ctx.matchedClause().asScala.map { clause => { if (clause.matchedAction().DELETE() != null) { DeleteAction(Option(clause.matchedCond).map(expression)) @@ -435,12 +430,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } } - val notMatchedClauses = ctx.notMatchedClause() - if (notMatchedClauses.size() > 1) { - throw new ParseException("There should be at most 1 'WHEN NOT MATCHED' clause.", - notMatchedClauses.get(1)) - } - val notMatchedActions = notMatchedClauses.asScala.map { + val notMatchedActions = ctx.notMatchedClause().asScala.map { clause => { if (clause.notMatchedAction().INSERT() != null) { val condition = Option(clause.notMatchedCond).map(expression) @@ -468,13 +458,15 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging throw new ParseException("There must be at least one WHEN clause in a MERGE statement", ctx) } // children being empty means that the condition is not set - if (matchedActions.length == 2 && matchedActions.head.children.isEmpty) { - throw new ParseException("When there are 2 MATCHED clauses in a MERGE statement, " + - "the first MATCHED clause must have a condition", ctx) - } - if (matchedActions.groupBy(_.getClass).mapValues(_.size).exists(_._2 > 1)) { - throw new ParseException( - "UPDATE and DELETE can appear at most once in MATCHED clauses in a MERGE statement", ctx) + val matchedActionSize = matchedActions.length + if (matchedActionSize >= 2 && !matchedActions.init.forall(_.condition.nonEmpty)) { + throw new ParseException("When there are more than one MATCHED clauses in a MERGE " + + "statement, only the last MATCHED clause can omit the condition.", ctx) + } + val notMatchedActionSize = notMatchedActions.length + if (notMatchedActionSize >= 2 && !notMatchedActions.init.forall(_.condition.nonEmpty)) { + throw new ParseException("When there are more than one NOT MATCHED clauses in a MERGE " + + "statement, only the last NOT MATCHED clause can omit the condition.", ctx) } MergeIntoTable( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 579157a6f2f2e..b4120d9f64cc5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -346,25 +346,25 @@ case class MergeIntoTable( override def children: Seq[LogicalPlan] = Seq(targetTable, sourceTable) } -sealed abstract class MergeAction( - condition: Option[Expression]) extends Expression with Unevaluable { +sealed abstract class MergeAction extends Expression with Unevaluable { + def condition: Option[Expression] override def foldable: Boolean = false override def nullable: Boolean = false override def dataType: DataType = throw new UnresolvedException(this, "nullable") override def children: Seq[Expression] = condition.toSeq } -case class DeleteAction(condition: Option[Expression]) extends MergeAction(condition) +case class DeleteAction(condition: Option[Expression]) extends MergeAction case class UpdateAction( condition: Option[Expression], - assignments: Seq[Assignment]) extends MergeAction(condition) { + assignments: Seq[Assignment]) extends MergeAction { override def children: Seq[Expression] = condition.toSeq ++ assignments } case class InsertAction( condition: Option[Expression], - assignments: Seq[Assignment]) extends MergeAction(condition) { + assignments: Seq[Assignment]) extends MergeAction { override def children: Seq[Expression] = condition.toSeq ++ assignments } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 6499b5d8e7974..e802449a69743 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1134,58 +1134,74 @@ class DDLParserSuite extends AnalysisTest { } } - test("merge into table: at most two matched clauses") { - val exc = intercept[ParseException] { - parsePlan( - """ - |MERGE INTO testcat1.ns1.ns2.tbl AS target - |USING testcat2.ns1.ns2.tbl AS source - |ON target.col1 = source.col1 - |WHEN MATCHED AND (target.col2='delete') THEN DELETE - |WHEN MATCHED AND (target.col2='update1') THEN UPDATE SET target.col2 = source.col2 - |WHEN MATCHED AND (target.col2='update2') THEN UPDATE SET target.col2 = source.col2 - |WHEN NOT MATCHED AND (target.col2='insert') - |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2) - """.stripMargin) - } - - assert(exc.getMessage.contains("There should be at most 2 'WHEN MATCHED' clauses.")) + test("merge into table: multi matched and not matched clauses") { + parseCompare( + """ + |MERGE INTO testcat1.ns1.ns2.tbl AS target + |USING testcat2.ns1.ns2.tbl AS source + |ON target.col1 = source.col1 + |WHEN MATCHED AND (target.col2='delete') THEN DELETE + |WHEN MATCHED AND (target.col2='update1') THEN UPDATE SET target.col2 = 1 + |WHEN MATCHED AND (target.col2='update2') THEN UPDATE SET target.col2 = 2 + |WHEN NOT MATCHED AND (target.col2='insert1') + |THEN INSERT (target.col1, target.col2) values (source.col1, 1) + |WHEN NOT MATCHED AND (target.col2='insert2') + |THEN INSERT (target.col1, target.col2) values (source.col1, 2) + """.stripMargin, + MergeIntoTable( + SubqueryAlias("target", UnresolvedRelation(Seq("testcat1", "ns1", "ns2", "tbl"))), + SubqueryAlias("source", UnresolvedRelation(Seq("testcat2", "ns1", "ns2", "tbl"))), + EqualTo(UnresolvedAttribute("target.col1"), UnresolvedAttribute("source.col1")), + Seq(DeleteAction(Some(EqualTo(UnresolvedAttribute("target.col2"), Literal("delete")))), + UpdateAction(Some(EqualTo(UnresolvedAttribute("target.col2"), Literal("update1"))), + Seq(Assignment(UnresolvedAttribute("target.col2"), Literal(1)))), + UpdateAction(Some(EqualTo(UnresolvedAttribute("target.col2"), Literal("update2"))), + Seq(Assignment(UnresolvedAttribute("target.col2"), Literal(2))))), + Seq(InsertAction(Some(EqualTo(UnresolvedAttribute("target.col2"), Literal("insert1"))), + Seq(Assignment(UnresolvedAttribute("target.col1"), UnresolvedAttribute("source.col1")), + Assignment(UnresolvedAttribute("target.col2"), Literal(1)))), + InsertAction(Some(EqualTo(UnresolvedAttribute("target.col2"), Literal("insert2"))), + Seq(Assignment(UnresolvedAttribute("target.col1"), UnresolvedAttribute("source.col1")), + Assignment(UnresolvedAttribute("target.col2"), Literal(2))))))) } - test("merge into table: at most one not matched clause") { + test("merge into table: only the last matched clause can omit the condition") { val exc = intercept[ParseException] { parsePlan( """ |MERGE INTO testcat1.ns1.ns2.tbl AS target |USING testcat2.ns1.ns2.tbl AS source |ON target.col1 = source.col1 - |WHEN MATCHED AND (target.col2='delete') THEN DELETE - |WHEN MATCHED AND (target.col2='update1') THEN UPDATE SET target.col2 = source.col2 - |WHEN NOT MATCHED AND (target.col2='insert1') - |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2) - |WHEN NOT MATCHED AND (target.col2='insert2') + |WHEN MATCHED AND (target.col2 == 'update1') THEN UPDATE SET target.col2 = 1 + |WHEN MATCHED THEN UPDATE SET target.col2 = 2 + |WHEN MATCHED THEN DELETE + |WHEN NOT MATCHED AND (target.col2='insert') |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2) """.stripMargin) } - assert(exc.getMessage.contains("There should be at most 1 'WHEN NOT MATCHED' clause.")) + assert(exc.getMessage.contains("only the last MATCHED clause can omit the condition")) } - test("merge into table: the first matched clause must have a condition if there's a second") { + test("merge into table: only the last not matched clause can omit the condition") { val exc = intercept[ParseException] { parsePlan( """ |MERGE INTO testcat1.ns1.ns2.tbl AS target |USING testcat2.ns1.ns2.tbl AS source |ON target.col1 = source.col1 + |WHEN MATCHED AND (target.col2 == 'update') THEN UPDATE SET target.col2 = source.col2 |WHEN MATCHED THEN DELETE - |WHEN MATCHED THEN UPDATE SET target.col2 = source.col2 - |WHEN NOT MATCHED AND (target.col2='insert') + |WHEN NOT MATCHED AND (target.col2='insert1') + |THEN INSERT (target.col1, target.col2) values (source.col1, 1) + |WHEN NOT MATCHED + |THEN INSERT (target.col1, target.col2) values (source.col1, 2) + |WHEN NOT MATCHED |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2) """.stripMargin) } - assert(exc.getMessage.contains("the first MATCHED clause must have a condition")) + assert(exc.getMessage.contains("only the last NOT MATCHED clause can omit the condition")) } test("merge into table: there must be a when (not) matched condition") { @@ -1201,26 +1217,6 @@ class DDLParserSuite extends AnalysisTest { assert(exc.getMessage.contains("There must be at least one WHEN clause in a MERGE statement")) } - test("merge into table: there can be only a single use DELETE or UPDATE") { - Seq("UPDATE SET *", "DELETE").foreach { op => - val exc = intercept[ParseException] { - parsePlan( - s""" - |MERGE INTO testcat1.ns1.ns2.tbl AS target - |USING testcat2.ns1.ns2.tbl AS source - |ON target.col1 = source.col1 - |WHEN MATCHED AND (target.col2='delete') THEN $op - |WHEN MATCHED THEN $op - |WHEN NOT MATCHED AND (target.col2='insert') - |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2) - """.stripMargin) - } - - assert(exc.getMessage.contains( - "UPDATE and DELETE can appear at most once in MATCHED clauses")) - } - } - test("show tables") { comparePlans( parsePlan("SHOW TABLES"), From 5472170a2b35864c617bdb846ff7123533765a16 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 30 Jun 2020 08:09:18 +0000 Subject: [PATCH 094/384] [SPARK-29999][SS][FOLLOWUP] Fix test to check the actual metadata log directory ### What changes were proposed in this pull request? This patch fixes the missed spot - the test initializes FileStreamSinkLog with its "output" directory instead of "metadata" directory, hence the verification against sink log was no-op. ### Why are the changes needed? Without the fix, the verification against sink log was no-op. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Checked with debugger in test, and verified `allFiles()` returns non-zero entries. (It returned zero entry, as there's no metadata.) Closes #28930 from HeartSaVioR/SPARK-29999-FOLLOWUP-fix-test. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Wenchen Fan --- .../execution/streaming/FileStreamSink.scala | 19 +++++++++++-------- .../sql/streaming/FileStreamSinkSuite.scala | 10 ++++++---- 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index 32245470d8f5d..ecaf4f8160a06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -45,8 +45,7 @@ object FileStreamSink extends Logging { val hdfsPath = new Path(singlePath) val fs = hdfsPath.getFileSystem(hadoopConf) if (fs.isDirectory(hdfsPath)) { - val metadataPath = new Path(hdfsPath, metadataDir) - checkEscapedMetadataPath(fs, metadataPath, sqlConf) + val metadataPath = getMetadataLogPath(fs, hdfsPath, sqlConf) fs.exists(metadataPath) } else { false @@ -55,6 +54,12 @@ object FileStreamSink extends Logging { } } + def getMetadataLogPath(fs: FileSystem, path: Path, sqlConf: SQLConf): Path = { + val metadataDir = new Path(path, FileStreamSink.metadataDir) + FileStreamSink.checkEscapedMetadataPath(fs, metadataDir, sqlConf) + metadataDir + } + def checkEscapedMetadataPath(fs: FileSystem, metadataPath: Path, sqlConf: SQLConf): Unit = { if (sqlConf.getConf(SQLConf.STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED) && StreamExecution.containsSpecialCharsInPath(metadataPath)) { @@ -125,14 +130,12 @@ class FileStreamSink( partitionColumnNames: Seq[String], options: Map[String, String]) extends Sink with Logging { + import FileStreamSink._ + private val hadoopConf = sparkSession.sessionState.newHadoopConf() private val basePath = new Path(path) - private val logPath = { - val metadataDir = new Path(basePath, FileStreamSink.metadataDir) - val fs = metadataDir.getFileSystem(hadoopConf) - FileStreamSink.checkEscapedMetadataPath(fs, metadataDir, sparkSession.sessionState.conf) - metadataDir - } + private val logPath = getMetadataLogPath(basePath.getFileSystem(hadoopConf), basePath, + sparkSession.sessionState.conf) private val fileLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, logPath.toString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index a25451bef62fd..4ccab58d24fed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -555,10 +555,12 @@ abstract class FileStreamSinkSuite extends StreamTest { } } - val fs = new Path(outputDir.getCanonicalPath).getFileSystem( - spark.sessionState.newHadoopConf()) - val sinkLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, spark, - outputDir.getCanonicalPath) + val outputDirPath = new Path(outputDir.getCanonicalPath) + val hadoopConf = spark.sessionState.newHadoopConf() + val fs = outputDirPath.getFileSystem(hadoopConf) + val logPath = FileStreamSink.getMetadataLogPath(fs, outputDirPath, conf) + + val sinkLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, spark, logPath.toString) val allFiles = sinkLog.allFiles() // only files from non-empty partition should be logged From 165c948e3297fd5ebcee771fcbf7a0c20fccfaae Mon Sep 17 00:00:00 2001 From: TJX2014 Date: Tue, 30 Jun 2020 08:56:59 -0500 Subject: [PATCH 095/384] [SPARK-32068][WEBUI] Correct task lauchtime show issue due to timezone in stage tab MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? `formatDate` in utils.js `org/apache/spark/ui/static/utils.js` is partly refactored. ### Why are the changes needed? In branch-2.4,task launch time is returned as html string from driver, while in branch-3.x,this is returned in JSON Object as`Date`type from `org.apache.spark.status.api.v1.TaskData` Due to: LaunchTime from jersey server in spark driver is correct, which will be converted to date string like `2020-06-28T02:57:42.605GMT` in json object, then the formatDate in utils.js treat it as date.split(".")[0].replace("T", " "). So `2020-06-28T02:57:42.605GMT` will be converted to `2020-06-28 02:57:42`, but correct is `2020-06-28 10:57:42` in GMT+8 timezone. ![选区_071](https://user-images.githubusercontent.com/7149304/85937186-b6d36780-b933-11ea-8382-80a3891f1c2a.png) ![选区_070](https://user-images.githubusercontent.com/7149304/85937190-bcc94880-b933-11ea-8860-2083c97ea269.png) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual test. Closes #28918 from TJX2014/master-SPARK-32068-ui-task-lauch-time-tz. Authored-by: TJX2014 Signed-off-by: Thomas Graves --- .../resources/org/apache/spark/ui/static/utils.js | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js index 6fc34a9e1f7ea..2e46111bf1ba0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/utils.js +++ b/core/src/main/resources/org/apache/spark/ui/static/utils.js @@ -56,13 +56,17 @@ function formatTimeMillis(timeMillis) { return "-"; } else { var dt = new Date(timeMillis); + return formatDateString(dt); + } +} + +function formatDateString(dt) { return dt.getFullYear() + "-" + padZeroes(dt.getMonth() + 1) + "-" + padZeroes(dt.getDate()) + " " + padZeroes(dt.getHours()) + ":" + padZeroes(dt.getMinutes()) + ":" + padZeroes(dt.getSeconds()); - } } function getTimeZone() { @@ -161,7 +165,10 @@ function setDataTableDefaults() { function formatDate(date) { if (date <= 0) return "-"; - else return date.split(".")[0].replace("T", " "); + else { + var dt = new Date(date.replace("GMT", "Z")) + return formatDateString(dt); + } } function createRESTEndPointForExecutorsPage(appId) { From 5176707ac3a451158e5705bfb9a070de2d6c9cab Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 30 Jun 2020 09:28:54 -0700 Subject: [PATCH 096/384] [MINOR][DOCS] Fix a typo for a configuration property of resources allocation ### What changes were proposed in this pull request? This PR fixes a typo for a configuration property in the `spark-standalone.md`. `spark.driver.resourcesfile` should be `spark.driver.resourcesFile`. I look for similar typo but this is the only typo. ### Why are the changes needed? The property name is wrong. ### Does this PR introduce _any_ user-facing change? Yes. The property name is corrected. ### How was this patch tested? I confirmed the spell of the property name is the correct from the property name defined in o.a.s.internal.config.package.scala. Closes #28958 from sarutak/fix-resource-typo. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- docs/spark-standalone.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 1f70d46d587a8..f3c479ba26547 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -359,7 +359,7 @@ Spark Standalone has 2 parts, the first is configuring the resources for the Wor The user must configure the Workers to have a set of resources available so that it can assign them out to Executors. The spark.worker.resource.{resourceName}.amount is used to control the amount of each resource the worker has allocated. The user must also specify either spark.worker.resourcesFile or spark.worker.resource.{resourceName}.discoveryScript to specify how the Worker discovers the resources its assigned. See the descriptions above for each of those to see which method works best for your setup. -The second part is running an application on Spark Standalone. The only special case from the standard Spark resource configs is when you are running the Driver in client mode. For a Driver in client mode, the user can specify the resources it uses via spark.driver.resourcesfile or spark.driver.resource.{resourceName}.discoveryScript. If the Driver is running on the same host as other Drivers, please make sure the resources file or discovery script only returns resources that do not conflict with other Drivers running on the same node. +The second part is running an application on Spark Standalone. The only special case from the standard Spark resource configs is when you are running the Driver in client mode. For a Driver in client mode, the user can specify the resources it uses via spark.driver.resourcesFile or spark.driver.resource.{resourceName}.discoveryScript. If the Driver is running on the same host as other Drivers, please make sure the resources file or discovery script only returns resources that do not conflict with other Drivers running on the same node. Note, the user does not need to specify a discovery script when submitting an application as the Worker will start each Executor with the resources it allocates to it. From 67cb7eaa6572770de0a6cc2f871eacdb15a572b2 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 30 Jun 2020 10:30:22 -0700 Subject: [PATCH 097/384] [SPARK-31336][SQL] Support Oracle Kerberos login in JDBC connector ### What changes were proposed in this pull request? When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it. This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues. In this PR I've added Oracle support. What this PR contains: * Added `OracleConnectionProvider` * Added `OracleConnectionProviderSuite` ### Why are the changes needed? Missing JDBC kerberos support. ### Does this PR introduce _any_ user-facing change? Yes, now user is able to connect to Oracle using kerberos. ### How was this patch tested? * Additional + existing unit tests * Test on cluster manually Closes #28863 from gaborgsomogyi/SPARK-31336. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- sql/core/pom.xml | 5 ++ .../jdbc/connection/ConnectionProvider.scala | 4 ++ .../connection/OracleConnectionProvider.scala | 62 +++++++++++++++++++ .../OracleConnectionProviderSuite.scala | 28 +++++++++ 4 files changed, 99 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0855fa13fa79a..c2ed4c079d3cf 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -150,6 +150,11 @@ mssql-jdbc test + + com.oracle.database.jdbc + ojdbc8 + test + org.apache.parquet parquet-avro diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala index 6c310ced37883..ce45be442ccc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala @@ -64,6 +64,10 @@ private[jdbc] object ConnectionProvider extends Logging { logDebug("MS SQL connection provider found") new MSSQLConnectionProvider(driver, options) + case OracleConnectionProvider.driverClass => + logDebug("Oracle connection provider found") + new OracleConnectionProvider(driver, options) + case _ => throw new IllegalArgumentException(s"Driver ${options.driverClass} does not support " + "Kerberos authentication") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala new file mode 100644 index 0000000000000..c2b71b35b8128 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc.connection + +import java.security.PrivilegedExceptionAction +import java.sql.{Connection, Driver} +import java.util.Properties + +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions + +private[sql] class OracleConnectionProvider(driver: Driver, options: JDBCOptions) + extends SecureConnectionProvider(driver, options) { + override val appEntry: String = "kprb5module" + + override def getConnection(): Connection = { + setAuthenticationConfigIfNeeded() + UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + OracleConnectionProvider.super.getConnection() + } + } + ) + } + + override def getAdditionalProperties(): Properties = { + val result = new Properties() + // This prop is needed to turn on kerberos authentication in the JDBC driver. + // The possible values can be found in AnoServices public interface + // The value is coming from AUTHENTICATION_KERBEROS5 final String in driver version 19.6.0.0 + result.put("oracle.net.authentication_services", "(KERBEROS5)"); + result + } + + override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { + val (parent, configEntry) = getConfigWithAppEntry() + if (configEntry == null || configEntry.isEmpty) { + setAuthenticationConfig(parent) + } + } +} + +private[sql] object OracleConnectionProvider { + val driverClass = "oracle.jdbc.OracleDriver" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala new file mode 100644 index 0000000000000..13cde32ddbe4e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc.connection + +class OracleConnectionProviderSuite extends ConnectionProviderSuiteBase { + test("setAuthenticationConfigIfNeeded must set authentication if not set") { + val driver = registerDriver(OracleConnectionProvider.driverClass) + val provider = new OracleConnectionProvider(driver, + options("jdbc:oracle:thin:@//localhost/xe")) + + testSecureConnectionProvider(provider) + } +} From bbd0275dfd532b51f946fa33bc7d4354e90a27dc Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 30 Jun 2020 11:18:16 -0700 Subject: [PATCH 098/384] [MINOR][SQL] Fix spaces in JDBC connection providers ### What changes were proposed in this pull request? JDBC connection providers implementation formatted in a wrong way. In this PR I've fixed the formatting. ### Why are the changes needed? Wrong spacing in JDBC connection providers. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #28945 from gaborgsomogyi/provider_spacing. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- .../datasources/jdbc/connection/BasicConnectionProvider.scala | 2 +- .../datasources/jdbc/connection/DB2ConnectionProvider.scala | 2 +- .../datasources/jdbc/connection/MariaDBConnectionProvider.scala | 2 +- .../jdbc/connection/PostgresConnectionProvider.scala | 2 +- .../datasources/jdbc/connection/SecureConnectionProvider.scala | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala index c21e16bcf1280..16b244cc617ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions private[jdbc] class BasicConnectionProvider(driver: Driver, options: JDBCOptions) - extends ConnectionProvider { + extends ConnectionProvider { def getConnection(): Connection = { val properties = getAdditionalProperties() options.asConnectionProperties.entrySet().asScala.foreach { e => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala index cf9729639c03c..095821cf83890 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions private[sql] class DB2ConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { + extends SecureConnectionProvider(driver, options) { override val appEntry: String = "JaasClient" override def getConnection(): Connection = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala index 589f13cf6ad5f..3c0286654a8ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala @@ -22,7 +22,7 @@ import java.sql.Driver import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { + extends SecureConnectionProvider(driver, options) { override val appEntry: String = { "Krb5ConnectorContext" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala index 73034dcb9c2e0..fa9232e00bd88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala @@ -23,7 +23,7 @@ import java.util.Properties import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions private[jdbc] class PostgresConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { + extends SecureConnectionProvider(driver, options) { override val appEntry: String = { val parseURL = driver.getClass.getMethod("parseURL", classOf[String], classOf[Properties]) val properties = parseURL.invoke(driver, options.url, null).asInstanceOf[Properties] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala index fa75fc8c28fbf..24eec63a7244f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.SecurityUtils private[connection] object SecurityConfigurationLock private[jdbc] abstract class SecureConnectionProvider(driver: Driver, options: JDBCOptions) - extends BasicConnectionProvider(driver, options) with Logging { + extends BasicConnectionProvider(driver, options) with Logging { override def getConnection(): Connection = { setAuthenticationConfigIfNeeded() super.getConnection() From dd03c31ea5daf6c7de58d0e047ca47fa8e715f02 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 1 Jul 2020 13:17:49 +0900 Subject: [PATCH 099/384] [SPARK-32088][PYTHON][FOLLOWUP] Replace `collect()` by `show()` in the example for `timestamp_seconds` ### What changes were proposed in this pull request? Modify the example for `timestamp_seconds` and replace `collect()` by `show()`. ### Why are the changes needed? The SQL config `spark.sql.session.timeZone` doesn't influence on the `collect` in the example. The code below demonstrates that: ``` $ export TZ="UTC" ``` ```python >>> from pyspark.sql.functions import timestamp_seconds >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() [Row(ts=datetime.datetime(2008, 12, 25, 15, 30))] ``` The expected time is **07:30 but we get 15:30**. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running the modified example via: ``` $ ./python/run-tests --modules=pyspark-sql ``` Closes #28959 from MaxGekk/SPARK-32088-fix-timezone-issue-followup. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- python/pyspark/sql/functions.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index b0498d0298785..b5a7c18904b14 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1433,8 +1433,12 @@ def timestamp_seconds(col): >>> from pyspark.sql.functions import timestamp_seconds >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) - >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() - [Row(ts=datetime.datetime(2008, 12, 25, 7, 30))] + >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).show() + +-------------------+ + | ts| + +-------------------+ + |2008-12-25 07:30:00| + +-------------------+ >>> spark.conf.unset("spark.sql.session.timeZone") """ From 8194d9ef788278ec23902da851f2a3c95f5f71bf Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 1 Jul 2020 14:15:02 +0900 Subject: [PATCH 100/384] [SPARK-32142][SQL][TESTS] Keep the original tests and codes to avoid potential conflicts in dev ### What changes were proposed in this pull request? This PR proposes to partially reverts back in the tests and some codes at https://github.com/apache/spark/pull/27728 without touching any behaivours. Most of changes in tests are back before #27728 by combining `withNestedDataFrame` and `withParquetDataFrame`. Basically, it addresses the comments https://github.com/apache/spark/pull/27728#discussion_r397655390, and my own comment in another PR at https://github.com/apache/spark/pull/28761#discussion_r446761037 ### Why are the changes needed? For maintenance purpose and to avoid a potential conflicts during backports. And also in case when other codes are matched with this. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually tested. Closes #28955 from HyukjinKwon/SPARK-25556-followup. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../parquet/ParquetFilterSuite.scala | 789 +++++++++--------- .../datasources/parquet/ParquetIOSuite.scala | 20 +- .../datasources/parquet/ParquetTest.scala | 14 +- 3 files changed, 411 insertions(+), 412 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index d20a07f420e87..8b922aaed4e68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -22,6 +22,9 @@ import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.time.{LocalDate, LocalDateTime, ZoneId} +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate, Operators} import org.apache.parquet.filter2.predicate.FilterApi._ import org.apache.parquet.filter2.predicate.Operators.{Column => _, _} @@ -106,10 +109,18 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } /** - * Takes single level `inputDF` dataframe to generate multi-level nested - * dataframes as new test data. + * Takes a sequence of products `data` to generate multi-level nested + * dataframes as new test data. It tests both non-nested and nested dataframes + * which are written and read back with Parquet datasource. + * + * This is different from [[ParquetTest.withParquetDataFrame]] which does not + * test nested cases. */ - private def withNestedDataFrame(inputDF: DataFrame) + private def withNestedParquetDataFrame[T <: Product: ClassTag: TypeTag](data: Seq[T]) + (runTest: (DataFrame, String, Any => Any) => Unit): Unit = + withNestedParquetDataFrame(spark.createDataFrame(data))(runTest) + + private def withNestedParquetDataFrame(inputDF: DataFrame) (runTest: (DataFrame, String, Any => Any) => Unit): Unit = { assert(inputDF.schema.fields.length == 1) assert(!inputDF.schema.fields.head.dataType.isInstanceOf[StructType]) @@ -138,8 +149,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared "`a.b`.`c.d`", // one level nesting with column names containing `dots` (x: Any) => Row(x) ) - ).foreach { case (df, colName, resultFun) => - runTest(df, colName, resultFun) + ).foreach { case (newDF, colName, resultFun) => + withTempPath { file => + newDF.write.format(dataSourceName).save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath) { df => runTest(df, colName, resultFun) } + } } } @@ -155,7 +169,9 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared import testImplicits._ val df = data.map(i => Tuple1(Timestamp.valueOf(i))).toDF() - withNestedDataFrame(df) { case (inputDF, colName, fun) => + withNestedParquetDataFrame(df) { case (parquetDF, colName, fun) => + implicit val df: DataFrame = parquetDF + def resultFun(tsStr: String): Any = { val parsed = if (java8Api) { LocalDateTime.parse(tsStr.replace(" ", "T")) @@ -166,36 +182,35 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } fun(parsed) } - withParquetDataFrame(inputDF) { implicit df => - val tsAttr = df(colName).expr - assert(df(colName).expr.dataType === TimestampType) - - checkFilterPredicate(tsAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(tsAttr.isNotNull, classOf[NotEq[_]], - data.map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(tsAttr === ts1.ts, classOf[Eq[_]], resultFun(ts1)) - checkFilterPredicate(tsAttr <=> ts1.ts, classOf[Eq[_]], resultFun(ts1)) - checkFilterPredicate(tsAttr =!= ts1.ts, classOf[NotEq[_]], - Seq(ts2, ts3, ts4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(tsAttr < ts2.ts, classOf[Lt[_]], resultFun(ts1)) - checkFilterPredicate(tsAttr > ts1.ts, classOf[Gt[_]], - Seq(ts2, ts3, ts4).map(i => Row.apply(resultFun(i)))) - checkFilterPredicate(tsAttr <= ts1.ts, classOf[LtEq[_]], resultFun(ts1)) - checkFilterPredicate(tsAttr >= ts4.ts, classOf[GtEq[_]], resultFun(ts4)) - - checkFilterPredicate(Literal(ts1.ts) === tsAttr, classOf[Eq[_]], resultFun(ts1)) - checkFilterPredicate(Literal(ts1.ts) <=> tsAttr, classOf[Eq[_]], resultFun(ts1)) - checkFilterPredicate(Literal(ts2.ts) > tsAttr, classOf[Lt[_]], resultFun(ts1)) - checkFilterPredicate(Literal(ts3.ts) < tsAttr, classOf[Gt[_]], resultFun(ts4)) - checkFilterPredicate(Literal(ts1.ts) >= tsAttr, classOf[LtEq[_]], resultFun(ts1)) - checkFilterPredicate(Literal(ts4.ts) <= tsAttr, classOf[GtEq[_]], resultFun(ts4)) - - checkFilterPredicate(!(tsAttr < ts4.ts), classOf[GtEq[_]], resultFun(ts4)) - checkFilterPredicate(tsAttr < ts2.ts || tsAttr > ts3.ts, classOf[Operators.Or], - Seq(Row(resultFun(ts1)), Row(resultFun(ts4)))) - } + + val tsAttr = df(colName).expr + assert(df(colName).expr.dataType === TimestampType) + + checkFilterPredicate(tsAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(tsAttr.isNotNull, classOf[NotEq[_]], + data.map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(tsAttr === ts1.ts, classOf[Eq[_]], resultFun(ts1)) + checkFilterPredicate(tsAttr <=> ts1.ts, classOf[Eq[_]], resultFun(ts1)) + checkFilterPredicate(tsAttr =!= ts1.ts, classOf[NotEq[_]], + Seq(ts2, ts3, ts4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(tsAttr < ts2.ts, classOf[Lt[_]], resultFun(ts1)) + checkFilterPredicate(tsAttr > ts1.ts, classOf[Gt[_]], + Seq(ts2, ts3, ts4).map(i => Row.apply(resultFun(i)))) + checkFilterPredicate(tsAttr <= ts1.ts, classOf[LtEq[_]], resultFun(ts1)) + checkFilterPredicate(tsAttr >= ts4.ts, classOf[GtEq[_]], resultFun(ts4)) + + checkFilterPredicate(Literal(ts1.ts) === tsAttr, classOf[Eq[_]], resultFun(ts1)) + checkFilterPredicate(Literal(ts1.ts) <=> tsAttr, classOf[Eq[_]], resultFun(ts1)) + checkFilterPredicate(Literal(ts2.ts) > tsAttr, classOf[Lt[_]], resultFun(ts1)) + checkFilterPredicate(Literal(ts3.ts) < tsAttr, classOf[Gt[_]], resultFun(ts4)) + checkFilterPredicate(Literal(ts1.ts) >= tsAttr, classOf[LtEq[_]], resultFun(ts1)) + checkFilterPredicate(Literal(ts4.ts) <= tsAttr, classOf[GtEq[_]], resultFun(ts4)) + + checkFilterPredicate(!(tsAttr < ts4.ts), classOf[GtEq[_]], resultFun(ts4)) + checkFilterPredicate(tsAttr < ts2.ts || tsAttr > ts3.ts, classOf[Operators.Or], + Seq(Row(resultFun(ts1)), Row(resultFun(ts4)))) } } @@ -226,272 +241,264 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared test("filter pushdown - boolean") { val data = (true :: false :: Nil).map(b => Tuple1.apply(Option(b))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val booleanAttr = df(colName).expr - assert(df(colName).expr.dataType === BooleanType) - - checkFilterPredicate(booleanAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(booleanAttr.isNotNull, classOf[NotEq[_]], - Seq(Row(resultFun(true)), Row(resultFun(false)))) - - checkFilterPredicate(booleanAttr === true, classOf[Eq[_]], resultFun(true)) - checkFilterPredicate(booleanAttr <=> true, classOf[Eq[_]], resultFun(true)) - checkFilterPredicate(booleanAttr =!= true, classOf[NotEq[_]], resultFun(false)) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val booleanAttr = df(colName).expr + assert(df(colName).expr.dataType === BooleanType) + + checkFilterPredicate(booleanAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(booleanAttr.isNotNull, classOf[NotEq[_]], + Seq(Row(resultFun(true)), Row(resultFun(false)))) + + checkFilterPredicate(booleanAttr === true, classOf[Eq[_]], resultFun(true)) + checkFilterPredicate(booleanAttr <=> true, classOf[Eq[_]], resultFun(true)) + checkFilterPredicate(booleanAttr =!= true, classOf[NotEq[_]], resultFun(false)) } } test("filter pushdown - tinyint") { val data = (1 to 4).map(i => Tuple1(Option(i.toByte))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val tinyIntAttr = df(colName).expr - assert(df(colName).expr.dataType === ByteType) - - checkFilterPredicate(tinyIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(tinyIntAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(tinyIntAttr === 1.toByte, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(tinyIntAttr <=> 1.toByte, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(tinyIntAttr =!= 1.toByte, classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(tinyIntAttr < 2.toByte, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(tinyIntAttr > 3.toByte, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(tinyIntAttr <= 1.toByte, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(tinyIntAttr >= 4.toByte, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(Literal(1.toByte) === tinyIntAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(1.toByte) <=> tinyIntAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(2.toByte) > tinyIntAttr, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(Literal(3.toByte) < tinyIntAttr, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(Literal(1.toByte) >= tinyIntAttr, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(Literal(4.toByte) <= tinyIntAttr, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(!(tinyIntAttr < 4.toByte), classOf[GtEq[_]], resultFun(4)) - checkFilterPredicate(tinyIntAttr < 2.toByte || tinyIntAttr > 3.toByte, - classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4)))) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val tinyIntAttr = df(colName).expr + assert(df(colName).expr.dataType === ByteType) + + checkFilterPredicate(tinyIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(tinyIntAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(tinyIntAttr === 1.toByte, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(tinyIntAttr <=> 1.toByte, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(tinyIntAttr =!= 1.toByte, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(tinyIntAttr < 2.toByte, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(tinyIntAttr > 3.toByte, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(tinyIntAttr <= 1.toByte, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(tinyIntAttr >= 4.toByte, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1.toByte) === tinyIntAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1.toByte) <=> tinyIntAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2.toByte) > tinyIntAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3.toByte) < tinyIntAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1.toByte) >= tinyIntAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4.toByte) <= tinyIntAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(tinyIntAttr < 4.toByte), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(tinyIntAttr < 2.toByte || tinyIntAttr > 3.toByte, + classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4)))) } } test("filter pushdown - smallint") { val data = (1 to 4).map(i => Tuple1(Option(i.toShort))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val smallIntAttr = df(colName).expr - assert(df(colName).expr.dataType === ShortType) - - checkFilterPredicate(smallIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(smallIntAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(smallIntAttr === 1.toShort, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(smallIntAttr <=> 1.toShort, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(smallIntAttr =!= 1.toShort, classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(smallIntAttr < 2.toShort, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(smallIntAttr > 3.toShort, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(smallIntAttr <= 1.toShort, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(smallIntAttr >= 4.toShort, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(Literal(1.toShort) === smallIntAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(1.toShort) <=> smallIntAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(2.toShort) > smallIntAttr, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(Literal(3.toShort) < smallIntAttr, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(Literal(1.toShort) >= smallIntAttr, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(Literal(4.toShort) <= smallIntAttr, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(!(smallIntAttr < 4.toShort), classOf[GtEq[_]], resultFun(4)) - checkFilterPredicate(smallIntAttr < 2.toShort || smallIntAttr > 3.toShort, - classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4)))) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val smallIntAttr = df(colName).expr + assert(df(colName).expr.dataType === ShortType) + + checkFilterPredicate(smallIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(smallIntAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(smallIntAttr === 1.toShort, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(smallIntAttr <=> 1.toShort, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(smallIntAttr =!= 1.toShort, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(smallIntAttr < 2.toShort, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(smallIntAttr > 3.toShort, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(smallIntAttr <= 1.toShort, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(smallIntAttr >= 4.toShort, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1.toShort) === smallIntAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1.toShort) <=> smallIntAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2.toShort) > smallIntAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3.toShort) < smallIntAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1.toShort) >= smallIntAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4.toShort) <= smallIntAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(smallIntAttr < 4.toShort), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(smallIntAttr < 2.toShort || smallIntAttr > 3.toShort, + classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4)))) } } test("filter pushdown - integer") { val data = (1 to 4).map(i => Tuple1(Option(i))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val intAttr = df(colName).expr - assert(df(colName).expr.dataType === IntegerType) - - checkFilterPredicate(intAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(intAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(intAttr === 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(intAttr <=> 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(intAttr =!= 1, classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(intAttr < 2, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(intAttr > 3, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(intAttr <= 1, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(intAttr >= 4, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(Literal(1) === intAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(1) <=> intAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(2) > intAttr, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(Literal(3) < intAttr, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(Literal(1) >= intAttr, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(Literal(4) <= intAttr, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(!(intAttr < 4), classOf[GtEq[_]], resultFun(4)) - checkFilterPredicate(intAttr < 2 || intAttr > 3, classOf[Operators.Or], - Seq(Row(resultFun(1)), Row(resultFun(4)))) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val intAttr = df(colName).expr + assert(df(colName).expr.dataType === IntegerType) + + checkFilterPredicate(intAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(intAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(intAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(intAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(intAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(intAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(intAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(intAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(intAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === intAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> intAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > intAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < intAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= intAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= intAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(intAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(intAttr < 2 || intAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) } } test("filter pushdown - long") { val data = (1 to 4).map(i => Tuple1(Option(i.toLong))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val longAttr = df(colName).expr - assert(df(colName).expr.dataType === LongType) - - checkFilterPredicate(longAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(longAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(longAttr === 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(longAttr <=> 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(longAttr =!= 1, classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(longAttr < 2, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(longAttr > 3, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(longAttr <= 1, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(longAttr >= 4, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(Literal(1) === longAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(1) <=> longAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(2) > longAttr, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(Literal(3) < longAttr, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(Literal(1) >= longAttr, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(Literal(4) <= longAttr, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(!(longAttr < 4), classOf[GtEq[_]], resultFun(4)) - checkFilterPredicate(longAttr < 2 || longAttr > 3, classOf[Operators.Or], - Seq(Row(resultFun(1)), Row(resultFun(4)))) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val longAttr = df(colName).expr + assert(df(colName).expr.dataType === LongType) + + checkFilterPredicate(longAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(longAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(longAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(longAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(longAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(longAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(longAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(longAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(longAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === longAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> longAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > longAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < longAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= longAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= longAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(longAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(longAttr < 2 || longAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) } } test("filter pushdown - float") { val data = (1 to 4).map(i => Tuple1(Option(i.toFloat))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val floatAttr = df(colName).expr - assert(df(colName).expr.dataType === FloatType) - - checkFilterPredicate(floatAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(floatAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(floatAttr === 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(floatAttr <=> 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(floatAttr =!= 1, classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(floatAttr < 2, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(floatAttr > 3, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(floatAttr <= 1, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(floatAttr >= 4, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(Literal(1) === floatAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(1) <=> floatAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(2) > floatAttr, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(Literal(3) < floatAttr, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(Literal(1) >= floatAttr, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(Literal(4) <= floatAttr, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(!(floatAttr < 4), classOf[GtEq[_]], resultFun(4)) - checkFilterPredicate(floatAttr < 2 || floatAttr > 3, classOf[Operators.Or], - Seq(Row(resultFun(1)), Row(resultFun(4)))) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val floatAttr = df(colName).expr + assert(df(colName).expr.dataType === FloatType) + + checkFilterPredicate(floatAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(floatAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(floatAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(floatAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(floatAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(floatAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(floatAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(floatAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(floatAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === floatAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> floatAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > floatAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < floatAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= floatAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= floatAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(floatAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(floatAttr < 2 || floatAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) } } test("filter pushdown - double") { val data = (1 to 4).map(i => Tuple1(Option(i.toDouble))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val doubleAttr = df(colName).expr - assert(df(colName).expr.dataType === DoubleType) - - checkFilterPredicate(doubleAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(doubleAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(doubleAttr === 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(doubleAttr <=> 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(doubleAttr =!= 1, classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(doubleAttr < 2, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(doubleAttr > 3, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(doubleAttr <= 1, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(doubleAttr >= 4, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(Literal(1) === doubleAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(1) <=> doubleAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(2) > doubleAttr, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(Literal(3) < doubleAttr, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(Literal(1) >= doubleAttr, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(Literal(4) <= doubleAttr, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(!(doubleAttr < 4), classOf[GtEq[_]], resultFun(4)) - checkFilterPredicate(doubleAttr < 2 || doubleAttr > 3, classOf[Operators.Or], - Seq(Row(resultFun(1)), Row(resultFun(4)))) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val doubleAttr = df(colName).expr + assert(df(colName).expr.dataType === DoubleType) + + checkFilterPredicate(doubleAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(doubleAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(doubleAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(doubleAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(doubleAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(doubleAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(doubleAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(doubleAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(doubleAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === doubleAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> doubleAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > doubleAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < doubleAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= doubleAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= doubleAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(doubleAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(doubleAttr < 2 || doubleAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) } } test("filter pushdown - string") { val data = (1 to 4).map(i => Tuple1(Option(i.toString))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val stringAttr = df(colName).expr - assert(df(colName).expr.dataType === StringType) - - checkFilterPredicate(stringAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(stringAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i.toString)))) - - checkFilterPredicate(stringAttr === "1", classOf[Eq[_]], resultFun("1")) - checkFilterPredicate(stringAttr <=> "1", classOf[Eq[_]], resultFun("1")) - checkFilterPredicate(stringAttr =!= "1", classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i.toString)))) - - checkFilterPredicate(stringAttr < "2", classOf[Lt[_]], resultFun("1")) - checkFilterPredicate(stringAttr > "3", classOf[Gt[_]], resultFun("4")) - checkFilterPredicate(stringAttr <= "1", classOf[LtEq[_]], resultFun("1")) - checkFilterPredicate(stringAttr >= "4", classOf[GtEq[_]], resultFun("4")) - - checkFilterPredicate(Literal("1") === stringAttr, classOf[Eq[_]], resultFun("1")) - checkFilterPredicate(Literal("1") <=> stringAttr, classOf[Eq[_]], resultFun("1")) - checkFilterPredicate(Literal("2") > stringAttr, classOf[Lt[_]], resultFun("1")) - checkFilterPredicate(Literal("3") < stringAttr, classOf[Gt[_]], resultFun("4")) - checkFilterPredicate(Literal("1") >= stringAttr, classOf[LtEq[_]], resultFun("1")) - checkFilterPredicate(Literal("4") <= stringAttr, classOf[GtEq[_]], resultFun("4")) - - checkFilterPredicate(!(stringAttr < "4"), classOf[GtEq[_]], resultFun("4")) - checkFilterPredicate(stringAttr < "2" || stringAttr > "3", classOf[Operators.Or], - Seq(Row(resultFun("1")), Row(resultFun("4")))) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val stringAttr = df(colName).expr + assert(df(colName).expr.dataType === StringType) + + checkFilterPredicate(stringAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(stringAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i.toString)))) + + checkFilterPredicate(stringAttr === "1", classOf[Eq[_]], resultFun("1")) + checkFilterPredicate(stringAttr <=> "1", classOf[Eq[_]], resultFun("1")) + checkFilterPredicate(stringAttr =!= "1", classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i.toString)))) + + checkFilterPredicate(stringAttr < "2", classOf[Lt[_]], resultFun("1")) + checkFilterPredicate(stringAttr > "3", classOf[Gt[_]], resultFun("4")) + checkFilterPredicate(stringAttr <= "1", classOf[LtEq[_]], resultFun("1")) + checkFilterPredicate(stringAttr >= "4", classOf[GtEq[_]], resultFun("4")) + + checkFilterPredicate(Literal("1") === stringAttr, classOf[Eq[_]], resultFun("1")) + checkFilterPredicate(Literal("1") <=> stringAttr, classOf[Eq[_]], resultFun("1")) + checkFilterPredicate(Literal("2") > stringAttr, classOf[Lt[_]], resultFun("1")) + checkFilterPredicate(Literal("3") < stringAttr, classOf[Gt[_]], resultFun("4")) + checkFilterPredicate(Literal("1") >= stringAttr, classOf[LtEq[_]], resultFun("1")) + checkFilterPredicate(Literal("4") <= stringAttr, classOf[GtEq[_]], resultFun("4")) + + checkFilterPredicate(!(stringAttr < "4"), classOf[GtEq[_]], resultFun("4")) + checkFilterPredicate(stringAttr < "2" || stringAttr > "3", classOf[Operators.Or], + Seq(Row(resultFun("1")), Row(resultFun("4")))) } } @@ -501,38 +508,37 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } val data = (1 to 4).map(i => Tuple1(Option(i.b))) - import testImplicits._ - withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val binaryAttr: Expression = df(colName).expr - assert(df(colName).expr.dataType === BinaryType) - - checkFilterPredicate(binaryAttr === 1.b, classOf[Eq[_]], resultFun(1.b)) - checkFilterPredicate(binaryAttr <=> 1.b, classOf[Eq[_]], resultFun(1.b)) - - checkFilterPredicate(binaryAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(binaryAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i.b)))) - - checkFilterPredicate(binaryAttr =!= 1.b, classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i.b)))) - - checkFilterPredicate(binaryAttr < 2.b, classOf[Lt[_]], resultFun(1.b)) - checkFilterPredicate(binaryAttr > 3.b, classOf[Gt[_]], resultFun(4.b)) - checkFilterPredicate(binaryAttr <= 1.b, classOf[LtEq[_]], resultFun(1.b)) - checkFilterPredicate(binaryAttr >= 4.b, classOf[GtEq[_]], resultFun(4.b)) - - checkFilterPredicate(Literal(1.b) === binaryAttr, classOf[Eq[_]], resultFun(1.b)) - checkFilterPredicate(Literal(1.b) <=> binaryAttr, classOf[Eq[_]], resultFun(1.b)) - checkFilterPredicate(Literal(2.b) > binaryAttr, classOf[Lt[_]], resultFun(1.b)) - checkFilterPredicate(Literal(3.b) < binaryAttr, classOf[Gt[_]], resultFun(4.b)) - checkFilterPredicate(Literal(1.b) >= binaryAttr, classOf[LtEq[_]], resultFun(1.b)) - checkFilterPredicate(Literal(4.b) <= binaryAttr, classOf[GtEq[_]], resultFun(4.b)) - - checkFilterPredicate(!(binaryAttr < 4.b), classOf[GtEq[_]], resultFun(4.b)) - checkFilterPredicate(binaryAttr < 2.b || binaryAttr > 3.b, classOf[Operators.Or], - Seq(Row(resultFun(1.b)), Row(resultFun(4.b)))) - } + withNestedParquetDataFrame(data) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val binaryAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === BinaryType) + + checkFilterPredicate(binaryAttr === 1.b, classOf[Eq[_]], resultFun(1.b)) + checkFilterPredicate(binaryAttr <=> 1.b, classOf[Eq[_]], resultFun(1.b)) + + checkFilterPredicate(binaryAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(binaryAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i.b)))) + + checkFilterPredicate(binaryAttr =!= 1.b, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i.b)))) + + checkFilterPredicate(binaryAttr < 2.b, classOf[Lt[_]], resultFun(1.b)) + checkFilterPredicate(binaryAttr > 3.b, classOf[Gt[_]], resultFun(4.b)) + checkFilterPredicate(binaryAttr <= 1.b, classOf[LtEq[_]], resultFun(1.b)) + checkFilterPredicate(binaryAttr >= 4.b, classOf[GtEq[_]], resultFun(4.b)) + + checkFilterPredicate(Literal(1.b) === binaryAttr, classOf[Eq[_]], resultFun(1.b)) + checkFilterPredicate(Literal(1.b) <=> binaryAttr, classOf[Eq[_]], resultFun(1.b)) + checkFilterPredicate(Literal(2.b) > binaryAttr, classOf[Lt[_]], resultFun(1.b)) + checkFilterPredicate(Literal(3.b) < binaryAttr, classOf[Gt[_]], resultFun(4.b)) + checkFilterPredicate(Literal(1.b) >= binaryAttr, classOf[LtEq[_]], resultFun(1.b)) + checkFilterPredicate(Literal(4.b) <= binaryAttr, classOf[GtEq[_]], resultFun(4.b)) + + checkFilterPredicate(!(binaryAttr < 4.b), classOf[GtEq[_]], resultFun(4.b)) + checkFilterPredicate(binaryAttr < 2.b || binaryAttr > 3.b, classOf[Operators.Or], + Seq(Row(resultFun(1.b)), Row(resultFun(4.b)))) } } @@ -546,56 +552,57 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared Seq(false, true).foreach { java8Api => withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { - val df = data.map(i => Tuple1(Date.valueOf(i))).toDF() - withNestedDataFrame(df) { case (inputDF, colName, fun) => + val dates = data.map(i => Tuple1(Date.valueOf(i))).toDF() + withNestedParquetDataFrame(dates) { case (inputDF, colName, fun) => + implicit val df: DataFrame = inputDF + def resultFun(dateStr: String): Any = { val parsed = if (java8Api) LocalDate.parse(dateStr) else Date.valueOf(dateStr) fun(parsed) } - withParquetDataFrame(inputDF) { implicit df => - val dateAttr: Expression = df(colName).expr - assert(df(colName).expr.dataType === DateType) - - checkFilterPredicate(dateAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(dateAttr.isNotNull, classOf[NotEq[_]], - data.map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(dateAttr === "2018-03-18".date, classOf[Eq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(dateAttr <=> "2018-03-18".date, classOf[Eq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(dateAttr =!= "2018-03-18".date, classOf[NotEq[_]], - Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(dateAttr < "2018-03-19".date, classOf[Lt[_]], - resultFun("2018-03-18")) - checkFilterPredicate(dateAttr > "2018-03-20".date, classOf[Gt[_]], - resultFun("2018-03-21")) - checkFilterPredicate(dateAttr <= "2018-03-18".date, classOf[LtEq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(dateAttr >= "2018-03-21".date, classOf[GtEq[_]], - resultFun("2018-03-21")) - - checkFilterPredicate(Literal("2018-03-18".date) === dateAttr, classOf[Eq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(Literal("2018-03-18".date) <=> dateAttr, classOf[Eq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(Literal("2018-03-19".date) > dateAttr, classOf[Lt[_]], - resultFun("2018-03-18")) - checkFilterPredicate(Literal("2018-03-20".date) < dateAttr, classOf[Gt[_]], - resultFun("2018-03-21")) - checkFilterPredicate(Literal("2018-03-18".date) >= dateAttr, classOf[LtEq[_]], - resultFun("2018-03-18")) - checkFilterPredicate(Literal("2018-03-21".date) <= dateAttr, classOf[GtEq[_]], - resultFun("2018-03-21")) - - checkFilterPredicate(!(dateAttr < "2018-03-21".date), classOf[GtEq[_]], - resultFun("2018-03-21")) - checkFilterPredicate( - dateAttr < "2018-03-19".date || dateAttr > "2018-03-20".date, - classOf[Operators.Or], - Seq(Row(resultFun("2018-03-18")), Row(resultFun("2018-03-21")))) - } + + val dateAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === DateType) + + checkFilterPredicate(dateAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(dateAttr.isNotNull, classOf[NotEq[_]], + data.map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(dateAttr === "2018-03-18".date, classOf[Eq[_]], + resultFun("2018-03-18")) + checkFilterPredicate(dateAttr <=> "2018-03-18".date, classOf[Eq[_]], + resultFun("2018-03-18")) + checkFilterPredicate(dateAttr =!= "2018-03-18".date, classOf[NotEq[_]], + Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(dateAttr < "2018-03-19".date, classOf[Lt[_]], + resultFun("2018-03-18")) + checkFilterPredicate(dateAttr > "2018-03-20".date, classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate(dateAttr <= "2018-03-18".date, classOf[LtEq[_]], + resultFun("2018-03-18")) + checkFilterPredicate(dateAttr >= "2018-03-21".date, classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate(Literal("2018-03-18".date) === dateAttr, classOf[Eq[_]], + resultFun("2018-03-18")) + checkFilterPredicate(Literal("2018-03-18".date) <=> dateAttr, classOf[Eq[_]], + resultFun("2018-03-18")) + checkFilterPredicate(Literal("2018-03-19".date) > dateAttr, classOf[Lt[_]], + resultFun("2018-03-18")) + checkFilterPredicate(Literal("2018-03-20".date) < dateAttr, classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate(Literal("2018-03-18".date) >= dateAttr, classOf[LtEq[_]], + resultFun("2018-03-18")) + checkFilterPredicate(Literal("2018-03-21".date) <= dateAttr, classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate(!(dateAttr < "2018-03-21".date), classOf[GtEq[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + dateAttr < "2018-03-19".date || dateAttr > "2018-03-20".date, + classOf[Operators.Or], + Seq(Row(resultFun("2018-03-18")), Row(resultFun("2018-03-21")))) } } } @@ -603,7 +610,9 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared test("filter pushdown - timestamp") { Seq(true, false).foreach { java8Api => - withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> "CORRECTED") { // spark.sql.parquet.outputTimestampType = TIMESTAMP_MILLIS val millisData = Seq( "1000-06-14 08:28:53.123", @@ -630,11 +639,14 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> ParquetOutputTimestampType.INT96.toString) { import testImplicits._ - withParquetDataFrame( - millisData.map(i => Tuple1(Timestamp.valueOf(i))).toDF()) { implicit df => - val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) - assertResult(None) { - createParquetFilters(schema).createFilter(sources.IsNull("_1")) + withTempPath { file => + millisData.map(i => Tuple1(Timestamp.valueOf(i))).toDF + .write.format(dataSourceName).save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath) { df => + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + assertResult(None) { + createParquetFilters(schema).createFilter(sources.IsNull("_1")) + } } } } @@ -653,36 +665,36 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared val rdd = spark.sparkContext.parallelize((1 to 4).map(i => Row(new java.math.BigDecimal(i)))) val dataFrame = spark.createDataFrame(rdd, StructType.fromDDL(s"a decimal($precision, 2)")) - withNestedDataFrame(dataFrame) { case (inputDF, colName, resultFun) => - withParquetDataFrame(inputDF) { implicit df => - val decimalAttr: Expression = df(colName).expr - assert(df(colName).expr.dataType === DecimalType(precision, 2)) - - checkFilterPredicate(decimalAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate(decimalAttr.isNotNull, classOf[NotEq[_]], - (1 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(decimalAttr === 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(decimalAttr <=> 1, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(decimalAttr =!= 1, classOf[NotEq[_]], - (2 to 4).map(i => Row.apply(resultFun(i)))) - - checkFilterPredicate(decimalAttr < 2, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(decimalAttr > 3, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(decimalAttr <= 1, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(decimalAttr >= 4, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(Literal(1) === decimalAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(1) <=> decimalAttr, classOf[Eq[_]], resultFun(1)) - checkFilterPredicate(Literal(2) > decimalAttr, classOf[Lt[_]], resultFun(1)) - checkFilterPredicate(Literal(3) < decimalAttr, classOf[Gt[_]], resultFun(4)) - checkFilterPredicate(Literal(1) >= decimalAttr, classOf[LtEq[_]], resultFun(1)) - checkFilterPredicate(Literal(4) <= decimalAttr, classOf[GtEq[_]], resultFun(4)) - - checkFilterPredicate(!(decimalAttr < 4), classOf[GtEq[_]], resultFun(4)) - checkFilterPredicate(decimalAttr < 2 || decimalAttr > 3, classOf[Operators.Or], - Seq(Row(resultFun(1)), Row(resultFun(4)))) - } + withNestedParquetDataFrame(dataFrame) { case (inputDF, colName, resultFun) => + implicit val df: DataFrame = inputDF + + val decimalAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === DecimalType(precision, 2)) + + checkFilterPredicate(decimalAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(decimalAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(decimalAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(decimalAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(decimalAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(decimalAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(decimalAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(decimalAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(decimalAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === decimalAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> decimalAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > decimalAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < decimalAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= decimalAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= decimalAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(decimalAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(decimalAttr < 2 || decimalAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) } } } @@ -1195,8 +1207,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } test("SPARK-16371 Do not push down filters when inner name and outer name are the same") { - import testImplicits._ - withParquetDataFrame((1 to 4).map(i => Tuple1(Tuple1(i))).toDF()) { implicit df => + withParquetDataFrame((1 to 4).map(i => Tuple1(Tuple1(i)))) { implicit df => // Here the schema becomes as below: // // root @@ -1336,10 +1347,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } test("filter pushdown - StringStartsWith") { - withParquetDataFrame { - import testImplicits._ - (1 to 4).map(i => Tuple1(i + "str" + i)).toDF() - } { implicit df => + withParquetDataFrame((1 to 4).map(i => Tuple1(i + "str" + i))) { implicit df => checkFilterPredicate( '_1.startsWith("").asInstanceOf[Predicate], classOf[UserDefinedByInstance[_, _]], @@ -1385,10 +1393,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } // SPARK-28371: make sure filter is null-safe. - withParquetDataFrame { - import testImplicits._ - Seq(Tuple1[String](null)).toDF() - } { implicit df => + withParquetDataFrame(Seq(Tuple1[String](null))) { implicit df => checkFilterPredicate( '_1.startsWith("blah").asInstanceOf[Predicate], classOf[UserDefinedByInstance[_, _]], @@ -1607,7 +1612,7 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { expected: Seq[Row]): Unit = { val output = predicate.collect { case a: Attribute => a }.distinct - Seq(("parquet", true), ("", false)).map { case (pushdownDsList, nestedPredicatePushdown) => + Seq(("parquet", true), ("", false)).foreach { case (pushdownDsList, nestedPredicatePushdown) => withSQLConf( SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", 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 79c32976f02ec..2dc8a062bb73d 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 @@ -85,7 +85,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession * Writes `data` to a Parquet file, reads it back and check file contents. */ protected def checkParquetFile[T <: Product : ClassTag: TypeTag](data: Seq[T]): Unit = { - withParquetDataFrame(data.toDF())(r => checkAnswer(r, data.map(Row.fromTuple))) + withParquetDataFrame(data)(r => checkAnswer(r, data.map(Row.fromTuple))) } test("basic data types (without binary)") { @@ -97,7 +97,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession test("raw binary") { val data = (1 to 4).map(i => Tuple1(Array.fill(3)(i.toByte))) - withParquetDataFrame(data.toDF()) { df => + withParquetDataFrame(data) { df => assertResult(data.map(_._1.mkString(",")).sorted) { df.collect().map(_.getAs[Array[Byte]](0).mkString(",")).sorted } @@ -200,7 +200,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession testStandardAndLegacyModes("struct") { val data = (1 to 4).map(i => Tuple1((i, s"val_$i"))) - withParquetDataFrame(data.toDF()) { df => + withParquetDataFrame(data) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(struct) => Row(Row(struct.productIterator.toSeq: _*)) @@ -217,7 +217,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession ) ) } - withParquetDataFrame(data.toDF()) { df => + withParquetDataFrame(data) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(array) => Row(array.map(struct => Row(struct.productIterator.toSeq: _*))) @@ -236,7 +236,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession ) ) } - withParquetDataFrame(data.toDF()) { df => + withParquetDataFrame(data) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(array) => Row(array.map { case Tuple1(Tuple1(str)) => Row(Row(str))}) @@ -246,7 +246,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession testStandardAndLegacyModes("nested struct with array of array as field") { val data = (1 to 4).map(i => Tuple1((i, Seq(Seq(s"val_$i"))))) - withParquetDataFrame(data.toDF()) { df => + withParquetDataFrame(data) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(struct) => Row(Row(struct.productIterator.toSeq: _*)) @@ -263,7 +263,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession ) ) } - withParquetDataFrame(data.toDF()) { df => + withParquetDataFrame(data) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(m) => Row(m.map { case (k, v) => Row(k.productIterator.toSeq: _*) -> v }) @@ -280,7 +280,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession ) ) } - withParquetDataFrame(data.toDF()) { df => + withParquetDataFrame(data) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(m) => Row(m.mapValues(struct => Row(struct.productIterator.toSeq: _*))) @@ -296,7 +296,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession null.asInstanceOf[java.lang.Float], null.asInstanceOf[java.lang.Double]) - withParquetDataFrame((allNulls :: Nil).toDF()) { df => + withParquetDataFrame(allNulls :: Nil) { df => val rows = df.collect() assert(rows.length === 1) assert(rows.head === Row(Seq.fill(5)(null): _*)) @@ -309,7 +309,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession None.asInstanceOf[Option[Long]], None.asInstanceOf[Option[String]]) - withParquetDataFrame((allNones :: Nil).toDF()) { df => + withParquetDataFrame(allNones :: Nil) { df => val rows = df.collect() assert(rows.length === 1) assert(rows.head === Row(Seq.fill(3)(null): _*)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index 105f025adc0ad..db8ee724c01c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -63,18 +63,12 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest { (f: String => Unit): Unit = withDataSourceFile(data)(f) /** - * Writes `df` dataframe to a Parquet file and reads it back as a [[DataFrame]], + * Writes `data` to a Parquet file and reads it back as a [[DataFrame]], * which is then passed to `f`. The Parquet file will be deleted after `f` returns. */ - protected def withParquetDataFrame(df: DataFrame, testVectorized: Boolean = true) - (f: DataFrame => Unit): Unit = { - withTempPath { file => - withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> "CORRECTED") { - df.write.format(dataSourceName).save(file.getCanonicalPath) - } - readFile(file.getCanonicalPath, testVectorized)(f) - } - } + protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag] + (data: Seq[T], testVectorized: Boolean = true) + (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f) /** * Writes `data` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a From 02f3b80d3a277e0c19a66c28d935fa41da7b3307 Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Tue, 30 Jun 2020 23:33:50 -0700 Subject: [PATCH 101/384] [SPARK-32131][SQL] Fix AnalysisException messages at UNION/EXCEPT/MINUS operations ### What changes were proposed in this pull request? fix error exception messages during exceptions on Union and set operations ### Why are the changes needed? Union and set operations can only be performed on tables with the compatible column types,while when we have more than two column, the exception messages will have wrong column index. Steps to reproduce: ``` drop table if exists test1; drop table if exists test2; drop table if exists test3; create table if not exists test1(id int, age int, name timestamp); create table if not exists test2(id int, age timestamp, name timestamp); create table if not exists test3(id int, age int, name int); insert into test1 select 1,2,'2020-01-01 01:01:01'; insert into test2 select 1,'2020-01-01 01:01:01','2020-01-01 01:01:01'; insert into test3 select 1,3,4; ``` Query1: ```sql select * from test1 except select * from test2; ``` Result1: ``` Error: org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types. timestamp <> int at the second column of the second table;; 'Except false :- Project [id#620, age#621, name#622] : +- SubqueryAlias `default`.`test1` : +- HiveTableRelation `default`.`test1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#620, age#621, name#622] +- Project [id#623, age#624, name#625] +- SubqueryAlias `default`.`test2` +- HiveTableRelation `default`.`test2`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#623, age#624, name#625] (state=,code=0) ``` Query2: ```sql select * from test1 except select * from test3; ``` Result2: ``` Error: org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types int <> timestamp at the 2th column of the second table; ``` the above query1 has the right exception message the above query2 have the wrong errors information, it may need to change to the following ``` Error: org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types. int <> timestamp at the third column of the second table ``` ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? unit test Closes #28951 from GuoPhilipse/32131-correct-error-messages. Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Co-authored-by: GuoPhilipse Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/CheckAnalysis.scala | 3 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 53 +++++++++++++++++++ 2 files changed, 55 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 9e325d0c2e4e1..9c99acaa994b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -337,7 +337,8 @@ trait CheckAnalysis extends PredicateHelper { def ordinalNumber(i: Int): String = i match { case 0 => "first" case 1 => "second" - case i => s"${i}th" + case 2 => "third" + case i => s"${i + 1}th" } val ref = dataTypes(operator.children.head) operator.children.tail.zipWithIndex.foreach { case (child, ti) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 189152374b0d1..c15ec49e14282 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -831,4 +831,57 @@ class AnalysisSuite extends AnalysisTest with Matchers { } } } + + test("SPARK-32131: Fix wrong column index when we have more than two columns" + + " during union and set operations" ) { + val firstTable = LocalRelation( + AttributeReference("a", StringType)(), + AttributeReference("b", DoubleType)(), + AttributeReference("c", IntegerType)(), + AttributeReference("d", FloatType)()) + + val secondTable = LocalRelation( + AttributeReference("a", StringType)(), + AttributeReference("b", TimestampType)(), + AttributeReference("c", IntegerType)(), + AttributeReference("d", FloatType)()) + + val thirdTable = LocalRelation( + AttributeReference("a", StringType)(), + AttributeReference("b", DoubleType)(), + AttributeReference("c", TimestampType)(), + AttributeReference("d", FloatType)()) + + val fourthTable = LocalRelation( + AttributeReference("a", StringType)(), + AttributeReference("b", DoubleType)(), + AttributeReference("c", IntegerType)(), + AttributeReference("d", TimestampType)()) + + val r1 = Union(firstTable, secondTable) + val r2 = Union(firstTable, thirdTable) + val r3 = Union(firstTable, fourthTable) + val r4 = Except(firstTable, secondTable, isAll = false) + val r5 = Intersect(firstTable, secondTable, isAll = false) + + assertAnalysisError(r1, + Seq("Union can only be performed on tables with the compatible column types. " + + "timestamp <> double at the second column of the second table")) + + assertAnalysisError(r2, + Seq("Union can only be performed on tables with the compatible column types. " + + "timestamp <> int at the third column of the second table")) + + assertAnalysisError(r3, + Seq("Union can only be performed on tables with the compatible column types. " + + "timestamp <> float at the 4th column of the second table")) + + assertAnalysisError(r4, + Seq("Except can only be performed on tables with the compatible column types. " + + "timestamp <> double at the second column of the second table")) + + assertAnalysisError(r5, + Seq("Intersect can only be performed on tables with the compatible column types. " + + "timestamp <> double at the second column of the second table")) + } } From a4ba34434c55c7f0eafa32d8c7ea6e052813f293 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Wed, 1 Jul 2020 07:13:10 +0000 Subject: [PATCH 102/384] [SPARK-32095][SQL] Update documentation to reflect usage of updated statistics ### What changes were proposed in this pull request? Update documentation to reflect changes in https://github.com/apache/spark/commit/faf220aad9051c224a630e678c54098861f6b482 I've changed the documentation to reflect updated statistics may be used to improve query plan. ### Why are the changes needed? I believe the documentation is stale and misleading. ### Does this PR introduce _any_ user-facing change? Yes, this is a javadoc documentation fix. ### How was this patch tested? Doc fix. Closes #28925 from emkornfield/spark-32095. Authored-by: Micah Kornfield Signed-off-by: Wenchen Fan --- .../spark/sql/connector/read/SupportsReportStatistics.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java index b839fd5a4a726..1e0c9ca7c7e4b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java @@ -23,9 +23,9 @@ * A mix in interface for {@link Scan}. Data sources can implement this interface to * report statistics to Spark. * - * As of Spark 2.4, statistics are reported to the optimizer before any operator is pushed to the - * data source. Implementations that return more accurate statistics based on pushed operators will - * not improve query performance until the planner can push operators before getting stats. + * As of Spark 3.0, statistics are reported to the optimizer after operators are pushed to the + * data source. Implementations may return more accurate statistics based on pushed operators + * which may improve query performance by providing better information to the optimizer. * * @since 3.0.0 */ From 15fb5d76777347fc328191321c9712adb1ee7b64 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 1 Jul 2020 12:00:15 +0000 Subject: [PATCH 103/384] [SPARK-28169][SQL] Convert scan predicate condition to CNF ### What changes were proposed in this pull request? Spark can't push down scan predicate condition of **Or**: Such as if I have a table `default.test`, it's partition col is `dt`, If we use query : ``` select * from default.test where dt=20190625 or (dt = 20190626 and id in (1,2,3) ) ``` In this case, Spark will resolve **Or** condition as one expression, and since this expr has reference of "id", then it can't been push down. Base on pr https://github.com/apache/spark/pull/28733, In my PR , for SQL like `select * from default.test` `where dt = 20190626 or (dt = 20190627 and xxx="a") ` For this condition `dt = 20190626 or (dt = 20190627 and xxx="a" )`, it will been converted to CNF ``` (dt = 20190626 or dt = 20190627) and (dt = 20190626 or xxx = "a" ) ``` then condition `dt = 20190626 or dt = 20190627` will be push down when partition pruning ### Why are the changes needed? Optimize partition pruning ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? Added UT Closes #28805 from AngersZhuuuu/cnf-for-partition-pruning. Lead-authored-by: angerszhu Co-authored-by: AngersZhuuuu Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/predicates.scala | 40 ++++++++-- .../PushCNFPredicateThroughJoin.scala | 2 +- .../ConjunctiveNormalFormPredicateSuite.scala | 10 +-- .../sql/execution/DataSourceScanExec.scala | 2 +- .../PruneFileSourcePartitions.scala | 9 ++- .../execution/PruneHiveTablePartitions.scala | 10 ++- .../PruneFileSourcePartitionsSuite.scala | 14 +++- .../PruneHiveTablePartitionsSuite.scala | 16 ++-- .../execution/PrunePartitionSuiteBase.scala | 76 +++++++++++++++++++ 9 files changed, 152 insertions(+), 27 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PrunePartitionSuiteBase.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 05a5ff45b8fb1..527618b8e2c5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -211,7 +211,9 @@ trait PredicateHelper extends Logging { * @return the CNF result as sequence of disjunctive expressions. If the number of expressions * exceeds threshold on converting `Or`, `Seq.empty` is returned. */ - protected def conjunctiveNormalForm(condition: Expression): Seq[Expression] = { + protected def conjunctiveNormalForm( + condition: Expression, + groupExpsFunc: Seq[Expression] => Seq[Expression]): Seq[Expression] = { val postOrderNodes = postOrderTraversal(condition) val resultStack = new mutable.Stack[Seq[Expression]] val maxCnfNodeCount = SQLConf.get.maxCnfNodeCount @@ -226,8 +228,8 @@ trait PredicateHelper extends Logging { // For each side, there is no need to expand predicates of the same references. // So here we can aggregate predicates of the same qualifier as one single predicate, // for reducing the size of pushed down predicates and corresponding codegen. - val right = groupExpressionsByQualifier(resultStack.pop()) - val left = groupExpressionsByQualifier(resultStack.pop()) + val right = groupExpsFunc(resultStack.pop()) + val left = groupExpsFunc(resultStack.pop()) // Stop the loop whenever the result exceeds the `maxCnfNodeCount` if (left.size * right.size > maxCnfNodeCount) { logInfo(s"As the result size exceeds the threshold $maxCnfNodeCount. " + @@ -249,8 +251,36 @@ trait PredicateHelper extends Logging { resultStack.top } - private def groupExpressionsByQualifier(expressions: Seq[Expression]): Seq[Expression] = { - expressions.groupBy(_.references.map(_.qualifier)).map(_._2.reduceLeft(And)).toSeq + /** + * Convert an expression to conjunctive normal form when pushing predicates through Join, + * when expand predicates, we can group by the qualifier avoiding generate unnecessary + * expression to control the length of final result since there are multiple tables. + * + * @param condition condition need to be converted + * @return the CNF result as sequence of disjunctive expressions. If the number of expressions + * exceeds threshold on converting `Or`, `Seq.empty` is returned. + */ + def CNFWithGroupExpressionsByQualifier(condition: Expression): Seq[Expression] = { + conjunctiveNormalForm(condition, (expressions: Seq[Expression]) => + expressions.groupBy(_.references.map(_.qualifier)).map(_._2.reduceLeft(And)).toSeq) + } + + /** + * Convert an expression to conjunctive normal form for predicate pushdown and partition pruning. + * When expanding predicates, this method groups expressions by their references for reducing + * the size of pushed down predicates and corresponding codegen. In partition pruning strategies, + * we split filters by [[splitConjunctivePredicates]] and partition filters by judging if it's + * references is subset of partCols, if we combine expressions group by reference when expand + * predicate of [[Or]], it won't impact final predicate pruning result since + * [[splitConjunctivePredicates]] won't split [[Or]] expression. + * + * @param condition condition need to be converted + * @return the CNF result as sequence of disjunctive expressions. If the number of expressions + * exceeds threshold on converting `Or`, `Seq.empty` is returned. + */ + def CNFWithGroupExpressionsByReference(condition: Expression): Seq[Expression] = { + conjunctiveNormalForm(condition, (expressions: Seq[Expression]) => + expressions.groupBy(e => AttributeSet(e.references)).map(_._2.reduceLeft(And)).toSeq) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala index 109e5f993c02e..47e9527ead7c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala @@ -38,7 +38,7 @@ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelpe def apply(plan: LogicalPlan): LogicalPlan = plan transform { case j @ Join(left, right, joinType, Some(joinCondition), hint) if canPushThrough(joinType) => - val predicates = conjunctiveNormalForm(joinCondition) + val predicates = CNFWithGroupExpressionsByQualifier(joinCondition) if (predicates.isEmpty) { j } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala index b449ed5cc0d07..793abccd79405 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala @@ -43,7 +43,7 @@ class ConjunctiveNormalFormPredicateSuite extends SparkFunSuite with PredicateHe // Check CNF conversion with expected expression, assuming the input has non-empty result. private def checkCondition(input: Expression, expected: Expression): Unit = { - val cnf = conjunctiveNormalForm(input) + val cnf = CNFWithGroupExpressionsByQualifier(input) assert(cnf.nonEmpty) val result = cnf.reduceLeft(And) assert(result.semanticEquals(expected)) @@ -113,14 +113,14 @@ class ConjunctiveNormalFormPredicateSuite extends SparkFunSuite with PredicateHe Seq(8, 9, 10, 35, 36, 37).foreach { maxCount => withSQLConf(SQLConf.MAX_CNF_NODE_COUNT.key -> maxCount.toString) { if (maxCount < 36) { - assert(conjunctiveNormalForm(input).isEmpty) + assert(CNFWithGroupExpressionsByQualifier(input).isEmpty) } else { - assert(conjunctiveNormalForm(input).nonEmpty) + assert(CNFWithGroupExpressionsByQualifier(input).nonEmpty) } if (maxCount < 9) { - assert(conjunctiveNormalForm(input2).isEmpty) + assert(CNFWithGroupExpressionsByQualifier(input2).isEmpty) } else { - assert(conjunctiveNormalForm(input2).nonEmpty) + assert(CNFWithGroupExpressionsByQualifier(input2).nonEmpty) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 458e11b97db6b..78808ff21394c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -207,7 +207,7 @@ case class FileSourceScanExec( private def isDynamicPruningFilter(e: Expression): Boolean = e.find(_.isInstanceOf[PlanExpression[_]]).isDefined - @transient private lazy val selectedPartitions: Array[PartitionDirectory] = { + @transient lazy val selectedPartitions: Array[PartitionDirectory] = { val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) val startTime = System.nanoTime() val ret = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index a7129fb14d1a6..576a826faf894 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -39,7 +39,8 @@ import org.apache.spark.sql.types.StructType * its underlying [[FileScan]]. And the partition filters will be removed in the filters of * returned logical plan. */ -private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { +private[sql] object PruneFileSourcePartitions + extends Rule[LogicalPlan] with PredicateHelper { private def getPartitionKeyFiltersAndDataFilters( sparkSession: SparkSession, @@ -87,8 +88,12 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { _, _)) if filters.nonEmpty && fsRelation.partitionSchemaOption.isDefined => + val predicates = CNFWithGroupExpressionsByReference(filters.reduceLeft(And)) + val finalPredicates = if (predicates.nonEmpty) predicates else filters val (partitionKeyFilters, _) = getPartitionKeyFiltersAndDataFilters( - fsRelation.sparkSession, logicalRelation, partitionSchema, filters, logicalRelation.output) + fsRelation.sparkSession, logicalRelation, partitionSchema, finalPredicates, + logicalRelation.output) + if (partitionKeyFilters.nonEmpty) { val prunedFileIndex = catalogFileIndex.filterPartitions(partitionKeyFilters.toSeq) val prunedFsRelation = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala index da6e4c52cf3a7..c4885f2842597 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala @@ -21,8 +21,8 @@ import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.CastSupport -import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTable, CatalogTablePartition, ExternalCatalogUtils, HiveTableRelation} -import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, ExpressionSet, SubqueryExpression} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, ExpressionSet, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule @@ -41,7 +41,7 @@ import org.apache.spark.sql.internal.SQLConf * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source. */ private[sql] class PruneHiveTablePartitions(session: SparkSession) - extends Rule[LogicalPlan] with CastSupport { + extends Rule[LogicalPlan] with CastSupport with PredicateHelper { override val conf: SQLConf = session.sessionState.conf @@ -103,7 +103,9 @@ private[sql] class PruneHiveTablePartitions(session: SparkSession) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case op @ PhysicalOperation(projections, filters, relation: HiveTableRelation) if filters.nonEmpty && relation.isPartitioned && relation.prunedPartitions.isEmpty => - val partitionKeyFilters = getPartitionKeyFilters(filters, relation) + val predicates = CNFWithGroupExpressionsByReference(filters.reduceLeft(And)) + val finalPredicates = if (predicates.nonEmpty) predicates else filters + val partitionKeyFilters = getPartitionKeyFilters(finalPredicates, relation) if (partitionKeyFilters.nonEmpty) { val newPartitions = prunePartitions(relation, partitionKeyFilters) val newTableMeta = updateTableMeta(relation.tableMeta, newPartitions) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index c9c36992906a8..24aecb0274ece 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -19,22 +19,22 @@ package org.apache.spark.sql.hive.execution import org.scalatest.Matchers._ -import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.functions.broadcast -import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType -class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { +class PruneFileSourcePartitionsSuite extends PrunePartitionSuiteBase { + + override def format: String = "parquet" object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("PruneFileSourcePartitions", Once, PruneFileSourcePartitions) :: Nil @@ -108,4 +108,10 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te } } } + + override def getScanExecPartitionSize(plan: SparkPlan): Long = { + plan.collectFirst { + case p: FileSourceScanExec => p + }.get.selectedPartitions.length + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala index e41709841a736..c29e889c3a941 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.execution.SparkPlan -class PruneHiveTablePartitionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { +class PruneHiveTablePartitionsSuite extends PrunePartitionSuiteBase { + + override def format(): String = "hive" object Optimize extends RuleExecutor[LogicalPlan] { val batches = @@ -32,7 +32,7 @@ class PruneHiveTablePartitionsSuite extends QueryTest with SQLTestUtils with Tes EliminateSubqueryAliases, new PruneHiveTablePartitions(spark)) :: Nil } - test("SPARK-15616 statistics pruned after going throuhg PruneHiveTablePartitions") { + test("SPARK-15616: statistics pruned after going through PruneHiveTablePartitions") { withTable("test", "temp") { sql( s""" @@ -54,4 +54,10 @@ class PruneHiveTablePartitionsSuite extends QueryTest with SQLTestUtils with Tes Optimize.execute(analyzed2).stats.sizeInBytes) } } + + override def getScanExecPartitionSize(plan: SparkPlan): Long = { + plan.collectFirst { + case p: HiveTableScanExec => p + }.get.prunedPartitions.size + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PrunePartitionSuiteBase.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PrunePartitionSuiteBase.scala new file mode 100644 index 0000000000000..d088061cdc6e5 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PrunePartitionSuiteBase.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils + +abstract class PrunePartitionSuiteBase extends QueryTest with SQLTestUtils with TestHiveSingleton { + + protected def format: String + + test("SPARK-28169: Convert scan predicate condition to CNF") { + withTempView("temp") { + withTable("t") { + sql( + s""" + |CREATE TABLE t(i INT, p STRING) + |USING $format + |PARTITIONED BY (p)""".stripMargin) + + spark.range(0, 1000, 1).selectExpr("id as col") + .createOrReplaceTempView("temp") + + for (part <- Seq(1, 2, 3, 4)) { + sql( + s""" + |INSERT OVERWRITE TABLE t PARTITION (p='$part') + |SELECT col FROM temp""".stripMargin) + } + + assertPrunedPartitions( + "SELECT * FROM t WHERE p = '1' OR (p = '2' AND i = 1)", 2) + assertPrunedPartitions( + "SELECT * FROM t WHERE (p = '1' AND i = 2) OR (i = 1 OR p = '2')", 4) + assertPrunedPartitions( + "SELECT * FROM t WHERE (p = '1' AND i = 2) OR (p = '3' AND i = 3 )", 2) + assertPrunedPartitions( + "SELECT * FROM t WHERE (p = '1' AND i = 2) OR (p = '2' OR p = '3')", 3) + assertPrunedPartitions( + "SELECT * FROM t", 4) + assertPrunedPartitions( + "SELECT * FROM t WHERE p = '1' AND i = 2", 1) + assertPrunedPartitions( + """ + |SELECT i, COUNT(1) FROM ( + |SELECT * FROM t WHERE p = '1' OR (p = '2' AND i = 1) + |) tmp GROUP BY i + """.stripMargin, 2) + } + } + } + + protected def assertPrunedPartitions(query: String, expected: Long): Unit = { + val plan = sql(query).queryExecution.sparkPlan + assert(getScanExecPartitionSize(plan) == expected) + } + + protected def getScanExecPartitionSize(plan: SparkPlan): Long +} From f7d9e3d16288305cf020ec8961788c6348bc652c Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 1 Jul 2020 08:09:07 -0500 Subject: [PATCH 104/384] [SPARK-23631][ML][PYSPARK] Add summary to RandomForestClassificationModel ### What changes were proposed in this pull request? Add summary to RandomForestClassificationModel... ### Why are the changes needed? so user can get a summary of this classification model, and retrieve common metrics such as accuracy, weightedTruePositiveRate, roc (for binary), pr curves (for binary), etc. ### Does this PR introduce _any_ user-facing change? Yes ``` RandomForestClassificationModel.summary RandomForestClassificationModel.evaluate ``` ### How was this patch tested? Add new tests Closes #28913 from huaxingao/rf_summary. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../ClassificationSummary.scala | 2 +- .../spark/ml/classification/Classifier.scala | 23 +++ .../spark/ml/classification/LinearSVC.scala | 21 -- .../classification/LogisticRegression.scala | 29 +-- .../ProbabilisticClassifier.scala | 22 +++ .../RandomForestClassifier.scala | 181 +++++++++++++++++- .../LogisticRegressionSuite.scala | 2 +- .../RandomForestClassifierSuite.scala | 110 +++++++++++ python/pyspark/ml/classification.py | 79 +++++++- .../pyspark/ml/tests/test_training_summary.py | 79 +++++++- 10 files changed, 496 insertions(+), 52 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala index e9ea38161d3c0..9f3428db484c2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala @@ -44,7 +44,7 @@ private[classification] trait ClassificationSummary extends Serializable { @Since("3.1.0") def labelCol: String - /** Field in "predictions" which gives the weight of each instance as a vector. */ + /** Field in "predictions" which gives the weight of each instance. */ @Since("3.1.0") def weightCol: String diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 1f3f291644f93..233e8e5bcdc88 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -22,6 +22,7 @@ import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util.{MetadataUtils, SchemaUtils} import org.apache.spark.rdd.RDD @@ -269,4 +270,26 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * @return predicted label */ protected def raw2prediction(rawPrediction: Vector): Double = rawPrediction.argmax + + /** + * If the rawPrediction and prediction columns are set, this method returns the current model, + * otherwise it generates new columns for them and sets them as columns on a new copy of + * the current model + */ + private[classification] def findSummaryModel(): + (ClassificationModel[FeaturesType, M], String, String) = { + val model = if ($(rawPredictionCol).isEmpty && $(predictionCol).isEmpty) { + copy(ParamMap.empty) + .setRawPredictionCol("rawPrediction_" + java.util.UUID.randomUUID.toString) + .setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) + } else if ($(rawPredictionCol).isEmpty) { + copy(ParamMap.empty).setRawPredictionCol("rawPrediction_" + + java.util.UUID.randomUUID.toString) + } else if ($(predictionCol).isEmpty) { + copy(ParamMap.empty).setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) + } else { + this + } + (model, model.getRawPredictionCol, model.getPredictionCol) + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 1659bbb1d34b3..4adc527c89b36 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -394,27 +394,6 @@ class LinearSVCModel private[classification] ( @Since("3.1.0") override def summary: LinearSVCTrainingSummary = super.summary - /** - * If the rawPrediction and prediction columns are set, this method returns the current model, - * otherwise it generates new columns for them and sets them as columns on a new copy of - * the current model - */ - private[classification] def findSummaryModel(): (LinearSVCModel, String, String) = { - val model = if ($(rawPredictionCol).isEmpty && $(predictionCol).isEmpty) { - copy(ParamMap.empty) - .setRawPredictionCol("rawPrediction_" + java.util.UUID.randomUUID.toString) - .setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) - } else if ($(rawPredictionCol).isEmpty) { - copy(ParamMap.empty).setRawPredictionCol("rawPrediction_" + - java.util.UUID.randomUUID.toString) - } else if ($(predictionCol).isEmpty) { - copy(ParamMap.empty).setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) - } else { - this - } - (model, model.getRawPredictionCol, model.getPredictionCol) - } - /** * Evaluates the model on a test dataset. * diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 20d619334f7b9..47b3e2de7695c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -1158,27 +1158,6 @@ class LogisticRegressionModel private[spark] ( s"(numClasses=${numClasses}), use summary instead.") } - /** - * If the probability and prediction columns are set, this method returns the current model, - * otherwise it generates new columns for them and sets them as columns on a new copy of - * the current model - */ - private[classification] def findSummaryModel(): - (LogisticRegressionModel, String, String) = { - val model = if ($(probabilityCol).isEmpty && $(predictionCol).isEmpty) { - copy(ParamMap.empty) - .setProbabilityCol("probability_" + java.util.UUID.randomUUID.toString) - .setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) - } else if ($(probabilityCol).isEmpty) { - copy(ParamMap.empty).setProbabilityCol("probability_" + java.util.UUID.randomUUID.toString) - } else if ($(predictionCol).isEmpty) { - copy(ParamMap.empty).setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) - } else { - this - } - (model, model.getProbabilityCol, model.getPredictionCol) - } - /** * Evaluates the model on a test dataset. * @@ -1451,7 +1430,7 @@ sealed trait BinaryLogisticRegressionTrainingSummary extends BinaryLogisticRegre * double. * @param labelCol field in "predictions" which gives the true label of each instance. * @param featuresCol field in "predictions" which gives the features of each instance as a vector. - * @param weightCol field in "predictions" which gives the weight of each instance as a vector. + * @param weightCol field in "predictions" which gives the weight of each instance. * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. */ private class LogisticRegressionTrainingSummaryImpl( @@ -1476,7 +1455,7 @@ private class LogisticRegressionTrainingSummaryImpl( * double. * @param labelCol field in "predictions" which gives the true label of each instance. * @param featuresCol field in "predictions" which gives the features of each instance as a vector. - * @param weightCol field in "predictions" which gives the weight of each instance as a vector. + * @param weightCol field in "predictions" which gives the weight of each instance. */ private class LogisticRegressionSummaryImpl( @transient override val predictions: DataFrame, @@ -1497,7 +1476,7 @@ private class LogisticRegressionSummaryImpl( * double. * @param labelCol field in "predictions" which gives the true label of each instance. * @param featuresCol field in "predictions" which gives the features of each instance as a vector. - * @param weightCol field in "predictions" which gives the weight of each instance as a vector. + * @param weightCol field in "predictions" which gives the weight of each instance. * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. */ private class BinaryLogisticRegressionTrainingSummaryImpl( @@ -1522,7 +1501,7 @@ private class BinaryLogisticRegressionTrainingSummaryImpl( * each class as a double. * @param labelCol field in "predictions" which gives the true label of each instance. * @param featuresCol field in "predictions" which gives the features of each instance as a vector. - * @param weightCol field in "predictions" which gives the weight of each instance as a vector. + * @param weightCol field in "predictions" which gives the weight of each instance. */ private class BinaryLogisticRegressionSummaryImpl( predictions: DataFrame, diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 9758e3ca72c38..1caaeccd7b0d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.{DenseVector, Vector, VectorUDT} +import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.sql.{DataFrame, Dataset} @@ -229,6 +230,27 @@ abstract class ProbabilisticClassificationModel[ argMax } } + + /** + *If the probability and prediction columns are set, this method returns the current model, + * otherwise it generates new columns for them and sets them as columns on a new copy of + * the current model + */ + override private[classification] def findSummaryModel(): + (ProbabilisticClassificationModel[FeaturesType, M], String, String) = { + val model = if ($(probabilityCol).isEmpty && $(predictionCol).isEmpty) { + copy(ParamMap.empty) + .setProbabilityCol("probability_" + java.util.UUID.randomUUID.toString) + .setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) + } else if ($(probabilityCol).isEmpty) { + copy(ParamMap.empty).setProbabilityCol("probability_" + java.util.UUID.randomUUID.toString) + } else if ($(predictionCol).isEmpty) { + copy(ParamMap.empty).setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) + } else { + this + } + (model, model.getProbabilityCol, model.getPredictionCol) + } } private[ml] object ProbabilisticClassificationModel { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index a316e472d9674..f9ce62b91924b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -166,7 +166,35 @@ class RandomForestClassifier @Since("1.4.0") ( val numFeatures = trees.head.numFeatures instr.logNumClasses(numClasses) instr.logNumFeatures(numFeatures) - new RandomForestClassificationModel(uid, trees, numFeatures, numClasses) + createModel(dataset, trees, numFeatures, numClasses) + } + + private def createModel( + dataset: Dataset[_], + trees: Array[DecisionTreeClassificationModel], + numFeatures: Int, + numClasses: Int): RandomForestClassificationModel = { + val model = copyValues(new RandomForestClassificationModel(uid, trees, numFeatures, numClasses)) + val weightColName = if (!isDefined(weightCol)) "weightCol" else $(weightCol) + + val (summaryModel, probabilityColName, predictionColName) = model.findSummaryModel() + val rfSummary = if (numClasses <= 2) { + new BinaryRandomForestClassificationTrainingSummaryImpl( + summaryModel.transform(dataset), + probabilityColName, + predictionColName, + $(labelCol), + weightColName, + Array(0.0)) + } else { + new RandomForestClassificationTrainingSummaryImpl( + summaryModel.transform(dataset), + predictionColName, + $(labelCol), + weightColName, + Array(0.0)) + } + model.setSummary(Some(rfSummary)) } @Since("1.4.1") @@ -204,7 +232,8 @@ class RandomForestClassificationModel private[ml] ( @Since("1.5.0") override val numClasses: Int) extends ProbabilisticClassificationModel[Vector, RandomForestClassificationModel] with RandomForestClassifierParams with TreeEnsembleModel[DecisionTreeClassificationModel] - with MLWritable with Serializable { + with MLWritable with Serializable + with HasTrainingSummary[RandomForestClassificationTrainingSummary] { require(_trees.nonEmpty, "RandomForestClassificationModel requires at least 1 tree.") @@ -228,6 +257,44 @@ class RandomForestClassificationModel private[ml] ( @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights + /** + * Gets summary of model on training set. An exception is thrown + * if `hasSummary` is false. + */ + @Since("3.1.0") + override def summary: RandomForestClassificationTrainingSummary = super.summary + + /** + * Gets summary of model on training set. An exception is thrown + * if `hasSummary` is false or it is a multiclass model. + */ + @Since("3.1.0") + def binarySummary: BinaryRandomForestClassificationTrainingSummary = summary match { + case b: BinaryRandomForestClassificationTrainingSummary => b + case _ => + throw new RuntimeException("Cannot create a binary summary for a non-binary model" + + s"(numClasses=${numClasses}), use summary instead.") + } + + /** + * Evaluates the model on a test dataset. + * + * @param dataset Test dataset to evaluate model on. + */ + @Since("3.1.0") + def evaluate(dataset: Dataset[_]): RandomForestClassificationSummary = { + val weightColName = if (!isDefined(weightCol)) "weightCol" else $(weightCol) + // Handle possible missing or invalid prediction columns + val (summaryModel, probabilityColName, predictionColName) = findSummaryModel() + if (numClasses > 2) { + new RandomForestClassificationSummaryImpl(summaryModel.transform(dataset), + predictionColName, $(labelCol), weightColName) + } else { + new BinaryRandomForestClassificationSummaryImpl(summaryModel.transform(dataset), + probabilityColName, predictionColName, $(labelCol), weightColName) + } + } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { var outputSchema = super.transformSchema(schema) @@ -388,3 +455,113 @@ object RandomForestClassificationModel extends MLReadable[RandomForestClassifica new RandomForestClassificationModel(uid, newTrees, numFeatures, numClasses) } } + +/** + * Abstraction for multiclass RandomForestClassification results for a given model. + */ +sealed trait RandomForestClassificationSummary extends ClassificationSummary { + /** + * Convenient method for casting to BinaryRandomForestClassificationSummary. + * This method will throw an Exception if the summary is not a binary summary. + */ + @Since("3.1.0") + def asBinary: BinaryRandomForestClassificationSummary = this match { + case b: BinaryRandomForestClassificationSummary => b + case _ => + throw new RuntimeException("Cannot cast to a binary summary.") + } +} + +/** + * Abstraction for multiclass RandomForestClassification training results. + */ +sealed trait RandomForestClassificationTrainingSummary extends RandomForestClassificationSummary + with TrainingSummary + +/** + * Abstraction for BinaryRandomForestClassification results for a given model. + */ +sealed trait BinaryRandomForestClassificationSummary extends BinaryClassificationSummary + +/** + * Abstraction for BinaryRandomForestClassification training results. + */ +sealed trait BinaryRandomForestClassificationTrainingSummary extends + BinaryRandomForestClassificationSummary with RandomForestClassificationTrainingSummary + +/** + * Multiclass RandomForestClassification training results. + * + * @param predictions dataframe output by the model's `transform` method. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. + */ +private class RandomForestClassificationTrainingSummaryImpl( + predictions: DataFrame, + predictionCol: String, + labelCol: String, + weightCol: String, + override val objectiveHistory: Array[Double]) + extends RandomForestClassificationSummaryImpl( + predictions, predictionCol, labelCol, weightCol) + with RandomForestClassificationTrainingSummary + +/** + * Multiclass RandomForestClassification results for a given model. + * + * @param predictions dataframe output by the model's `transform` method. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + */ +private class RandomForestClassificationSummaryImpl( + @transient override val predictions: DataFrame, + override val predictionCol: String, + override val labelCol: String, + override val weightCol: String) + extends RandomForestClassificationSummary + +/** + * Binary RandomForestClassification training results. + * + * @param predictions dataframe output by the model's `transform` method. + * @param scoreCol field in "predictions" which gives the probability of each class as a vector. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. + */ +private class BinaryRandomForestClassificationTrainingSummaryImpl( + predictions: DataFrame, + scoreCol: String, + predictionCol: String, + labelCol: String, + weightCol: String, + override val objectiveHistory: Array[Double]) + extends BinaryRandomForestClassificationSummaryImpl( + predictions, scoreCol, predictionCol, labelCol, weightCol) + with BinaryRandomForestClassificationTrainingSummary + +/** + * Binary RandomForestClassification for a given model. + * + * @param predictions dataframe output by the model's `transform` method. + * @param scoreCol field in "predictions" which gives the prediction of + * each class as a vector. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + */ +private class BinaryRandomForestClassificationSummaryImpl( + predictions: DataFrame, + override val scoreCol: String, + predictionCol: String, + labelCol: String, + weightCol: String) + extends RandomForestClassificationSummaryImpl( + predictions, predictionCol, labelCol, weightCol) + with BinaryRandomForestClassificationSummary diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index ecee531c88a8f..56eadff6df078 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -342,7 +342,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { blorModel2.summary.asBinary.weightedPrecision relTol 1e-6) assert(blorModel.summary.asBinary.weightedRecall ~== blorModel2.summary.asBinary.weightedRecall relTol 1e-6) - assert(blorModel.summary.asBinary.asBinary.areaUnderROC ~== + assert(blorModel.summary.asBinary.areaUnderROC ~== blorModel2.summary.asBinary.areaUnderROC relTol 1e-6) assert(mlorSummary.accuracy ~== mlorSummary2.accuracy relTol 1e-6) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index e30e93ad4628c..645a436fa0ad6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions._ /** * Test suite for [[RandomForestClassifier]]. @@ -296,6 +297,115 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { } } + test("summary for binary and multiclass") { + val arr = new Array[LabeledPoint](300) + for (i <- 0 until 300) { + if (i < 100) { + arr(i) = new LabeledPoint(0.0, Vectors.dense(2.0, 2.0)) + } else if (i < 200) { + arr(i) = new LabeledPoint(1.0, Vectors.dense(1.0, 2.0)) + } else { + arr(i) = new LabeledPoint(2.0, Vectors.dense(0.0, 2.0)) + } + } + val rdd = sc.parallelize(arr) + val multinomialDataset = spark.createDataFrame(rdd) + + val rf = new RandomForestClassifier() + + val brfModel = rf.fit(binaryDataset) + assert(brfModel.summary.isInstanceOf[BinaryRandomForestClassificationTrainingSummary]) + assert(brfModel.summary.asBinary.isInstanceOf[BinaryRandomForestClassificationTrainingSummary]) + assert(brfModel.binarySummary.isInstanceOf[RandomForestClassificationTrainingSummary]) + assert(brfModel.summary.totalIterations === 0) + assert(brfModel.binarySummary.totalIterations === 0) + + val mrfModel = rf.fit(multinomialDataset) + assert(mrfModel.summary.isInstanceOf[RandomForestClassificationTrainingSummary]) + withClue("cannot get binary summary for multiclass model") { + intercept[RuntimeException] { + mrfModel.binarySummary + } + } + withClue("cannot cast summary to binary summary multiclass model") { + intercept[RuntimeException] { + mrfModel.summary.asBinary + } + } + assert(mrfModel.summary.totalIterations === 0) + + val brfSummary = brfModel.evaluate(binaryDataset) + val mrfSummary = mrfModel.evaluate(multinomialDataset) + assert(brfSummary.isInstanceOf[BinaryRandomForestClassificationSummary]) + assert(mrfSummary.isInstanceOf[RandomForestClassificationSummary]) + + assert(brfSummary.accuracy === brfModel.summary.accuracy) + assert(brfSummary.weightedPrecision === brfModel.summary.weightedPrecision) + assert(brfSummary.weightedRecall === brfModel.summary.weightedRecall) + assert(brfSummary.asBinary.areaUnderROC ~== brfModel.summary.asBinary.areaUnderROC relTol 1e-6) + + // verify instance weight works + val rf2 = new RandomForestClassifier() + .setWeightCol("weight") + + val binaryDatasetWithWeight = + binaryDataset.select(col("label"), col("features"), lit(2.5).as("weight")) + + val multinomialDatasetWithWeight = + multinomialDataset.select(col("label"), col("features"), lit(10.0).as("weight")) + + val brfModel2 = rf2.fit(binaryDatasetWithWeight) + assert(brfModel2.summary.isInstanceOf[BinaryRandomForestClassificationTrainingSummary]) + assert(brfModel2.summary.asBinary.isInstanceOf[BinaryRandomForestClassificationTrainingSummary]) + assert(brfModel2.binarySummary.isInstanceOf[BinaryRandomForestClassificationTrainingSummary]) + + val mrfModel2 = rf2.fit(multinomialDatasetWithWeight) + assert(mrfModel2.summary.isInstanceOf[RandomForestClassificationTrainingSummary]) + withClue("cannot get binary summary for multiclass model") { + intercept[RuntimeException] { + mrfModel2.binarySummary + } + } + withClue("cannot cast summary to binary summary multiclass model") { + intercept[RuntimeException] { + mrfModel2.summary.asBinary + } + } + + val brfSummary2 = brfModel2.evaluate(binaryDatasetWithWeight) + val mrfSummary2 = mrfModel2.evaluate(multinomialDatasetWithWeight) + assert(brfSummary2.isInstanceOf[BinaryRandomForestClassificationSummary]) + assert(mrfSummary2.isInstanceOf[RandomForestClassificationSummary]) + + assert(brfSummary2.accuracy === brfModel2.summary.accuracy) + assert(brfSummary2.weightedPrecision === brfModel2.summary.weightedPrecision) + assert(brfSummary2.weightedRecall === brfModel2.summary.weightedRecall) + assert(brfSummary2.asBinary.areaUnderROC ~== + brfModel2.summary.asBinary.areaUnderROC relTol 1e-6) + + assert(brfSummary.accuracy ~== brfSummary2.accuracy relTol 1e-6) + assert(brfSummary.weightedPrecision ~== brfSummary2.weightedPrecision relTol 1e-6) + assert(brfSummary.weightedRecall ~== brfSummary2.weightedRecall relTol 1e-6) + assert(brfSummary.asBinary.areaUnderROC ~== brfSummary2.asBinary.areaUnderROC relTol 1e-6) + + assert(brfModel.summary.asBinary.accuracy ~== + brfModel2.summary.asBinary.accuracy relTol 1e-6) + assert(brfModel.summary.asBinary.weightedPrecision ~== + brfModel2.summary.asBinary.weightedPrecision relTol 1e-6) + assert(brfModel.summary.asBinary.weightedRecall ~== + brfModel2.summary.asBinary.weightedRecall relTol 1e-6) + assert(brfModel.summary.asBinary.areaUnderROC ~== + brfModel2.summary.asBinary.areaUnderROC relTol 1e-6) + + assert(mrfSummary.accuracy ~== mrfSummary2.accuracy relTol 1e-6) + assert(mrfSummary.weightedPrecision ~== mrfSummary2.weightedPrecision relTol 1e-6) + assert(mrfSummary.weightedRecall ~== mrfSummary2.weightedRecall relTol 1e-6) + + assert(mrfModel.summary.accuracy ~== mrfModel2.summary.accuracy relTol 1e-6) + assert(mrfModel.summary.weightedPrecision ~== mrfModel2.summary.weightedPrecision relTol 1e-6) + assert(mrfModel.summary.weightedRecall ~==mrfModel2.summary.weightedRecall relTol 1e-6) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index bdd37c99df0a8..d70932a1bc6fc 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -46,6 +46,9 @@ 'DecisionTreeClassifier', 'DecisionTreeClassificationModel', 'GBTClassifier', 'GBTClassificationModel', 'RandomForestClassifier', 'RandomForestClassificationModel', + 'RandomForestClassificationSummary', 'RandomForestClassificationTrainingSummary', + 'BinaryRandomForestClassificationSummary', + 'BinaryRandomForestClassificationTrainingSummary', 'NaiveBayes', 'NaiveBayesModel', 'MultilayerPerceptronClassifier', 'MultilayerPerceptronClassificationModel', 'OneVsRest', 'OneVsRestModel', @@ -1762,7 +1765,7 @@ def setMinWeightFractionPerNode(self, value): class RandomForestClassificationModel(_TreeEnsembleModel, _JavaProbabilisticClassificationModel, _RandomForestClassifierParams, JavaMLWritable, - JavaMLReadable): + JavaMLReadable, HasTrainingSummary): """ Model fitted by RandomForestClassifier. @@ -1790,6 +1793,80 @@ def trees(self): """Trees in this ensemble. Warning: These have null parent Estimators.""" return [DecisionTreeClassificationModel(m) for m in list(self._call_java("trees"))] + @property + @since("3.1.0") + def summary(self): + """ + Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + trained on the training set. An exception is thrown if `trainingSummary is None`. + """ + if self.hasSummary: + if self.numClasses <= 2: + return BinaryRandomForestClassificationTrainingSummary( + super(RandomForestClassificationModel, self).summary) + else: + return RandomForestClassificationTrainingSummary( + super(RandomForestClassificationModel, self).summary) + else: + raise RuntimeError("No training summary available for this %s" % + self.__class__.__name__) + + @since("3.1.0") + def evaluate(self, dataset): + """ + Evaluates the model on a test dataset. + + :param dataset: + Test dataset to evaluate model on, where dataset is an + instance of :py:class:`pyspark.sql.DataFrame` + """ + if not isinstance(dataset, DataFrame): + raise ValueError("dataset must be a DataFrame but got %s." % type(dataset)) + java_rf_summary = self._call_java("evaluate", dataset) + if self.numClasses <= 2: + return BinaryRandomForestClassificationSummary(java_rf_summary) + else: + return RandomForestClassificationSummary(java_rf_summary) + + +class RandomForestClassificationSummary(_ClassificationSummary): + """ + Abstraction for RandomForestClassification Results for a given model. + .. versionadded:: 3.1.0 + """ + pass + + +@inherit_doc +class RandomForestClassificationTrainingSummary(RandomForestClassificationSummary, + _TrainingSummary): + """ + Abstraction for RandomForestClassificationTraining Training results. + .. versionadded:: 3.1.0 + """ + pass + + +@inherit_doc +class BinaryRandomForestClassificationSummary(_BinaryClassificationSummary): + """ + BinaryRandomForestClassification results for a given model. + + .. versionadded:: 3.1.0 + """ + pass + + +@inherit_doc +class BinaryRandomForestClassificationTrainingSummary(BinaryRandomForestClassificationSummary, + RandomForestClassificationTrainingSummary): + """ + BinaryRandomForestClassification training results for a given model. + + .. versionadded:: 3.1.0 + """ + pass + class _GBTClassifierParams(_GBTParams, _HasVarianceImpurity): """ diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index 19acd194f4ddf..7d905793188bf 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -22,7 +22,9 @@ basestring = str from pyspark.ml.classification import BinaryLogisticRegressionSummary, LinearSVC, \ - LinearSVCSummary, LogisticRegression, LogisticRegressionSummary + LinearSVCSummary, BinaryRandomForestClassificationSummary, LogisticRegression, \ + LogisticRegressionSummary, RandomForestClassificationSummary, \ + RandomForestClassifier from pyspark.ml.clustering import BisectingKMeans, GaussianMixture, KMeans from pyspark.ml.linalg import Vectors from pyspark.ml.regression import GeneralizedLinearRegression, LinearRegression @@ -235,6 +237,81 @@ def test_linear_svc_summary(self): self.assertTrue(isinstance(sameSummary, LinearSVCSummary)) self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) + def test_binary_randomforest_classification_summary(self): + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], []))], + ["label", "weight", "features"]) + rf = RandomForestClassifier(weightCol="weight") + model = rf.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.predictionCol, "prediction") + self.assertEqual(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertTrue(isinstance(s.roc, DataFrame)) + self.assertAlmostEqual(s.areaUnderROC, 1.0, 2) + self.assertTrue(isinstance(s.pr, DataFrame)) + self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame)) + self.assertTrue(isinstance(s.precisionByThreshold, DataFrame)) + self.assertTrue(isinstance(s.recallByThreshold, DataFrame)) + self.assertAlmostEqual(s.accuracy, 1.0, 2) + self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2) + self.assertAlmostEqual(s.weightedRecall, 1.0, 2) + self.assertAlmostEqual(s.weightedPrecision, 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2) + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, BinaryRandomForestClassificationSummary)) + self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) + + def test_multiclass_randomforest_classification_summary(self): + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], [])), + (2.0, 2.0, Vectors.dense(2.0)), + (2.0, 2.0, Vectors.dense(1.9))], + ["label", "weight", "features"]) + rf = RandomForestClassifier(weightCol="weight") + model = rf.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.predictionCol, "prediction") + self.assertEqual(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertAlmostEqual(s.accuracy, 1.0, 2) + self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2) + self.assertAlmostEqual(s.weightedRecall, 1.0, 2) + self.assertAlmostEqual(s.weightedPrecision, 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2) + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, RandomForestClassificationSummary)) + self.assertFalse(isinstance(sameSummary, BinaryRandomForestClassificationSummary)) + self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) + def test_gaussian_mixture_summary(self): data = [(Vectors.dense(1.0),), (Vectors.dense(5.0),), (Vectors.dense(10.0),), (Vectors.sparse(1, [], []),)] From 7dbd90b68d4f886ef53205b031a2f238edced4da Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 1 Jul 2020 08:03:46 -0700 Subject: [PATCH 105/384] [SPARK-31797][SQL][FOLLOWUP] TIMESTAMP_SECONDS supports fractional input ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/28534 , to make `TIMESTAMP_SECONDS` function support fractional input as well. ### Why are the changes needed? Previously the cast function can cast fractional values to timestamp. Now we suggest users to ues these new functions, and we need to cover all the cast use cases. ### Does this PR introduce _any_ user-facing change? Yes, now `TIMESTAMP_SECONDS` function accepts fractional input. ### How was this patch tested? new tests Closes #28956 from cloud-fan/follow. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../expressions/datetimeExpressions.scala | 62 ++++++-- .../expressions/DateExpressionsSuite.scala | 136 +++++++++++++++--- .../resources/sql-tests/inputs/datetime.sql | 7 +- .../sql-tests/results/ansi/datetime.sql.out | 29 +++- .../sql-tests/results/datetime-legacy.sql.out | 29 +++- .../sql-tests/results/datetime.sql.out | 29 +++- ...cast #3-0-9f6fcc8c1a29c793c2238bad91453e9f | 2 + ...cast #4-0-e9286317470d42e9f8122bc98a2c1ce1 | 2 + .../sql/hive/execution/HiveQuerySuite.scala | 10 +- 9 files changed, 267 insertions(+), 39 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-9f6fcc8c1a29c793c2238bad91453e9f create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-e9286317470d42e9f8122bc98a2c1ce1 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 0e4ff4f9f2cb4..a1277217b1b3a 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 @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.expressions import java.text.ParseException import java.time.{DateTimeException, LocalDate, LocalDateTime, ZoneId} import java.time.format.DateTimeParseException -import java.time.temporal.IsoFields import java.util.Locale import org.apache.commons.text.StringEscapeUtils @@ -386,7 +385,7 @@ case class DayOfYear(child: Expression) extends GetDateField { override val funcName = "getDayInYear" } -abstract class NumberToTimestampBase extends UnaryExpression +abstract class IntegralToTimestampBase extends UnaryExpression with ExpectsInputTypes with NullIntolerant { protected def upScaleFactor: Long @@ -408,19 +407,66 @@ abstract class NumberToTimestampBase extends UnaryExpression } } +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.", + usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds (can be fractional) since UTC epoch.", examples = """ Examples: > SELECT _FUNC_(1230219000); 2008-12-25 07:30:00 + > SELECT _FUNC_(1230219000.123); + 2008-12-25 07:30:00.123 """, group = "datetime_funcs", since = "3.1.0") -case class SecondsToTimestamp(child: Expression) - extends NumberToTimestampBase { +// scalastyle:on line.size.limit +case class SecondsToTimestamp(child: Expression) extends UnaryExpression + with ExpectsInputTypes with NullIntolerant { + + override def inputTypes: Seq[AbstractDataType] = Seq(NumericType) + + override def dataType: DataType = TimestampType - override def upScaleFactor: Long = MICROS_PER_SECOND + override def nullable: Boolean = child.dataType match { + case _: FloatType | _: DoubleType => true + case _ => child.nullable + } + + @transient + private lazy val evalFunc: Any => Any = child.dataType match { + case _: IntegralType => input => + Math.multiplyExact(input.asInstanceOf[Number].longValue(), MICROS_PER_SECOND) + case _: DecimalType => input => + val operand = new java.math.BigDecimal(MICROS_PER_SECOND) + input.asInstanceOf[Decimal].toJavaBigDecimal.multiply(operand).longValueExact() + case _: FloatType => input => + val f = input.asInstanceOf[Float] + if (f.isNaN || f.isInfinite) null else (f * MICROS_PER_SECOND).toLong + case _: DoubleType => input => + val d = input.asInstanceOf[Double] + if (d.isNaN || d.isInfinite) null else (d * MICROS_PER_SECOND).toLong + } + + override def nullSafeEval(input: Any): Any = evalFunc(input) + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = child.dataType match { + case _: IntegralType => + defineCodeGen(ctx, ev, c => s"java.lang.Math.multiplyExact($c, ${MICROS_PER_SECOND}L)") + case _: DecimalType => + val operand = s"new java.math.BigDecimal($MICROS_PER_SECOND)" + defineCodeGen(ctx, ev, c => s"$c.toJavaBigDecimal().multiply($operand).longValueExact()") + case other => + nullSafeCodeGen(ctx, ev, c => { + val typeStr = CodeGenerator.boxedType(other) + s""" + |if ($typeStr.isNaN($c) || $typeStr.isInfinite($c)) { + | ${ev.isNull} = true; + |} else { + | ${ev.value} = (long)($c * $MICROS_PER_SECOND); + |} + |""".stripMargin + }) + } override def prettyName: String = "timestamp_seconds" } @@ -437,7 +483,7 @@ case class SecondsToTimestamp(child: Expression) since = "3.1.0") // scalastyle:on line.size.limit case class MillisToTimestamp(child: Expression) - extends NumberToTimestampBase { + extends IntegralToTimestampBase { override def upScaleFactor: Long = MICROS_PER_MILLIS @@ -456,7 +502,7 @@ case class MillisToTimestamp(child: Expression) since = "3.1.0") // scalastyle:on line.size.limit case class MicrosToTimestamp(child: Expression) - extends NumberToTimestampBase { + extends IntegralToTimestampBase { override def upScaleFactor: Long = 1L 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 4edf95d8f994b..85492084d51ac 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 @@ -1142,28 +1142,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("SPARK-31710:Adds TIMESTAMP_SECONDS, " + - "TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions") { - checkEvaluation(SecondsToTimestamp(Literal(1230219000)), 1230219000L * MICROS_PER_SECOND) - checkEvaluation(SecondsToTimestamp(Literal(-1230219000)), -1230219000L * MICROS_PER_SECOND) - checkEvaluation(SecondsToTimestamp(Literal(null, IntegerType)), null) - checkEvaluation(MillisToTimestamp(Literal(1230219000123L)), 1230219000123L * MICROS_PER_MILLIS) - checkEvaluation(MillisToTimestamp( - Literal(-1230219000123L)), -1230219000123L * MICROS_PER_MILLIS) - checkEvaluation(MillisToTimestamp(Literal(null, IntegerType)), null) - checkEvaluation(MicrosToTimestamp(Literal(1230219000123123L)), 1230219000123123L) - checkEvaluation(MicrosToTimestamp(Literal(-1230219000123123L)), -1230219000123123L) - checkEvaluation(MicrosToTimestamp(Literal(null, IntegerType)), null) - checkExceptionInExpression[ArithmeticException]( - SecondsToTimestamp(Literal(1230219000123123L)), "long overflow") - checkExceptionInExpression[ArithmeticException]( - SecondsToTimestamp(Literal(-1230219000123123L)), "long overflow") - checkExceptionInExpression[ArithmeticException]( - MillisToTimestamp(Literal(92233720368547758L)), "long overflow") - checkExceptionInExpression[ArithmeticException]( - MillisToTimestamp(Literal(-92233720368547758L)), "long overflow") - } - test("Consistent error handling for datetime formatting and parsing functions") { def checkException[T <: Exception : ClassTag](c: String): Unit = { @@ -1194,4 +1172,118 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { new ParseToTimestamp(Literal("11:11 PM"), Literal("mm:ss a")).child, Timestamp.valueOf("1970-01-01 12:11:11.0")) } + + def testIntegralInput(testFunc: Number => Unit): Unit = { + def checkResult(input: Long): Unit = { + if (input.toByte == input) { + testFunc(input.toByte) + } else if (input.toShort == input) { + testFunc(input.toShort) + } else if (input.toInt == input) { + testFunc(input.toInt) + } else { + testFunc(input) + } + } + checkResult(0) + checkResult(Byte.MaxValue) + checkResult(Byte.MinValue) + checkResult(Short.MaxValue) + checkResult(Short.MinValue) + checkResult(Int.MaxValue) + checkResult(Int.MinValue) + checkResult(Int.MaxValue.toLong + 100) + checkResult(Int.MinValue.toLong - 100) + } + + test("TIMESTAMP_SECONDS") { + def testIntegralFunc(value: Number): Unit = { + checkEvaluation( + SecondsToTimestamp(Literal(value)), + Instant.ofEpochSecond(value.longValue())) + } + + // test null input + checkEvaluation( + SecondsToTimestamp(Literal(null, IntegerType)), + null) + + // test integral input + testIntegralInput(testIntegralFunc) + // test overflow + checkExceptionInExpression[ArithmeticException]( + SecondsToTimestamp(Literal(Long.MaxValue, LongType)), EmptyRow, "long overflow") + + def testFractionalInput(input: String): Unit = { + Seq(input.toFloat, input.toDouble, Decimal(input)).foreach { value => + checkEvaluation( + SecondsToTimestamp(Literal(value)), + (input.toDouble * MICROS_PER_SECOND).toLong) + } + } + + testFractionalInput("1.0") + testFractionalInput("-1.0") + testFractionalInput("1.234567") + testFractionalInput("-1.234567") + + // test overflow for decimal input + checkExceptionInExpression[ArithmeticException]( + SecondsToTimestamp(Literal(Decimal("9" * 38))), "Overflow" + ) + // test truncation error for decimal input + checkExceptionInExpression[ArithmeticException]( + SecondsToTimestamp(Literal(Decimal("0.1234567"))), "Rounding necessary" + ) + + // test NaN + checkEvaluation( + SecondsToTimestamp(Literal(Double.NaN)), + null) + checkEvaluation( + SecondsToTimestamp(Literal(Float.NaN)), + null) + // double input can truncate + checkEvaluation( + SecondsToTimestamp(Literal(123.456789123)), + Instant.ofEpochSecond(123, 456789000)) + } + + test("TIMESTAMP_MILLIS") { + def testIntegralFunc(value: Number): Unit = { + checkEvaluation( + MillisToTimestamp(Literal(value)), + Instant.ofEpochMilli(value.longValue())) + } + + // test null input + checkEvaluation( + MillisToTimestamp(Literal(null, IntegerType)), + null) + + // test integral input + testIntegralInput(testIntegralFunc) + // test overflow + checkExceptionInExpression[ArithmeticException]( + MillisToTimestamp(Literal(Long.MaxValue, LongType)), EmptyRow, "long overflow") + } + + test("TIMESTAMP_MICROS") { + def testIntegralFunc(value: Number): Unit = { + checkEvaluation( + MicrosToTimestamp(Literal(value)), + value.longValue()) + } + + // test null input + checkEvaluation( + MicrosToTimestamp(Literal(null, IntegerType)), + null) + + // test integral input + testIntegralInput(testIntegralFunc) + // test max/min input + testIntegralFunc(Long.MaxValue) + testIntegralFunc(Long.MinValue) + } } 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 06765627f5545..bdf11f51db532 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -2,13 +2,18 @@ -- [SPARK-31710] TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS to timestamp transfer select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null); +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)); select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null); select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null); --- overflow exception: +-- overflow exception select TIMESTAMP_SECONDS(1230219000123123); select TIMESTAMP_SECONDS(-1230219000123123); select TIMESTAMP_MILLIS(92233720368547758); select TIMESTAMP_MILLIS(-92233720368547758); +-- truncate exception +select TIMESTAMP_SECONDS(0.1234567); +-- truncation is OK for float/double +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)); -- [SPARK-16836] current_date and current_timestamp literals select current_date = current_date(), current_timestamp = current_timestamp(); 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 26adb40ce1b14..484b67677a91b 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,15 +1,23 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 103 +-- Number of queries: 106 -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + -- !query select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) -- !query schema @@ -62,6 +70,23 @@ java.lang.ArithmeticException long overflow +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + -- !query select current_date = current_date(), current_timestamp = current_timestamp() -- !query schema 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 index 15092f0a27c1f..edb49e575f52e 100644 --- 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 @@ -1,15 +1,23 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 103 +-- Number of queries: 106 -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + -- !query select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) -- !query schema @@ -62,6 +70,23 @@ java.lang.ArithmeticException long overflow +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + -- !query select current_date = current_date(), current_timestamp = current_timestamp() -- !query schema 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 b80f36e9c2347..9f9351a4809af 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,15 +1,23 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 103 +-- Number of queries: 106 -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + -- !query select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) -- !query schema @@ -62,6 +70,23 @@ java.lang.ArithmeticException long overflow +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + -- !query select current_date = current_date(), current_timestamp = current_timestamp() -- !query schema diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-9f6fcc8c1a29c793c2238bad91453e9f b/sql/hive/src/test/resources/golden/timestamp cast #3-0-9f6fcc8c1a29c793c2238bad91453e9f new file mode 100644 index 0000000000000..f99e724db6af8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #3-0-9f6fcc8c1a29c793c2238bad91453e9f @@ -0,0 +1,2 @@ +1.2 + diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-e9286317470d42e9f8122bc98a2c1ce1 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-e9286317470d42e9f8122bc98a2c1ce1 new file mode 100644 index 0000000000000..decdb1d30e6a6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #4-0-e9286317470d42e9f8122bc98a2c1ce1 @@ -0,0 +1,2 @@ +-1.2 + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2b42444ceeaa1..e5628c33b5ec8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -564,12 +564,18 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd assert(-1 == res.get(0)) } - test("timestamp cast #3") { + createQueryTest("timestamp cast #3", + "SELECT CAST(TIMESTAMP_SECONDS(1.2) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #4", + "SELECT CAST(TIMESTAMP_SECONDS(-1.2) AS DOUBLE) FROM src LIMIT 1") + + test("timestamp cast #5") { val res = sql("SELECT CAST(TIMESTAMP_SECONDS(1200) AS INT) FROM src LIMIT 1").collect().head assert(1200 == res.getInt(0)) } - test("timestamp cast #4") { + test("timestamp cast #6") { val res = sql("SELECT CAST(TIMESTAMP_SECONDS(-1200) AS INT) FROM src LIMIT 1").collect().head assert(-1200 == res.getInt(0)) } From f843a5bf7ce625fd99dddbca51fcca357933a4a6 Mon Sep 17 00:00:00 2001 From: Eren Avsarogullari Date: Wed, 1 Jul 2020 11:33:18 -0700 Subject: [PATCH 106/384] [SPARK-32026][CORE][TEST] Add PrometheusServletSuite ### What changes were proposed in this pull request? This PR aims to add `PrometheusServletSuite`. ### Why are the changes needed? This improves the test coverage. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the newly added test suite. Closes #28865 from erenavsarogullari/spark_driver_prometheus_metrics_improvement. Authored-by: Eren Avsarogullari Signed-off-by: Dongjoon Hyun --- .../metrics/sink/PrometheusServletSuite.scala | 73 +++++++++++++++++++ 1 file changed, 73 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/metrics/sink/PrometheusServletSuite.scala diff --git a/core/src/test/scala/org/apache/spark/metrics/sink/PrometheusServletSuite.scala b/core/src/test/scala/org/apache/spark/metrics/sink/PrometheusServletSuite.scala new file mode 100644 index 0000000000000..080ca0e41f793 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/metrics/sink/PrometheusServletSuite.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics.sink + +import java.util.Properties + +import scala.collection.JavaConverters._ + +import com.codahale.metrics.{Counter, Gauge, MetricRegistry} +import org.scalatest.PrivateMethodTester + +import org.apache.spark.SparkFunSuite + +class PrometheusServletSuite extends SparkFunSuite with PrivateMethodTester { + + test("register metrics") { + val sink = createPrometheusServlet() + + val gauge = new Gauge[Double] { + override def getValue: Double = 5.0 + } + + val counter = new Counter + counter.inc(10) + + sink.registry.register("gauge1", gauge) + sink.registry.register("gauge2", gauge) + sink.registry.register("counter1", counter) + + val metricGaugeKeys = sink.registry.getGauges.keySet.asScala + assert(metricGaugeKeys.equals(Set("gauge1", "gauge2")), + "Should contain 2 gauges metrics registered") + + val metricCounterKeys = sink.registry.getCounters.keySet.asScala + assert(metricCounterKeys.equals(Set("counter1")), + "Should contain 1 counter metric registered") + + val gaugeValues = sink.registry.getGauges.values.asScala + assert(gaugeValues.size == 2) + gaugeValues.foreach(gauge => assert(gauge.getValue == 5.0)) + + val counterValues = sink.registry.getCounters.values.asScala + assert(counterValues.size == 1) + counterValues.foreach(counter => assert(counter.getCount == 10)) + } + + test("normalize key") { + val key = "local-1592132938718.driver.LiveListenerBus." + + "listenerProcessingTime.org.apache.spark.HeartbeatReceiver" + val sink = createPrometheusServlet() + val suffix = sink invokePrivate PrivateMethod[String]('normalizeKey)(key) + assert(suffix == "metrics_local_1592132938718_driver_LiveListenerBus_" + + "listenerProcessingTime_org_apache_spark_HeartbeatReceiver_") + } + + private def createPrometheusServlet(): PrometheusServlet = + new PrometheusServlet(new Properties, new MetricRegistry, securityMgr = null) +} From 6edb20df834f7f9b85c1559ef78a3d0d2272e4df Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 2 Jul 2020 06:09:54 +0800 Subject: [PATCH 107/384] [SPARK-31935][SQL][FOLLOWUP] Hadoop file system config should be effective in data source options ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/28760 to fix the remaining issues: 1. should consider data source options when refreshing cache by path at the end of `InsertIntoHadoopFsRelationCommand` 2. should consider data source options when inferring schema for file source 3. should consider data source options when getting the qualified path in file source v2. ### Why are the changes needed? We didn't catch these issues in https://github.com/apache/spark/pull/28760, because the test case is to check error when initializing the file system. If we initialize the file system multiple times during a simple read/write action, the test case actually only test the first time. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? rewrite the test to make sure the entire data source read/write action can succeed. Closes #28948 from cloud-fan/fix. Authored-by: Wenchen Fan Signed-off-by: Gengliang Wang --- .../spark/sql/v2/avro/AvroDataSourceV2.scala | 4 +-- .../spark/sql/execution/CacheManager.scala | 15 ++++++--- .../InsertIntoHadoopFsRelationCommand.scala | 2 +- .../datasources/SchemaMergeUtils.scala | 4 ++- .../execution/datasources/orc/OrcUtils.scala | 2 +- .../parquet/ParquetFileFormat.scala | 3 +- .../datasources/parquet/ParquetUtils.scala | 2 +- .../datasources/v2/FileDataSourceV2.scala | 13 +++++--- .../datasources/v2/csv/CSVDataSourceV2.scala | 4 +-- .../v2/json/JsonDataSourceV2.scala | 4 +-- .../datasources/v2/orc/OrcDataSourceV2.scala | 4 +-- .../v2/parquet/ParquetDataSourceV2.scala | 4 +-- .../v2/text/TextDataSourceV2.scala | 4 +-- .../spark/sql/FileBasedDataSourceSuite.scala | 31 +++++++++++-------- .../datasources/orc/OrcSourceSuite.scala | 4 +-- .../spark/sql/hive/orc/OrcFileFormat.scala | 4 +-- 16 files changed, 59 insertions(+), 45 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala index c6f52d676422c..969dee0a39696 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala @@ -31,13 +31,13 @@ class AvroDataSourceV2 extends FileDataSourceV2 { override def getTable(options: CaseInsensitiveStringMap): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) AvroTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) } override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) AvroTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 52cec8b202885..7d86c48015406 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -248,12 +248,17 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { * `HadoopFsRelation` node(s) as part of its logical plan. */ def recacheByPath(spark: SparkSession, resourcePath: String): Unit = { - val (fs, qualifiedPath) = { - val path = new Path(resourcePath) - val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) - (fs, fs.makeQualified(path)) - } + val path = new Path(resourcePath) + val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) + recacheByPath(spark, path, fs) + } + /** + * Tries to re-cache all the cache entries that contain `resourcePath` in one or more + * `HadoopFsRelation` node(s) as part of its logical plan. + */ + def recacheByPath(spark: SparkSession, resourcePath: Path, fs: FileSystem): Unit = { + val qualifiedPath = fs.makeQualified(resourcePath) recacheByCondition(spark, _.plan.find(lookupAndRefresh(_, fs, qualifiedPath)).isDefined) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index f11972115e09f..fe733f4238e1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -192,7 +192,7 @@ case class InsertIntoHadoopFsRelationCommand( // refresh cached files in FileIndex fileIndex.foreach(_.refresh()) // refresh data cache if table is cached - sparkSession.catalog.refreshByPath(outputPath.toString) + sparkSession.sharedState.cacheManager.recacheByPath(sparkSession, outputPath, fs) if (catalogTable.nonEmpty) { CommandUtils.updateTableStats(sparkSession, catalogTable.get) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala index 99882b0f7c7b0..28097c35401c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala @@ -32,10 +32,12 @@ object SchemaMergeUtils extends Logging { */ def mergeSchemasInParallel( sparkSession: SparkSession, + parameters: Map[String, String], files: Seq[FileStatus], schemaReader: (Seq[FileStatus], Configuration, Boolean) => Seq[StructType]) : Option[StructType] = { - val serializedConf = new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()) + val serializedConf = new SerializableConfiguration( + sparkSession.sessionState.newHadoopConfWithOptions(parameters)) // !! HACK ALERT !! // Here is a hack for Parquet, but it can be used by Orc as well. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index eea9b2a8f9613..d274bcd0edd2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -109,7 +109,7 @@ object OrcUtils extends Logging { val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) if (orcOptions.mergeSchema) { SchemaMergeUtils.mergeSchemasInParallel( - sparkSession, files, OrcUtils.readOrcSchemasInParallel) + sparkSession, options, files, OrcUtils.readOrcSchemasInParallel) } else { OrcUtils.readSchema(sparkSession, files) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 71874104fcf4f..68f49f9442579 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -475,6 +475,7 @@ object ParquetFileFormat extends Logging { * S3 nodes). */ def mergeSchemasInParallel( + parameters: Map[String, String], filesToTouch: Seq[FileStatus], sparkSession: SparkSession): Option[StructType] = { val assumeBinaryIsString = sparkSession.sessionState.conf.isParquetBinaryAsString @@ -490,7 +491,7 @@ object ParquetFileFormat extends Logging { .map(ParquetFileFormat.readSchemaFromFooter(_, converter)) } - SchemaMergeUtils.mergeSchemasInParallel(sparkSession, filesToTouch, reader) + SchemaMergeUtils.mergeSchemasInParallel(sparkSession, parameters, filesToTouch, reader) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala index 7e7dba92f37b5..b91d75c55c513 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -104,7 +104,7 @@ object ParquetUtils { .orElse(filesByType.data.headOption) .toSeq } - ParquetFileFormat.mergeSchemasInParallel(filesToTouch, sparkSession) + ParquetFileFormat.mergeSchemasInParallel(parameters, filesToTouch, sparkSession) } case class FileTypes( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index 30a964d7e643f..bbe8835049fa4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -18,7 +18,10 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util +import scala.collection.JavaConverters._ + import com.fasterxml.jackson.databind.ObjectMapper +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession @@ -53,14 +56,16 @@ trait FileDataSourceV2 extends TableProvider with DataSourceRegister { paths ++ Option(map.get("path")).toSeq } - protected def getTableName(paths: Seq[String]): String = { - val name = shortName() + " " + paths.map(qualifiedPathName).mkString(",") + protected def getTableName(map: CaseInsensitiveStringMap, paths: Seq[String]): String = { + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions( + map.asCaseSensitiveMap().asScala.toMap) + val name = shortName() + " " + paths.map(qualifiedPathName(_, hadoopConf)).mkString(",") Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, name) } - private def qualifiedPathName(path: String): String = { + private def qualifiedPathName(path: String, hadoopConf: Configuration): String = { val hdfsPath = new Path(path) - val fs = hdfsPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) + val fs = hdfsPath.getFileSystem(hadoopConf) hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toString } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala index 1f99d4282f6da..69d001b4a615c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala @@ -31,13 +31,13 @@ class CSVDataSourceV2 extends FileDataSourceV2 { override def getTable(options: CaseInsensitiveStringMap): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) CSVTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) } override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) CSVTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala index 7a0949e586cd8..9c4e3b8c78026 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala @@ -31,13 +31,13 @@ class JsonDataSourceV2 extends FileDataSourceV2 { override def getTable(options: CaseInsensitiveStringMap): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) JsonTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) } override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) JsonTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 8665af33b976a..fa2febdc5a984 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -31,13 +31,13 @@ class OrcDataSourceV2 extends FileDataSourceV2 { override def getTable(options: CaseInsensitiveStringMap): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) OrcTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) } override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) OrcTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala index 8cb6186c12ff3..7e7ca964de28f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala @@ -31,13 +31,13 @@ class ParquetDataSourceV2 extends FileDataSourceV2 { override def getTable(options: CaseInsensitiveStringMap): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) ParquetTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) } override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) ParquetTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala index 049c717effa26..43bcb61f25962 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala @@ -31,13 +31,13 @@ class TextDataSourceV2 extends FileDataSourceV2 { override def getTable(options: CaseInsensitiveStringMap): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) TextTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) } override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { val paths = getPaths(options) - val tableName = getTableName(paths) + val tableName = getTableName(options, paths) TextTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index d8157d3c779b9..231a8f2aa7ddd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -18,12 +18,14 @@ package org.apache.spark.sql import java.io.{File, FileNotFoundException} +import java.net.URI import java.nio.file.{Files, StandardOpenOption} import java.util.Locale import scala.collection.mutable -import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{LocalFileSystem, Path} import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} @@ -845,19 +847,15 @@ class FileBasedDataSourceSuite extends QueryTest test("SPARK-31935: Hadoop file system config should be effective in data source options") { Seq("parquet", "").foreach { format => - withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> format) { + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> format, + "fs.file.impl" -> classOf[FakeFileSystemRequiringDSOption].getName, + "fs.file.impl.disable.cache" -> "true") { withTempDir { dir => - val path = dir.getCanonicalPath - val defaultFs = "nonexistFS://nonexistFS" - val expectMessage = "No FileSystem for scheme nonexistFS" - val message1 = intercept[java.io.IOException] { - spark.range(10).write.option("fs.defaultFS", defaultFs).parquet(path) - }.getMessage - assert(message1.filterNot(Set(':', '"').contains) == expectMessage) - val message2 = intercept[java.io.IOException] { - spark.read.option("fs.defaultFS", defaultFs).parquet(path) - }.getMessage - assert(message2.filterNot(Set(':', '"').contains) == expectMessage) + val path = "file:" + dir.getCanonicalPath.stripPrefix("file:") + spark.range(10).write.option("ds_option", "value").mode("overwrite").parquet(path) + checkAnswer( + spark.read.option("ds_option", "value").parquet(path), spark.range(10).toDF()) } } } @@ -932,3 +930,10 @@ object TestingUDT { override def userClass: Class[NullData] = classOf[NullData] } } + +class FakeFileSystemRequiringDSOption extends LocalFileSystem { + override def initialize(name: URI, conf: Configuration): Unit = { + super.initialize(name, conf) + require(conf.get("ds_option", "") == "value") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 73873684f6aaf..b70fd7476ed98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -213,9 +213,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { Seq(fs.listStatus(path1), fs.listStatus(path2), fs.listStatus(path3)).flatten val schema = SchemaMergeUtils.mergeSchemasInParallel( - spark, - fileStatuses, - schemaReader) + spark, Map.empty, fileStatuses, schemaReader) assert(schema.isDefined) assert(schema.get == StructType(Seq( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 7f2eb14956dc1..356b92b4652b3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -70,9 +70,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) if (orcOptions.mergeSchema) { SchemaMergeUtils.mergeSchemasInParallel( - sparkSession, - files, - OrcFileOperator.readOrcSchemasInParallel) + sparkSession, options, files, OrcFileOperator.readOrcSchemasInParallel) } else { val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles OrcFileOperator.readSchema( From bcf23307f4fd70590ea10e5e9edb6e9de1f76125 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 1 Jul 2020 15:45:39 -0700 Subject: [PATCH 108/384] [SPARK-32130][SQL] Disable the JSON option `inferTimestamp` by default ### What changes were proposed in this pull request? Set the JSON option `inferTimestamp` to `false` if an user don't pass it as datasource option. ### Why are the changes needed? To prevent perf regression while inferring schemas from JSON with potential timestamps fields. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? - Modified existing tests in `JsonSuite` and `JsonInferSchemaSuite`. - Regenerated results of `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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 | Closes #28966 from MaxGekk/json-inferTimestamps-disable-by-default. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 6 +- .../spark/sql/catalyst/json/JSONOptions.scala | 2 +- .../catalyst/json/JsonInferSchemaSuite.scala | 56 +++++++----- .../JsonBenchmark-jdk11-results.txt | 86 +++++++++---------- sql/core/benchmarks/JsonBenchmark-results.txt | 86 +++++++++---------- .../datasources/json/JsonSuite.scala | 6 +- 6 files changed, 130 insertions(+), 112 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index d6550c30b9553..0c84db38afafc 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -31,7 +31,11 @@ license: | - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. - In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can set `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710. - + +## Upgrading from Spark SQL 3.0 to 3.0.1 + +- In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Since version 3.0.1, the timestamp type inference is disabled by default. Set the JSON option `inferTimestamp` to `true` to enable such type inference. + ## Upgrading from Spark SQL 2.4 to 3.0 ### Dataset/DataFrame APIs diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index f9222f5af54da..70a673bb42457 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -133,7 +133,7 @@ private[sql] class JSONOptions( * Enables inferring of TimestampType from strings matched to the timestamp pattern * defined by the timestampFormat option. */ - val inferTimestamp: Boolean = parameters.get("inferTimestamp").map(_.toBoolean).getOrElse(true) + val inferTimestamp: Boolean = parameters.get("inferTimestamp").map(_.toBoolean).getOrElse(false) /** Build a Jackson [[JsonFactory]] using JSON options. */ def buildJsonFactory(): JsonFactory = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala index bce917c80f93c..8290b38e33934 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala @@ -35,22 +35,29 @@ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { assert(inferSchema.inferField(parser) === expectedType) } - def checkTimestampType(pattern: String, json: String): Unit = { - checkType(Map("timestampFormat" -> pattern), json, TimestampType) + def checkTimestampType(pattern: String, json: String, inferTimestamp: Boolean): Unit = { + checkType( + Map("timestampFormat" -> pattern, "inferTimestamp" -> inferTimestamp.toString), + json, + if (inferTimestamp) TimestampType else StringType) } test("inferring timestamp type") { - Seq("legacy", "corrected").foreach { legacyParserPolicy => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) { - checkTimestampType("yyyy", """{"a": "2018"}""") - checkTimestampType("yyyy=MM", """{"a": "2018=12"}""") - checkTimestampType("yyyy MM dd", """{"a": "2018 12 02"}""") - checkTimestampType( - "yyyy-MM-dd'T'HH:mm:ss.SSS", - """{"a": "2018-12-02T21:04:00.123"}""") - checkTimestampType( - "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXX", - """{"a": "2018-12-02T21:04:00.123567+01:00"}""") + Seq(true, false).foreach { inferTimestamp => + Seq("legacy", "corrected").foreach { legacyParserPolicy => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) { + checkTimestampType("yyyy", """{"a": "2018"}""", inferTimestamp) + checkTimestampType("yyyy=MM", """{"a": "2018=12"}""", inferTimestamp) + checkTimestampType("yyyy MM dd", """{"a": "2018 12 02"}""", inferTimestamp) + checkTimestampType( + "yyyy-MM-dd'T'HH:mm:ss.SSS", + """{"a": "2018-12-02T21:04:00.123"}""", + inferTimestamp) + checkTimestampType( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXX", + """{"a": "2018-12-02T21:04:00.123567+01:00"}""", + inferTimestamp) + } } } } @@ -71,16 +78,19 @@ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("skip decimal type inferring") { - Seq("legacy", "corrected").foreach { legacyParserPolicy => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) { - checkType( - options = Map( - "prefersDecimal" -> "false", - "timestampFormat" -> "yyyyMMdd.HHmmssSSS" - ), - json = """{"a": "20181202.210400123"}""", - dt = TimestampType - ) + Seq(true, false).foreach { inferTimestamp => + Seq("legacy", "corrected").foreach { legacyParserPolicy => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) { + checkType( + options = Map( + "prefersDecimal" -> "false", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS", + "inferTimestamp" -> inferTimestamp.toString + ), + json = """{"a": "20181202.210400123"}""", + dt = if (inferTimestamp) TimestampType else StringType + ) + } } } } diff --git a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt index d0cd591da4c94..ff370847a2e3e 100644 --- a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt @@ -7,106 +7,106 @@ 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 JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 68879 68993 116 1.5 688.8 1.0X -UTF-8 is set 115270 115602 455 0.9 1152.7 0.6X +No encoding 69219 69342 116 1.4 692.2 1.0X +UTF-8 is set 143950 143986 55 0.7 1439.5 0.5X Preparing data for benchmarking ... 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 47452 47538 113 2.1 474.5 1.0X -UTF-8 is set 77330 77354 30 1.3 773.3 0.6X +No encoding 57828 57913 136 1.7 578.3 1.0X +UTF-8 is set 83649 83711 60 1.2 836.5 0.7X Preparing data for benchmarking ... 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 60470 60900 534 0.2 6047.0 1.0X -UTF-8 is set 104733 104931 189 0.1 10473.3 0.6X +No encoding 64560 65193 1023 0.2 6456.0 1.0X +UTF-8 is set 102925 103174 216 0.1 10292.5 0.6X Preparing data for benchmarking ... 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 130302 131072 976 0.0 260604.6 1.0X -UTF-8 is set 150860 151284 377 0.0 301720.1 0.9X +No encoding 131002 132316 1160 0.0 262003.1 1.0X +UTF-8 is set 152128 152371 332 0.0 304256.5 0.9X Preparing data for benchmarking ... 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 18619 18684 99 0.5 1861.9 1.0X -Select 1 column 24227 24270 38 0.4 2422.7 0.8X +Select 10 columns 19376 19514 160 0.5 1937.6 1.0X +Select 1 column 24089 24156 58 0.4 2408.9 0.8X Preparing data for benchmarking ... 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 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 +Short column without encoding 8131 8219 103 1.2 813.1 1.0X +Short column with UTF-8 13464 13508 44 0.7 1346.4 0.6X +Wide column without encoding 108012 108598 914 0.1 10801.2 0.1X +Wide column with UTF-8 150988 151369 412 0.1 15098.8 0.1X Preparing data for benchmarking ... 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 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 +Text read 753 765 18 13.3 75.3 1.0X +from_json 23182 23446 230 0.4 2318.2 0.0X +json_tuple 31129 31304 181 0.3 3112.9 0.0X +get_json_object 22821 23073 225 0.4 2282.1 0.0X Preparing data for benchmarking ... 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 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 +Text read 3078 3101 26 16.2 61.6 1.0X +schema inferring 30225 30434 333 1.7 604.5 0.1X +parsing 32237 32308 63 1.6 644.7 0.1X Preparing data for benchmarking ... 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 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 +Text read 10835 10900 86 4.6 216.7 1.0X +Schema inferring 37720 37805 110 1.3 754.4 0.3X +Parsing without charset 35464 35538 100 1.4 709.3 0.3X +Parsing with UTF-8 67311 67738 381 0.7 1346.2 0.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 Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +Create a dataset of timestamps 2208 2222 14 4.5 220.8 1.0X +to_json(timestamp) 14299 14570 285 0.7 1429.9 0.2X +write timestamps to files 12955 12969 13 0.8 1295.5 0.2X +Create a dataset of dates 2297 2323 30 4.4 229.7 1.0X +to_json(date) 8509 8561 74 1.2 850.9 0.3X +write dates to files 6786 6827 45 1.5 678.6 0.3X 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 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 +read timestamp text from files 2598 2613 18 3.8 259.8 1.0X +read timestamps from files 42007 42028 19 0.2 4200.7 0.1X +infer timestamps from files 18102 18120 28 0.6 1810.2 0.1X +read date text from files 2355 2360 5 4.2 235.5 1.1X +read date from files 17420 17458 33 0.6 1742.0 0.1X +timestamp strings 3099 3101 3 3.2 309.9 0.8X +parse timestamps from Dataset[String] 48188 48215 25 0.2 4818.8 0.1X +infer timestamps from Dataset[String] 22929 22988 102 0.4 2292.9 0.1X +date strings 4090 4103 11 2.4 409.0 0.6X +parse dates from Dataset[String] 24952 25068 139 0.4 2495.2 0.1X +from_json(timestamp) 66038 66352 413 0.2 6603.8 0.0X +from_json(date) 43755 43782 27 0.2 4375.5 0.1X diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt index 46d2410fb47c3..0e4ce9003f70b 100644 --- a/sql/core/benchmarks/JsonBenchmark-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -7,106 +7,106 @@ 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 JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 63981 64044 56 1.6 639.8 1.0X -UTF-8 is set 112672 113350 962 0.9 1126.7 0.6X +No encoding 64950 65182 306 1.5 649.5 1.0X +UTF-8 is set 129566 129796 229 0.8 1295.7 0.5X Preparing data for benchmarking ... 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 51256 51449 180 2.0 512.6 1.0X -UTF-8 is set 83694 83859 148 1.2 836.9 0.6X +No encoding 50896 51277 372 2.0 509.0 1.0X +UTF-8 is set 89712 89763 49 1.1 897.1 0.6X Preparing data for benchmarking ... 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 58440 59097 569 0.2 5844.0 1.0X -UTF-8 is set 102746 102883 198 0.1 10274.6 0.6X +No encoding 59415 59785 372 0.2 5941.5 1.0X +UTF-8 is set 103059 103165 156 0.1 10305.9 0.6X Preparing data for benchmarking ... 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 128982 129304 356 0.0 257965.0 1.0X -UTF-8 is set 147247 147415 231 0.0 294494.1 0.9X +No encoding 132951 133122 288 0.0 265901.9 1.0X +UTF-8 is set 149318 149441 107 0.0 298635.3 0.9X Preparing data for benchmarking ... 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 18837 19048 331 0.5 1883.7 1.0X -Select 1 column 24707 24723 14 0.4 2470.7 0.8X +Select 10 columns 18491 18552 85 0.5 1849.1 1.0X +Select 1 column 25908 25946 65 0.4 2590.8 0.7X Preparing data for benchmarking ... 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 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 +Short column without encoding 9264 9307 49 1.1 926.4 1.0X +Short column with UTF-8 14707 14727 17 0.7 1470.7 0.6X +Wide column without encoding 141138 141347 276 0.1 14113.8 0.1X +Wide column with UTF-8 179601 180035 664 0.1 17960.1 0.1X Preparing data for benchmarking ... 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 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 +Text read 1173 1184 9 8.5 117.3 1.0X +from_json 23432 23738 338 0.4 2343.2 0.1X +json_tuple 32573 32851 358 0.3 3257.3 0.0X +get_json_object 22442 22489 47 0.4 2244.2 0.1X Preparing data for benchmarking ... 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 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 +Text read 5656 5680 31 8.8 113.1 1.0X +schema inferring 33283 33337 64 1.5 665.7 0.2X +parsing 41771 41929 178 1.2 835.4 0.1X Preparing data for benchmarking ... 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 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 +Text read 9626 9668 39 5.2 192.5 1.0X +Schema inferring 39489 39579 91 1.3 789.8 0.2X +Parsing without charset 38096 38232 125 1.3 761.9 0.3X +Parsing with UTF-8 64565 64725 165 0.8 1291.3 0.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 Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +Create a dataset of timestamps 1898 1912 13 5.3 189.8 1.0X +to_json(timestamp) 20011 20092 119 0.5 2001.1 0.1X +write timestamps to files 13388 13427 35 0.7 1338.8 0.1X +Create a dataset of dates 2351 2368 18 4.3 235.1 0.8X +to_json(date) 11884 11913 40 0.8 1188.4 0.2X +write dates to files 7317 7326 9 1.4 731.7 0.3X 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 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 +read timestamp text from files 2316 2324 13 4.3 231.6 1.0X +read timestamps from files 43712 43900 165 0.2 4371.2 0.1X +infer timestamps from files 19302 19328 38 0.5 1930.2 0.1X +read date text from files 2090 2099 11 4.8 209.0 1.1X +read date from files 18914 18940 44 0.5 1891.4 0.1X +timestamp strings 3785 3793 11 2.6 378.5 0.6X +parse timestamps from Dataset[String] 51177 51353 160 0.2 5117.7 0.0X +infer timestamps from Dataset[String] 27907 28119 186 0.4 2790.7 0.1X +date strings 4446 4452 6 2.2 444.6 0.5X +parse dates from Dataset[String] 28124 28172 55 0.4 2812.4 0.1X +from_json(timestamp) 71432 71827 354 0.1 7143.2 0.0X +from_json(date) 46497 46651 163 0.2 4649.7 0.0X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 6344ec6be4878..c7448b12626be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2610,7 +2610,9 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson } test("inferring timestamp type") { - def schemaOf(jsons: String*): StructType = spark.read.json(jsons.toDS).schema + def schemaOf(jsons: String*): StructType = { + spark.read.option("inferTimestamp", true).json(jsons.toDS).schema + } assert(schemaOf( """{"a":"2018-12-17T10:11:12.123-01:00"}""", @@ -2633,6 +2635,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.json" val timestampsWithFormat = spark.read .option("timestampFormat", "dd/MM/yyyy HH:mm") + .option("inferTimestamp", true) .json(datesRecords) assert(timestampsWithFormat.schema === customSchema) @@ -2645,6 +2648,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson val readBack = spark.read .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .option("inferTimestamp", true) .json(timestampsWithFormatPath) assert(readBack.schema === customSchema) From 2a52a1b300381afe394923fdd2d20ced053f0bd0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 1 Jul 2020 16:14:51 -0700 Subject: [PATCH 109/384] [SPARK-32056][SQL][FOLLOW-UP] Coalesce partitions for repartiotion hint and sql when AQE is enabled ### What changes were proposed in this pull request? As the followup of #28900, this patch extends coalescing partitions to repartitioning using hints and SQL syntax without specifying number of partitions, when AQE is enabled. ### Why are the changes needed? When repartitionning using hints and SQL syntax, we should follow the shuffling behavior of repartition by expression/range to coalesce partitions when AQE is enabled. ### Does this PR introduce _any_ user-facing change? Yes. After this change, if users don't specify the number of partitions when repartitioning using `REPARTITION`/`REPARTITION_BY_RANGE` hint or `DISTRIBUTE BY`/`CLUSTER BY`, AQE will coalesce partitions. ### How was this patch tested? Unit tests. Closes #28952 from viirya/SPARK-32056-sql. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/ResolveHints.scala | 16 ++-- .../catalyst/analysis/ResolveHintsSuite.scala | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 2 +- .../sql/execution/SparkSqlParserSuite.scala | 6 +- .../adaptive/AdaptiveQueryExecSuite.scala | 81 +++++++++++++++---- 5 files changed, 78 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index 81de086e78f91..4cbff62e16cc1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -183,7 +183,7 @@ object ResolveHints { val hintName = hint.name.toUpperCase(Locale.ROOT) def createRepartitionByExpression( - numPartitions: Int, partitionExprs: Seq[Any]): RepartitionByExpression = { + numPartitions: Option[Int], partitionExprs: Seq[Any]): RepartitionByExpression = { val sortOrders = partitionExprs.filter(_.isInstanceOf[SortOrder]) if (sortOrders.nonEmpty) throw new IllegalArgumentException( s"""Invalid partitionExprs specified: $sortOrders @@ -208,11 +208,11 @@ object ResolveHints { throw new AnalysisException(s"$hintName Hint expects a partition number as a parameter") case param @ Seq(IntegerLiteral(numPartitions), _*) if shuffle => - createRepartitionByExpression(numPartitions, param.tail) + createRepartitionByExpression(Some(numPartitions), param.tail) case param @ Seq(numPartitions: Int, _*) if shuffle => - createRepartitionByExpression(numPartitions, param.tail) + createRepartitionByExpression(Some(numPartitions), param.tail) case param @ Seq(_*) if shuffle => - createRepartitionByExpression(conf.numShufflePartitions, param) + createRepartitionByExpression(None, param) } } @@ -224,7 +224,7 @@ object ResolveHints { val hintName = hint.name.toUpperCase(Locale.ROOT) def createRepartitionByExpression( - numPartitions: Int, partitionExprs: Seq[Any]): RepartitionByExpression = { + numPartitions: Option[Int], partitionExprs: Seq[Any]): RepartitionByExpression = { val invalidParams = partitionExprs.filter(!_.isInstanceOf[UnresolvedAttribute]) if (invalidParams.nonEmpty) { throw new AnalysisException(s"$hintName Hint parameter should include columns, but " + @@ -239,11 +239,11 @@ object ResolveHints { hint.parameters match { case param @ Seq(IntegerLiteral(numPartitions), _*) => - createRepartitionByExpression(numPartitions, param.tail) + createRepartitionByExpression(Some(numPartitions), param.tail) case param @ Seq(numPartitions: Int, _*) => - createRepartitionByExpression(numPartitions, param.tail) + createRepartitionByExpression(Some(numPartitions), param.tail) case param @ Seq(_*) => - createRepartitionByExpression(conf.numShufflePartitions, param) + createRepartitionByExpression(None, param) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index d3bd5d07a0932..513f1d001f757 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -163,7 +163,7 @@ class ResolveHintsSuite extends AnalysisTest { checkAnalysis( UnresolvedHint("REPARTITION", Seq(UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( - Seq(AttributeReference("a", IntegerType)()), testRelation, conf.numShufflePartitions)) + Seq(AttributeReference("a", IntegerType)()), testRelation, None)) val e = intercept[IllegalArgumentException] { checkAnalysis( @@ -187,7 +187,7 @@ class ResolveHintsSuite extends AnalysisTest { "REPARTITION_BY_RANGE", Seq(UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), - testRelation, conf.numShufflePartitions)) + testRelation, None)) val errMsg2 = "REPARTITION Hint parameter should include columns, but" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 078813b7d631d..3a2c673229c20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -746,7 +746,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ctx: QueryOrganizationContext, expressions: Seq[Expression], query: LogicalPlan): LogicalPlan = { - RepartitionByExpression(expressions, query, conf.numShufflePartitions) + RepartitionByExpression(expressions, query, None) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 06574a9f8fd2c..1991f139e48c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -199,20 +199,20 @@ class SparkSqlParserSuite extends AnalysisTest { assertEqual(s"$baseSql distribute by a, b", RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil, basePlan, - numPartitions = newConf.numShufflePartitions)) + None)) assertEqual(s"$baseSql distribute by a sort by b", Sort(SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, global = false, RepartitionByExpression(UnresolvedAttribute("a") :: Nil, basePlan, - numPartitions = newConf.numShufflePartitions))) + None))) assertEqual(s"$baseSql cluster by a, b", Sort(SortOrder(UnresolvedAttribute("a"), Ascending) :: SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, global = false, RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil, basePlan, - numPartitions = newConf.numShufflePartitions))) + None))) } test("pipeline concatenation") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 27d9748476c98..c696d3f648ed1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -23,7 +23,7 @@ import java.net.URI import org.apache.log4j.Level import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} -import org.apache.spark.sql.{QueryTest, Row, SparkSession, Strategy} +import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession, Strategy} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} import org.apache.spark.sql.execution.{PartialReducerPartitionSpec, ReusedSubqueryExec, ShuffledRowRDD, SparkPlan} @@ -130,6 +130,17 @@ class AdaptiveQueryExecSuite assert(numShuffles === (numLocalReaders.length + numShufflesWithoutLocalReader)) } + private def checkInitialPartitionNum(df: Dataset[_], numPartition: Int): Unit = { + // repartition obeys initialPartitionNum when adaptiveExecutionEnabled + val plan = df.queryExecution.executedPlan + assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffle.size == 1) + assert(shuffle(0).outputPartitioning.numPartitions == numPartition) + } + test("Change merge join to broadcast join") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", @@ -1040,14 +1051,8 @@ class AdaptiveQueryExecSuite assert(partitionsNum1 < 10) assert(partitionsNum2 < 10) - // repartition obeys initialPartitionNum when adaptiveExecutionEnabled - val plan = df1.queryExecution.executedPlan - assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) - val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { - case s: ShuffleExchangeExec => s - } - assert(shuffle.size == 1) - assert(shuffle(0).outputPartitioning.numPartitions == 10) + checkInitialPartitionNum(df1, 10) + checkInitialPartitionNum(df2, 10) } else { assert(partitionsNum1 === 10) assert(partitionsNum2 === 10) @@ -1081,14 +1086,8 @@ class AdaptiveQueryExecSuite assert(partitionsNum1 < 10) assert(partitionsNum2 < 10) - // repartition obeys initialPartitionNum when adaptiveExecutionEnabled - val plan = df1.queryExecution.executedPlan - assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) - val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { - case s: ShuffleExchangeExec => s - } - assert(shuffle.size == 1) - assert(shuffle(0).outputPartitioning.numPartitions == 10) + checkInitialPartitionNum(df1, 10) + checkInitialPartitionNum(df2, 10) } else { assert(partitionsNum1 === 10) assert(partitionsNum2 === 10) @@ -1100,4 +1099,52 @@ class AdaptiveQueryExecSuite } } } + + test("SPARK-31220, SPARK-32056: repartition using sql and hint with AQE") { + Seq(true, false).foreach { enableAQE => + withTempView("test") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10", + SQLConf.SHUFFLE_PARTITIONS.key -> "10") { + + spark.range(10).toDF.createTempView("test") + + val df1 = spark.sql("SELECT /*+ REPARTITION(id) */ * from test") + val df2 = spark.sql("SELECT /*+ REPARTITION_BY_RANGE(id) */ * from test") + val df3 = spark.sql("SELECT * from test DISTRIBUTE BY id") + val df4 = spark.sql("SELECT * from test CLUSTER BY id") + + val partitionsNum1 = df1.rdd.collectPartitions().length + val partitionsNum2 = df2.rdd.collectPartitions().length + val partitionsNum3 = df3.rdd.collectPartitions().length + val partitionsNum4 = df4.rdd.collectPartitions().length + + if (enableAQE) { + assert(partitionsNum1 < 10) + assert(partitionsNum2 < 10) + assert(partitionsNum3 < 10) + assert(partitionsNum4 < 10) + + checkInitialPartitionNum(df1, 10) + checkInitialPartitionNum(df2, 10) + checkInitialPartitionNum(df3, 10) + checkInitialPartitionNum(df4, 10) + } else { + assert(partitionsNum1 === 10) + assert(partitionsNum2 === 10) + assert(partitionsNum3 === 10) + assert(partitionsNum4 === 10) + } + + // Don't coalesce partitions if the number of partitions is specified. + val df5 = spark.sql("SELECT /*+ REPARTITION(10, id) */ * from test") + val df6 = spark.sql("SELECT /*+ REPARTITION_BY_RANGE(10, id) */ * from test") + assert(df5.rdd.collectPartitions().length == 10) + assert(df6.rdd.collectPartitions().length == 10) + } + } + } + } } From ced8e0e66226636a4bfbd58ba05f2c7f7f252d1a Mon Sep 17 00:00:00 2001 From: Rajat Ahuja Date: Wed, 1 Jul 2020 18:28:14 -0700 Subject: [PATCH 110/384] [SPARK-29465][YARN][WEBUI] Adding Check to not to set UI port (spark.ui.port) property if mentioned explicitly ## What changes were proposed in this pull request? When a Spark Job launched in Cluster mode with Yarn, Application Master sets spark.ui.port port to 0 which means Driver's web UI gets any random port even if we want to explicitly set the Port range for Driver's Web UI ## Why are the changes needed? We access Spark Web UI via Knox Proxy, and there are firewall restrictions due to which we can not access Spark Web UI since Web UI port range gets random port even if we set explicitly. This Change will check if there is a specified port range explicitly mentioned so that it does not assign a random port. ## Does this PR introduce any user-facing change? No ## How was this patch tested? Local Tested. Closes #28880 from rajatahujaatinmobi/ahujarajat261/SPARK-32039-change-yarn-webui-port-range-with-property-latest-spark. Authored-by: Rajat Ahuja Signed-off-by: Dongjoon Hyun --- .../apache/spark/deploy/yarn/ApplicationMaster.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9a6a43914bca3..5ca624a8d66cb 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -211,9 +211,13 @@ private[spark] class ApplicationMaster( final def run(): Int = { try { val attemptID = if (isClusterMode) { - // Set the web ui port to be ephemeral for yarn so we don't conflict with - // other spark processes running on the same box - System.setProperty(UI_PORT.key, "0") + // Set the web ui port to be ephemeral for yarn if not set explicitly + // so we don't conflict with other spark processes running on the same box + // If set explicitly, Web UI will attempt to run on UI_PORT and try + // incrementally until UI_PORT + `spark.port.maxRetries` + if (System.getProperty(UI_PORT.key) == null) { + System.setProperty(UI_PORT.key, "0") + } // Set the master and deploy mode property to match the requested mode. System.setProperty("spark.master", "yarn") From 3f7780d30d712e6d3894bacb5e80113c7a4bcc09 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 2 Jul 2020 13:56:43 +0900 Subject: [PATCH 111/384] [SPARK-32136][SQL] NormalizeFloatingNumbers should work on null struct ### What changes were proposed in this pull request? This patch fixes wrong groupBy result if the grouping key is a null-value struct. ### Why are the changes needed? `NormalizeFloatingNumbers` reconstructs a struct if input expression is StructType. If the input struct is null, it will reconstruct a struct with null-value fields, instead of null. ### Does this PR introduce _any_ user-facing change? Yes, fixing incorrect groupBy result. ### How was this patch tested? Unit test. Closes #28962 from viirya/SPARK-32136. Authored-by: Liang-Chi Hsieh Signed-off-by: HyukjinKwon --- .../optimizer/NormalizeFloatingNumbers.scala | 5 +++-- .../apache/spark/sql/DataFrameAggregateSuite.scala | 12 ++++++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala index 43738204c6704..8d5dbc7dc90eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, KnownFloatingPointNormalized, LambdaFunction, NamedLambdaVariable, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, If, IsNull, KnownFloatingPointNormalized, LambdaFunction, Literal, NamedLambdaVariable, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, Window} @@ -123,7 +123,8 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { val fields = expr.dataType.asInstanceOf[StructType].fields.indices.map { i => normalize(GetStructField(expr, i)) } - CreateStruct(fields) + val struct = CreateStruct(fields) + KnownFloatingPointNormalized(If(IsNull(expr), Literal(null, struct.dataType), struct)) case _ if expr.dataType.isInstanceOf[ArrayType] => val ArrayType(et, containsNull) = expr.dataType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index f7438f3ffec04..09f30bb5e2c77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -1028,4 +1028,16 @@ class DataFrameAggregateSuite extends QueryTest checkAnswer(df, Row("abellina", 2) :: Row("mithunr", 1) :: Nil) } } + + test("SPARK-32136: NormalizeFloatingNumbers should work on null struct") { + val df = Seq( + A(None), + A(Some(B(None))), + A(Some(B(Some(1.0))))).toDF + val groupBy = df.groupBy("b").agg(count("*")) + checkAnswer(groupBy, Row(null, 1) :: Row(Row(null), 1) :: Row(Row(1.0), 1) :: Nil) + } } + +case class B(c: Option[Double]) +case class A(b: Option[B]) From 7fda184f0fc39613fb68e912c189c54b93c638e6 Mon Sep 17 00:00:00 2001 From: pancheng <379377944@qq.com> Date: Thu, 2 Jul 2020 19:21:11 +0900 Subject: [PATCH 112/384] [SPARK-32121][SHUFFLE] Support Windows OS in ExecutorDiskUtils ### What changes were proposed in this pull request? Correct file seprate use in `ExecutorDiskUtils.createNormalizedInternedPathname` on Windows ### Why are the changes needed? `ExternalShuffleBlockResolverSuite` failed on Windows, see detail at: https://issues.apache.org/jira/browse/SPARK-32121 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The existed test suite. Closes #28940 from pan3793/SPARK-32121. Lead-authored-by: pancheng <379377944@qq.com> Co-authored-by: chengpan Signed-off-by: HyukjinKwon --- .../network/shuffle/ExecutorDiskUtils.java | 18 ++++++++++++--- .../ExternalShuffleBlockResolverSuite.java | 22 +++++++++++++------ 2 files changed, 30 insertions(+), 10 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java index 13f6046dd856b..6549cac011feb 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java @@ -23,11 +23,19 @@ import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang3.SystemUtils; import org.apache.spark.network.util.JavaUtils; public class ExecutorDiskUtils { - private static final Pattern MULTIPLE_SEPARATORS = Pattern.compile(File.separator + "{2,}"); + private static final Pattern MULTIPLE_SEPARATORS; + static { + if (SystemUtils.IS_OS_WINDOWS) { + MULTIPLE_SEPARATORS = Pattern.compile("[/\\\\]+"); + } else { + MULTIPLE_SEPARATORS = Pattern.compile("/{2,}"); + } + } /** * Hashes a filename into the corresponding local directory, in a manner consistent with @@ -50,14 +58,18 @@ public static File getFile(String[] localDirs, int subDirsPerLocalDir, String fi * the internal code in java.io.File would normalize it later, creating a new "foo/bar" * String copy. Unfortunately, we cannot just reuse the normalization code that java.io.File * uses, since it is in the package-private class java.io.FileSystem. + * + * On Windows, separator "\" is used instead of "/". + * + * "\\" is a legal character in path name on Unix-like OS, but illegal on Windows. */ @VisibleForTesting static String createNormalizedInternedPathname(String dir1, String dir2, String fname) { String pathname = dir1 + File.separator + dir2 + File.separator + fname; Matcher m = MULTIPLE_SEPARATORS.matcher(pathname); - pathname = m.replaceAll("/"); + pathname = m.replaceAll(Matcher.quoteReplacement(File.separator)); // A single trailing slash needs to be taken care of separately - if (pathname.length() > 1 && pathname.endsWith("/")) { + if (pathname.length() > 1 && pathname.charAt(pathname.length() - 1) == File.separatorChar) { pathname = pathname.substring(0, pathname.length() - 1); } return pathname.intern(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 09b31430b1eb9..6515b6ca035f7 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.io.CharStreams; +import org.apache.commons.lang3.SystemUtils; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -146,12 +147,19 @@ public void jsonSerializationOfExecutorRegistration() throws IOException { @Test public void testNormalizeAndInternPathname() { - assertPathsMatch("/foo", "bar", "baz", "/foo/bar/baz"); - assertPathsMatch("//foo/", "bar/", "//baz", "/foo/bar/baz"); - assertPathsMatch("foo", "bar", "baz///", "foo/bar/baz"); - assertPathsMatch("/foo/", "/bar//", "/baz", "/foo/bar/baz"); - assertPathsMatch("/", "", "", "/"); - assertPathsMatch("/", "/", "/", "/"); + String sep = File.separator; + String expectedPathname = sep + "foo" + sep + "bar" + sep + "baz"; + assertPathsMatch("/foo", "bar", "baz", expectedPathname); + assertPathsMatch("//foo/", "bar/", "//baz", expectedPathname); + assertPathsMatch("/foo/", "/bar//", "/baz", expectedPathname); + assertPathsMatch("foo", "bar", "baz///", "foo" + sep + "bar" + sep + "baz"); + assertPathsMatch("/", "", "", sep); + assertPathsMatch("/", "/", "/", sep); + if (SystemUtils.IS_OS_WINDOWS) { + assertPathsMatch("/foo\\/", "bar", "baz", expectedPathname); + } else { + assertPathsMatch("/foo\\/", "bar", "baz", sep + "foo\\" + sep + "bar" + sep + "baz"); + } } private void assertPathsMatch(String p1, String p2, String p3, String expectedPathname) { @@ -160,6 +168,6 @@ private void assertPathsMatch(String p1, String p2, String p3, String expectedPa assertEquals(expectedPathname, normPathname); File file = new File(normPathname); String returnedPath = file.getPath(); - assertTrue(normPathname == returnedPath); + assertEquals(normPathname, returnedPath); } } From 45fe6b62a73540ff010317fc7518b007206707d6 Mon Sep 17 00:00:00 2001 From: animenon Date: Thu, 2 Jul 2020 21:02:00 +0900 Subject: [PATCH 113/384] [MINOR][DOCS] Pyspark getActiveSession docstring ### What changes were proposed in this pull request? Minor fix so that the documentation of `getActiveSession` is fixed. The sample code snippet doesn't come up formatted rightly, added spacing for this to be fixed. Also added return to docs. ### Why are the changes needed? The sample code is getting mixed up as description in the docs. [Current Doc Link](http://spark.apache.org/docs/latest/api/python/pyspark.sql.html?highlight=getactivesession#pyspark.sql.SparkSession.getActiveSession) ![image](https://user-images.githubusercontent.com/6907950/86331522-d7b6f800-bc66-11ea-998c-42085f5e5b04.png) ### Does this PR introduce _any_ user-facing change? Yes, documentation of getActiveSession is fixed. And added description about return. ### How was this patch tested? Adding a spacing between description and code seems to fix the issue. Closes #28978 from animenon/docs_minor. Authored-by: animenon Signed-off-by: HyukjinKwon --- python/pyspark/sql/session.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index be4fa20a04327..61891c478dbe4 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -265,7 +265,10 @@ def newSession(self): @since(3.0) def getActiveSession(cls): """ - Returns the active SparkSession for the current thread, returned by the builder. + Returns the active SparkSession for the current thread, returned by the builder + + :return: :class:`SparkSession` if an active session exists for the current thread + >>> s = SparkSession.getActiveSession() >>> l = [('Alice', 1)] >>> rdd = s.sparkContext.parallelize(l) From f83415629b18d628f72a32285f0afc24f29eaa1e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 2 Jul 2020 21:04:26 +0900 Subject: [PATCH 114/384] [MINOR][TEST][SQL] Make in-limit.sql more robust ### What changes were proposed in this pull request? For queries like `t1d in (SELECT t2d FROM t2 ORDER BY t2c LIMIT 2)`, the result can be non-deterministic as the result of the subquery may output different results (it's not sorted by `t2d` and it has shuffle). This PR makes the test more robust by sorting the output column. ### Why are the changes needed? avoid flaky test ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? N/A Closes #28976 from cloud-fan/small. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon --- .../sql-tests/inputs/subquery/in-subquery/in-limit.sql | 4 ++-- .../sql-tests/results/subquery/in-subquery/in-limit.sql.out | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql index 481b5e8cc7700..0a16f118f0455 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql @@ -72,7 +72,7 @@ SELECT Count(DISTINCT( t1a )), FROM t1 WHERE t1d IN (SELECT t2d FROM t2 - ORDER BY t2c + ORDER BY t2c, t2d LIMIT 2) GROUP BY t1b ORDER BY t1b DESC NULLS FIRST @@ -93,7 +93,7 @@ SELECT Count(DISTINCT( t1a )), FROM t1 WHERE t1d NOT IN (SELECT t2d FROM t2 - ORDER BY t2b DESC nulls first + ORDER BY t2b DESC nulls first, t2d LIMIT 1) GROUP BY t1b ORDER BY t1b NULLS last diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out index 1c335445114c7..e24538b9138ba 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out @@ -103,7 +103,7 @@ SELECT Count(DISTINCT( t1a )), FROM t1 WHERE t1d IN (SELECT t2d FROM t2 - ORDER BY t2c + ORDER BY t2c, t2d LIMIT 2) GROUP BY t1b ORDER BY t1b DESC NULLS FIRST @@ -136,7 +136,7 @@ SELECT Count(DISTINCT( t1a )), FROM t1 WHERE t1d NOT IN (SELECT t2d FROM t2 - ORDER BY t2b DESC nulls first + ORDER BY t2b DESC nulls first, t2d LIMIT 1) GROUP BY t1b ORDER BY t1b NULLS last From f082a7996add1d6edbae6e0ee2a1409834813c0c Mon Sep 17 00:00:00 2001 From: stczwd Date: Thu, 2 Jul 2020 14:49:40 +0000 Subject: [PATCH 115/384] [SPARK-31100][SQL] Check namespace existens when setting namespace ## What changes were proposed in this pull request? Check the namespace existence while calling "use namespace", and throw NoSuchNamespaceException if namespace not exists. ### Why are the changes needed? Users need to know that the namespace does not exist when they try to set a wrong namespace. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Run all suites and add a test for this Closes #27900 from stczwd/SPARK-31100. Authored-by: stczwd Signed-off-by: Wenchen Fan --- .../connector/catalog/CatalogManager.scala | 15 ++++++---- .../catalog/CatalogManagerSuite.scala | 16 +++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 28 +++++++++++++++---- 3 files changed, 48 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index a81d8f79d6fcc..c6d21540f27d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -42,6 +42,7 @@ class CatalogManager( defaultSessionCatalog: CatalogPlugin, val v1SessionCatalog: SessionCatalog) extends Logging { import CatalogManager.SESSION_CATALOG_NAME + import CatalogV2Util._ private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] @@ -106,13 +107,15 @@ class CatalogManager( } def setCurrentNamespace(namespace: Array[String]): Unit = synchronized { - if (currentCatalog.name() == SESSION_CATALOG_NAME) { - if (namespace.length != 1) { + currentCatalog match { + case _ if isSessionCatalog(currentCatalog) && namespace.length == 1 => + v1SessionCatalog.setCurrentDatabase(namespace.head) + case _ if isSessionCatalog(currentCatalog) => throw new NoSuchNamespaceException(namespace) - } - v1SessionCatalog.setCurrentDatabase(namespace.head) - } else { - _currentNamespace = Some(namespace) + case catalog: SupportsNamespaces if !catalog.namespaceExists(namespace) => + throw new NoSuchNamespaceException(namespace) + case _ => + _currentNamespace = Some(namespace) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index 17d326019f86b..7dd0753fcf777 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -19,9 +19,12 @@ package org.apache.spark.sql.connector.catalog import java.net.URI +import scala.collection.JavaConverters._ + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, FakeV2SessionCatalog, NoSuchNamespaceException} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.connector.InMemoryTableCatalog import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -108,6 +111,19 @@ class CatalogManagerSuite extends SparkFunSuite { assert(v1SessionCatalog.getCurrentDatabase == "default") catalogManager.setCurrentNamespace(Array("test2")) assert(v1SessionCatalog.getCurrentDatabase == "default") + + // Check namespace existence if currentCatalog implements SupportsNamespaces. + conf.setConfString("spark.sql.catalog.testCatalog", classOf[InMemoryTableCatalog].getName) + catalogManager.setCurrentCatalog("testCatalog") + catalogManager.currentCatalog.asInstanceOf[InMemoryTableCatalog] + .createNamespace(Array("test3"), Map.empty[String, String].asJava) + assert(v1SessionCatalog.getCurrentDatabase == "default") + catalogManager.setCurrentNamespace(Array("test3")) + assert(v1SessionCatalog.getCurrentDatabase == "default") + + intercept[NoSuchNamespaceException] { + catalogManager.setCurrentNamespace(Array("ns1", "ns2")) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 8462ce5a6c44f..f7f4df8f2d2e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1479,11 +1479,26 @@ class DataSourceV2SQLSuite assert(exception.getMessage.contains("Database 'ns1' not found")) } - test("Use: v2 catalog is used and namespace does not exist") { - // Namespaces are not required to exist for v2 catalogs. - sql("USE testcat.ns1.ns2") - val catalogManager = spark.sessionState.catalogManager - assert(catalogManager.currentNamespace === Array("ns1", "ns2")) + test("SPARK-31100: Use: v2 catalog that implements SupportsNamespaces is used " + + "and namespace not exists") { + // Namespaces are required to exist for v2 catalogs that implements SupportsNamespaces. + val exception = intercept[NoSuchNamespaceException] { + sql("USE testcat.ns1.ns2") + } + assert(exception.getMessage.contains("Namespace 'ns1.ns2' not found")) + } + + test("SPARK-31100: Use: v2 catalog that does not implement SupportsNameSpaces is used " + + "and namespace does not exist") { + // Namespaces are not required to exist for v2 catalogs + // that does not implement SupportsNamespaces. + withSQLConf("spark.sql.catalog.dummy" -> classOf[BasicInMemoryTableCatalog].getName) { + val catalogManager = spark.sessionState.catalogManager + + sql("USE dummy.ns1") + assert(catalogManager.currentCatalog.name() == "dummy") + assert(catalogManager.currentNamespace === Array("ns1")) + } } test("ShowCurrentNamespace: basic tests") { @@ -1505,6 +1520,8 @@ class DataSourceV2SQLSuite sql("USE testcat") testShowCurrentNamespace("testcat", "") + + sql("CREATE NAMESPACE testcat.ns1.ns2") sql("USE testcat.ns1.ns2") testShowCurrentNamespace("testcat", "ns1.ns2") } @@ -2342,6 +2359,7 @@ class DataSourceV2SQLSuite spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) val sessionCatalogName = CatalogManager.SESSION_CATALOG_NAME + sql("CREATE NAMESPACE testcat.ns1.ns2") sql("USE testcat.ns1.ns2") sql("CREATE TABLE t USING foo AS SELECT 1 col") checkAnswer(spark.table("t"), Row(1)) From 0acad589e120cd777b25c03777a3cce4ef704422 Mon Sep 17 00:00:00 2001 From: TJX2014 Date: Thu, 2 Jul 2020 10:15:10 -0700 Subject: [PATCH 116/384] [SPARK-32156][SPARK-31061][TESTS][SQL] Refactor two similar test cases from in HiveExternalCatalogSuite ### What changes were proposed in this pull request? 1.Merge two similar tests for SPARK-31061 and make the code clean. 2.Fix table alter issue due to lose path. ### Why are the changes needed? Because this two tests for SPARK-31061 is very similar and could be merged. And the first test case should use `rawTable` instead of `parquetTable` to alter. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test. Closes #28980 from TJX2014/master-follow-merge-spark-31061-test-case. Authored-by: TJX2014 Signed-off-by: Dongjoon Hyun --- .../sql/hive/HiveExternalCatalogSuite.scala | 54 +++++++------------ 1 file changed, 19 insertions(+), 35 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 473a93bf129df..270595b0011e9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -181,41 +181,25 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { "INSERT overwrite directory \"fs://localhost/tmp\" select 1 as a")) } - test("SPARK-31061: alterTable should be able to change table provider") { + test("SPARK-31061: alterTable should be able to change table provider/hive") { val catalog = newBasicCatalog() - val parquetTable = CatalogTable( - identifier = TableIdentifier("parq_tbl", Some("db1")), - tableType = CatalogTableType.MANAGED, - storage = storageFormat.copy(locationUri = Some(new URI("file:/some/path"))), - schema = new StructType().add("col1", "int").add("col2", "string"), - provider = Some("parquet")) - catalog.createTable(parquetTable, ignoreIfExists = false) - - val rawTable = externalCatalog.getTable("db1", "parq_tbl") - assert(rawTable.provider === Some("parquet")) - - val fooTable = parquetTable.copy(provider = Some("foo")) - catalog.alterTable(fooTable) - val alteredTable = externalCatalog.getTable("db1", "parq_tbl") - assert(alteredTable.provider === Some("foo")) - } - - test("SPARK-31061: alterTable should be able to change table provider from hive") { - val catalog = newBasicCatalog() - val hiveTable = CatalogTable( - identifier = TableIdentifier("parq_tbl", Some("db1")), - tableType = CatalogTableType.MANAGED, - storage = storageFormat, - schema = new StructType().add("col1", "int").add("col2", "string"), - provider = Some("hive")) - catalog.createTable(hiveTable, ignoreIfExists = false) - - val rawTable = externalCatalog.getTable("db1", "parq_tbl") - assert(rawTable.provider === Some("hive")) - - val fooTable = rawTable.copy(provider = Some("foo")) - catalog.alterTable(fooTable) - val alteredTable = externalCatalog.getTable("db1", "parq_tbl") - assert(alteredTable.provider === Some("foo")) + Seq("parquet", "hive").foreach( provider => { + val tableDDL = CatalogTable( + identifier = TableIdentifier("parq_tbl", Some("db1")), + tableType = CatalogTableType.MANAGED, + storage = storageFormat, + schema = new StructType().add("col1", "int"), + provider = Some(provider)) + catalog.dropTable("db1", "parq_tbl", true, true) + catalog.createTable(tableDDL, ignoreIfExists = false) + + val rawTable = externalCatalog.getTable("db1", "parq_tbl") + assert(rawTable.provider === Some(provider)) + + val fooTable = rawTable.copy(provider = Some("foo")) + catalog.alterTable(fooTable) + val alteredTable = externalCatalog.getTable("db1", "parq_tbl") + assert(alteredTable.provider === Some("foo")) + }) } } From 42f01e314b4874236544cc8b94bef766269385ee Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 2 Jul 2020 13:26:57 -0700 Subject: [PATCH 117/384] [SPARK-32130][SQL][FOLLOWUP] Enable timestamps inference in JsonBenchmark ### What changes were proposed in this pull request? Set the JSON option `inferTimestamp` to `true` for the cases that measure perf of timestamp inference. ### Why are the changes needed? The PR https://github.com/apache/spark/pull/28966 disabled timestamp inference by default. As a consequence, some benchmarks don't measure perf of timestamp inference from JSON fields. This PR explicitly enable such inference. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By re-generating results of `JsonBenchmark`. Closes #28981 from MaxGekk/json-inferTimestamps-disable-by-default-followup. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- .../JsonBenchmark-jdk11-results.txt | 86 +++++++++---------- sql/core/benchmarks/JsonBenchmark-results.txt | 86 +++++++++---------- .../datasources/json/JsonBenchmark.scala | 4 +- 3 files changed, 88 insertions(+), 88 deletions(-) diff --git a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt index ff370847a2e3e..2d506f03d9f7e 100644 --- a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt @@ -7,106 +7,106 @@ 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 JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 69219 69342 116 1.4 692.2 1.0X -UTF-8 is set 143950 143986 55 0.7 1439.5 0.5X +No encoding 73307 73400 141 1.4 733.1 1.0X +UTF-8 is set 143834 143925 152 0.7 1438.3 0.5X Preparing data for benchmarking ... 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 57828 57913 136 1.7 578.3 1.0X -UTF-8 is set 83649 83711 60 1.2 836.5 0.7X +No encoding 50894 51065 292 2.0 508.9 1.0X +UTF-8 is set 98462 99455 1173 1.0 984.6 0.5X Preparing data for benchmarking ... 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 64560 65193 1023 0.2 6456.0 1.0X -UTF-8 is set 102925 103174 216 0.1 10292.5 0.6X +No encoding 64011 64969 1001 0.2 6401.1 1.0X +UTF-8 is set 102757 102984 311 0.1 10275.7 0.6X Preparing data for benchmarking ... 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 131002 132316 1160 0.0 262003.1 1.0X -UTF-8 is set 152128 152371 332 0.0 304256.5 0.9X +No encoding 132559 133561 1010 0.0 265117.3 1.0X +UTF-8 is set 151458 152129 611 0.0 302915.4 0.9X Preparing data for benchmarking ... 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 19376 19514 160 0.5 1937.6 1.0X -Select 1 column 24089 24156 58 0.4 2408.9 0.8X +Select 10 columns 21148 21202 87 0.5 2114.8 1.0X +Select 1 column 24701 24724 21 0.4 2470.1 0.9X Preparing data for benchmarking ... 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 8131 8219 103 1.2 813.1 1.0X -Short column with UTF-8 13464 13508 44 0.7 1346.4 0.6X -Wide column without encoding 108012 108598 914 0.1 10801.2 0.1X -Wide column with UTF-8 150988 151369 412 0.1 15098.8 0.1X +Short column without encoding 6945 6998 59 1.4 694.5 1.0X +Short column with UTF-8 11510 11569 51 0.9 1151.0 0.6X +Wide column without encoding 95004 95795 790 0.1 9500.4 0.1X +Wide column with UTF-8 149223 149409 276 0.1 14922.3 0.0X Preparing data for benchmarking ... 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 753 765 18 13.3 75.3 1.0X -from_json 23182 23446 230 0.4 2318.2 0.0X -json_tuple 31129 31304 181 0.3 3112.9 0.0X -get_json_object 22821 23073 225 0.4 2282.1 0.0X +Text read 649 652 3 15.4 64.9 1.0X +from_json 22284 22393 99 0.4 2228.4 0.0X +json_tuple 32310 32824 484 0.3 3231.0 0.0X +get_json_object 22111 22751 568 0.5 2211.1 0.0X Preparing data for benchmarking ... 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 3078 3101 26 16.2 61.6 1.0X -schema inferring 30225 30434 333 1.7 604.5 0.1X -parsing 32237 32308 63 1.6 644.7 0.1X +Text read 2894 2903 8 17.3 57.9 1.0X +schema inferring 26724 26785 62 1.9 534.5 0.1X +parsing 37502 37632 131 1.3 750.0 0.1X Preparing data for benchmarking ... 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 10835 10900 86 4.6 216.7 1.0X -Schema inferring 37720 37805 110 1.3 754.4 0.3X -Parsing without charset 35464 35538 100 1.4 709.3 0.3X -Parsing with UTF-8 67311 67738 381 0.7 1346.2 0.2X +Text read 10994 11010 16 4.5 219.9 1.0X +Schema inferring 45654 45677 37 1.1 913.1 0.2X +Parsing without charset 34476 34559 73 1.5 689.5 0.3X +Parsing with UTF-8 56987 57002 13 0.9 1139.7 0.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 Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2208 2222 14 4.5 220.8 1.0X -to_json(timestamp) 14299 14570 285 0.7 1429.9 0.2X -write timestamps to files 12955 12969 13 0.8 1295.5 0.2X -Create a dataset of dates 2297 2323 30 4.4 229.7 1.0X -to_json(date) 8509 8561 74 1.2 850.9 0.3X -write dates to files 6786 6827 45 1.5 678.6 0.3X +Create a dataset of timestamps 2150 2188 35 4.7 215.0 1.0X +to_json(timestamp) 17874 18080 294 0.6 1787.4 0.1X +write timestamps to files 12518 12538 34 0.8 1251.8 0.2X +Create a dataset of dates 2298 2310 18 4.4 229.8 0.9X +to_json(date) 11673 11703 27 0.9 1167.3 0.2X +write dates to files 7121 7135 12 1.4 712.1 0.3X 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 2598 2613 18 3.8 259.8 1.0X -read timestamps from files 42007 42028 19 0.2 4200.7 0.1X -infer timestamps from files 18102 18120 28 0.6 1810.2 0.1X -read date text from files 2355 2360 5 4.2 235.5 1.1X -read date from files 17420 17458 33 0.6 1742.0 0.1X -timestamp strings 3099 3101 3 3.2 309.9 0.8X -parse timestamps from Dataset[String] 48188 48215 25 0.2 4818.8 0.1X -infer timestamps from Dataset[String] 22929 22988 102 0.4 2292.9 0.1X -date strings 4090 4103 11 2.4 409.0 0.6X -parse dates from Dataset[String] 24952 25068 139 0.4 2495.2 0.1X -from_json(timestamp) 66038 66352 413 0.2 6603.8 0.0X -from_json(date) 43755 43782 27 0.2 4375.5 0.1X +read timestamp text from files 2616 2641 34 3.8 261.6 1.0X +read timestamps from files 37481 37517 58 0.3 3748.1 0.1X +infer timestamps from files 84774 84964 201 0.1 8477.4 0.0X +read date text from files 2362 2365 3 4.2 236.2 1.1X +read date from files 16583 16612 29 0.6 1658.3 0.2X +timestamp strings 3927 3963 40 2.5 392.7 0.7X +parse timestamps from Dataset[String] 52827 53004 243 0.2 5282.7 0.0X +infer timestamps from Dataset[String] 101108 101644 769 0.1 10110.8 0.0X +date strings 4886 4906 26 2.0 488.6 0.5X +parse dates from Dataset[String] 27623 27694 62 0.4 2762.3 0.1X +from_json(timestamp) 71764 71887 124 0.1 7176.4 0.0X +from_json(date) 46200 46314 99 0.2 4620.0 0.1X diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt index 0e4ce9003f70b..c22118f91b3fc 100644 --- a/sql/core/benchmarks/JsonBenchmark-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -7,106 +7,106 @@ 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 JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 64950 65182 306 1.5 649.5 1.0X -UTF-8 is set 129566 129796 229 0.8 1295.7 0.5X +No encoding 63839 64000 263 1.6 638.4 1.0X +UTF-8 is set 124633 124945 429 0.8 1246.3 0.5X Preparing data for benchmarking ... 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 50896 51277 372 2.0 509.0 1.0X -UTF-8 is set 89712 89763 49 1.1 897.1 0.6X +No encoding 51720 51901 157 1.9 517.2 1.0X +UTF-8 is set 91161 91190 25 1.1 911.6 0.6X Preparing data for benchmarking ... 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 59415 59785 372 0.2 5941.5 1.0X -UTF-8 is set 103059 103165 156 0.1 10305.9 0.6X +No encoding 58486 59038 714 0.2 5848.6 1.0X +UTF-8 is set 103045 103350 358 0.1 10304.5 0.6X Preparing data for benchmarking ... 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 132951 133122 288 0.0 265901.9 1.0X -UTF-8 is set 149318 149441 107 0.0 298635.3 0.9X +No encoding 134909 135024 105 0.0 269818.6 1.0X +UTF-8 is set 154418 154593 155 0.0 308836.7 0.9X Preparing data for benchmarking ... 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 18491 18552 85 0.5 1849.1 1.0X -Select 1 column 25908 25946 65 0.4 2590.8 0.7X +Select 10 columns 19538 19620 70 0.5 1953.8 1.0X +Select 1 column 26142 26159 15 0.4 2614.2 0.7X Preparing data for benchmarking ... 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 9264 9307 49 1.1 926.4 1.0X -Short column with UTF-8 14707 14727 17 0.7 1470.7 0.6X -Wide column without encoding 141138 141347 276 0.1 14113.8 0.1X -Wide column with UTF-8 179601 180035 664 0.1 17960.1 0.1X +Short column without encoding 8103 8162 53 1.2 810.3 1.0X +Short column with UTF-8 13104 13150 58 0.8 1310.4 0.6X +Wide column without encoding 135280 135593 375 0.1 13528.0 0.1X +Wide column with UTF-8 175189 175483 278 0.1 17518.9 0.0X Preparing data for benchmarking ... 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 1173 1184 9 8.5 117.3 1.0X -from_json 23432 23738 338 0.4 2343.2 0.1X -json_tuple 32573 32851 358 0.3 3257.3 0.0X -get_json_object 22442 22489 47 0.4 2244.2 0.1X +Text read 1225 1234 8 8.2 122.5 1.0X +from_json 22482 22552 95 0.4 2248.2 0.1X +json_tuple 30203 30338 146 0.3 3020.3 0.0X +get_json_object 22219 22245 26 0.5 2221.9 0.1X Preparing data for benchmarking ... 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 5656 5680 31 8.8 113.1 1.0X -schema inferring 33283 33337 64 1.5 665.7 0.2X -parsing 41771 41929 178 1.2 835.4 0.1X +Text read 5897 5904 10 8.5 117.9 1.0X +schema inferring 30282 30340 50 1.7 605.6 0.2X +parsing 33304 33577 289 1.5 666.1 0.2X Preparing data for benchmarking ... 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 9626 9668 39 5.2 192.5 1.0X -Schema inferring 39489 39579 91 1.3 789.8 0.2X -Parsing without charset 38096 38232 125 1.3 761.9 0.3X -Parsing with UTF-8 64565 64725 165 0.8 1291.3 0.1X +Text read 9710 9757 80 5.1 194.2 1.0X +Schema inferring 35929 35939 9 1.4 718.6 0.3X +Parsing without charset 39175 39227 87 1.3 783.5 0.2X +Parsing with UTF-8 59188 59294 109 0.8 1183.8 0.2X 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 1898 1912 13 5.3 189.8 1.0X -to_json(timestamp) 20011 20092 119 0.5 2001.1 0.1X -write timestamps to files 13388 13427 35 0.7 1338.8 0.1X -Create a dataset of dates 2351 2368 18 4.3 235.1 0.8X -to_json(date) 11884 11913 40 0.8 1188.4 0.2X -write dates to files 7317 7326 9 1.4 731.7 0.3X +Create a dataset of timestamps 1967 1977 9 5.1 196.7 1.0X +to_json(timestamp) 17086 17304 371 0.6 1708.6 0.1X +write timestamps to files 12691 12716 28 0.8 1269.1 0.2X +Create a dataset of dates 2192 2217 39 4.6 219.2 0.9X +to_json(date) 10541 10656 137 0.9 1054.1 0.2X +write dates to files 7259 7311 46 1.4 725.9 0.3X 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 2316 2324 13 4.3 231.6 1.0X -read timestamps from files 43712 43900 165 0.2 4371.2 0.1X -infer timestamps from files 19302 19328 38 0.5 1930.2 0.1X -read date text from files 2090 2099 11 4.8 209.0 1.1X -read date from files 18914 18940 44 0.5 1891.4 0.1X -timestamp strings 3785 3793 11 2.6 378.5 0.6X -parse timestamps from Dataset[String] 51177 51353 160 0.2 5117.7 0.0X -infer timestamps from Dataset[String] 27907 28119 186 0.4 2790.7 0.1X -date strings 4446 4452 6 2.2 444.6 0.5X -parse dates from Dataset[String] 28124 28172 55 0.4 2812.4 0.1X -from_json(timestamp) 71432 71827 354 0.1 7143.2 0.0X -from_json(date) 46497 46651 163 0.2 4649.7 0.0X +read timestamp text from files 2318 2326 13 4.3 231.8 1.0X +read timestamps from files 43345 43627 258 0.2 4334.5 0.1X +infer timestamps from files 89570 89621 59 0.1 8957.0 0.0X +read date text from files 2099 2107 9 4.8 209.9 1.1X +read date from files 18000 18065 98 0.6 1800.0 0.1X +timestamp strings 3937 3956 32 2.5 393.7 0.6X +parse timestamps from Dataset[String] 56001 56429 539 0.2 5600.1 0.0X +infer timestamps from Dataset[String] 109410 109963 559 0.1 10941.0 0.0X +date strings 4530 4540 9 2.2 453.0 0.5X +parse dates from Dataset[String] 29723 29767 72 0.3 2972.3 0.1X +from_json(timestamp) 74106 74619 728 0.1 7410.6 0.0X +from_json(date) 46599 46632 32 0.2 4659.9 0.0X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala index 56930880ed5da..0dbd6b5754afb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala @@ -430,7 +430,7 @@ object JsonBenchmark extends SqlBasedBenchmark { } readBench.addCase("infer timestamps from files", numIters) { _ => - spark.read.json(timestampDir).noop() + spark.read.option("inferTimestamp", true).json(timestampDir).noop() } val dateSchema = new StructType().add("date", DateType) @@ -460,7 +460,7 @@ object JsonBenchmark extends SqlBasedBenchmark { } readBench.addCase("infer timestamps from Dataset[String]", numIters) { _ => - spark.read.json(timestampStr).noop() + spark.read.option("inferTimestamp", true).json(timestampStr).noop() } def dateStr: Dataset[String] = { From 492d5d174a435c624bd87af9ee3621f4f1c8d1c5 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 4 Jul 2020 19:01:07 -0700 Subject: [PATCH 118/384] [SPARK-32171][SQL][DOCS] Change file locations for use db and refresh table ### What changes were proposed in this pull request? docs/sql-ref-syntax-qry-select-usedb.md -> docs/sql-ref-syntax-ddl-usedb.md docs/sql-ref-syntax-aux-refresh-table.md -> docs/sql-ref-syntax-aux-cache-refresh-table.md ### Why are the changes needed? usedb belongs to DDL. Its location should be consistent with other DDL commands file locations similar reason for refresh table ### Does this PR introduce _any_ user-facing change? before change, when clicking USE DATABASE, the side bar menu shows select commands Screen Shot 2020-07-04 at 9 05 35 AM after change, when clicking USE DATABASE, the side bar menu shows DDL commands Screen Shot 2020-07-04 at 9 06 06 AM before change, when clicking refresh table, the side bar menu shows Auxiliary statements Screen Shot 2020-07-04 at 9 30 40 AM after change, when clicking refresh table, the side bar menu shows Cache statements Screen Shot 2020-07-04 at 9 35 21 AM ### How was this patch tested? Manually build and check Closes #28995 from huaxingao/docs_fix. Authored-by: Huaxin Gao Signed-off-by: Huaxin Gao --- docs/_data/menu-sql.yaml | 4 ++-- docs/sql-ref-syntax-aux-cache-cache-table.md | 2 +- docs/sql-ref-syntax-aux-cache-clear-cache.md | 2 +- ...esh-table.md => sql-ref-syntax-aux-cache-refresh-table.md} | 0 docs/sql-ref-syntax-aux-cache-refresh.md | 2 +- docs/sql-ref-syntax-aux-cache-uncache-table.md | 2 +- docs/sql-ref-syntax-aux-cache.md | 2 +- ...syntax-qry-select-usedb.md => sql-ref-syntax-ddl-usedb.md} | 0 docs/sql-ref-syntax-ddl.md | 2 +- docs/sql-ref-syntax.md | 4 ++-- 10 files changed, 10 insertions(+), 10 deletions(-) rename docs/{sql-ref-syntax-aux-refresh-table.md => sql-ref-syntax-aux-cache-refresh-table.md} (100%) rename docs/{sql-ref-syntax-qry-select-usedb.md => sql-ref-syntax-ddl-usedb.md} (100%) diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 219e6809a96f0..eea657e684495 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -139,7 +139,7 @@ - text: REPAIR TABLE url: sql-ref-syntax-ddl-repair-table.html - text: USE DATABASE - url: sql-ref-syntax-qry-select-usedb.html + url: sql-ref-syntax-ddl-usedb.html - text: Data Manipulation Statements url: sql-ref-syntax-dml.html subitems: @@ -207,7 +207,7 @@ - text: CLEAR CACHE url: sql-ref-syntax-aux-cache-clear-cache.html - text: REFRESH TABLE - url: sql-ref-syntax-aux-refresh-table.html + url: sql-ref-syntax-aux-cache-refresh-table.html - text: REFRESH url: sql-ref-syntax-aux-cache-refresh.html - text: DESCRIBE diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md index 193e209d792b3..fdef3d657dfa3 100644 --- a/docs/sql-ref-syntax-aux-cache-cache-table.md +++ b/docs/sql-ref-syntax-aux-cache-cache-table.md @@ -78,5 +78,5 @@ CACHE TABLE testCache OPTIONS ('storageLevel' 'DISK_ONLY') SELECT * FROM testDat * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) -* [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) +* [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-cache-clear-cache.md b/docs/sql-ref-syntax-aux-cache-clear-cache.md index ee33e6a98296d..a27cd83c146a3 100644 --- a/docs/sql-ref-syntax-aux-cache-clear-cache.md +++ b/docs/sql-ref-syntax-aux-cache-clear-cache.md @@ -39,5 +39,5 @@ CLEAR CACHE; * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) -* [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) +* [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-refresh-table.md b/docs/sql-ref-syntax-aux-cache-refresh-table.md similarity index 100% rename from docs/sql-ref-syntax-aux-refresh-table.md rename to docs/sql-ref-syntax-aux-cache-refresh-table.md diff --git a/docs/sql-ref-syntax-aux-cache-refresh.md b/docs/sql-ref-syntax-aux-cache-refresh.md index 82bc12da5d1ac..b10e6fb47aaf7 100644 --- a/docs/sql-ref-syntax-aux-cache-refresh.md +++ b/docs/sql-ref-syntax-aux-cache-refresh.md @@ -53,4 +53,4 @@ REFRESH "hdfs://path/to/table"; * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) -* [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) +* [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md index c5a8fbbe08281..96a691e4c3931 100644 --- a/docs/sql-ref-syntax-aux-cache-uncache-table.md +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -48,5 +48,5 @@ UNCACHE TABLE t1; * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) -* [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) +* [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md index 418b8cc3403b5..0ccb1c61a0da5 100644 --- a/docs/sql-ref-syntax-aux-cache.md +++ b/docs/sql-ref-syntax-aux-cache.md @@ -22,5 +22,5 @@ license: | * [CACHE TABLE statement](sql-ref-syntax-aux-cache-cache-table.html) * [UNCACHE TABLE statement](sql-ref-syntax-aux-cache-uncache-table.html) * [CLEAR CACHE statement](sql-ref-syntax-aux-cache-clear-cache.html) - * [REFRESH TABLE statement](sql-ref-syntax-aux-refresh-table.html) + * [REFRESH TABLE statement](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH statement](sql-ref-syntax-aux-cache-refresh.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select-usedb.md b/docs/sql-ref-syntax-ddl-usedb.md similarity index 100% rename from docs/sql-ref-syntax-qry-select-usedb.md rename to docs/sql-ref-syntax-ddl-usedb.md diff --git a/docs/sql-ref-syntax-ddl.md b/docs/sql-ref-syntax-ddl.md index 82fbf0498a20f..cb3e04c0ec910 100644 --- a/docs/sql-ref-syntax-ddl.md +++ b/docs/sql-ref-syntax-ddl.md @@ -34,4 +34,4 @@ Data Definition Statements are used to create or modify the structure of databas * [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) * [TRUNCATE TABLE](sql-ref-syntax-ddl-truncate-table.html) * [REPAIR TABLE](sql-ref-syntax-ddl-repair-table.html) - * [USE DATABASE](sql-ref-syntax-qry-select-usedb.html) + * [USE DATABASE](sql-ref-syntax-ddl-usedb.html) diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index d78a01fd655a2..4bf1858428d98 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -36,7 +36,7 @@ Spark SQL is Apache Spark's module for working with structured data. The SQL Syn * [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) * [REPAIR TABLE](sql-ref-syntax-ddl-repair-table.html) * [TRUNCATE TABLE](sql-ref-syntax-ddl-truncate-table.html) - * [USE DATABASE](sql-ref-syntax-qry-select-usedb.html) + * [USE DATABASE](sql-ref-syntax-ddl-usedb.html) ### DML Statements @@ -82,7 +82,7 @@ Spark SQL is Apache Spark's module for working with structured data. The SQL Syn * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) - * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) + * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) * [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) From 3726aab6400ffbe2373e6baeb65d1866cdf3abc4 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 5 Jul 2020 19:09:04 -0700 Subject: [PATCH 119/384] [SPARK-32177][WEBUI] Remove the weird line from near the Spark logo on mouseover in the WebUI ### What changes were proposed in this pull request? This PR changes `webui.css` to fix a style issue on moving mouse cursor on the Spark logo. ### Why are the changes needed? In the webui, the Spark logo is on the top right side. When we move mouse cursor on the logo, a weird underline appears near the logo. logo_with_line ### Does this PR introduce _any_ user-facing change? Yes. After this change applied, no more weird line shown even if mouse cursor moves on the logo. removed-line-from-logo ### How was this patch tested? By moving mouse cursor on the Spark logo and confirmed no more weird line there. Closes #29003 from sarutak/fix-logo-underline. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- core/src/main/resources/org/apache/spark/ui/static/webui.css | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index f7f8a0e0e9061..d4394ebcfd258 100755 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -80,6 +80,10 @@ a:not([href]):hover { padding: 0; } +.navbar-brand a:hover { + text-decoration: none; +} + .navbar .navbar-nav .nav-link { height: 50px; padding: 10px 15px 10px; From 0e33b5ecde12452cbd5e5f4c54bf0b6388b502bd Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jul 2020 12:03:08 +0900 Subject: [PATCH 120/384] [SPARK-32178][TESTS] Disable test-dependencies.sh from Jenkins jobs ### What changes were proposed in this pull request? This PR aims to disable dependency tests(test-dependencies.sh) from Jenkins. ### Why are the changes needed? - First of all, GitHub Action provides the same test capability already and stabler. - Second, currently, `test-dependencies.sh` fails very frequently in AmpLab Jenkins environment. For example, in the following irrelevant PR, it fails 5 times during 6 hours. - https://github.com/apache/spark/pull/29001 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins without `test-dependencies.sh` invocation. Closes #29004 from dongjoon-hyun/SPARK-32178. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- dev/run-tests.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index ec04c37857d96..ca502b2818847 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -648,7 +648,7 @@ def main(): # if "DOCS" in changed_modules and test_env == "amplab_jenkins": # build_spark_documentation() - if any(m.should_run_build_tests for m in test_modules): + if any(m.should_run_build_tests for m in test_modules) and test_env != "amplab_jenkins": run_build_tests() # spark build From dea7bc464db6724d89e8b31264e054f82ceb4e5f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 5 Jul 2020 20:12:41 -0700 Subject: [PATCH 121/384] [SPARK-32100][CORE][TESTS][FOLLOWUP] Reduce the required test resources ### What changes were proposed in this pull request? This PR aims to reduce the required test resources in WorkerDecommissionExtendedSuite. ### Why are the changes needed? When Jenkins farms is crowded, the following failure happens currently [here](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-3.2-hive-2.3/890/testReport/junit/org.apache.spark.scheduler/WorkerDecommissionExtendedSuite/Worker_decommission_and_executor_idle_timeout/) ``` java.util.concurrent.TimeoutException: Can't find 20 executors before 60000 milliseconds elapsed at org.apache.spark.TestUtils$.waitUntilExecutorsUp(TestUtils.scala:326) at org.apache.spark.scheduler.WorkerDecommissionExtendedSuite.$anonfun$new$2(WorkerDecommissionExtendedSuite.scala:45) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #29001 from dongjoon-hyun/SPARK-32100-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../scheduler/WorkerDecommissionExtendedSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala index 02c72fa349a79..4de5aaeab5c51 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala @@ -32,17 +32,17 @@ import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend class WorkerDecommissionExtendedSuite extends SparkFunSuite with LocalSparkContext { private val conf = new org.apache.spark.SparkConf() .setAppName(getClass.getName) - .set(SPARK_MASTER, "local-cluster[20,1,512]") + .set(SPARK_MASTER, "local-cluster[5,1,512]") .set(EXECUTOR_MEMORY, "512m") .set(DYN_ALLOCATION_ENABLED, true) .set(DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED, true) - .set(DYN_ALLOCATION_INITIAL_EXECUTORS, 20) + .set(DYN_ALLOCATION_INITIAL_EXECUTORS, 5) .set(WORKER_DECOMMISSION_ENABLED, true) test("Worker decommission and executor idle timeout") { sc = new SparkContext(conf.set(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "10s")) withSpark(sc) { sc => - TestUtils.waitUntilExecutorsUp(sc, 20, 60000) + TestUtils.waitUntilExecutorsUp(sc, 5, 60000) val rdd1 = sc.parallelize(1 to 10, 2) val rdd2 = rdd1.map(x => (1, x)) val rdd3 = rdd2.reduceByKey(_ + _) @@ -54,10 +54,10 @@ class WorkerDecommissionExtendedSuite extends SparkFunSuite with LocalSparkConte } } - test("Decommission 19 executors from 20 executors in total") { + test("Decommission 4 executors from 5 executors in total") { sc = new SparkContext(conf) withSpark(sc) { sc => - TestUtils.waitUntilExecutorsUp(sc, 20, 60000) + TestUtils.waitUntilExecutorsUp(sc, 5, 60000) val rdd1 = sc.parallelize(1 to 100000, 200) val rdd2 = rdd1.map(x => (x % 100, x)) val rdd3 = rdd2.reduceByKey(_ + _) From 59a70879c04cf34442d3dde6e94e30c196279afe Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 6 Jul 2020 10:34:31 +0000 Subject: [PATCH 122/384] [SPARK-32145][SQL][TEST-HIVE1.2][TEST-HADOOP2.7] ThriftCLIService.GetOperationStatus should include exception's stack trace to the error message ### What changes were proposed in this pull request? In https://issues.apache.org/jira/browse/SPARK-29283, we only show the error message of root cause to end-users through JDBC client. In some cases, it erases the straightaway messages that we intentionally make to help them for better understanding. The root cause is somehow obscure for JDBC end-users who only writing SQL queries. e.g ``` Error running query: org.apache.spark.sql.AnalysisException: The second argument of 'date_sub' function needs to be an integer.; ``` is better than just ``` Caused by: java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2 ``` We should do as Hive does in https://issues.apache.org/jira/browse/HIVE-14368 In general, this PR partially reverts SPARK-29283, ports HIVE-14368, and improves test coverage ### Why are the changes needed? 1. Do the same as Hive 2.3 and later for getting an error message in ThriftCLIService.GetOperationStatus 2. The root cause is somehow obscure for JDBC end-users who only writing SQL queries. 3. Consistency with `spark-sql` script ### Does this PR introduce _any_ user-facing change? Yes, when running queries using thrift server and an error occurs, you will get the full stack traces instead of only the message of the root cause ### How was this patch tested? add unit test Closes #28963 from yaooqinn/SPARK-32145. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../SparkExecuteStatementOperation.scala | 19 +++-------- .../SparkGetCatalogsOperation.scala | 24 ++----------- .../SparkGetColumnsOperation.scala | 19 ++--------- .../SparkGetFunctionsOperation.scala | 18 ++-------- .../SparkGetSchemasOperation.scala | 18 ++-------- .../SparkGetTableTypesOperation.scala | 20 ++--------- .../SparkGetTablesOperation.scala | 25 +++----------- .../SparkGetTypeInfoOperation.scala | 22 ++---------- .../hive/thriftserver/SparkOperation.scala | 14 +++++++- .../thriftserver/SharedThriftServer.scala | 34 +++++++++++++++++-- .../ThriftServerWithSparkContextSuite.scala | 34 +++++++++++++++++++ .../service/cli/thrift/ThriftCLIService.java | 3 +- .../service/cli/thrift/ThriftCLIService.java | 3 +- 13 files changed, 106 insertions(+), 147 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 1404ece76449e..eae5d5d4bcfa9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -18,26 +18,22 @@ package org.apache.spark.sql.hive.thriftserver import java.security.PrivilegedExceptionAction -import java.sql.{Date, Timestamp} -import java.util.{Arrays, Map => JMap, UUID} +import java.util.{Arrays, Map => JMap} import java.util.concurrent.RejectedExecutionException import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.shims.Utils import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext} import org.apache.spark.sql.execution.HiveResult.{getTimeFormatters, toHiveString, TimeFormatters} -import org.apache.spark.sql.execution.command.SetCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -315,16 +311,11 @@ private[hive] class SparkExecuteStatementOperation( } else { logError(s"Error executing query with $statementId, currentState $currentState, ", e) setState(OperationState.ERROR) + HiveThriftServer2.eventManager.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error running query: " + root.toString, root) + case _: HiveSQLException => throw e + case _ => throw new HiveSQLException("Error running query: " + e.toString, e) } } } finally { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index 55070e035b944..01ef78cde8956 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -17,17 +17,13 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.UUID - -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType -import org.apache.hive.service.cli.{HiveSQLException, OperationState} +import org.apache.hive.service.cli.OperationState import org.apache.hive.service.cli.operation.GetCatalogsOperation import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own GetCatalogsOperation @@ -62,22 +58,8 @@ private[hive] class SparkGetCatalogsOperation( authorizeMetaGets(HiveOperationType.GET_CATALOGS, null) } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get catalogs operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting catalogs: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index ca8ad5e6ad134..d42732f426681 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.hive.thriftserver.ThriftserverShimUtils.toJavaSQLType import org.apache.spark.sql.types.StructType -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own SparkGetColumnsOperation @@ -122,22 +121,8 @@ private[hive] class SparkGetColumnsOperation( } } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get columns operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting columns: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index f5e647bfd4f38..cf5dbae93a365 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -98,22 +98,8 @@ private[hive] class SparkGetFunctionsOperation( } } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get functions operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting functions: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 74220986fcd34..16fd502048e80 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -81,22 +81,8 @@ private[hive] class SparkGetSchemasOperation( rowSet.addRow(Array[AnyRef](globalTempViewDb, DEFAULT_HIVE_CATALOG)) } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get schemas operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting schemas: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index 1cf9c3a731af5..9e31b8baad78e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTableTypesOperation @@ -28,7 +27,6 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.catalog.CatalogTableType -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own GetTableTypesOperation @@ -69,22 +67,8 @@ private[hive] class SparkGetTableTypesOperation( rowSet.addRow(Array[AnyRef](tableType)) } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get table types operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting table types: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index a1d21e2d60c63..0d4b9b392f074 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -17,14 +17,12 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.{List => JList, UUID} +import java.util.{List => JList} import java.util.regex.Pattern import scala.collection.JavaConverters._ -import org.apache.commons.lang3.exception.ExceptionUtils -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils} import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTablesOperation import org.apache.hive.service.cli.session.HiveSession @@ -33,7 +31,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.hive.HiveUtils -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own GetTablesOperation @@ -111,22 +108,8 @@ private[hive] class SparkGetTablesOperation( } } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get tables operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting tables: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index e38139d60df60..c2568ad4ada0a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -19,15 +19,13 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType -import org.apache.hive.service.cli.{HiveSQLException, OperationState} +import org.apache.hive.service.cli.OperationState import org.apache.hive.service.cli.operation.GetTypeInfoOperation import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own GetTypeInfoOperation @@ -87,22 +85,8 @@ private[hive] class SparkGetTypeInfoOperation( rowSet.addRow(rowData) }) setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get type info with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting type info: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index 3da568cfa256e..0acd1b3e9899a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -import org.apache.hive.service.cli.OperationState +import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.Operation import org.apache.spark.SparkContext @@ -93,4 +93,16 @@ private[hive] trait SparkOperation extends Operation with Logging { case t => throw new IllegalArgumentException(s"Unknown table type is found: $t") } + + protected def onError(): PartialFunction[Throwable, Unit] = { + case e: Throwable => + logError(s"Error executing get catalogs operation with $statementId", e) + super.setState(OperationState.ERROR) + HiveThriftServer2.eventManager.onStatementError( + statementId, e.getMessage, Utils.exceptionString(e)) + e match { + case _: HiveSQLException => throw e + case _ => throw new HiveSQLException(s"Error operating $getType ${e.getMessage}", e) + } + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index 3d7933fba17d8..5f17607585521 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File import java.sql.{DriverManager, Statement} +import java.util import scala.collection.JavaConverters._ import scala.concurrent.duration._ @@ -27,7 +28,12 @@ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hive.service.cli.thrift.ThriftCLIService +import org.apache.hive.jdbc.HttpBasicAuthInterceptor +import org.apache.hive.service.auth.PlainSaslHelper +import org.apache.hive.service.cli.thrift.{ThriftCLIService, ThriftCLIServiceClient} +import org.apache.http.impl.client.HttpClientBuilder +import org.apache.thrift.protocol.TBinaryProtocol +import org.apache.thrift.transport.{THttpClient, TSocket} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils @@ -76,8 +82,9 @@ trait SharedThriftServer extends SharedSparkSession { s"jdbc:hive2://localhost:$serverPort/" } + protected def user: String = System.getProperty("user.name") + protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = { - val user = System.getProperty("user.name") require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } @@ -91,6 +98,29 @@ trait SharedThriftServer extends SharedSparkSession { } } + protected def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { + require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") + val transport = mode match { + case ServerMode.binary => + val rawTransport = new TSocket("localhost", serverPort) + PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + case ServerMode.http => + val interceptor = new HttpBasicAuthInterceptor( + user, + "anonymous", + null, null, true, new util.HashMap[String, String]()) + new THttpClient( + s"http://localhost:$serverPort/cliservice", + HttpClientBuilder.create.addInterceptorFirst(interceptor).build()) + } + + val protocol = new TBinaryProtocol(transport) + val client = new ThriftCLIServiceClient(new ThriftserverShimUtils.Client(protocol)) + + transport.open() + try f(client) finally transport.close() + } + private def startThriftServer(attempt: Int): Unit = { logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt") val sqlContext = spark.newSession().sqlContext diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 1382eb2d79f38..fd3a638c4fa44 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark.sql.hive.thriftserver +import java.sql.SQLException + +import org.apache.hive.service.cli.HiveSQLException + trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("the scratch dir will be deleted during server start but recreated with new operation") { @@ -45,6 +49,36 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { assert(cacheManager.isEmpty) } } + + test("Full stack traces as error message for jdbc or thrift client") { + val sql = "select date_sub(date'2011-11-11', '1.2')" + withCLIServiceClient { client => + val sessionHandle = client.openSession(user, "") + + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] + val e = intercept[HiveSQLException] { + client.executeStatement( + sessionHandle, + sql, + confOverlay) + } + + assert(e.getMessage + .contains("The second argument of 'date_sub' function needs to be an integer.")) + assert(!e.getMessage.contains("" + + "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + } + + withJdbcStatement { statement => + val e = intercept[SQLException] { + statement.executeQuery(sql) + } + assert(e.getMessage + .contains("The second argument of 'date_sub' function needs to be an integer.")) + assert(e.getMessage.contains("" + + "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + } + } } diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 783e5795aca76..ff533769b5b84 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -564,7 +564,8 @@ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) th if (opException != null) { resp.setSqlState(opException.getSQLState()); resp.setErrorCode(opException.getErrorCode()); - resp.setErrorMessage(opException.getMessage()); + resp.setErrorMessage(org.apache.hadoop.util.StringUtils + .stringifyException(opException)); } resp.setStatus(OK_STATUS); } catch (Exception e) { diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index e46799a1c427d..914d6d3612596 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -566,7 +566,8 @@ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) th if (opException != null) { resp.setSqlState(opException.getSQLState()); resp.setErrorCode(opException.getErrorCode()); - resp.setErrorMessage(opException.getMessage()); + resp.setErrorMessage(org.apache.hadoop.util.StringUtils + .stringifyException(opException)); } resp.setStatus(OK_STATUS); } catch (Exception e) { From 1d1809636b23409df2e81a6df81367d30a57f225 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Mon, 6 Jul 2020 21:39:41 +0900 Subject: [PATCH 123/384] [SPARK-32162][PYTHON][TESTS] Improve error message of Pandas grouped map test with window ### What changes were proposed in this pull request? Improve the error message in test GroupedMapInPandasTests.test_grouped_over_window_with_key to show the incorrect values. ### Why are the changes needed? This test failure has come up often in Arrow testing because it tests a struct with timestamp values through a Pandas UDF. The current error message is not helpful as it doesn't show the incorrect values, only that it failed. This change will instead raise an assertion error with the incorrect values on a failure. Before: ``` ====================================================================== FAIL: test_grouped_over_window_with_key (pyspark.sql.tests.test_pandas_grouped_map.GroupedMapInPandasTests) ---------------------------------------------------------------------- Traceback (most recent call last): File "/spark/python/pyspark/sql/tests/test_pandas_grouped_map.py", line 588, in test_grouped_over_window_with_key self.assertTrue(all([r[0] for r in result])) AssertionError: False is not true ``` After: ``` ====================================================================== ERROR: test_grouped_over_window_with_key (pyspark.sql.tests.test_pandas_grouped_map.GroupedMapInPandasTests) ---------------------------------------------------------------------- ... AssertionError: {'start': datetime.datetime(2018, 3, 20, 0, 0), 'end': datetime.datetime(2018, 3, 25, 0, 0)}, != {'start': datetime.datetime(2020, 3, 20, 0, 0), 'end': datetime.datetime(2020, 3, 25, 0, 0)} ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Improved existing test Closes #28987 from BryanCutler/pandas-grouped-map-test-output-SPARK-32162. Authored-by: Bryan Cutler Signed-off-by: HyukjinKwon --- .../sql/tests/test_pandas_grouped_map.py | 57 ++++++++++++------- 1 file changed, 35 insertions(+), 22 deletions(-) diff --git a/python/pyspark/sql/tests/test_pandas_grouped_map.py b/python/pyspark/sql/tests/test_pandas_grouped_map.py index 76119432662ba..cc6167e619285 100644 --- a/python/pyspark/sql/tests/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_grouped_map.py @@ -545,13 +545,13 @@ def f(pdf): def test_grouped_over_window_with_key(self): - data = [(0, 1, "2018-03-10T00:00:00+00:00", False), - (1, 2, "2018-03-11T00:00:00+00:00", False), - (2, 2, "2018-03-12T00:00:00+00:00", False), - (3, 3, "2018-03-15T00:00:00+00:00", False), - (4, 3, "2018-03-16T00:00:00+00:00", False), - (5, 3, "2018-03-17T00:00:00+00:00", False), - (6, 3, "2018-03-21T00:00:00+00:00", False)] + data = [(0, 1, "2018-03-10T00:00:00+00:00", [0]), + (1, 2, "2018-03-11T00:00:00+00:00", [0]), + (2, 2, "2018-03-12T00:00:00+00:00", [0]), + (3, 3, "2018-03-15T00:00:00+00:00", [0]), + (4, 3, "2018-03-16T00:00:00+00:00", [0]), + (5, 3, "2018-03-17T00:00:00+00:00", [0]), + (6, 3, "2018-03-21T00:00:00+00:00", [0])] expected_window = [ {'start': datetime.datetime(2018, 3, 10, 0, 0), @@ -562,30 +562,43 @@ def test_grouped_over_window_with_key(self): 'end': datetime.datetime(2018, 3, 25, 0, 0)}, ] - expected = {0: (1, expected_window[0]), - 1: (2, expected_window[0]), - 2: (2, expected_window[0]), - 3: (3, expected_window[1]), - 4: (3, expected_window[1]), - 5: (3, expected_window[1]), - 6: (3, expected_window[2])} + expected_key = {0: (1, expected_window[0]), + 1: (2, expected_window[0]), + 2: (2, expected_window[0]), + 3: (3, expected_window[1]), + 4: (3, expected_window[1]), + 5: (3, expected_window[1]), + 6: (3, expected_window[2])} + + # id -> array of group with len of num records in window + expected = {0: [1], + 1: [2, 2], + 2: [2, 2], + 3: [3, 3, 3], + 4: [3, 3, 3], + 5: [3, 3, 3], + 6: [3]} df = self.spark.createDataFrame(data, ['id', 'group', 'ts', 'result']) df = df.select(col('id'), col('group'), col('ts').cast('timestamp'), col('result')) - @pandas_udf(df.schema, PandasUDFType.GROUPED_MAP) def f(key, pdf): group = key[0] window_range = key[1] - # Result will be True if group and window range equal to expected - is_expected = pdf.id.apply(lambda id: (expected[id][0] == group and - expected[id][1] == window_range)) - return pdf.assign(result=is_expected) - result = df.groupby('group', window('ts', '5 days')).apply(f).select('result').collect() + # Make sure the key with group and window values are correct + for _, i in pdf.id.iteritems(): + assert expected_key[i][0] == group, "{} != {}".format(expected_key[i][0], group) + assert expected_key[i][1] == window_range, \ + "{} != {}".format(expected_key[i][1], window_range) - # Check that all group and window_range values from udf matched expected - self.assertTrue(all([r[0] for r in result])) + return pdf.assign(result=[[group] * len(pdf)] * len(pdf)) + + result = df.groupby('group', window('ts', '5 days')).applyInPandas(f, df.schema)\ + .select('id', 'result').collect() + + for r in result: + self.assertListEqual(expected[r[0]], r[1]) def test_case_insensitive_grouping_column(self): # SPARK-31915: case-insensitive grouping column should work. From 3fe3365292962fc354eb19670a4067df212d31cf Mon Sep 17 00:00:00 2001 From: sidedoorleftroad Date: Mon, 6 Jul 2020 09:20:31 -0700 Subject: [PATCH 124/384] [SPARK-32172][CORE] Use createDirectory instead of mkdir ### What changes were proposed in this pull request? Use Files.createDirectory() to create local directory instead of File.mkdir() in DiskBlockManager. Many times, we will see such error log information like "Failed to create local dir in xxxxxx". But there is no clear information indicating why the directory creation failed. When Files.createDirectory() fails to create a local directory, it can give specific error information for subsequent troubleshooting(also throws IOException). ### Why are the changes needed? Throw clear error message when creating directory fails. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? `DiskBlockManagerSuite` Closes #28997 from sidedoorleftroad/SPARK-32172. Authored-by: sidedoorleftroad Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/storage/DiskBlockManager.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index f2113947f6bf5..bf76eef443e81 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.io.{File, IOException} +import java.nio.file.Files import java.util.UUID import org.apache.spark.SparkConf @@ -69,8 +70,8 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea old } else { val newDir = new File(localDirs(dirId), "%02x".format(subDirId)) - if (!newDir.exists() && !newDir.mkdir()) { - throw new IOException(s"Failed to create local dir in $newDir.") + if (!newDir.exists()) { + Files.createDirectory(newDir.toPath) } subDirs(dirId)(subDirId) = newDir newDir From 5d296ed39e3dd79ddb10c68657e773adba40a5e0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 6 Jul 2020 20:07:33 -0700 Subject: [PATCH 125/384] [SPARK-32167][SQL] Fix GetArrayStructFields to respect inner field's nullability together ### What changes were proposed in this pull request? Fix nullability of `GetArrayStructFields`. It should consider both the original array's `containsNull` and the inner field's nullability. ### Why are the changes needed? Fix a correctness issue. ### Does this PR introduce _any_ user-facing change? Yes. See the added test. ### How was this patch tested? a new UT and end-to-end test Closes #28992 from cloud-fan/bug. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../expressions/complexTypeExtractors.scala | 2 +- .../expressions/ComplexTypeSuite.scala | 26 +++++++++++++++++++ .../expressions/SelectedFieldSuite.scala | 8 +++--- .../apache/spark/sql/ComplexTypesSuite.scala | 11 ++++++++ 4 files changed, 42 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index 9c600c9d39cf7..89ff4facd25a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -57,7 +57,7 @@ object ExtractValue { val fieldName = v.toString val ordinal = findField(fields, fieldName, resolver) GetArrayStructFields(child, fields(ordinal).copy(name = fieldName), - ordinal, fields.length, containsNull) + ordinal, fields.length, containsNull || fields(ordinal).nullable) case (_: ArrayType, _) => GetArrayItem(child, extraction) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 3df7d02fb6604..dbe43709d1d35 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext @@ -159,6 +160,31 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(getArrayStructFields(nullArrayStruct, "a"), null) } + test("SPARK-32167: nullability of GetArrayStructFields") { + val resolver = SQLConf.get.resolver + + val array1 = ArrayType( + new StructType().add("a", "int", nullable = true), + containsNull = false) + val data1 = Literal.create(Seq(Row(null)), array1) + val get1 = ExtractValue(data1, Literal("a"), resolver).asInstanceOf[GetArrayStructFields] + assert(get1.containsNull) + + val array2 = ArrayType( + new StructType().add("a", "int", nullable = false), + containsNull = true) + val data2 = Literal.create(Seq(null), array2) + val get2 = ExtractValue(data2, Literal("a"), resolver).asInstanceOf[GetArrayStructFields] + assert(get2.containsNull) + + val array3 = ArrayType( + new StructType().add("a", "int", nullable = false), + containsNull = false) + val data3 = Literal.create(Seq(Row(1)), array3) + val get3 = ExtractValue(data3, Literal("a"), resolver).asInstanceOf[GetArrayStructFields] + assert(!get3.containsNull) + } + test("CreateArray") { val intSeq = Seq(5, 10, 15, 20, 25) val longSeq = intSeq.map(_.toLong) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala index 3c826e812b5cc..76d6890cc8f6f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala @@ -254,13 +254,13 @@ class SelectedFieldSuite extends AnalysisTest { StructField("col3", ArrayType(StructType( StructField("field1", StructType( StructField("subfield1", IntegerType, nullable = false) :: Nil)) - :: Nil), containsNull = false), nullable = false) + :: Nil), containsNull = true), nullable = false) } testSelect(arrayWithStructAndMap, "col3.field2['foo'] as foo") { StructField("col3", ArrayType(StructType( StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) - :: Nil), containsNull = false), nullable = false) + :: Nil), containsNull = true), nullable = false) } // |-- col1: string (nullable = false) @@ -471,7 +471,7 @@ class SelectedFieldSuite extends AnalysisTest { testSelect(mapWithArrayOfStructKey, "map_keys(col2)[0].field1 as foo") { StructField("col2", MapType( ArrayType(StructType( - StructField("field1", StringType) :: Nil), containsNull = false), + StructField("field1", StringType) :: Nil), containsNull = true), ArrayType(StructType( StructField("field3", StructType( StructField("subfield3", IntegerType) :: @@ -482,7 +482,7 @@ class SelectedFieldSuite extends AnalysisTest { StructField("col2", MapType( ArrayType(StructType( StructField("field2", StructType( - StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = false), + StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = true), ArrayType(StructType( StructField("field3", StructType( StructField("subfield3", IntegerType) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala index 6b503334f9f23..bdcf7230e3211 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql +import scala.collection.JavaConverters._ + import org.apache.spark.sql.catalyst.expressions.CreateNamedStruct import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{ArrayType, StructType} class ComplexTypesSuite extends QueryTest with SharedSparkSession { + import testImplicits._ override def beforeAll(): Unit = { super.beforeAll() @@ -106,4 +110,11 @@ class ComplexTypesSuite extends QueryTest with SharedSparkSession { checkAnswer(df1, Row(10, 12) :: Row(11, 13) :: Nil) checkNamedStruct(df.queryExecution.optimizedPlan, expectedCount = 0) } + + test("SPARK-32167: get field from an array of struct") { + val innerStruct = new StructType().add("i", "int", nullable = true) + val schema = new StructType().add("arr", ArrayType(innerStruct, containsNull = false)) + val df = spark.createDataFrame(List(Row(Seq(Row(1), Row(null)))).asJava, schema) + checkAnswer(df.select($"arr".getField("i")), Row(Seq(1, null))) + } } From 2e23da2bda47dcfa8d143d317aff56860b77fe80 Mon Sep 17 00:00:00 2001 From: ulysses Date: Tue, 7 Jul 2020 13:39:04 +0000 Subject: [PATCH 126/384] [SPARK-31975][SQL] Show AnalysisException when WindowFunction is used without WindowExpression ### What changes were proposed in this pull request? Add WindowFunction check at `CheckAnalysis`. ### Why are the changes needed? Provide friendly error msg. **BEFORE** ```scala scala> sql("select rank() from values(1)").show java.lang.UnsupportedOperationException: Cannot generate code for expression: rank() ``` **AFTER** ```scala scala> sql("select rank() from values(1)").show org.apache.spark.sql.AnalysisException: Window function rank() requires an OVER clause.;; Project [rank() AS RANK()#3] +- LocalRelation [col1#2] ``` ### Does this PR introduce _any_ user-facing change? Yes, user wiill be given a better error msg. ### How was this patch tested? Pass the newly added UT. Closes #28808 from ulysses-you/SPARK-31975. Authored-by: ulysses Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 5 +++++ .../spark/sql/catalyst/analysis/AnalysisSuite.scala | 11 +++++++++++ 2 files changed, 16 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 9c99acaa994b8..43dd0979eed7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -158,6 +158,11 @@ trait CheckAnalysis extends PredicateHelper { case g: GroupingID => failAnalysis("grouping_id() can only be used with GroupingSets/Cube/Rollup") + case e: Expression if e.children.exists(_.isInstanceOf[WindowFunction]) && + !e.isInstanceOf[WindowExpression] => + val w = e.children.find(_.isInstanceOf[WindowFunction]).get + failAnalysis(s"Window function $w requires an OVER clause.") + case w @ WindowExpression(AggregateExpression(_, _, true, _, _), _) => failAnalysis(s"Distinct window functions are not supported: $w") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index c15ec49e14282..c0be49af2107d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -884,4 +884,15 @@ class AnalysisSuite extends AnalysisTest with Matchers { Seq("Intersect can only be performed on tables with the compatible column types. " + "timestamp <> double at the second column of the second table")) } + + test("SPARK-31975: Throw user facing error when use WindowFunction directly") { + assertAnalysisError(testRelation2.select(RowNumber()), + Seq("Window function row_number() requires an OVER clause.")) + + assertAnalysisError(testRelation2.select(Sum(RowNumber())), + Seq("Window function row_number() requires an OVER clause.")) + + assertAnalysisError(testRelation2.select(RowNumber() + 1), + Seq("Window function row_number() requires an OVER clause.")) + } } From 75d342858a71293b7c2cbbd5cec2eb417ed3852c Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 7 Jul 2020 07:26:40 -0700 Subject: [PATCH 127/384] [SPARK-32209][SQL] Re-use GetTimestamp in ParseToDate ### What changes were proposed in this pull request? Replace the combination of expressions `SecondsToTimestamp` and `UnixTimestamp` by `GetTimestamp` in `ParseToDate`. ### Why are the changes needed? Eliminate unnecessary parsing overhead in: **string -> timestamp -> long (seconds) -> timestamp -> date**. After the changes, the chain will be: **string -> timestamp -> date**. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By existing test suites such as `DateFunctionsSuite`. Closes #28999 from MaxGekk/ParseToDate-parse-timestamp. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 a1277217b1b3a..551cbc3161cc1 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 @@ -1450,8 +1450,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr extends RuntimeReplaceable { def this(left: Expression, format: Expression) { - this(left, Option(format), - Cast(SecondsToTimestamp(UnixTimestamp(left, format)), DateType)) + this(left, Option(format), Cast(GetTimestamp(left, format), DateType)) } def this(left: Expression) = { From 8d5c0947f88d1ecae6e342488a280f70ec26d074 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Tue, 7 Jul 2020 08:30:15 -0700 Subject: [PATCH 128/384] [SPARK-32164][ML] GeneralizedLinearRegressionSummary optimization ### What changes were proposed in this pull request? 1, GeneralizedLinearRegressionSummary compute several statistics on single pass 2, LinearRegressionSummary use metrics.count ### Why are the changes needed? avoid extra passes on the dataset ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? existing testsuites Closes #28990 from zhengruifeng/glr_summary_opt. Authored-by: zhengruifeng Signed-off-by: Huaxin Gao --- .../GeneralizedLinearRegression.scala | 50 ++++++++++++++----- .../ml/regression/LinearRegression.scala | 2 +- .../mllib/evaluation/RegressionMetrics.scala | 2 + 3 files changed, 40 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 0ee895a95a288..8336df8e34ae0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -1220,10 +1220,41 @@ class GeneralizedLinearRegressionSummary private[regression] ( private[regression] lazy val link: Link = familyLink.link + /** + * summary row containing: + * numInstances, weightSum, deviance, rss, weighted average of label - offset. + */ + private lazy val glrSummary = { + val devUDF = udf { (label: Double, pred: Double, weight: Double) => + family.deviance(label, pred, weight) + } + val devCol = sum(devUDF(label, prediction, weight)) + + val rssCol = if (model.getFamily.toLowerCase(Locale.ROOT) != Binomial.name && + model.getFamily.toLowerCase(Locale.ROOT) != Poisson.name) { + val rssUDF = udf { (label: Double, pred: Double, weight: Double) => + (label - pred) * (label - pred) * weight / family.variance(pred) + } + sum(rssUDF(label, prediction, weight)) + } else { + lit(Double.NaN) + } + + val avgCol = if (model.getFitIntercept && + (!model.hasOffsetCol || (model.hasOffsetCol && family == Gaussian && link == Identity))) { + sum((label - offset) * weight) / sum(weight) + } else { + lit(Double.NaN) + } + + predictions + .select(count(label), sum(weight), devCol, rssCol, avgCol) + .head() + } + /** Number of instances in DataFrame predictions. */ @Since("2.2.0") - lazy val numInstances: Long = predictions.count() - + lazy val numInstances: Long = glrSummary.getLong(0) /** * Name of features. If the name cannot be retrieved from attributes, @@ -1335,9 +1366,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ if (!model.hasOffsetCol || (model.hasOffsetCol && family == Gaussian && link == Identity)) { - val agg = predictions.agg(sum(weight.multiply( - label.minus(offset))), sum(weight)).first() - link.link(agg.getDouble(0) / agg.getDouble(1)) + link.link(glrSummary.getDouble(4)) } else { // Create empty feature column and fit intercept only model using param setting from model val featureNull = "feature_" + java.util.UUID.randomUUID.toString @@ -1362,12 +1391,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( * The deviance for the fitted model. */ @Since("2.0.0") - lazy val deviance: Double = { - predictions.select(label, prediction, weight).rdd.map { - case Row(label: Double, pred: Double, weight: Double) => - family.deviance(label, pred, weight) - }.sum() - } + lazy val deviance: Double = glrSummary.getDouble(2) /** * The dispersion of the fitted model. @@ -1381,14 +1405,14 @@ class GeneralizedLinearRegressionSummary private[regression] ( model.getFamily.toLowerCase(Locale.ROOT) == Poisson.name) { 1.0 } else { - val rss = pearsonResiduals.agg(sum(pow(col("pearsonResiduals"), 2.0))).first().getDouble(0) + val rss = glrSummary.getDouble(3) rss / degreesOfFreedom } /** Akaike Information Criterion (AIC) for the fitted model. */ @Since("2.0.0") lazy val aic: Double = { - val weightSum = predictions.select(weight).agg(sum(weight)).first().getDouble(0) + val weightSum = glrSummary.getDouble(1) val t = predictions.select( label, prediction, weight).rdd.map { case Row(label: Double, pred: Double, weight: Double) => diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index d9f09c097292a..de559142a9261 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -1037,7 +1037,7 @@ class LinearRegressionSummary private[regression] ( } /** Number of instances in DataFrame predictions */ - lazy val numInstances: Long = predictions.count() + lazy val numInstances: Long = metrics.count /** Degrees of freedom */ @Since("2.2.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index b697d2746ce7b..7938427544bd9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -131,4 +131,6 @@ class RegressionMetrics @Since("2.0.0") ( 1 - SSerr / SStot } } + + private[spark] def count: Long = summary.count } From 4bbc343a4c0ff3d4b5443bd65adf2df55b9245ee Mon Sep 17 00:00:00 2001 From: "fqaiser94@gmail.com" Date: Tue, 7 Jul 2020 16:34:03 +0000 Subject: [PATCH 129/384] [SPARK-31317][SQL] Add withField method to Column ### What changes were proposed in this pull request? Added a new `withField` method to the `Column` class. This method should allow users to add or replace a `StructField` in a `StructType` column (with very similar semantics to the `withColumn` method on `Dataset`). ### Why are the changes needed? Often Spark users have to work with deeply nested data e.g. to fix a data quality issue with an existing `StructField`. To do this with the existing Spark APIs, users have to rebuild the entire struct column. For example, let's say you have the following deeply nested data structure which has a data quality issue (`5` is missing): ``` import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ val data = spark.createDataFrame(sc.parallelize( Seq(Row(Row(Row(1, 2, 3), Row(Row(4, null, 6), Row(7, 8, 9), Row(10, 11, 12)), Row(13, 14, 15))))), StructType(Seq( StructField("a", StructType(Seq( StructField("a", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))), StructField("b", StructType(Seq( StructField("a", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))), StructField("b", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))), StructField("c", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))) ))), StructField("c", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))) )))))).cache data.show(false) +---------------------------------+ |a | +---------------------------------+ |[[1, 2, 3], [[4,, 6], [7, 8, 9]]]| +---------------------------------+ ``` Currently, to replace the missing value users would have to do something like this: ``` val result = data.withColumn("a", struct( $"a.a", struct( struct( $"a.b.a.a", lit(5).as("b"), $"a.b.a.c" ).as("a"), $"a.b.b", $"a.b.c" ).as("b"), $"a.c" )) result.show(false) +---------------------------------------------------------------+ |a | +---------------------------------------------------------------+ |[[1, 2, 3], [[4, 5, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]| +---------------------------------------------------------------+ ``` As you can see above, with the existing methods users must call the `struct` function and list all fields, including fields they don't want to change. This is not ideal as: >this leads to complex, fragile code that cannot survive schema evolution. [SPARK-16483](https://issues.apache.org/jira/browse/SPARK-16483) In contrast, with the method added in this PR, a user could simply do something like this: ``` val result = data.withColumn("a", 'a.withField("b.a.b", lit(5))) result.show(false) +---------------------------------------------------------------+ |a | +---------------------------------------------------------------+ |[[1, 2, 3], [[4, 5, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]| +---------------------------------------------------------------+ ``` This is the first of maybe a few methods that could be added to the `Column` class to make it easier to manipulate nested data. Other methods under discussion in [SPARK-22231](https://issues.apache.org/jira/browse/SPARK-22231) include `drop` and `renameField`. However, these should be added in a separate PR. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? New unit tests were added. Jenkins must pass them. ### Related JIRAs: - https://issues.apache.org/jira/browse/SPARK-22231 - https://issues.apache.org/jira/browse/SPARK-16483 Closes #27066 from fqaiser94/SPARK-22231-withField. Lead-authored-by: fqaiser94@gmail.com Co-authored-by: fqaiser94 Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../expressions/complexTypeCreator.scala | 58 ++ .../sql/catalyst/optimizer/ComplexTypes.scala | 13 +- .../sql/catalyst/optimizer/Optimizer.scala | 7 +- .../sql/catalyst/optimizer/WithFields.scala | 42 ++ .../optimizer/CombineWithFieldsSuite.scala | 76 +++ .../optimizer/complexTypesSuite.scala | 57 ++ .../scala/org/apache/spark/sql/Column.scala | 66 +++ .../spark/sql/ColumnExpressionSuite.scala | 499 ++++++++++++++++++ 8 files changed, 815 insertions(+), 3 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/WithFields.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineWithFieldsSuite.scala 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 1b4a705e804f1..cf7cc3a5e16ff 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 @@ -539,3 +539,61 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E override def prettyName: String = "str_to_map" } + +/** + * Adds/replaces field in struct by name. + */ +case class WithFields( + structExpr: Expression, + names: Seq[String], + valExprs: Seq[Expression]) extends Unevaluable { + + assert(names.length == valExprs.length) + + override def checkInputDataTypes(): TypeCheckResult = { + if (!structExpr.dataType.isInstanceOf[StructType]) { + TypeCheckResult.TypeCheckFailure( + "struct argument should be struct type, got: " + structExpr.dataType.catalogString) + } else { + TypeCheckResult.TypeCheckSuccess + } + } + + override def children: Seq[Expression] = structExpr +: valExprs + + override def dataType: StructType = evalExpr.dataType.asInstanceOf[StructType] + + override def foldable: Boolean = structExpr.foldable && valExprs.forall(_.foldable) + + override def nullable: Boolean = structExpr.nullable + + override def prettyName: String = "with_fields" + + lazy val evalExpr: Expression = { + val existingExprs = structExpr.dataType.asInstanceOf[StructType].fieldNames.zipWithIndex.map { + case (name, i) => (name, GetStructField(KnownNotNull(structExpr), i).asInstanceOf[Expression]) + } + + val addOrReplaceExprs = names.zip(valExprs) + + val resolver = SQLConf.get.resolver + val newExprs = addOrReplaceExprs.foldLeft(existingExprs) { + case (resultExprs, newExpr @ (newExprName, _)) => + if (resultExprs.exists(x => resolver(x._1, newExprName))) { + resultExprs.map { + case (name, _) if resolver(name, newExprName) => newExpr + case x => x + } + } else { + resultExprs :+ newExpr + } + }.flatMap { case (name, expr) => Seq(Literal(name), expr) } + + val expr = CreateNamedStruct(newExprs) + if (structExpr.nullable) { + If(IsNull(structExpr), Literal(null, expr.dataType), expr) + } else { + expr + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala index f79dabf758c14..1c33a2c7c3136 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala @@ -39,7 +39,18 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] { // Remove redundant field extraction. case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) => createNamedStruct.valExprs(ordinal) - + case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) => + val name = w.dataType(ordinal).name + val matches = names.zip(valExprs).filter(_._1 == name) + if (matches.nonEmpty) { + // return last matching element as that is the final value for the field being extracted. + // For example, if a user submits a query like this: + // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")` + // we want to return `lit(2)` (and not `lit(1)`). + matches.last._2 + } else { + GetStructField(struct, ordinal, maybeName) + } // Remove redundant array indexing. case GetArrayStructFields(CreateArray(elems, useStringTypeWhenEmpty), field, ordinal, _, _) => // Instead of selecting the field on the entire array, select it from each member 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 e800ee3b93f51..1b141572cc7f9 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 @@ -107,6 +107,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateSerialization, RemoveRedundantAliases, RemoveNoopOperators, + CombineWithFields, SimplifyExtractValueOps, CombineConcats) ++ extendedOperatorOptimizationRules @@ -207,7 +208,8 @@ abstract class Optimizer(catalogManager: CatalogManager) CollapseProject, RemoveNoopOperators) :+ // This batch must be executed after the `RewriteSubquery` batch, which creates joins. - Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) + Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) :+ + Batch("ReplaceWithFieldsExpression", Once, ReplaceWithFieldsExpression) // remove any batches with no rules. this may happen when subclasses do not add optional rules. batches.filter(_.rules.nonEmpty) @@ -240,7 +242,8 @@ abstract class Optimizer(catalogManager: CatalogManager) PullupCorrelatedPredicates.ruleName :: RewriteCorrelatedScalarSubquery.ruleName :: RewritePredicateSubquery.ruleName :: - NormalizeFloatingNumbers.ruleName :: Nil + NormalizeFloatingNumbers.ruleName :: + ReplaceWithFieldsExpression.ruleName :: Nil /** * Optimize all the subqueries inside expression. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/WithFields.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/WithFields.scala new file mode 100644 index 0000000000000..05c90864e4bb0 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/WithFields.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions.WithFields +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule + + +/** + * Combines all adjacent [[WithFields]] expression into a single [[WithFields]] expression. + */ +object CombineWithFields extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case WithFields(WithFields(struct, names1, valExprs1), names2, valExprs2) => + WithFields(struct, names1 ++ names2, valExprs1 ++ valExprs2) + } +} + +/** + * Replaces [[WithFields]] expression with an evaluable expression. + */ +object ReplaceWithFieldsExpression extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case w: WithFields => w.evalExpr + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineWithFieldsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineWithFieldsSuite.scala new file mode 100644 index 0000000000000..a3e0bbc57e639 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineWithFieldsSuite.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, WithFields} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + + +class CombineWithFieldsSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("CombineWithFields", FixedPoint(10), CombineWithFields) :: Nil + } + + private val testRelation = LocalRelation('a.struct('a1.int)) + + test("combines two WithFields") { + val originalQuery = testRelation + .select(Alias( + WithFields( + WithFields( + 'a, + Seq("b1"), + Seq(Literal(4))), + Seq("c1"), + Seq(Literal(5))), "out")()) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .select(Alias(WithFields('a, Seq("b1", "c1"), Seq(Literal(4), Literal(5))), "out")()) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("combines three WithFields") { + val originalQuery = testRelation + .select(Alias( + WithFields( + WithFields( + WithFields( + 'a, + Seq("b1"), + Seq(Literal(4))), + Seq("c1"), + Seq(Literal(5))), + Seq("d1"), + Seq(Literal(6))), "out")()) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .select(Alias(WithFields('a, Seq("b1", "c1", "d1"), Seq(4, 5, 6).map(Literal(_))), "out")()) + .analyze + + comparePlans(optimized, correctAnswer) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala index d55746002783a..c71e7dbe7d6f9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala @@ -452,4 +452,61 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](2, 1), BinaryType)), "2") checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](3, 4))), null) } + + private val structAttr = 'struct1.struct('a.int) + private val testStructRelation = LocalRelation(structAttr) + + test("simplify GetStructField on WithFields that is not changing the attribute being extracted") { + val query = testStructRelation.select( + GetStructField(WithFields('struct1, Seq("b"), Seq(Literal(1))), 0, Some("a")) as "outerAtt") + val expected = testStructRelation.select(GetStructField('struct1, 0, Some("a")) as "outerAtt") + checkRule(query, expected) + } + + test("simplify GetStructField on WithFields that is changing the attribute being extracted") { + val query = testStructRelation.select( + GetStructField(WithFields('struct1, Seq("b"), Seq(Literal(1))), 1, Some("b")) as "outerAtt") + val expected = testStructRelation.select(Literal(1) as "outerAtt") + checkRule(query, expected) + } + + test( + "simplify GetStructField on WithFields that is changing the attribute being extracted twice") { + val query = testStructRelation + .select(GetStructField(WithFields('struct1, Seq("b", "b"), Seq(Literal(1), Literal(2))), 1, + Some("b")) as "outerAtt") + val expected = testStructRelation.select(Literal(2) as "outerAtt") + checkRule(query, expected) + } + + test("collapse multiple GetStructField on the same WithFields") { + val query = testStructRelation + .select(WithFields('struct1, Seq("b"), Seq(Literal(2))) as "struct2") + .select( + GetStructField('struct2, 0, Some("a")) as "struct1A", + GetStructField('struct2, 1, Some("b")) as "struct1B") + val expected = testStructRelation.select( + GetStructField('struct1, 0, Some("a")) as "struct1A", + Literal(2) as "struct1B") + checkRule(query, expected) + } + + test("collapse multiple GetStructField on different WithFields") { + val query = testStructRelation + .select( + WithFields('struct1, Seq("b"), Seq(Literal(2))) as "struct2", + WithFields('struct1, Seq("b"), Seq(Literal(3))) as "struct3") + .select( + GetStructField('struct2, 0, Some("a")) as "struct2A", + GetStructField('struct2, 1, Some("b")) as "struct2B", + GetStructField('struct3, 0, Some("a")) as "struct3A", + GetStructField('struct3, 1, Some("b")) as "struct3B") + val expected = testStructRelation + .select( + GetStructField('struct1, 0, Some("a")) as "struct2A", + Literal(2) as "struct2B", + GetStructField('struct1, 0, Some("a")) as "struct3A", + Literal(3) as "struct3B") + checkRule(query, expected) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index e6f7b1d723af6..da542c67d9c51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -871,6 +871,72 @@ class Column(val expr: Expression) extends Logging { */ def getItem(key: Any): Column = withExpr { UnresolvedExtractValue(expr, Literal(key)) } + // scalastyle:off line.size.limit + /** + * An expression that adds/replaces field in `StructType` by name. + * + * {{{ + * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + * df.select($"struct_col".withField("c", lit(3))) + * // result: {"a":1,"b":2,"c":3} + * + * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + * df.select($"struct_col".withField("b", lit(3))) + * // result: {"a":1,"b":3} + * + * val df = sql("SELECT CAST(NULL AS struct) struct_col") + * df.select($"struct_col".withField("c", lit(3))) + * // result: null of type struct + * + * val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col") + * df.select($"struct_col".withField("b", lit(100))) + * // result: {"a":1,"b":100,"b":100} + * + * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + * df.select($"struct_col".withField("a.c", lit(3))) + * // result: {"a":{"a":1,"b":2,"c":3}} + * + * val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") + * df.select($"struct_col".withField("a.c", lit(3))) + * // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields + * }}} + * + * @group expr_ops + * @since 3.1.0 + */ + // scalastyle:on line.size.limit + def withField(fieldName: String, col: Column): Column = withExpr { + require(fieldName != null, "fieldName cannot be null") + require(col != null, "col cannot be null") + + val nameParts = if (fieldName.isEmpty) { + fieldName :: Nil + } else { + CatalystSqlParser.parseMultipartIdentifier(fieldName) + } + withFieldHelper(expr, nameParts, Nil, col.expr) + } + + private def withFieldHelper( + struct: Expression, + namePartsRemaining: Seq[String], + namePartsDone: Seq[String], + value: Expression) : WithFields = { + val name = namePartsRemaining.head + if (namePartsRemaining.length == 1) { + WithFields(struct, name :: Nil, value :: Nil) + } else { + val newNamesRemaining = namePartsRemaining.tail + val newNamesDone = namePartsDone :+ name + val newValue = withFieldHelper( + struct = UnresolvedExtractValue(struct, Literal(name)), + namePartsRemaining = newNamesRemaining, + namePartsDone = newNamesDone, + value = value) + WithFields(struct, name :: Nil, newValue :: Nil) + } + } + /** * An expression that gets a field by name in a `StructType`. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index fa06484a73d95..131ab1b94f59e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -923,4 +923,503 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { val inSet = InSet(Literal("a"), Set("a", "b").map(UTF8String.fromString)) assert(inSet.sql === "('a' IN ('a', 'b'))") } + + def checkAnswerAndSchema( + df: => DataFrame, + expectedAnswer: Seq[Row], + expectedSchema: StructType): Unit = { + + checkAnswer(df, expectedAnswer) + assert(df.schema == expectedSchema) + } + + private lazy val structType = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false))) + + private lazy val structLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, null, 3)) :: Nil), + StructType(Seq(StructField("a", structType, nullable = false)))) + + private lazy val nullStructLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("a", structType, nullable = true)))) + + private lazy val structLevel2: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(1, null, 3))) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", structType, nullable = false))), + nullable = false)))) + + private lazy val nullStructLevel2: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(null)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", structType, nullable = true))), + nullable = false)))) + + private lazy val structLevel3: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(Row(1, null, 3)))) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", structType, nullable = false))), + nullable = false))), + nullable = false)))) + + test("withField should throw an exception if called on a non-StructType column") { + intercept[AnalysisException] { + testData.withColumn("key", $"key".withField("a", lit(2))) + }.getMessage should include("struct argument should be struct type, got: int") + } + + test("withField should throw an exception if either fieldName or col argument are null") { + intercept[IllegalArgumentException] { + structLevel1.withColumn("a", $"a".withField(null, lit(2))) + }.getMessage should include("fieldName cannot be null") + + intercept[IllegalArgumentException] { + structLevel1.withColumn("a", $"a".withField("b", null)) + }.getMessage should include("col cannot be null") + + intercept[IllegalArgumentException] { + structLevel1.withColumn("a", $"a".withField(null, null)) + }.getMessage should include("fieldName cannot be null") + } + + test("withField should throw an exception if any intermediate structs don't exist") { + intercept[AnalysisException] { + structLevel2.withColumn("a", 'a.withField("x.b", lit(2))) + }.getMessage should include("No such struct field x in a") + + intercept[AnalysisException] { + structLevel3.withColumn("a", 'a.withField("a.x.b", lit(2))) + }.getMessage should include("No such struct field x in a") + } + + test("withField should throw an exception if intermediate field is not a struct") { + intercept[AnalysisException] { + structLevel1.withColumn("a", 'a.withField("b.a", lit(2))) + }.getMessage should include("struct argument should be struct type, got: int") + } + + test("withField should throw an exception if intermediate field reference is ambiguous") { + intercept[AnalysisException] { + val structLevel2: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", structType, nullable = false), + StructField("a", structType, nullable = false))), + nullable = false)))) + + structLevel2.withColumn("a", 'a.withField("a.b", lit(2))) + }.getMessage should include("Ambiguous reference to fields") + } + + test("withField should add field with no name") { + checkAnswerAndSchema( + structLevel1.withColumn("a", $"a".withField("", lit(4))), + Row(Row(1, null, 3, 4)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("", IntegerType, nullable = false))), + nullable = false)))) + } + + test("withField should add field to struct") { + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("d", lit(4))), + Row(Row(1, null, 3, 4)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false))), + nullable = false)))) + } + + test("withField should add field to null struct") { + checkAnswerAndSchema( + nullStructLevel1.withColumn("a", $"a".withField("d", lit(4))), + Row(null) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false))), + nullable = true)))) + } + + test("withField should add field to nested null struct") { + checkAnswerAndSchema( + nullStructLevel2.withColumn("a", $"a".withField("a.d", lit(4))), + Row(Row(null)) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false))), + nullable = true))), + nullable = false)))) + } + + test("withField should add null field to struct") { + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("d", lit(null).cast(IntegerType))), + Row(Row(1, null, 3, null)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = true))), + nullable = false)))) + } + + test("withField should add multiple fields to struct") { + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("d", lit(4)).withField("e", lit(5))), + Row(Row(1, null, 3, 4, 5)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = false)))) + } + + test("withField should add field to nested struct") { + Seq( + structLevel2.withColumn("a", 'a.withField("a.d", lit(4))), + structLevel2.withColumn("a", 'a.withField("a", $"a.a".withField("d", lit(4)))) + ).foreach { df => + checkAnswerAndSchema( + df, + Row(Row(Row(1, null, 3, 4))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("withField should add field to deeply nested struct") { + checkAnswerAndSchema( + structLevel3.withColumn("a", 'a.withField("a.a.d", lit(4))), + Row(Row(Row(Row(1, null, 3, 4)))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false))), + nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("withField should replace field in struct") { + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("b", lit(2))), + Row(Row(1, 2, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + } + + test("withField should replace field in null struct") { + checkAnswerAndSchema( + nullStructLevel1.withColumn("a", 'a.withField("b", lit("foo"))), + Row(null) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true)))) + } + + test("withField should replace field in nested null struct") { + checkAnswerAndSchema( + nullStructLevel2.withColumn("a", $"a".withField("a.b", lit("foo"))), + Row(Row(null)) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true))), + nullable = false)))) + } + + test("withField should replace field with null value in struct") { + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("c", lit(null).cast(IntegerType))), + Row(Row(1, null, null)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = true))), + nullable = false)))) + } + + test("withField should replace multiple fields in struct") { + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("a", lit(10)).withField("b", lit(20))), + Row(Row(10, 20, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + } + + test("withField should replace field in nested struct") { + Seq( + structLevel2.withColumn("a", $"a".withField("a.b", lit(2))), + structLevel2.withColumn("a", 'a.withField("a", $"a.a".withField("b", lit(2)))) + ).foreach { df => + checkAnswerAndSchema( + df, + Row(Row(Row(1, 2, 3))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("withField should replace field in deeply nested struct") { + checkAnswerAndSchema( + structLevel3.withColumn("a", $"a".withField("a.a.b", lit(2))), + Row(Row(Row(Row(1, 2, 3)))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("withField should replace all fields with given name in struct") { + val structLevel1 = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2, 3)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("b", lit(100))), + Row(Row(1, 100, 100)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false)))) + } + + test("withField should replace fields in struct in given order") { + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("b", lit(2)).withField("b", lit(20))), + Row(Row(1, 20, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + } + + test("withField should add field and then replace same field in struct") { + checkAnswerAndSchema( + structLevel1.withColumn("a", 'a.withField("d", lit(4)).withField("d", lit(5))), + Row(Row(1, null, 3, 5)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false))), + nullable = false)))) + } + + test("withField should handle fields with dots in their name if correctly quoted") { + val df: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(1, null, 3))) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a.b", StructType(Seq( + StructField("c.d", IntegerType, nullable = false), + StructField("e.f", IntegerType, nullable = true), + StructField("g.h", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + + checkAnswerAndSchema( + df.withColumn("a", 'a.withField("`a.b`.`e.f`", lit(2))), + Row(Row(Row(1, 2, 3))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a.b", StructType(Seq( + StructField("c.d", IntegerType, nullable = false), + StructField("e.f", IntegerType, nullable = false), + StructField("g.h", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + + intercept[AnalysisException] { + df.withColumn("a", 'a.withField("a.b.e.f", lit(2))) + }.getMessage should include("No such struct field a in a.b") + } + + private lazy val mixedCaseStructLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 1)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("B", IntegerType, nullable = false))), + nullable = false)))) + + test("withField should replace field in struct even if casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswerAndSchema( + mixedCaseStructLevel1.withColumn("a", 'a.withField("A", lit(2))), + Row(Row(2, 1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("A", IntegerType, nullable = false), + StructField("B", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswerAndSchema( + mixedCaseStructLevel1.withColumn("a", 'a.withField("b", lit(2))), + Row(Row(1, 2)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false)))) + } + } + + test("withField should add field to struct because casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswerAndSchema( + mixedCaseStructLevel1.withColumn("a", 'a.withField("A", lit(2))), + Row(Row(1, 1, 2)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("B", IntegerType, nullable = false), + StructField("A", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswerAndSchema( + mixedCaseStructLevel1.withColumn("a", 'a.withField("b", lit(2))), + Row(Row(1, 1, 2)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("B", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false)))) + } + } + + private lazy val mixedCaseStructLevel2: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(1, 1), Row(1, 1))) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false), + StructField("B", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + + test("withField should replace nested field in struct even if casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswerAndSchema( + mixedCaseStructLevel2.withColumn("a", 'a.withField("A.a", lit(2))), + Row(Row(Row(2, 1), Row(1, 1))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("A", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false), + StructField("B", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + + checkAnswerAndSchema( + mixedCaseStructLevel2.withColumn("a", 'a.withField("b.a", lit(2))), + Row(Row(Row(1, 1), Row(2, 1))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false), + StructField("b", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("withField should throw an exception because casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + intercept[AnalysisException] { + mixedCaseStructLevel2.withColumn("a", 'a.withField("A.a", lit(2))) + }.getMessage should include("No such struct field A in a, B") + + intercept[AnalysisException] { + mixedCaseStructLevel2.withColumn("a", 'a.withField("b.a", lit(2))) + }.getMessage should include("No such struct field b in a, B") + } + } } From eb8eda7d3232fb3bae3e3321260414e6459c8ee9 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 7 Jul 2020 09:38:08 -0700 Subject: [PATCH 130/384] [SPARK-32211][SQL] Pin mariadb-plugin-gssapi-server version to fix MariaDBKrbIntegrationSuite ### What changes were proposed in this pull request? `MariaDBKrbIntegrationSuite` fails because the docker image contains MariaDB version `1:10.4.12+maria~bionic` but `1:10.4.13+maria~bionic` came out and `mariadb-plugin-gssapi-server` installation triggered unwanted database upgrade inside the docker image. The main problem is that the docker image scripts are prepared to handle `1:10.4.12+maria~bionic` version and not any future development. ### Why are the changes needed? Failing test. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Executed `MariaDBKrbIntegrationSuite` manually. Closes #29025 from gaborgsomogyi/SPARK-32211. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- .../src/test/resources/mariadb_docker_entrypoint.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh b/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh index 00885a3b62327..343bc01651318 100755 --- a/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh +++ b/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh @@ -18,7 +18,7 @@ dpkg-divert --add /bin/systemctl && ln -sT /bin/true /bin/systemctl apt update -apt install -y mariadb-plugin-gssapi-server +apt install -y mariadb-plugin-gssapi-server=1:10.4.12+maria~bionic echo "gssapi_keytab_path=/docker-entrypoint-initdb.d/mariadb.keytab" >> /etc/mysql/mariadb.conf.d/auth_gssapi.cnf echo "gssapi_principal_name=mariadb/__IP_ADDRESS_REPLACE_ME__@EXAMPLE.COM" >> /etc/mysql/mariadb.conf.d/auth_gssapi.cnf docker-entrypoint.sh mysqld From 90b9099064059e414bf3535f7f250aad384288b5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 7 Jul 2020 11:17:53 -0700 Subject: [PATCH 131/384] [SPARK-32163][SQL] Nested pruning should work even with cosmetic variations ### What changes were proposed in this pull request? This patch proposes to deal with cosmetic variations when processing nested column extractors in `NestedColumnAliasing`. Currently if cosmetic variations are in the nested column extractors, the query is not optimized. ### Why are the changes needed? If the expressions extracting nested fields have cosmetic variations like qualifier difference, currently nested column pruning cannot work well. For example, two attributes which are semantically the same, are referred in a query, but the nested column extractors of them are treated differently when we deal with nested column pruning. ### Does this PR introduce _any_ user-facing change? Yes, fixing a bug in nested column pruning. ### How was this patch tested? Unit test. Closes #28988 from viirya/SPARK-32163. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../optimizer/NestedColumnAliasing.scala | 11 +++++++--- .../datasources/SchemaPruningSuite.scala | 20 +++++++++++++++++++ 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index 118f41f9cd232..0c8666b72cace 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -149,10 +149,12 @@ object NestedColumnAliasing { case _ => false } + // Note that when we group by extractors with their references, we should remove + // cosmetic variations. val exclusiveAttrSet = AttributeSet(exclusiveAttrs ++ otherRootReferences) val aliasSub = nestedFieldReferences.asInstanceOf[Seq[ExtractValue]] .filter(!_.references.subsetOf(exclusiveAttrSet)) - .groupBy(_.references.head) + .groupBy(_.references.head.canonicalized.asInstanceOf[Attribute]) .flatMap { case (attr, nestedFields: Seq[ExtractValue]) => // Remove redundant `ExtractValue`s if they share the same parent nest field. // For example, when `a.b` and `a.b.c` are in project list, we only need to alias `a.b`. @@ -174,9 +176,12 @@ object NestedColumnAliasing { // If all nested fields of `attr` are used, we don't need to introduce new aliases. // By default, ColumnPruning rule uses `attr` already. + // Note that we need to remove cosmetic variations first, so we only count a + // nested field once. if (nestedFieldToAlias.nonEmpty && - nestedFieldToAlias - .map { case (nestedField, _) => totalFieldNum(nestedField.dataType) } + dedupNestedFields.map(_.canonicalized) + .distinct + .map { nestedField => totalFieldNum(nestedField.dataType) } .sum < totalFieldNum(attr.dataType)) { Some(attr.exprId -> nestedFieldToAlias) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 8b859e951b9b9..d51eafa5a8aed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -497,6 +497,26 @@ abstract class SchemaPruningSuite Row(Row("Janet", null, "Jones"), "Jones") ::Nil) } + testSchemaPruning("SPARK-32163: nested pruning should work even with cosmetic variations") { + withTempView("contact_alias") { + sql("select * from contacts") + .repartition(100, col("name.first"), col("name.last")) + .selectExpr("name").createOrReplaceTempView("contact_alias") + + val query1 = sql("select name.first from contact_alias") + checkScan(query1, "struct>") + checkAnswer(query1, Row("Jane") :: Row("John") :: Row("Jim") :: Row("Janet") ::Nil) + + sql("select * from contacts") + .select(explode(col("friends.first")), col("friends")) + .createOrReplaceTempView("contact_alias") + + val query2 = sql("select friends.middle, col from contact_alias") + checkScan(query2, "struct>>") + checkAnswer(query2, Row(Array("Z."), "Susan") :: Nil) + } + } + protected def testSchemaPruning(testName: String)(testThunk: => Unit): Unit = { test(s"Spark vectorized reader - without partition data column - $testName") { withSQLConf(vectorizedReaderEnabledKey -> "true") { From 1261fac67457afab38bfad42e62d7bf93c661204 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 7 Jul 2020 14:09:40 -0700 Subject: [PATCH 132/384] [SPARK-31710][SQL][FOLLOWUP] Allow cast numeric to timestamp by default ### What changes were proposed in this pull request? 1. Set the SQL config `spark.sql.legacy.allowCastNumericToTimestamp` to `true` by default 2. Remove explicit sets of `spark.sql.legacy.allowCastNumericToTimestamp` to `true` in the cast suites. ### Why are the changes needed? To avoid breaking changes in minor versions (in the upcoming Spark 3.1.0) according to the the semantic versioning guidelines (https://spark.apache.org/versioning-policy.html) ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? By `CastSuite`. Closes #29012 from MaxGekk/allow-cast-numeric-to-timestamp. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 2 - .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/catalyst/expressions/CastSuite.scala | 84 ++++++++----------- 3 files changed, 38 insertions(+), 50 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 0c84db38afafc..d3138ae319160 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -30,8 +30,6 @@ license: | - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. - - In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can set `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710. - ## Upgrading from Spark SQL 3.0 to 3.0.1 - In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Since version 3.0.1, the timestamp type inference is disabled by default. Set the JSON option `inferTimestamp` to `true` to enable such type inference. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3149d14c1ddcc..31dd943eeba2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2634,7 +2634,7 @@ object SQLConf { "when false, forbid the cast, more details in SPARK-31710") .version("3.1.0") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED = buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 76ec450a4d7c6..4ab288a34cb08 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -49,9 +49,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } protected def checkNullCast(from: DataType, to: DataType): Unit = { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) - } + checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) } test("null cast") { @@ -240,9 +238,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkCast(1.5, 1.5f) checkCast(1.5, "1.5") - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) - } + checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) } test("cast from string") { @@ -309,19 +305,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), LongType), 5.toLong) - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - 5.toShort) - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - null) - checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), - ByteType), TimestampType), LongType), StringType), ShortType), - 5.toShort) - } + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + 5.toShort) + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + null) + checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), + ByteType), TimestampType), LongType), StringType), ShortType), + 5.toShort) checkEvaluation(cast("23", DoubleType), 23d) checkEvaluation(cast("23", IntegerType), 23) @@ -383,31 +377,29 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(ts, FloatType), 15.003f) checkEvaluation(cast(ts, DoubleType), 15.003) - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - checkEvaluation(cast(cast(tss, ShortType), TimestampType), - fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation(cast(cast(tss, IntegerType), TimestampType), - fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation(cast(cast(tss, LongType), TimestampType), - fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), - millis.toFloat / MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), - millis.toDouble / MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), - Decimal(1)) + checkEvaluation(cast(cast(tss, ShortType), TimestampType), + fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation(cast(cast(tss, IntegerType), TimestampType), + fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation(cast(cast(tss, LongType), TimestampType), + fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), + millis.toFloat / MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), + millis.toDouble / MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), + Decimal(1)) - // A test for higher precision than millis - checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) + // A test for higher precision than millis + checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) - checkEvaluation(cast(Double.NaN, TimestampType), null) - checkEvaluation(cast(1.0 / 0.0, TimestampType), null) - checkEvaluation(cast(Float.NaN, TimestampType), null) - checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) - } + checkEvaluation(cast(Double.NaN, TimestampType), null) + checkEvaluation(cast(1.0 / 0.0, TimestampType), null) + checkEvaluation(cast(Float.NaN, TimestampType), null) + checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) } test("cast from array") { @@ -1036,10 +1028,8 @@ class CastSuite extends CastSuiteBase { test("cast from int 2") { checkEvaluation(cast(1, LongType), 1.toLong) - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) - checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) - } + checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) + checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) @@ -1323,7 +1313,7 @@ class CastSuite extends CastSuiteBase { } } - test("SPARK-31710:fail casting from numeric to timestamp by default") { + test("SPARK-31710: fail casting from numeric to timestamp if it is forbidden") { Seq(true, false).foreach { enable => withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) { assert(cast(2.toByte, TimestampType).resolved == enable) From 8b0a54e6ffadd6326fd254f67ebeb1673eb8f33b Mon Sep 17 00:00:00 2001 From: Ali Smesseim Date: Wed, 8 Jul 2020 09:28:16 +0900 Subject: [PATCH 133/384] [SPARK-32057][SQL][TEST-HIVE1.2][TEST-HADOOP2.7] ExecuteStatement: cancel and close should not transiently ERROR ### What changes were proposed in this pull request? #28671 introduced a change where the order in which CANCELED state for SparkExecuteStatementOperation is set was changed. Before setting the state to CANCELED, `cleanup()` was called which kills the jobs, causing an exception to be thrown inside `execute()`. This causes the state to transiently become ERROR before being set to CANCELED. This PR fixes the order. ### Why are the changes needed? Bug: wrong operation state is set. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test in SparkExecuteStatementOperationSuite.scala. Closes #28912 from alismess-db/execute-statement-operation-cleanup-order. Authored-by: Ali Smesseim Signed-off-by: HyukjinKwon --- project/SparkBuild.scala | 1 + .../SparkExecuteStatementOperation.scala | 2 +- .../hive/thriftserver/SparkOperation.scala | 2 +- .../SparkExecuteStatementOperationSuite.scala | 81 ++++++++++++++++++- 4 files changed, 83 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 04a3fc4b63050..60c54dfc98a58 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -476,6 +476,7 @@ object SparkParallelTestGrouping { "org.apache.spark.ml.classification.LogisticRegressionSuite", "org.apache.spark.ml.classification.LinearSVCSuite", "org.apache.spark.sql.SQLQueryTestSuite", + "org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperationSuite", "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite", "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index eae5d5d4bcfa9..57ed15a76a893 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -333,8 +333,8 @@ private[hive] class SparkExecuteStatementOperation( synchronized { if (!getStatus.getState.isTerminal) { logInfo(s"Cancel query with $statementId") - cleanup() setState(OperationState.CANCELED) + cleanup() HiveThriftServer2.eventManager.onStatementCanceled(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index 0acd1b3e9899a..446669d08e76b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -46,8 +46,8 @@ private[hive] trait SparkOperation extends Operation with Logging { } abstract override def close(): Unit = { - cleanup() super.close() + cleanup() logInfo(s"Close statement with $statementId") HiveThriftServer2.eventManager.onOperationClosed(statementId) } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala index 13df3fabc4919..4c2f29e0bf394 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala @@ -17,10 +17,25 @@ package org.apache.spark.sql.hive.thriftserver +import java.util +import java.util.concurrent.Semaphore + +import scala.concurrent.duration._ + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hive.service.cli.OperationState +import org.apache.hive.service.cli.session.{HiveSession, HiveSessionImpl} +import org.mockito.Mockito.{doReturn, mock, spy, when, RETURNS_DEEP_STUBS} +import org.mockito.invocation.InvocationOnMock + import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2EventManager +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, NullType, StringType, StructField, StructType} -class SparkExecuteStatementOperationSuite extends SparkFunSuite { +class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSparkSession { + test("SPARK-17112 `select null` via JDBC triggers IllegalArgumentException in ThriftServer") { val field1 = StructField("NULL", NullType) val field2 = StructField("(IF(true, NULL, NULL))", NullType) @@ -42,4 +57,68 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite { assert(columns.get(1).getType().getName == "INT") assert(columns.get(1).getComment() == "") } + + Seq( + (OperationState.CANCELED, (_: SparkExecuteStatementOperation).cancel()), + (OperationState.CLOSED, (_: SparkExecuteStatementOperation).close()) + ).foreach { case (finalState, transition) => + test("SPARK-32057 SparkExecuteStatementOperation should not transiently become ERROR " + + s"before being set to $finalState") { + val hiveSession = new HiveSessionImpl(ThriftserverShimUtils.testedProtocolVersions.head, + "username", "password", new HiveConf, "ip address") + hiveSession.open(new util.HashMap) + + HiveThriftServer2.eventManager = mock(classOf[HiveThriftServer2EventManager]) + + val spySqlContext = spy(sqlContext) + + // When cancel() is called on the operation, cleanup causes an exception to be thrown inside + // of execute(). This should not cause the state to become ERROR. The exception here will be + // triggered in our custom cleanup(). + val signal = new Semaphore(0) + val dataFrame = mock(classOf[DataFrame], RETURNS_DEEP_STUBS) + when(dataFrame.collect()).thenAnswer((_: InvocationOnMock) => { + signal.acquire() + throw new RuntimeException("Operation was cancelled by test cleanup.") + }) + val statement = "stmt" + doReturn(dataFrame, Nil: _*).when(spySqlContext).sql(statement) + + val executeStatementOperation = new MySparkExecuteStatementOperation(spySqlContext, + hiveSession, statement, signal, finalState) + + val run = new Thread() { + override def run(): Unit = executeStatementOperation.runInternal() + } + assert(executeStatementOperation.getStatus.getState === OperationState.INITIALIZED) + run.start() + eventually(timeout(5.seconds)) { + assert(executeStatementOperation.getStatus.getState === OperationState.RUNNING) + } + transition(executeStatementOperation) + run.join() + assert(executeStatementOperation.getStatus.getState === finalState) + } + } + + private class MySparkExecuteStatementOperation( + sqlContext: SQLContext, + hiveSession: HiveSession, + statement: String, + signal: Semaphore, + finalState: OperationState) + extends SparkExecuteStatementOperation(sqlContext, hiveSession, statement, + new util.HashMap, false) { + + override def cleanup(): Unit = { + super.cleanup() + signal.release() + // At this point, operation should already be in finalState (set by either close() or + // cancel()). We want to check if it stays in finalState after the exception thrown by + // releasing the semaphore propagates. We hence need to sleep for a short while. + Thread.sleep(1000) + // State should not be ERROR + assert(getStatus.getState === finalState) + } + } } From 365961155a655f19c9184b16ccd493838c848213 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 8 Jul 2020 09:36:06 +0900 Subject: [PATCH 134/384] [SPARK-32124][CORE][FOLLOW-UP] Use the invalid value Int.MinValue to fill the map index when the event logs from the old Spark version ### What changes were proposed in this pull request? Use the invalid value Int.MinValue to fill the map index when the event logs from the old Spark version. ### Why are the changes needed? Follow up PR for #28941. ### Does this PR introduce _any_ user-facing change? When we use the Spark version 3.0 history server reading the event log written by the old Spark version, we use the invalid value -2 to fill the map index. ### How was this patch tested? Existing UT. Closes #28965 from xuanyuanking/follow-up. Authored-by: Yuanjian Li Signed-off-by: Jungtaek Lim (HeartSaVioR) --- core/src/main/scala/org/apache/spark/TaskEndReason.scala | 3 ++- .../main/scala/org/apache/spark/util/JsonProtocol.scala | 8 ++++++-- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 2 +- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index b13028f868072..6606d317e7b86 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -90,7 +90,8 @@ case class FetchFailed( extends TaskFailedReason { override def toErrorString: String = { val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString - s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapIndex=$mapIndex, " + + val mapIndexString = if (mapIndex == Int.MinValue) "Unknown" else mapIndex.toString + s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapIndex=$mapIndexString, " + s"mapId=$mapId, reduceId=$reduceId, message=\n$message\n)" } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index ced3f9d15720d..f3372501f471b 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -1078,8 +1078,12 @@ private[spark] object JsonProtocol { val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") val shuffleId = (json \ "Shuffle ID").extract[Int] val mapId = (json \ "Map ID").extract[Long] - val mapIndex = (json \ "Map Index") match { - case JNothing => 0 + val mapIndex = json \ "Map Index" match { + case JNothing => + // Note, we use the invalid value Int.MinValue here to fill the map index for backward + // compatibility. Otherwise, the fetch failed event will be dropped when the history + // server loads the event log written by the Spark version before 3.0. + Int.MinValue case x => x.extract[Int] } val reduceId = (json \ "Reduce ID").extract[Int] diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 955589fc5b47b..c75e98f39758d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -334,7 +334,7 @@ class JsonProtocolSuite extends SparkFunSuite { val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) .removeField({ _._1 == "Map Index" }) val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, - 0, 19, "ignored") + Int.MinValue, 19, "ignored") assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } From b5297c43b0bd5a62a20fb047fdee24ebd63f939d Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 7 Jul 2020 18:58:01 -0700 Subject: [PATCH 135/384] [SPARK-20680][SQL] Spark-sql do not support for creating table with void column datatype ### What changes were proposed in this pull request? This is the new PR which to address the close one #17953 1. support "void" primitive data type in the `AstBuilder`, point it to `NullType` 2. forbid creating tables with VOID/NULL column type ### Why are the changes needed? 1. Spark is incompatible with hive void type. When Hive table schema contains void type, DESC table will throw an exception in Spark. >hive> create table bad as select 1 x, null z from dual; >hive> describe bad; OK x int z void In Spark2.0.x, the behaviour to read this view is normal: >spark-sql> describe bad; x int NULL z void NULL Time taken: 4.431 seconds, Fetched 2 row(s) But in lastest Spark version, it failed with SparkException: Cannot recognize hive type string: void >spark-sql> describe bad; 17/05/09 03:12:08 ERROR thriftserver.SparkSQLDriver: Failed in [describe bad] org.apache.spark.SparkException: Cannot recognize hive type string: void Caused by: org.apache.spark.sql.catalyst.parser.ParseException: DataType void() is not supported.(line 1, pos 0) == SQL == void ^^^ ... 61 more org.apache.spark.SparkException: Cannot recognize hive type string: void 2. Hive CTAS statements throws error when select clause has NULL/VOID type column since HIVE-11217 In Spark, creating table with a VOID/NULL column should throw readable exception message, include - create data source table (using parquet, json, ...) - create hive table (with or without stored as) - CTAS ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add unit tests Closes #28833 from LantaoJin/SPARK-20680_COPY. Authored-by: LantaoJin Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/types.py | 3 + .../catalyst/analysis/ResolveCatalogs.scala | 11 ++ .../sql/catalyst/parser/AstBuilder.scala | 1 + .../sql/connector/catalog/CatalogV2Util.scala | 21 ++- .../org/apache/spark/sql/types/NullType.scala | 4 + .../catalyst/parser/DataTypeParserSuite.scala | 1 + .../analysis/ResolveSessionCatalog.scala | 11 ++ .../sql/execution/datasources/rules.scala | 3 + .../sql-functions/sql-expression-schema.md | 2 +- .../sql-tests/results/ansi/literals.sql.out | 2 +- .../sql-tests/results/inline-table.sql.out | 2 +- .../sql-tests/results/literals.sql.out | 2 +- .../sql-tests/results/misc-functions.sql.out | 2 +- .../results/postgreSQL/select.sql.out | 4 +- .../sql-compatibility-functions.sql.out | 6 +- .../results/udf/udf-inline-table.sql.out | 2 +- .../spark/sql/FileBasedDataSourceSuite.scala | 2 +- .../spark/sql/hive/HiveStrategies.scala | 3 + .../sql/hive/execution/HiveDDLSuite.scala | 121 ++++++++++++++++++ .../sql/hive/orc/HiveOrcSourceSuite.scala | 2 +- 20 files changed, 191 insertions(+), 14 deletions(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 320a68dffe7a3..ddd13ca3a01be 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -116,6 +116,9 @@ class NullType(DataType): __metaclass__ = DataTypeSingleton + def simpleString(self): + return 'unknown' + class AtomicType(DataType): """An internal type used to represent everything that is not diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 2a0a944e4849c..a40604045978c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -34,6 +34,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case AlterTableAddColumnsStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => + cols.foreach(c => failNullType(c.dataType)) cols.foreach(c => failCharType(c.dataType)) val changes = cols.map { col => TableChange.addColumn( @@ -47,6 +48,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case AlterTableReplaceColumnsStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => + cols.foreach(c => failNullType(c.dataType)) cols.foreach(c => failCharType(c.dataType)) val changes: Seq[TableChange] = loadTable(catalog, tbl.asIdentifier) match { case Some(table) => @@ -69,6 +71,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case a @ AlterTableAlterColumnStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => + a.dataType.foreach(failNullType) a.dataType.foreach(failCharType) val colName = a.column.toArray val typeChange = a.dataType.map { newDataType => @@ -145,6 +148,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case c @ CreateTableStatement( NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + assertNoNullTypeInSchema(c.tableSchema) assertNoCharTypeInSchema(c.tableSchema) CreateV2Table( catalog.asTableCatalog, @@ -157,6 +161,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case c @ CreateTableAsSelectStatement( NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _) => + if (c.asSelect.resolved) { + assertNoNullTypeInSchema(c.asSelect.schema) + } CreateTableAsSelect( catalog.asTableCatalog, tbl.asIdentifier, @@ -172,6 +179,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case c @ ReplaceTableStatement( NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + assertNoNullTypeInSchema(c.tableSchema) assertNoCharTypeInSchema(c.tableSchema) ReplaceTable( catalog.asTableCatalog, @@ -184,6 +192,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case c @ ReplaceTableAsSelectStatement( NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _) => + if (c.asSelect.resolved) { + assertNoNullTypeInSchema(c.asSelect.schema) + } ReplaceTableAsSelect( catalog.asTableCatalog, tbl.asIdentifier, 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 d08bcb1420176..6b41a8b22fbee 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 @@ -2203,6 +2203,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging DecimalType(precision.getText.toInt, 0) case ("decimal" | "dec" | "numeric", precision :: scale :: Nil) => DecimalType(precision.getText.toInt, scale.getText.toInt) + case ("void", Nil) => NullType case ("interval", Nil) => CalendarIntervalType case (dt, params) => val dtStr = if (params.nonEmpty) s"$dt(${params.mkString(",")})" else dt diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index e1f329352592f..d130a13282cc8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.AlterTable import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.{ArrayType, DataType, HIVE_TYPE_STRING, HiveStringType, MapType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, HIVE_TYPE_STRING, HiveStringType, MapType, NullType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils @@ -346,4 +346,23 @@ private[sql] object CatalogV2Util { } } } + + def failNullType(dt: DataType): Unit = { + def containsNullType(dt: DataType): Boolean = dt match { + case ArrayType(et, _) => containsNullType(et) + case MapType(kt, vt, _) => containsNullType(kt) || containsNullType(vt) + case StructType(fields) => fields.exists(f => containsNullType(f.dataType)) + case _ => dt.isInstanceOf[NullType] + } + if (containsNullType(dt)) { + throw new AnalysisException( + "Cannot create tables with unknown type.") + } + } + + def assertNoNullTypeInSchema(schema: StructType): Unit = { + schema.foreach { f => + failNullType(f.dataType) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala index 14097a5280d50..6c9a1d69ca681 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala @@ -32,6 +32,10 @@ class NullType private() extends DataType { override def defaultSize: Int = 1 private[spark] override def asNullable: NullType = this + + // "null" is mainly used to represent a literal in Spark, + // it's better to avoid using it for data types. + override def simpleString: String = "unknown" } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala index d519fdf378786..655b1d26d6c90 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala @@ -61,6 +61,7 @@ class DataTypeParserSuite extends SparkFunSuite { checkDataType("varchAr(20)", StringType) checkDataType("cHaR(27)", StringType) checkDataType("BINARY", BinaryType) + checkDataType("void", NullType) checkDataType("interval", CalendarIntervalType) checkDataType("array", ArrayType(DoubleType, true)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index bf90875e511f8..bc3f38a35834d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -48,6 +48,7 @@ class ResolveSessionCatalog( override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case AlterTableAddColumnsStatement( nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => + cols.foreach(c => failNullType(c.dataType)) loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => if (!DDLUtils.isHiveTable(v1Table.v1Table)) { @@ -76,6 +77,7 @@ class ResolveSessionCatalog( case AlterTableReplaceColumnsStatement( nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => + cols.foreach(c => failNullType(c.dataType)) val changes: Seq[TableChange] = loadTable(catalog, tbl.asIdentifier) match { case Some(_: V1Table) => throw new AnalysisException("REPLACE COLUMNS is only supported with v2 tables.") @@ -100,6 +102,7 @@ class ResolveSessionCatalog( case a @ AlterTableAlterColumnStatement( nameParts @ SessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => + a.dataType.foreach(failNullType) loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => if (!DDLUtils.isHiveTable(v1Table.v1Table)) { @@ -268,6 +271,7 @@ class ResolveSessionCatalog( // session catalog and the table provider is not v2. case c @ CreateTableStatement( SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + assertNoNullTypeInSchema(c.tableSchema) val provider = c.provider.getOrElse(conf.defaultDataSourceName) if (!isV2Provider(provider)) { if (!DDLUtils.isHiveTable(Some(provider))) { @@ -292,6 +296,9 @@ class ResolveSessionCatalog( case c @ CreateTableAsSelectStatement( SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _) => + if (c.asSelect.resolved) { + assertNoNullTypeInSchema(c.asSelect.schema) + } val provider = c.provider.getOrElse(conf.defaultDataSourceName) if (!isV2Provider(provider)) { val tableDesc = buildCatalogTable(tbl.asTableIdentifier, new StructType, @@ -319,6 +326,7 @@ class ResolveSessionCatalog( // session catalog and the table provider is not v2. case c @ ReplaceTableStatement( SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + assertNoNullTypeInSchema(c.tableSchema) val provider = c.provider.getOrElse(conf.defaultDataSourceName) if (!isV2Provider(provider)) { throw new AnalysisException("REPLACE TABLE is only supported with v2 tables.") @@ -336,6 +344,9 @@ class ResolveSessionCatalog( case c @ ReplaceTableAsSelectStatement( SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _) => + if (c.asSelect.resolved) { + assertNoNullTypeInSchema(c.asSelect.schema) + } val provider = c.provider.getOrElse(conf.defaultDataSourceName) if (!isV2Provider(provider)) { throw new AnalysisException("REPLACE TABLE AS SELECT is only supported with v2 tables.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 95343e2872def..60cacda9f5f1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.CatalogV2Util.assertNoNullTypeInSchema import org.apache.spark.sql.connector.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 @@ -292,6 +293,8 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi "in the table definition of " + table.identifier, sparkSession.sessionState.conf.caseSensitiveAnalysis) + assertNoNullTypeInSchema(schema) + val normalizedPartCols = normalizePartitionColumns(schema, table) val normalizedBucketSpec = normalizeBucketSpec(schema, table) 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 8898a11ec08fb..c39adac4ac680 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 @@ -34,7 +34,7 @@ | org.apache.spark.sql.catalyst.expressions.Ascii | ascii | SELECT ascii('222') | struct | | org.apache.spark.sql.catalyst.expressions.Asin | asin | SELECT asin(0) | struct | | org.apache.spark.sql.catalyst.expressions.Asinh | asinh | SELECT asinh(0) | struct | -| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT assert_true(0 < 1) | struct | +| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT assert_true(0 < 1) | struct | | org.apache.spark.sql.catalyst.expressions.Atan | atan | SELECT atan(0) | struct | | org.apache.spark.sql.catalyst.expressions.Atan2 | atan2 | SELECT atan2(0, 0) | struct | | org.apache.spark.sql.catalyst.expressions.Atanh | atanh | SELECT atanh(0) | struct | diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index f6720f6c5faa4..02747718c91df 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -5,7 +5,7 @@ -- !query select null, Null, nUll -- !query schema -struct +struct -- !query output NULL NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out index 9943b93c431df..2dd6960682740 100644 --- a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -49,7 +49,7 @@ two 2 -- !query select * from values ("one", null), ("two", null) as data(a, b) -- !query schema -struct +struct -- !query output one NULL two NULL diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index f6720f6c5faa4..02747718c91df 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -5,7 +5,7 @@ -- !query select null, Null, nUll -- !query schema -struct +struct -- !query output NULL NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out index bd8ffb82ee129..8d34bf293ef2b 100644 --- a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out @@ -7,7 +7,7 @@ select typeof(null) -- !query schema struct -- !query output -null +unknown -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out index 1e59036b979b4..8b32bd6ce1995 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out @@ -308,7 +308,7 @@ struct<1:int> -- !query select foo.* from (select null) as foo -- !query schema -struct +struct -- !query output NULL @@ -316,7 +316,7 @@ NULL -- !query select foo.* from (select 'xyzzy',1,null) as foo -- !query schema -struct +struct -- !query output xyzzy 1 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out index 26a44a85841e0..b905f9e038619 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -5,7 +5,7 @@ -- !query SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null) -- !query schema -struct +struct -- !query output x y NULL @@ -21,7 +21,7 @@ NULL x -- !query SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null) -- !query schema -struct +struct -- !query output x y NULL @@ -29,7 +29,7 @@ x y NULL -- !query SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null) -- !query schema -struct +struct -- !query output y x NULL diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out index d78d347bc9802..0680a873fbf8f 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out @@ -49,7 +49,7 @@ two 2 -- !query select udf(a), b from values ("one", null), ("two", null) as data(a, b) -- !query schema -struct +struct -- !query output one NULL two NULL diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 231a8f2aa7ddd..daa262d581cb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -405,7 +405,7 @@ class FileBasedDataSourceSuite extends QueryTest "" } def errorMessage(format: String): String = { - s"$format data source does not support null data type." + s"$format data source does not support unknown data type." } withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> useV1List) { withTempDir { dir => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index b9c98f4ea15e9..2b1eb05e22cc7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, ScriptTransformation, Statistics} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.CatalogV2Util.assertNoNullTypeInSchema import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} import org.apache.spark.sql.execution.datasources.CreateTable @@ -225,6 +226,8 @@ case class RelationConversions( isConvertible(tableDesc) && SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_CTAS) => // validation is required to be done here before relation conversion. DDLUtils.checkDataColNames(tableDesc.copy(schema = query.schema)) + // This is for CREATE TABLE .. STORED AS PARQUET/ORC AS SELECT null + assertNoNullTypeInSchema(query.schema) OptimizedCreateHiveTableAsSelectCommand( tableDesc, query, query.output.map(_.name), mode) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index e8cf4ad5d9f28..774fb5b4b9ad5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.connector.FakeV2Provider import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} @@ -2309,6 +2310,126 @@ class HiveDDLSuite } } + test("SPARK-20680: Spark-sql do not support for unknown column datatype") { + withTable("t") { + withView("tabUnknownType") { + hiveClient.runSqlHive("CREATE TABLE t (t1 int)") + hiveClient.runSqlHive("INSERT INTO t VALUES (3)") + hiveClient.runSqlHive("CREATE VIEW tabUnknownType AS SELECT NULL AS col FROM t") + checkAnswer(spark.table("tabUnknownType"), Row(null)) + // No exception shows + val desc = spark.sql("DESC tabUnknownType").collect().toSeq + assert(desc.contains(Row("col", NullType.simpleString, null))) + } + } + + // Forbid CTAS with unknown type + withTable("t1", "t2", "t3") { + val e1 = intercept[AnalysisException] { + spark.sql("CREATE TABLE t1 USING PARQUET AS SELECT null as null_col") + }.getMessage + assert(e1.contains("Cannot create tables with unknown type")) + + val e2 = intercept[AnalysisException] { + spark.sql("CREATE TABLE t2 AS SELECT null as null_col") + }.getMessage + assert(e2.contains("Cannot create tables with unknown type")) + + val e3 = intercept[AnalysisException] { + spark.sql("CREATE TABLE t3 STORED AS PARQUET AS SELECT null as null_col") + }.getMessage + assert(e3.contains("Cannot create tables with unknown type")) + } + + // Forbid Replace table AS SELECT with unknown type + withTable("t") { + val v2Source = classOf[FakeV2Provider].getName + val e = intercept[AnalysisException] { + spark.sql(s"CREATE OR REPLACE TABLE t USING $v2Source AS SELECT null as null_col") + }.getMessage + assert(e.contains("Cannot create tables with unknown type")) + } + + // Forbid creating table with VOID type in Spark + withTable("t1", "t2", "t3", "t4") { + val e1 = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE t1 (v VOID) USING PARQUET") + }.getMessage + assert(e1.contains("Cannot create tables with unknown type")) + val e2 = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE t2 (v VOID) USING hive") + }.getMessage + assert(e2.contains("Cannot create tables with unknown type")) + val e3 = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE t3 (v VOID)") + }.getMessage + assert(e3.contains("Cannot create tables with unknown type")) + val e4 = intercept[AnalysisException] { + spark.sql(s"CREATE TABLE t4 (v VOID) STORED AS PARQUET") + }.getMessage + assert(e4.contains("Cannot create tables with unknown type")) + } + + // Forbid Replace table with VOID type + withTable("t") { + val v2Source = classOf[FakeV2Provider].getName + val e = intercept[AnalysisException] { + spark.sql(s"CREATE OR REPLACE TABLE t (v VOID) USING $v2Source") + }.getMessage + assert(e.contains("Cannot create tables with unknown type")) + } + + // Make sure spark.catalog.createTable with null type will fail + val schema1 = new StructType().add("c", NullType) + assertHiveTableNullType(schema1) + assertDSTableNullType(schema1) + + val schema2 = new StructType() + .add("c", StructType(Seq(StructField.apply("c1", NullType)))) + assertHiveTableNullType(schema2) + assertDSTableNullType(schema2) + + val schema3 = new StructType().add("c", ArrayType(NullType)) + assertHiveTableNullType(schema3) + assertDSTableNullType(schema3) + + val schema4 = new StructType() + .add("c", MapType(StringType, NullType)) + assertHiveTableNullType(schema4) + assertDSTableNullType(schema4) + + val schema5 = new StructType() + .add("c", MapType(NullType, StringType)) + assertHiveTableNullType(schema5) + assertDSTableNullType(schema5) + } + + private def assertHiveTableNullType(schema: StructType): Unit = { + withTable("t") { + val e = intercept[AnalysisException] { + spark.catalog.createTable( + tableName = "t", + source = "hive", + schema = schema, + options = Map("fileFormat" -> "parquet")) + }.getMessage + assert(e.contains("Cannot create tables with unknown type")) + } + } + + private def assertDSTableNullType(schema: StructType): Unit = { + withTable("t") { + val e = intercept[AnalysisException] { + spark.catalog.createTable( + tableName = "t", + source = "json", + schema = schema, + options = Map.empty[String, String]) + }.getMessage + assert(e.contains("Cannot create tables with unknown type")) + } + } + test("SPARK-21216: join with a streaming DataFrame") { import org.apache.spark.sql.execution.streaming.MemoryStream import testImplicits._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index 91fd8a47339fc..61c48c6f9c115 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -121,7 +121,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { msg = intercept[AnalysisException] { sql("select null").write.mode("overwrite").orc(orcDir) }.getMessage - assert(msg.contains("ORC data source does not support null data type.")) + assert(msg.contains("ORC data source does not support unknown data type.")) msg = intercept[AnalysisException] { spark.udf.register("testType", () => new IntervalData()) From 65286aec4b3c4e93d8beac6dd1b097ce97d53fd8 Mon Sep 17 00:00:00 2001 From: ulysses Date: Wed, 8 Jul 2020 11:30:47 +0900 Subject: [PATCH 136/384] [SPARK-30703][SQL][FOLLOWUP] Update SqlBase.g4 invalid comment ### What changes were proposed in this pull request? Modify the comment of `SqlBase.g4`. ### Why are the changes needed? `docs/sql-keywords.md` has already moved to `docs/sql-ref-ansi-compliance.md#sql-keywords`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No need. Closes #29033 from ulysses-you/SPARK-30703-FOLLOWUP. Authored-by: ulysses Signed-off-by: Takeshi Yamamuro --- .../antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 691fde8d48f94..b383e037e1ed8 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -1461,8 +1461,7 @@ nonReserved ; // NOTE: If you add a new token in the list below, you should update the list of keywords -// in `docs/sql-keywords.md`. If the token is a non-reserved keyword, -// please update `ansiNonReserved` and `nonReserved` as well. +// and reserved tag in `docs/sql-ref-ansi-compliance.md#sql-keywords`. //============================ // Start of the keywords list From 371b35d2e0ab08ebd853147c6673de3adfad0553 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 8 Jul 2020 17:46:25 +0900 Subject: [PATCH 137/384] [SPARK-32214][SQL] The type conversion function generated in makeFromJava for "other" type uses a wrong variable ### What changes were proposed in this pull request? This PR fixes an inconsistency in `EvaluatePython.makeFromJava`, which creates a type conversion function for some Java/Scala types. `other` is a type but it should actually pass `obj`: ```scala case other => (obj: Any) => nullSafeConvert(other)(PartialFunction.empty) ``` This does not change the output because it always returns `null` for unsupported datatypes. ### Why are the changes needed? To make the codes coherent, and consistent. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No behaviour change. Closes #29029 from sarutak/fix-makeFromJava. Authored-by: Kousuke Saruta Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/execution/python/EvaluatePython.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index 520afad287648..7fe3263630820 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -198,7 +198,7 @@ object EvaluatePython { case udt: UserDefinedType[_] => makeFromJava(udt.sqlType) - case other => (obj: Any) => nullSafeConvert(other)(PartialFunction.empty) + case other => (obj: Any) => nullSafeConvert(obj)(PartialFunction.empty) } private def nullSafeConvert(input: Any)(f: PartialFunction[Any, Any]): Any = { From 8e7fc04637bbb8d2fdc2c758746e0eaf496c4d92 Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Wed, 8 Jul 2020 21:58:45 +0900 Subject: [PATCH 138/384] [SPARK-32024][WEBUI] Update ApplicationStoreInfo.size during HistoryServerDiskManager initializing ### What changes were proposed in this pull request? Update ApplicationStoreInfo.size to real size during HistoryServerDiskManager initializing. ### Why are the changes needed? This PR is for fixing bug [32024](https://issues.apache.org/jira/browse/SPARK-32024). We found after history server restart, below error would randomly happen: "java.lang.IllegalStateException: Disk usage tracker went negative (now = -***, delta = -***)" from `HistoryServerDiskManager`. ![Capture](https://user-images.githubusercontent.com/10524738/85034468-fda4ae80-b136-11ea-9011-f0c3e6508002.JPG) **Cause**: Reading data from level db would trigger table file compaction, which may also trigger size of level db directory changes. This size change may not be recorded in LevelDB (`ApplicationStoreInfo` in `listing`). When service restarts, `currentUsage` is calculated from real directory size, but `ApplicationStoreInfo` are loaded from leveldb, then `currentUsage` may be less then sum of `ApplicationStoreInfo.size`. In `makeRoom()` function, `ApplicationStoreInfo.size` is used to update usage. Then `currentUsage` becomes negative after several round of `release()` and `lease()` (`makeRoom()`). **Reproduce**: we can reproduce this issue in dev environment by reducing config value of "spark.history.retainedApplications" and "spark.history.store.maxDiskUsage" to some small values. Here are steps: 1. start history server, load some applications and access some pages (maybe "stages" page to trigger leveldb compaction). 2. restart HS, and refresh pages. I also added an UT to simulate this case in `HistoryServerDiskManagerSuite`. **Benefit**: this change would help improve history server reliability. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add unit test and manually tested it. Closes #28859 from zhli1142015/update-ApplicationStoreInfo.size-during-disk-manager-initialize. Authored-by: Zhen Li Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../history/HistoryServerDiskManager.scala | 21 +++++++-- .../HistoryServerDiskManagerSuite.scala | 46 +++++++++++++++++++ 2 files changed, 64 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala index b1adc3c112ed3..a542d2b8cb27c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala @@ -75,14 +75,29 @@ private class HistoryServerDiskManager( // Go through the recorded store directories and remove any that may have been removed by // external code. - val orphans = listing.view(classOf[ApplicationStoreInfo]).asScala.filter { info => - !new File(info.path).exists() - }.toSeq + val (existences, orphans) = listing + .view(classOf[ApplicationStoreInfo]) + .asScala + .toSeq + .partition { info => + new File(info.path).exists() + } orphans.foreach { info => listing.delete(info.getClass(), info.path) } + // Reading level db would trigger table file compaction, then it may cause size of level db + // directory changed. When service restarts, "currentUsage" is calculated from real directory + // size. Update "ApplicationStoreInfo.size" to ensure "currentUsage" equals + // sum of "ApplicationStoreInfo.size". + existences.foreach { info => + val fileSize = sizeOf(new File(info.path)) + if (fileSize != info.size) { + listing.write(info.copy(size = fileSize)) + } + } + logInfo("Initialized disk manager: " + s"current usage = ${Utils.bytesToString(currentUsage.get())}, " + s"max usage = ${Utils.bytesToString(maxUsage)}") diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala index f78469e132490..b17880a733615 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala @@ -158,4 +158,50 @@ class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter { assert(manager.approximateSize(50L, true) > 50L) } + test("SPARK-32024: update ApplicationStoreInfo.size during initializing") { + val manager = mockManager() + val leaseA = manager.lease(2) + doReturn(3L).when(manager).sizeOf(meq(leaseA.tmpPath)) + val dstA = leaseA.commit("app1", None) + assert(manager.free() === 0) + assert(manager.committed() === 3) + // Listing store tracks dstA now. + assert(store.read(classOf[ApplicationStoreInfo], dstA.getAbsolutePath).size === 3) + + // Simulate: service restarts, new disk manager (manager1) is initialized. + val manager1 = mockManager() + // Simulate: event KVstore compaction before restart, directory size reduces. + doReturn(2L).when(manager1).sizeOf(meq(dstA)) + doReturn(2L).when(manager1).sizeOf(meq(new File(testDir, "apps"))) + manager1.initialize() + // "ApplicationStoreInfo.size" is updated for dstA. + assert(store.read(classOf[ApplicationStoreInfo], dstA.getAbsolutePath).size === 2) + assert(manager1.free() === 1) + // If "ApplicationStoreInfo.size" is not correctly updated, "IllegalStateException" + // would be thrown. + val leaseB = manager1.lease(2) + assert(manager1.free() === 1) + doReturn(2L).when(manager1).sizeOf(meq(leaseB.tmpPath)) + val dstB = leaseB.commit("app2", None) + assert(manager1.committed() === 2) + // Listing store tracks dstB only, dstA is evicted by "makeRoom()". + assert(store.read(classOf[ApplicationStoreInfo], dstB.getAbsolutePath).size === 2) + + val manager2 = mockManager() + // Simulate: cache entities are written after replaying, directory size increases. + doReturn(3L).when(manager2).sizeOf(meq(dstB)) + doReturn(3L).when(manager2).sizeOf(meq(new File(testDir, "apps"))) + manager2.initialize() + // "ApplicationStoreInfo.size" is updated for dstB. + assert(store.read(classOf[ApplicationStoreInfo], dstB.getAbsolutePath).size === 3) + assert(manager2.free() === 0) + val leaseC = manager2.lease(2) + doReturn(2L).when(manager2).sizeOf(meq(leaseC.tmpPath)) + val dstC = leaseC.commit("app3", None) + assert(manager2.free() === 1) + assert(manager2.committed() === 2) + // Listing store tracks dstC only, dstB is evicted by "makeRoom()". + assert(store.read(classOf[ApplicationStoreInfo], dstC.getAbsolutePath).size === 2) + } + } From f60b3b7e47695c74670d113c257342406c15808a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 8 Jul 2020 07:50:07 -0700 Subject: [PATCH 139/384] [MINOR][INFRA][R] Show the installed packages in R in a prettier way ### What changes were proposed in this pull request? This PR proposes to fix the AppVeyor configuration to show all installed R packages with name/versions. **Before:** ``` [1] '1.29' [1] '2.3' [1] '2.3.2' [1] '1.7.3' [1] '3.2.3' [1] '0.17.1' ``` **After:** ``` Package Version arrow arrow 0.17.1 askpass askpass 1.1 assertthat assertthat 0.2.1 backports backports 1.1.8 base64enc base64enc 0.1-3 bit bit 1.1-15.2 bit64 bit64 0.9-7 ... ``` ### Why are the changes needed? To show the package versions in a prettier way, and don't update the line every time when a package is added. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? AppVeyor CI should test it out. Closes #29038 from HyukjinKwon/minor-appveyor. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- appveyor.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/appveyor.yml b/appveyor.yml index a4da5f9040ded..1fd91daae9015 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -42,8 +42,8 @@ install: # Install maven and dependencies - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" - - cmd: R -e "packageVersion('knitr'); packageVersion('rmarkdown'); packageVersion('testthat'); packageVersion('e1071'); packageVersion('survival'); packageVersion('arrow')" + - cmd: Rscript -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" + - cmd: Rscript -e "pkg_list <- as.data.frame(installed.packages()[,c(1, 3:4)]); pkg_list[is.na(pkg_list$Priority), 1:2, drop = FALSE]" build_script: # '-Djna.nosys=true' is required to avoid kernel32.dll load failure. From 17997a57969ed935b543ab8b2af60e944f452f6f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 8 Jul 2020 14:11:18 -0700 Subject: [PATCH 140/384] [SPARK-32233][TESTS] Disable SBT unidoc generation testing in Jenkins ### What changes were proposed in this pull request? This PR aims to disable SBT `unidoc` generation testing in Jenkins environment because it's flaky in Jenkins environment and not used for the official documentation generation. Also, GitHub Action has the correct test coverage for the official documentation generation. - https://github.com/apache/spark/pull/28848#issuecomment-654577911 (amp-jenkins-worker-06) - https://github.com/apache/spark/pull/28926#issuecomment-654316537 (amp-jenkins-worker-06) - https://github.com/apache/spark/pull/28969#issuecomment-654918636 (amp-jenkins-worker-06) - https://github.com/apache/spark/pull/28975#issuecomment-654447955 (amp-jenkins-worker-05) - https://github.com/apache/spark/pull/28986#issuecomment-654416543 (amp-jenkins-worker-05) - https://github.com/apache/spark/pull/28992#issuecomment-654371469 (amp-jenkins-worker-06) - https://github.com/apache/spark/pull/28993#issuecomment-655289237 (amp-jenkins-worker-05) - https://github.com/apache/spark/pull/28999#issuecomment-653976760 (amp-jenkins-worker-04) - https://github.com/apache/spark/pull/29010#issuecomment-655246083 (amp-jenkins-worker-03) - https://github.com/apache/spark/pull/29013#issuecomment-654292483 (amp-jenkins-worker-04) - https://github.com/apache/spark/pull/29016#issuecomment-654495070 (amp-jenkins-worker-05) - https://github.com/apache/spark/pull/29025#issuecomment-654889938 (amp-jenkins-worker-04) - https://github.com/apache/spark/pull/29042#issuecomment-655587989 (amp-jenkins-worker-03) ### Why are the changes needed? Apache Spark `release-build.sh` generates the official document by using the following command. - https://github.com/apache/spark/blob/master/dev/create-release/release-build.sh#L341 ```bash PRODUCTION=1 RELEASE_VERSION="$SPARK_VERSION" jekyll build ``` And, this is executed by the following `unidoc` command for Scala/Java API doc. - https://github.com/apache/spark/blob/master/docs/_plugins/copy_api_dirs.rb#L30 ```ruby system("build/sbt -Pkinesis-asl clean compile unidoc") || raise("Unidoc generation failed") ``` However, the PR builder disabled `Jekyll build` and instead has a different test coverage. ```python # determine if docs were changed and if we're inside the amplab environment # note - the below commented out until *all* Jenkins workers can get `jekyll` installed # if "DOCS" in changed_modules and test_env == "amplab_jenkins": # build_spark_documentation() ``` ``` Building Unidoc API Documentation ======================================================================== [info] Building Spark unidoc using SBT with these arguments: -Phadoop-3.2 -Phive-2.3 -Pspark-ganglia-lgpl -Pkubernetes -Pmesos -Phadoop-cloud -Phive -Phive-thriftserver -Pkinesis-asl -Pyarn unidoc ``` ### Does this PR introduce _any_ user-facing change? No. (This is used only for testing and not used in the official doc generation.) ### How was this patch tested? Pass the Jenkins without doc generation invocation. Closes #29017 from dongjoon-hyun/SPARK-DOC-GEN. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/run-tests.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index ca502b2818847..223072cbe7bfb 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -363,7 +363,8 @@ def build_spark_assembly_sbt(extra_profiles, checkstyle=False): if checkstyle: run_java_style_checks(build_profiles) - build_spark_unidoc_sbt(extra_profiles) + if not os.environ.get("AMPLAB_JENKINS"): + build_spark_unidoc_sbt(extra_profiles) def build_apache_spark(build_tool, extra_profiles): From 3bb1ac597a6603e8224cb99349419d950ad7318e Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 8 Jul 2020 16:06:40 -0700 Subject: [PATCH 141/384] [SPARK-32168][SQL] Fix hidden partitioning correctness bug in SQL overwrite ### What changes were proposed in this pull request? When converting an `INSERT OVERWRITE` query to a v2 overwrite plan, Spark attempts to detect when a dynamic overwrite and a static overwrite will produce the same result so it can use the static overwrite. Spark incorrectly detects when dynamic and static overwrites are equivalent when there are hidden partitions, such as `days(ts)`. This updates the analyzer rule `ResolveInsertInto` to always use a dynamic overwrite when the mode is dynamic, and static when the mode is static. This avoids the problem by not trying to determine whether the two plans are equivalent and always using the one that corresponds to the partition overwrite mode. ### Why are the changes needed? This is a correctness bug. If a table has hidden partitions, all of the values for those partitions are dropped instead of dynamically overwriting changed partitions. This only affects SQL commands (not `DataFrameWriter`) writing to tables that have hidden partitions. It is also only a problem when the partition overwrite mode is dynamic. ### Does this PR introduce _any_ user-facing change? Yes, it fixes the correctness bug detailed above. ### How was this patch tested? * This updates the in-memory table to support a hidden partition transform, `days`, and adds a test case to `DataSourceV2SQLSuite` in which the table uses this hidden partition function. This test fails without the fix to `ResolveInsertInto`. * This updates the test case `InsertInto: overwrite - multiple static partitions - dynamic mode` in `InsertIntoTests`. The result of the SQL command is unchanged, but the SQL command will now use a dynamic overwrite so the test now uses `dynamicOverwriteTest`. Closes #28993 from rdblue/fix-insert-overwrite-v2-conversion. Authored-by: Ryan Blue Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../spark/sql/connector/InMemoryTable.scala | 66 ++++++++++++++++--- .../datasources/v2/BatchScanExec.scala | 2 +- .../spark/sql/DataFrameWriterV2Suite.scala | 7 -- .../sql/connector/DataSourceV2SQLSuite.scala | 38 ++++++++++- .../spark/sql/connector/InsertIntoTests.scala | 27 ++++---- 6 files changed, 107 insertions(+), 37 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d08a6382f738b..f92cf377bff12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1050,12 +1050,10 @@ class Analyzer( val staticPartitions = i.partitionSpec.filter(_._2.isDefined).mapValues(_.get) val query = addStaticPartitionColumns(r, i.query, staticPartitions) - val dynamicPartitionOverwrite = partCols.size > staticPartitions.size && - conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC if (!i.overwrite) { AppendData.byPosition(r, query) - } else if (dynamicPartitionOverwrite) { + } else if (conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC) { OverwritePartitionsDynamic.byPosition(r, query) } else { OverwriteByExpression.byPosition(r, query, staticDeleteExpression(r, staticPartitions)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 3d7026e180cd1..616fc72320caf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.connector +import java.time.{Instant, ZoneId} +import java.time.temporal.ChronoUnit import java.util import scala.collection.JavaConverters._ @@ -25,12 +27,13 @@ import scala.collection.mutable import org.scalatest.Assertions._ import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.connector.catalog._ -import org.apache.spark.sql.connector.expressions.{IdentityTransform, NamedReference, Transform} +import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, HoursTransform, IdentityTransform, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.sources.{And, EqualTo, Filter, IsNotNull} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{DataType, DateType, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap /** @@ -46,10 +49,15 @@ class InMemoryTable( private val allowUnsupportedTransforms = properties.getOrDefault("allow-unsupported-transforms", "false").toBoolean - partitioning.foreach { t => - if (!t.isInstanceOf[IdentityTransform] && !allowUnsupportedTransforms) { - throw new IllegalArgumentException(s"Transform $t must be IdentityTransform") - } + partitioning.foreach { + case _: IdentityTransform => + case _: YearsTransform => + case _: MonthsTransform => + case _: DaysTransform => + case _: HoursTransform => + case _: BucketTransform => + case t if !allowUnsupportedTransforms => + throw new IllegalArgumentException(s"Transform $t is not a supported transform") } // The key `Seq[Any]` is the partition values. @@ -66,8 +74,14 @@ class InMemoryTable( } } + private val UTC = ZoneId.of("UTC") + private val EPOCH_LOCAL_DATE = Instant.EPOCH.atZone(UTC).toLocalDate + private def getKey(row: InternalRow): Seq[Any] = { - def extractor(fieldNames: Array[String], schema: StructType, row: InternalRow): Any = { + def extractor( + fieldNames: Array[String], + schema: StructType, + row: InternalRow): (Any, DataType) = { val index = schema.fieldIndex(fieldNames(0)) val value = row.toSeq(schema).apply(index) if (fieldNames.length > 1) { @@ -78,10 +92,44 @@ class InMemoryTable( throw new IllegalArgumentException(s"Unsupported type, ${dataType.simpleString}") } } else { - value + (value, schema(index).dataType) } } - partCols.map(fieldNames => extractor(fieldNames, schema, row)) + + partitioning.map { + case IdentityTransform(ref) => + extractor(ref.fieldNames, schema, row)._1 + case YearsTransform(ref) => + extractor(ref.fieldNames, schema, row) match { + case (days: Int, DateType) => + ChronoUnit.YEARS.between(EPOCH_LOCAL_DATE, DateTimeUtils.daysToLocalDate(days)) + case (micros: Long, TimestampType) => + val localDate = DateTimeUtils.microsToInstant(micros).atZone(UTC).toLocalDate + ChronoUnit.YEARS.between(EPOCH_LOCAL_DATE, localDate) + } + case MonthsTransform(ref) => + extractor(ref.fieldNames, schema, row) match { + case (days: Int, DateType) => + ChronoUnit.MONTHS.between(EPOCH_LOCAL_DATE, DateTimeUtils.daysToLocalDate(days)) + case (micros: Long, TimestampType) => + val localDate = DateTimeUtils.microsToInstant(micros).atZone(UTC).toLocalDate + ChronoUnit.MONTHS.between(EPOCH_LOCAL_DATE, localDate) + } + case DaysTransform(ref) => + extractor(ref.fieldNames, schema, row) match { + case (days, DateType) => + days + case (micros: Long, TimestampType) => + ChronoUnit.DAYS.between(Instant.EPOCH, DateTimeUtils.microsToInstant(micros)) + } + case HoursTransform(ref) => + extractor(ref.fieldNames, schema, row) match { + case (micros: Long, TimestampType) => + ChronoUnit.HOURS.between(Instant.EPOCH, DateTimeUtils.microsToInstant(micros)) + } + case BucketTransform(numBuckets, ref) => + (extractor(ref.fieldNames, schema, row).hashCode() & Integer.MAX_VALUE) % numBuckets + } } def withData(data: Array[BufferedRows]): InMemoryTable = dataMap.synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala index e4e7887017a1d..c199df676ced3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala @@ -40,7 +40,7 @@ case class BatchScanExec( override def hashCode(): Int = batch.hashCode() - override lazy val partitions: Seq[InputPartition] = batch.planInputPartitions() + @transient override lazy val partitions: Seq[InputPartition] = batch.planInputPartitions() override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index ac2ebd8bd748b..508eefafd0754 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -336,7 +336,6 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source") .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) .writeTo("testcat.table_name") - .tableProperty("allow-unsupported-transforms", "true") .partitionedBy(years($"ts")) .create() @@ -350,7 +349,6 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source") .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) .writeTo("testcat.table_name") - .tableProperty("allow-unsupported-transforms", "true") .partitionedBy(months($"ts")) .create() @@ -364,7 +362,6 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source") .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) .writeTo("testcat.table_name") - .tableProperty("allow-unsupported-transforms", "true") .partitionedBy(days($"ts")) .create() @@ -378,7 +375,6 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source") .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) .writeTo("testcat.table_name") - .tableProperty("allow-unsupported-transforms", "true") .partitionedBy(hours($"ts")) .create() @@ -391,7 +387,6 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo test("Create: partitioned by bucket(4, id)") { spark.table("source") .writeTo("testcat.table_name") - .tableProperty("allow-unsupported-transforms", "true") .partitionedBy(bucket(4, $"id")) .create() @@ -596,7 +591,6 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo lit("2019-09-02 07:00:00.000000").cast("timestamp") as "modified", lit("America/Los_Angeles") as "timezone")) .writeTo("testcat.table_name") - .tableProperty("allow-unsupported-transforms", "true") .partitionedBy( years($"ts.created"), months($"ts.created"), days($"ts.created"), hours($"ts.created"), years($"ts.modified"), months($"ts.modified"), days($"ts.modified"), hours($"ts.modified") @@ -624,7 +618,6 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo lit("2019-09-02 07:00:00.000000").cast("timestamp") as "modified", lit("America/Los_Angeles") as "timezone")) .writeTo("testcat.table_name") - .tableProperty("allow-unsupported-transforms", "true") .partitionedBy(bucket(4, $"ts.timezone")) .create() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index f7f4df8f2d2e9..85aea3ce41ecc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.connector +import java.sql.Timestamp +import java.time.LocalDate + import scala.collection.JavaConverters._ import org.apache.spark.SparkException @@ -27,7 +30,7 @@ import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} -import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION +import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.SimpleScanSource import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType} @@ -1647,7 +1650,6 @@ class DataSourceV2SQLSuite """ |CREATE TABLE testcat.t (id int, `a.b` string) USING foo |CLUSTERED BY (`a.b`) INTO 4 BUCKETS - |OPTIONS ('allow-unsupported-transforms'=true) """.stripMargin) val testCatalog = catalog("testcat").asTableCatalog.asInstanceOf[InMemoryTableCatalog] @@ -2494,6 +2496,38 @@ class DataSourceV2SQLSuite } } + test("SPARK-32168: INSERT OVERWRITE - hidden days partition - dynamic mode") { + def testTimestamp(daysOffset: Int): Timestamp = { + Timestamp.valueOf(LocalDate.of(2020, 1, 1 + daysOffset).atStartOfDay()) + } + + withSQLConf(PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString) { + val t1 = s"${catalogAndNamespace}tbl" + withTable(t1) { + val df = spark.createDataFrame(Seq( + (testTimestamp(1), "a"), + (testTimestamp(2), "b"), + (testTimestamp(3), "c"))).toDF("ts", "data") + df.createOrReplaceTempView("source_view") + + sql(s"CREATE TABLE $t1 (ts timestamp, data string) " + + s"USING $v2Format PARTITIONED BY (days(ts))") + sql(s"INSERT INTO $t1 VALUES " + + s"(CAST(date_add('2020-01-01', 2) AS timestamp), 'dummy'), " + + s"(CAST(date_add('2020-01-01', 4) AS timestamp), 'keep')") + sql(s"INSERT OVERWRITE TABLE $t1 SELECT ts, data FROM source_view") + + val expected = spark.createDataFrame(Seq( + (testTimestamp(1), "a"), + (testTimestamp(2), "b"), + (testTimestamp(3), "c"), + (testTimestamp(4), "keep"))).toDF("ts", "data") + + verifyTable(t1, expected) + } + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index b88ad5218fcd2..2cc7a1f994645 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -446,21 +446,18 @@ trait InsertIntoSQLOnlyTests } } - test("InsertInto: overwrite - multiple static partitions - dynamic mode") { - // Since all partitions are provided statically, this should be supported by everyone - withSQLConf(PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString) { - val t1 = s"${catalogAndNamespace}tbl" - withTableAndData(t1) { view => - sql(s"CREATE TABLE $t1 (id bigint, data string, p int) " + - s"USING $v2Format PARTITIONED BY (id, p)") - sql(s"INSERT INTO $t1 VALUES (2L, 'dummy', 2), (4L, 'keep', 2)") - sql(s"INSERT OVERWRITE TABLE $t1 PARTITION (id = 2, p = 2) SELECT data FROM $view") - verifyTable(t1, Seq( - (2, "a", 2), - (2, "b", 2), - (2, "c", 2), - (4, "keep", 2)).toDF("id", "data", "p")) - } + dynamicOverwriteTest("InsertInto: overwrite - multiple static partitions - dynamic mode") { + val t1 = s"${catalogAndNamespace}tbl" + withTableAndData(t1) { view => + sql(s"CREATE TABLE $t1 (id bigint, data string, p int) " + + s"USING $v2Format PARTITIONED BY (id, p)") + sql(s"INSERT INTO $t1 VALUES (2L, 'dummy', 2), (4L, 'keep', 2)") + sql(s"INSERT OVERWRITE TABLE $t1 PARTITION (id = 2, p = 2) SELECT data FROM $view") + verifyTable(t1, Seq( + (2, "a", 2), + (2, "b", 2), + (2, "c", 2), + (4, "keep", 2)).toDF("id", "data", "p")) } } From d1d16d14bc04ee98a92359b61532e617886c7ca8 Mon Sep 17 00:00:00 2001 From: Warren Zhu Date: Wed, 8 Jul 2020 16:45:36 -0700 Subject: [PATCH 142/384] [SPARK-31723][CORE][TEST] Reenable one test case in HistoryServerSuite ### What changes were proposed in this pull request? Enable test("static relative links are prefixed with uiRoot (spark.ui.proxyBase)") ### Why are the changes needed? In Jira, the failed test is another one test("ajax rendered relative links are prefixed with uiRoot (spark.ui.proxyBase)"). This test has been fixed in 6a895d0 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Fix UT Closes #28970 from warrenzhu25/31723. Authored-by: Warren Zhu Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/deploy/history/HistoryServerSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 8737cd5bb3241..6ede98d55f094 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -313,8 +313,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers all (directSiteRelativeLinks) should not startWith (knoxBaseUrl) } - // TODO (SPARK-31723): re-enable it - ignore("static relative links are prefixed with uiRoot (spark.ui.proxyBase)") { + test("static relative links are prefixed with uiRoot (spark.ui.proxyBase)") { val uiRoot = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("/testwebproxybase") val page = new HistoryPage(server) val request = mock[HttpServletRequest] From 161cf2a12698bfebba94e0d406e0b110e4429b6b Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 9 Jul 2020 15:26:38 +0900 Subject: [PATCH 143/384] [SPARK-32024][WEBUI][FOLLOWUP] Quick fix on test failure on missing when statements ### What changes were proposed in this pull request? This patch fixes the test failure due to the missing when statements for destination path. Note that it didn't fail on master branch, because 245aee9 got rid of size call in destination path, but still good to not depend on 245aee9. ### Why are the changes needed? The build against branch-3.0 / branch-2.4 starts to fail after merging SPARK-32024 (#28859) and this patch will fix it. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Ran modified UT against master / branch-3.0 / branch-2.4. Closes #29046 from HeartSaVioR/QUICKFIX-SPARK-32024. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../spark/deploy/history/HistoryServerDiskManager.scala | 2 +- .../deploy/history/HistoryServerDiskManagerSuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala index a542d2b8cb27c..31f9d185174dc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala @@ -250,7 +250,7 @@ private class HistoryServerDiskManager( } } - private def appStorePath(appId: String, attemptId: Option[String]): File = { + private[history] def appStorePath(appId: String, attemptId: Option[String]): File = { val fileName = appId + attemptId.map("_" + _).getOrElse("") + ".ldb" new File(appStoreDir, fileName) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala index b17880a733615..9004e86323691 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala @@ -162,6 +162,8 @@ class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter { val manager = mockManager() val leaseA = manager.lease(2) doReturn(3L).when(manager).sizeOf(meq(leaseA.tmpPath)) + val dstPathA = manager.appStorePath("app1", None) + doReturn(3L).when(manager).sizeOf(meq(dstPathA)) val dstA = leaseA.commit("app1", None) assert(manager.free() === 0) assert(manager.committed() === 3) @@ -182,6 +184,8 @@ class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter { val leaseB = manager1.lease(2) assert(manager1.free() === 1) doReturn(2L).when(manager1).sizeOf(meq(leaseB.tmpPath)) + val dstPathB = manager.appStorePath("app2", None) + doReturn(2L).when(manager1).sizeOf(meq(dstPathB)) val dstB = leaseB.commit("app2", None) assert(manager1.committed() === 2) // Listing store tracks dstB only, dstA is evicted by "makeRoom()". @@ -197,6 +201,8 @@ class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter { assert(manager2.free() === 0) val leaseC = manager2.lease(2) doReturn(2L).when(manager2).sizeOf(meq(leaseC.tmpPath)) + val dstPathC = manager.appStorePath("app3", None) + doReturn(2L).when(manager2).sizeOf(meq(dstPathC)) val dstC = leaseC.commit("app3", None) assert(manager2.free() === 1) assert(manager2.committed() === 2) From cfecc2030d8b4774c1f4754fe81f57fbc61c9c75 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 9 Jul 2020 15:51:56 +0900 Subject: [PATCH 144/384] [SPARK-32160][CORE][PYSPARK] Disallow to create SparkContext in executors ### What changes were proposed in this pull request? This PR proposes to disallow to create `SparkContext` in executors, e.g., in UDFs. ### Why are the changes needed? Currently executors can create SparkContext, but shouldn't be able to create it. ```scala sc.range(0, 1).foreach { _ => new SparkContext(new SparkConf().setAppName("test").setMaster("local")) } ``` ### Does this PR introduce _any_ user-facing change? Yes, users won't be able to create `SparkContext` in executors. ### How was this patch tested? Addes tests. Closes #28986 from ueshin/issues/SPARK-32160/disallow_spark_context_in_executors. Authored-by: Takuya UESHIN Signed-off-by: HyukjinKwon --- .../scala/org/apache/spark/SparkContext.scala | 16 +++++++++ .../org/apache/spark/SparkContextSuite.scala | 12 +++++++ python/pyspark/context.py | 14 ++++++++ python/pyspark/tests/test_context.py | 8 +++++ .../org/apache/spark/sql/SparkSession.scala | 2 +- ...xternalAppendOnlyUnsafeRowArraySuite.scala | 35 +++++++++---------- 6 files changed, 67 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 38d7319b1f0ef..5078ef43adf68 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -83,6 +83,9 @@ class SparkContext(config: SparkConf) extends Logging { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() + // In order to prevent SparkContext from being created in executors. + SparkContext.assertOnDriver() + // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having started construction. // NOTE: this must be placed at the beginning of the SparkContext constructor. @@ -2554,6 +2557,19 @@ object SparkContext extends Logging { } } + /** + * Called to ensure that SparkContext is created or accessed only on the Driver. + * + * Throws an exception if a SparkContext is about to be created in executors. + */ + private def assertOnDriver(): Unit = { + if (TaskContext.get != null) { + // we're accessing it during task execution, fail. + throw new IllegalStateException( + "SparkContext should only be created and accessed on the driver.") + } + } + /** * This function may be used to get or instantiate a SparkContext and register it as a * singleton object. Because we can only have one active SparkContext per JVM, diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 30237fd576830..d111bb33ce8ff 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -934,6 +934,18 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } } + + test("SPARK-32160: Disallow to create SparkContext in executors") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) + + val error = intercept[SparkException] { + sc.range(0, 1).foreach { _ => + new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + } + }.getMessage() + + assert(error.contains("SparkContext should only be created and accessed on the driver.")) + } } object SparkContextSuite { diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 32d69edb171db..6d58e1d14484c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -38,6 +38,7 @@ from pyspark.storagelevel import StorageLevel from pyspark.resource.information import ResourceInformation from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix +from pyspark.taskcontext import TaskContext from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler @@ -118,6 +119,9 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ... ValueError:... """ + # In order to prevent SparkContext from being created in executors. + SparkContext._assert_on_driver() + self._callsite = first_spark_call() or CallSite(None, None, None) if gateway is not None and gateway.gateway_parameters.auth_token is None: raise ValueError( @@ -1145,6 +1149,16 @@ def resources(self): resources[name] = ResourceInformation(name, addrs) return resources + @staticmethod + def _assert_on_driver(): + """ + Called to ensure that SparkContext is created only on the Driver. + + Throws an exception if a SparkContext is about to be created in executors. + """ + if TaskContext.get() is not None: + raise Exception("SparkContext should only be created and accessed on the driver.") + def _test(): import atexit diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 5833bf9f96fb3..168299e385e78 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -267,6 +267,14 @@ def test_resources(self): resources = sc.resources self.assertEqual(len(resources), 0) + def test_disallow_to_create_spark_context_in_executors(self): + # SPARK-32160: SparkContext should not be created in executors. + with SparkContext("local-cluster[3, 1, 1024]") as sc: + with self.assertRaises(Exception) as context: + sc.range(2).foreach(lambda _: SparkContext()) + self.assertIn("SparkContext should only be created and accessed on the driver.", + str(context.exception)) + class ContextTestsWithResources(unittest.TestCase): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index ea1a9f12cd24b..9278eeeefe608 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -1087,7 +1087,7 @@ object SparkSession extends Logging { } private def assertOnDriver(): Unit = { - if (Utils.isTesting && TaskContext.get != null) { + if (TaskContext.get != null) { // we're accessing it during task execution, fail. throw new IllegalStateException( "SparkSession should only be created and accessed on the driver.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala index b29de9c4adbaa..98aba3ba25f17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala @@ -27,32 +27,29 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow class ExternalAppendOnlyUnsafeRowArraySuite extends SparkFunSuite with LocalSparkContext { private val random = new java.util.Random() - private var taskContext: TaskContext = _ - - override def afterAll(): Unit = try { - TaskContext.unset() - } finally { - super.afterAll() - } private def withExternalArray(inMemoryThreshold: Int, spillThreshold: Int) (f: ExternalAppendOnlyUnsafeRowArray => Unit): Unit = { sc = new SparkContext("local", "test", new SparkConf(false)) - taskContext = MemoryTestingUtils.fakeTaskContext(SparkEnv.get) + val taskContext = MemoryTestingUtils.fakeTaskContext(SparkEnv.get) TaskContext.setTaskContext(taskContext) - val array = new ExternalAppendOnlyUnsafeRowArray( - taskContext.taskMemoryManager(), - SparkEnv.get.blockManager, - SparkEnv.get.serializerManager, - taskContext, - 1024, - SparkEnv.get.memoryManager.pageSizeBytes, - inMemoryThreshold, - spillThreshold) - try f(array) finally { - array.clear() + try { + val array = new ExternalAppendOnlyUnsafeRowArray( + taskContext.taskMemoryManager(), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + taskContext, + 1024, + SparkEnv.get.memoryManager.pageSizeBytes, + inMemoryThreshold, + spillThreshold) + try f(array) finally { + array.clear() + } + } finally { + TaskContext.unset() } } From 8c5bee599d58fdb6d1c0335ec2de872f8256f0ba Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 9 Jul 2020 15:56:40 +0900 Subject: [PATCH 145/384] [SPARK-28067][SPARK-32018] Fix decimal overflow issues ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/27627 to fix the remaining issues. There are 2 issues fixed in this PR: 1. `UnsafeRow.setDecimal` can set an overflowed decimal and causes an error when reading it. The expected behavior is to return null. 2. The update/merge expression for decimal type in `Sum` is wrong. We shouldn't turn the `sum` value back to 0 after it becomes null due to overflow. This issue was hidden because: 2.1 for hash aggregate, the buffer is unsafe row. Due to the first bug, we fail when overflow happens, so there is no chance to mistakenly turn null back to 0. 2.2 for sort-based aggregate, the buffer is generic row. The decimal can overflow (the Decimal class has unlimited precision) and we don't have the null problem. If we only fix the first bug, then the second bug is exposed and test fails. If we only fix the second bug, there is no way to test it. This PR fixes these 2 bugs together. ### Why are the changes needed? Fix issues during decimal sum when overflow happens ### Does this PR introduce _any_ user-facing change? Yes. Now decimal sum can return null correctly for overflow under non-ansi mode. ### How was this patch tested? new test and updated test Closes #29026 from cloud-fan/decimal. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon --- .../sql/catalyst/expressions/UnsafeRow.java | 2 +- .../catalyst/expressions/aggregate/Sum.scala | 64 ++++++++++++------- .../org/apache/spark/sql/DataFrameSuite.scala | 14 +--- .../org/apache/spark/sql/UnsafeRowSuite.scala | 10 +++ 4 files changed, 54 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 034894bd86085..4dc5ce1de047b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -288,7 +288,7 @@ public void setDecimal(int ordinal, Decimal value, int precision) { Platform.putLong(baseObject, baseOffset + cursor, 0L); Platform.putLong(baseObject, baseOffset + cursor + 8, 0L); - if (value == null) { + if (value == null || !value.changePrecision(precision, value.scale())) { setNullAt(ordinal); // keep the offset for future update Platform.putLong(baseObject, getFieldOffset(ordinal), cursor << 32); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala index 6e850267100fb..a29ae2c8b65a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala @@ -58,13 +58,11 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast case _ => DoubleType } - private lazy val sumDataType = resultType - - private lazy val sum = AttributeReference("sum", sumDataType)() + private lazy val sum = AttributeReference("sum", resultType)() private lazy val isEmpty = AttributeReference("isEmpty", BooleanType, nullable = false)() - private lazy val zero = Literal.default(sumDataType) + private lazy val zero = Literal.default(resultType) override lazy val aggBufferAttributes = resultType match { case _: DecimalType => sum :: isEmpty :: Nil @@ -72,25 +70,38 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast } override lazy val initialValues: Seq[Expression] = resultType match { - case _: DecimalType => Seq(Literal(null, resultType), Literal(true, BooleanType)) + case _: DecimalType => Seq(zero, Literal(true, BooleanType)) case _ => Seq(Literal(null, resultType)) } override lazy val updateExpressions: Seq[Expression] = { - if (child.nullable) { - val updateSumExpr = coalesce(coalesce(sum, zero) + child.cast(sumDataType), sum) - resultType match { - case _: DecimalType => - Seq(updateSumExpr, isEmpty && child.isNull) - case _ => Seq(updateSumExpr) - } - } else { - val updateSumExpr = coalesce(sum, zero) + child.cast(sumDataType) - resultType match { - case _: DecimalType => - Seq(updateSumExpr, Literal(false, BooleanType)) - case _ => Seq(updateSumExpr) - } + resultType match { + case _: DecimalType => + // For decimal type, the initial value of `sum` is 0. We need to keep `sum` unchanged if + // the input is null, as SUM function ignores null input. The `sum` can only be null if + // overflow happens under non-ansi mode. + val sumExpr = if (child.nullable) { + If(child.isNull, sum, sum + KnownNotNull(child).cast(resultType)) + } else { + sum + child.cast(resultType) + } + // The buffer becomes non-empty after seeing the first not-null input. + val isEmptyExpr = if (child.nullable) { + isEmpty && child.isNull + } else { + Literal(false, BooleanType) + } + Seq(sumExpr, isEmptyExpr) + case _ => + // For non-decimal type, the initial value of `sum` is null, which indicates no value. + // We need `coalesce(sum, zero)` to start summing values. And we need an outer `coalesce` + // in case the input is nullable. The `sum` can only be null if there is no value, as + // non-decimal type can produce overflowed value under non-ansi mode. + if (child.nullable) { + Seq(coalesce(coalesce(sum, zero) + child.cast(resultType), sum)) + } else { + Seq(coalesce(sum, zero) + child.cast(resultType)) + } } } @@ -107,15 +118,20 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast * means we have seen atleast a value that was not null. */ override lazy val mergeExpressions: Seq[Expression] = { - val mergeSumExpr = coalesce(coalesce(sum.left, zero) + sum.right, sum.left) resultType match { case _: DecimalType => - val inputOverflow = !isEmpty.right && sum.right.isNull val bufferOverflow = !isEmpty.left && sum.left.isNull + val inputOverflow = !isEmpty.right && sum.right.isNull Seq( - If(inputOverflow || bufferOverflow, Literal.create(null, sumDataType), mergeSumExpr), + If( + bufferOverflow || inputOverflow, + Literal.create(null, resultType), + // If both the buffer and the input do not overflow, just add them, as they can't be + // null. See the comments inside `updateExpressions`: `sum` can only be null if + // overflow happens. + KnownNotNull(sum.left) + KnownNotNull(sum.right)), isEmpty.left && isEmpty.right) - case _ => Seq(mergeSumExpr) + case _ => Seq(coalesce(coalesce(sum.left, zero) + sum.right, sum.left)) } } @@ -128,7 +144,7 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast */ override lazy val evaluateExpression: Expression = resultType match { case d: DecimalType => - If(isEmpty, Literal.create(null, sumDataType), + If(isEmpty, Literal.create(null, resultType), CheckOverflowInSum(sum, d, !SQLConf.get.ansiEnabled)) case _ => sum } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 8359dff674a87..52ef5895ed9ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -195,22 +195,14 @@ class DataFrameSuite extends QueryTest private def assertDecimalSumOverflow( df: DataFrame, ansiEnabled: Boolean, expectedAnswer: Row): Unit = { if (!ansiEnabled) { - try { - checkAnswer(df, expectedAnswer) - } catch { - case e: SparkException if e.getCause.isInstanceOf[ArithmeticException] => - // This is an existing bug that we can write overflowed decimal to UnsafeRow but fail - // to read it. - assert(e.getCause.getMessage.contains("Decimal precision 39 exceeds max precision 38")) - } + checkAnswer(df, expectedAnswer) } else { val e = intercept[SparkException] { - df.collect + df.collect() } assert(e.getCause.isInstanceOf[ArithmeticException]) assert(e.getCause.getMessage.contains("cannot be represented as Decimal") || - e.getCause.getMessage.contains("Overflow in sum of decimals") || - e.getCause.getMessage.contains("Decimal precision 39 exceeds max precision 38")) + e.getCause.getMessage.contains("Overflow in sum of decimals")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala index a5f904c621e6e..9daa69ce9f155 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala @@ -178,4 +178,14 @@ class UnsafeRowSuite extends SparkFunSuite { // Makes sure hashCode on unsafe array won't crash unsafeRow.getArray(0).hashCode() } + + test("SPARK-32018: setDecimal with overflowed value") { + val d1 = new Decimal().set(BigDecimal("10000000000000000000")).toPrecision(38, 18) + val row = InternalRow.apply(d1) + val unsafeRow = UnsafeProjection.create(Array[DataType](DecimalType(38, 18))).apply(row) + assert(unsafeRow.getDecimal(0, 38, 18) === d1) + val d2 = (d1 * Decimal(10)).toPrecision(39, 18) + unsafeRow.setDecimal(0, d2, 38) + assert(unsafeRow.getDecimal(0, 38, 18) === null) + } } From 09cc6c51eaa489733551e0507d129b06d683207c Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Thu, 9 Jul 2020 16:14:33 +0900 Subject: [PATCH 146/384] [SPARK-32193][SQL][DOCS] Update regexp usage in SQL docs ### What changes were proposed in this pull request? update REGEXP usage and examples in sql-ref-syntx-qry-select-like.cmd ### Why are the changes needed? make the usage of REGEXP known to more users ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No tests Closes #29009 from GuoPhilipse/update-migrate-guide. Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Co-authored-by: GuoPhilipse Signed-off-by: Takeshi Yamamuro --- docs/sql-ref-syntax-qry-select-like.md | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-qry-select-like.md b/docs/sql-ref-syntax-qry-select-like.md index feb5eb7b3c80d..6211faa8d529e 100644 --- a/docs/sql-ref-syntax-qry-select-like.md +++ b/docs/sql-ref-syntax-qry-select-like.md @@ -26,7 +26,7 @@ A LIKE predicate is used to search for a specific pattern. ### Syntax ```sql -[ NOT ] { LIKE search_pattern [ ESCAPE esc_char ] | RLIKE regex_pattern } +[ NOT ] { LIKE search_pattern [ ESCAPE esc_char ] | [ RLIKE | REGEXP ] regex_pattern } ``` ### Parameters @@ -44,7 +44,7 @@ A LIKE predicate is used to search for a specific pattern. * **regex_pattern** - Specifies a regular expression search pattern to be searched by the `RLIKE` clause. + Specifies a regular expression search pattern to be searched by the `RLIKE` or `REGEXP` clause. ### Examples @@ -90,6 +90,14 @@ SELECT * FROM person WHERE name RLIKE 'M+'; |200|Mary|null| +---+----+----+ +SELECT * FROM person WHERE name REGEXP 'M+'; ++---+----+----+ +| id|name| age| ++---+----+----+ +|300|Mike| 80| +|200|Mary|null| ++---+----+----+ + SELECT * FROM person WHERE name LIKE '%\_%'; +---+------+---+ | id| name|age| From 526cb2d1ba2b4c07e10d7011367fdef24a40a927 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 9 Jul 2020 07:37:06 +0000 Subject: [PATCH 147/384] [SPARK-32148][SS] Fix stream-stream join issue on missing to copy reused unsafe row ### What changes were proposed in this pull request? This patch fixes the odd join result being occurred from stream-stream join for state store format V2. There're some spots on V2 path which leverage UnsafeProjection. As the result row is reused, the row should be copied to avoid changing value during reading (or make sure the caller doesn't affect by such behavior) but `SymmetricHashJoinStateManager.removeByValueCondition` violates the case. This patch makes `KeyWithIndexToValueRowConverterV2.convertValue` copy the row by itself so that callers don't need to take care about it. This patch doesn't change the behavior of `KeyWithIndexToValueRowConverterV2.convertToValueRow` to avoid double-copying, as the caller is expected to store the row which the implementation of state store will call `copy()`. This patch adds such behavior into each method doc in `KeyWithIndexToValueRowConverter`, so that further contributors can read through and make sure the change / new addition doesn't break the contract. ### Why are the changes needed? Stream-stream join with state store format V2 (newly added in Spark 3.0.0) has a serious correctness bug which brings indeterministic result. ### Does this PR introduce _any_ user-facing change? Yes, some of Spark 3.0.0 users using stream-stream join from the new checkpoint (as the bug exists to only v2 format path) may encounter wrong join result. This patch will fix it. ### How was this patch tested? Reported case is converted to the new UT, and confirmed UT passed. All UTs in StreamingInnerJoinSuite and StreamingOuterJoinSuite passed as well Closes #28975 from HeartSaVioR/SPARK-32148. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Wenchen Fan --- .../StreamingSymmetricHashJoinExec.scala | 4 ++ .../state/SymmetricHashJoinStateManager.scala | 17 ++++++- .../sql/streaming/StreamingJoinSuite.scala | 44 +++++++++++++++++++ 3 files changed, 64 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index dc5fc2e43143d..3d071df493cec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -295,6 +295,10 @@ case class StreamingSymmetricHashJoinExec( postJoinFilter(joinedRow.withLeft(leftKeyValue.value).withRight(rightValue)) } } + + // NOTE: we need to make sure `outerOutputIter` is evaluated "after" exhausting all of + // elements in `innerOutputIter`, because evaluation of `innerOutputIter` may update + // the match flag which the logic for outer join is relying on. val removedRowIter = leftSideJoiner.removeOldState() val outerOutputIter = removedRowIter.filterNot { kv => stateFormatVersion match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala index 1a0a43c083879..1a5b50dcc7901 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala @@ -451,10 +451,25 @@ class SymmetricHashJoinStateManager( } private trait KeyWithIndexToValueRowConverter { + /** Defines the schema of the value row (the value side of K-V in state store). */ def valueAttributes: Seq[Attribute] + /** + * Convert the value row to (actual value, match) pair. + * + * NOTE: implementations should ensure the result row is NOT reused during execution, so + * that caller can safely read the value in any time. + */ def convertValue(value: UnsafeRow): ValueAndMatchPair + /** + * Build the value row from (actual value, match) pair. This is expected to be called just + * before storing to the state store. + * + * NOTE: depending on the implementation, the result row "may" be reused during execution + * (to avoid initialization of object), so the caller should ensure that the logic doesn't + * affect by such behavior. Call copy() against the result row if needed. + */ def convertToValueRow(value: UnsafeRow, matched: Boolean): UnsafeRow } @@ -493,7 +508,7 @@ class SymmetricHashJoinStateManager( override def convertValue(value: UnsafeRow): ValueAndMatchPair = { if (value != null) { - ValueAndMatchPair(valueRowGenerator(value), + ValueAndMatchPair(valueRowGenerator(value).copy(), value.getBoolean(indexOrdinalInValueWithMatchedRow)) } else { null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index caca749f9dd1e..b182727408bbf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming import java.io.File +import java.sql.Timestamp import java.util.{Locale, UUID} import scala.util.Random @@ -996,4 +997,47 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with ) } } + + test("SPARK-32148 stream-stream join regression on Spark 3.0.0") { + val input1 = MemoryStream[(Timestamp, String, String)] + val df1 = input1.toDF + .selectExpr("_1 as eventTime", "_2 as id", "_3 as comment") + .withWatermark(s"eventTime", "2 minutes") + + val input2 = MemoryStream[(Timestamp, String, String)] + val df2 = input2.toDF + .selectExpr("_1 as eventTime", "_2 as id", "_3 as name") + .withWatermark(s"eventTime", "4 minutes") + + val joined = df1.as("left") + .join(df2.as("right"), + expr(""" + |left.id = right.id AND left.eventTime BETWEEN + | right.eventTime - INTERVAL 30 seconds AND + | right.eventTime + INTERVAL 30 seconds + """.stripMargin), + joinType = "leftOuter") + + val inputDataForInput1 = Seq( + (Timestamp.valueOf("2020-01-01 00:00:00"), "abc", "has no join partner"), + (Timestamp.valueOf("2020-01-02 00:00:00"), "abc", "joined with A"), + (Timestamp.valueOf("2020-01-02 01:00:00"), "abc", "joined with B")) + + val inputDataForInput2 = Seq( + (Timestamp.valueOf("2020-01-02 00:00:10"), "abc", "A"), + (Timestamp.valueOf("2020-01-02 00:59:59"), "abc", "B"), + (Timestamp.valueOf("2020-01-02 02:00:00"), "abc", "C")) + + val expectedOutput = Seq( + (Timestamp.valueOf("2020-01-01 00:00:00"), "abc", "has no join partner", null, null, null), + (Timestamp.valueOf("2020-01-02 00:00:00"), "abc", "joined with A", + Timestamp.valueOf("2020-01-02 00:00:10"), "abc", "A"), + (Timestamp.valueOf("2020-01-02 01:00:00"), "abc", "joined with B", + Timestamp.valueOf("2020-01-02 00:59:59"), "abc", "B")) + + testStream(joined)( + MultiAddData((input1, inputDataForInput1), (input2, inputDataForInput2)), + CheckNewAnswer(expectedOutput.head, expectedOutput.tail: _*) + ) + } } From c5bd0730a210276bac55d716fd6587c01e579c0e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 9 Jul 2020 17:18:39 +0900 Subject: [PATCH 148/384] [SPARK-32231][R][INFRA] Use Hadoop 3.2 winutils in AppVeyor build ### What changes were proposed in this pull request? This PR proposes to use Hadoop 3 winutils to make AppVeyor builds pass. Currently it's being failed as below https://ci.appveyor.com/project/ApacheSoftwareFoundation/spark/builds/33976604 ### Why are the changes needed? To recover the build in AppVeyor. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? AppVeyor build will test it out. Closes #29042 from HyukjinKwon/SPARK-32231. Lead-authored-by: Dongjoon Hyun Co-authored-by: Hyukjin Kwon Co-authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/appveyor-install-dependencies.ps1 | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index 87eb82935e4e0..e344a7fc23191 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -67,8 +67,8 @@ Function InstallRtools { Else { $gccPath = $env:GCC_PATH } - $env:PATH = $RtoolsDrive + '\Rtools40\bin;' + $RtoolsDrive + '\Rtools40\MinGW$(WIN)\bin;' + $RtoolsDrive + '\Rtools40\' + $gccPath + '\bin;' + $env:PATH - $env:BINPREF=$RtoolsDrive + '/Rtools40/mingw$(WIN)/bin/' + $env:PATH = $RtoolsDrive + '\Rtools40\bin;' + $RtoolsDrive + '\Rtools40\mingw64\bin;' + $RtoolsDrive + '\Rtools40\' + $gccPath + '\bin;' + $env:PATH + $env:BINPREF=$RtoolsDrive + '/Rtools40/mingw64/bin/' } # create tools directory outside of Spark directory @@ -95,22 +95,22 @@ $env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=1g" Pop-Location # ========================== Hadoop bin package -# This must match the version at https://github.com/steveloughran/winutils/tree/master/hadoop-2.7.1 -$hadoopVer = "2.7.1" +# This must match the version at https://github.com/cdarlint/winutils/tree/master/hadoop-3.2.0 +$hadoopVer = "3.2.0" $hadoopPath = "$tools\hadoop" if (!(Test-Path $hadoopPath)) { New-Item -ItemType Directory -Force -Path $hadoopPath | Out-Null } Push-Location $hadoopPath -Start-FileDownload "https://github.com/steveloughran/winutils/archive/master.zip" "winutils-master.zip" +Start-FileDownload "https://codeload.github.com/cdarlint/winutils/zip/master" "winutils-master.zip" # extract Invoke-Expression "7z.exe x winutils-master.zip" # add hadoop bin to environment variables -$env:HADOOP_HOME = "$hadoopPath/winutils-master/hadoop-$hadoopVer" -$env:Path += ";$env:HADOOP_HOME\bin" +$env:HADOOP_HOME = "$hadoopPath\winutils-master\hadoop-$hadoopVer" +$env:PATH = "$env:HADOOP_HOME\bin;" + $env:PATH Pop-Location From 1cb5bfc47a2b4fff824433f8cecabfbac7e050b6 Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Thu, 9 Jul 2020 08:42:20 +0000 Subject: [PATCH 149/384] [SPARK-32159][SQL] Fix integration between Aggregator[Array[_], _, _] and UnresolvedMapObjects Context: The fix for SPARK-27296 introduced by #25024 allows `Aggregator` objects to appear in queries. This works fine for aggregators with atomic input types, e.g. `Aggregator[Double, _, _]`. However it can cause a null pointer exception if the input type is `Array[_]`. This was historically considered an ignorable case for serialization of `UnresolvedMapObjects`, but the new ScalaAggregator class causes these expressions to be serialized over to executors because the resolve-and-bind is being deferred. ### What changes were proposed in this pull request? A new rule `ResolveEncodersInScalaAgg` that performs the resolution of the expressions contained in the encoders so that properly resolved expressions are serialized over to executors. ### Why are the changes needed? Applying an aggregator of the form `Aggregator[Array[_], _, _]` using `functions.udaf()` currently causes a null pointer error in Catalyst. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? A unit test has been added that does aggregation with array types for input, buffer, and output. I have done additional testing with my own custom aggregators in the spark REPL. Closes #28983 from erikerlandson/fix-spark-32159. Authored-by: Erik Erlandson Signed-off-by: Wenchen Fan --- .../expressions/objects/objects.scala | 7 +++ .../spark/sql/execution/aggregate/udaf.scala | 27 +++++++++-- .../sql/expressions/UserDefinedFunction.scala | 3 +- .../internal/BaseSessionStateBuilder.scala | 2 + .../sql/hive/HiveSessionStateBuilder.scala | 2 + .../sql/hive/execution/UDAQuerySuite.scala | 47 +++++++++++++------ 6 files changed, 68 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index d5de95c65e49e..ab2f66b1a53e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -678,6 +678,13 @@ object MapObjects { elementType: DataType, elementNullable: Boolean = true, customCollectionCls: Option[Class[_]] = None): MapObjects = { + // UnresolvedMapObjects does not serialize its 'function' field. + // If an array expression or array Encoder is not correctly resolved before + // serialization, this exception condition may occur. + require(function != null, + "MapObjects applied with a null function. " + + "Likely cause is failure to resolve an array expression or encoder. " + + "(See UnresolvedMapObjects)") val loopVar = LambdaVariable("MapObject", elementType, elementNullable) MapObjects(loopVar, function(loopVar), inputData, customCollectionCls) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index 544b90a736071..44bc9c2e3a9d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -27,6 +27,8 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.catalyst.expressions.aggregate.{ImperativeAggregate, TypedImperativeAggregate} import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateMutableProjection, GenerateSafeProjection} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction} import org.apache.spark.sql.types._ @@ -458,7 +460,8 @@ case class ScalaUDAF( case class ScalaAggregator[IN, BUF, OUT]( children: Seq[Expression], agg: Aggregator[IN, BUF, OUT], - inputEncoderNR: ExpressionEncoder[IN], + inputEncoder: ExpressionEncoder[IN], + bufferEncoder: ExpressionEncoder[BUF], nullable: Boolean = true, isDeterministic: Boolean = true, mutableAggBufferOffset: Int = 0, @@ -469,9 +472,8 @@ case class ScalaAggregator[IN, BUF, OUT]( with ImplicitCastInputTypes with Logging { - private[this] lazy val inputDeserializer = inputEncoderNR.resolveAndBind().createDeserializer() - private[this] lazy val bufferEncoder = - agg.bufferEncoder.asInstanceOf[ExpressionEncoder[BUF]].resolveAndBind() + // input and buffer encoders are resolved by ResolveEncodersInScalaAgg + private[this] lazy val inputDeserializer = inputEncoder.createDeserializer() private[this] lazy val bufferSerializer = bufferEncoder.createSerializer() private[this] lazy val bufferDeserializer = bufferEncoder.createDeserializer() private[this] lazy val outputEncoder = agg.outputEncoder.asInstanceOf[ExpressionEncoder[OUT]] @@ -479,7 +481,7 @@ case class ScalaAggregator[IN, BUF, OUT]( def dataType: DataType = outputEncoder.objSerializer.dataType - def inputTypes: Seq[DataType] = inputEncoderNR.schema.map(_.dataType) + def inputTypes: Seq[DataType] = inputEncoder.schema.map(_.dataType) override lazy val deterministic: Boolean = isDeterministic @@ -517,3 +519,18 @@ case class ScalaAggregator[IN, BUF, OUT]( override def nodeName: String = agg.getClass.getSimpleName } + +/** + * An extension rule to resolve encoder expressions from a [[ScalaAggregator]] + */ +object ResolveEncodersInScalaAgg extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case p if !p.resolved => p + case p => p.transformExpressionsUp { + case agg: ScalaAggregator[_, _, _] => + agg.copy( + inputEncoder = agg.inputEncoder.resolveAndBind(), + bufferEncoder = agg.bufferEncoder.resolveAndBind()) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 2ef6e3d291cef..6a20a46756f85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -150,7 +150,8 @@ private[sql] case class UserDefinedAggregator[IN, BUF, OUT]( // This is also used by udf.register(...) when it detects a UserDefinedAggregator def scalaAggregator(exprs: Seq[Expression]): ScalaAggregator[IN, BUF, OUT] = { val iEncoder = inputEncoder.asInstanceOf[ExpressionEncoder[IN]] - ScalaAggregator(exprs, aggregator, iEncoder, nullable, deterministic) + val bEncoder = aggregator.bufferEncoder.asInstanceOf[ExpressionEncoder[BUF]] + ScalaAggregator(exprs, aggregator, iEncoder, bEncoder, nullable, deterministic) } override def withName(name: String): UserDefinedAggregator[IN, BUF, OUT] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 3bbdbb002cca8..4ae12f8716752 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution.{ColumnarRule, QueryExecution, SparkOptimizer, SparkPlanner, SparkSqlParser} +import org.apache.spark.sql.execution.aggregate.ResolveEncodersInScalaAgg import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.command.CommandCheck import org.apache.spark.sql.execution.datasources._ @@ -175,6 +176,7 @@ abstract class BaseSessionStateBuilder( new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: + ResolveEncodersInScalaAgg +: new ResolveSessionCatalog( catalogManager, conf, catalog.isTempView, catalog.isTempFunction) +: customResolutionRules diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 64726755237a6..e25610757a69b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{SparkOptimizer, SparkPlanner} +import org.apache.spark.sql.execution.aggregate.ResolveEncodersInScalaAgg import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.command.CommandCheck import org.apache.spark.sql.execution.datasources._ @@ -76,6 +77,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: + ResolveEncodersInScalaAgg +: new ResolveSessionCatalog( catalogManager, conf, catalog.isTempView, catalog.isTempFunction) +: customResolutionRules diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala index e6856a58b0ea9..1f1a5568b0201 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala @@ -119,6 +119,27 @@ object CountSerDeAgg extends Aggregator[Int, CountSerDeSQL, CountSerDeSQL] { def outputEncoder: Encoder[CountSerDeSQL] = ExpressionEncoder[CountSerDeSQL]() } +object ArrayDataAgg extends Aggregator[Array[Double], Array[Double], Array[Double]] { + def zero: Array[Double] = Array(0.0, 0.0, 0.0) + def reduce(s: Array[Double], array: Array[Double]): Array[Double] = { + require(s.length == array.length) + for ( j <- 0 until s.length ) { + s(j) += array(j) + } + s + } + def merge(s1: Array[Double], s2: Array[Double]): Array[Double] = { + require(s1.length == s2.length) + for ( j <- 0 until s1.length ) { + s1(j) += s2(j) + } + s1 + } + def finish(s: Array[Double]): Array[Double] = s + def bufferEncoder: Encoder[Array[Double]] = ExpressionEncoder[Array[Double]] + def outputEncoder: Encoder[Array[Double]] = ExpressionEncoder[Array[Double]] +} + abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import testImplicits._ @@ -156,20 +177,11 @@ abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSi (3, null, null)).toDF("key", "value1", "value2") data2.write.saveAsTable("agg2") - val data3 = Seq[(Seq[Integer], Integer, Integer)]( - (Seq[Integer](1, 1), 10, -10), - (Seq[Integer](null), -60, 60), - (Seq[Integer](1, 1), 30, -30), - (Seq[Integer](1), 30, 30), - (Seq[Integer](2), 1, 1), - (null, -10, 10), - (Seq[Integer](2, 3), -1, null), - (Seq[Integer](2, 3), 1, 1), - (Seq[Integer](2, 3, 4), null, 1), - (Seq[Integer](null), 100, -10), - (Seq[Integer](3), null, 3), - (null, null, null), - (Seq[Integer](3), null, null)).toDF("key", "value1", "value2") + val data3 = Seq[(Seq[Double], Int)]( + (Seq(1.0, 2.0, 3.0), 0), + (Seq(4.0, 5.0, 6.0), 0), + (Seq(7.0, 8.0, 9.0), 0) + ).toDF("data", "dummy") data3.write.saveAsTable("agg3") val data4 = Seq[Boolean](true, false, true).toDF("boolvalues") @@ -184,6 +196,7 @@ abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSi spark.udf.register("mydoublesum", udaf(MyDoubleSumAgg)) spark.udf.register("mydoubleavg", udaf(MyDoubleAvgAgg)) spark.udf.register("longProductSum", udaf(LongProductSumAgg)) + spark.udf.register("arraysum", udaf(ArrayDataAgg)) } override def afterAll(): Unit = { @@ -354,6 +367,12 @@ abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSi Row(3, 0, null, 1, 3, 0, 0, 0, null, 1, 3, 0, 2, 2) :: Nil) } + test("SPARK-32159: array encoders should be resolved in analyzer") { + checkAnswer( + spark.sql("SELECT arraysum(data) FROM agg3"), + Row(Seq(12.0, 15.0, 18.0)) :: Nil) + } + test("verify aggregator ser/de behavior") { val data = sparkContext.parallelize((1 to 100).toSeq, 3).toDF("value1") val agg = udaf(CountSerDeAgg) From 523e238d2a45e05f9d10fb8fc5d1f035f9479d1e Mon Sep 17 00:00:00 2001 From: xiepengjie Date: Thu, 9 Jul 2020 09:33:54 -0500 Subject: [PATCH 150/384] [SPARK-32192][SQL] Print column name when throws ClassCastException ### What changes were proposed in this pull request? When somebody changed the type of partition's field, spark will throw ClassCastException. For example, we have a table like this: ``` drop table if exists cast_exception_test; create table cast_exception_test(c1 int, c2 string) partitioned by (dt string) stored as orc; insert into table cast_exception_test partition(dt='2020-04-08') values('1', 'jeff_1'); ``` If you change the field's type in hive, query the old partition, spark will throw ClassCastException, but hive will not: ``` -- change the field's type using hive alter table cast_exception_test change column c1 c1 string; -- hive correct, but spark throws ClassCastException select * from cast_exception_test where dt='2020-04-08'; ``` ### Why are the changes needed? When the table has many fields, we don's known which field has been changed. If we print out log about this exception, it will very helpful for us to troubleshoot. ### Does this PR introduce _any_ user-facing change? When the ClassCastException is caused by changed field's type, you can search which field has problem in exexutor logs: ``` 20/04/09 17:22:05 ERROR hive.HadoopTableReader: Exception thrown in field ``` ### How was this patch tested? First, prepare the test data, the table is partitioned and stored as orc: ``` drop table if exists cast_exception_test; create table cast_exception_test(c1 int, c2 string) partitioned by (dt string) stored as orc; insert into table cast_exception_test partition(dt='2020-04-08') values('1', 'jeff_1'); ``` Then, change the field's type in hive. ``` alter table cast_exception_test change column c1 c1 string; ``` Now the metadata of the table has been modified, but the partition's metadata which is stored in orc file or hive metastore's mysql is still old. So, query command throws ClassCastException in spark, because spark use table's metadata which is different from orc file's metadata. But hive use partition's metadata which is the same as orc file's metadata. If you query the old partition, spark will thrown ClassCastException, but hive will not: ``` select * from cast_exception_test where dt='2020-04-08'; ``` Closes #29010 from StefanXiepj/SPARK-32192. Authored-by: xiepengjie Signed-off-by: Sean Owen --- .../apache/spark/sql/hive/TableReader.scala | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 4d18eb6289418..8ddb979d7713a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -486,13 +486,19 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { var i = 0 val length = fieldRefs.length while (i < length) { - val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) - if (fieldValue == null) { - mutableRow.setNullAt(fieldOrdinals(i)) - } else { - unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i)) + try { + val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) + if (fieldValue == null) { + mutableRow.setNullAt(fieldOrdinals(i)) + } else { + unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i)) + } + i += 1 + } catch { + case ex: Throwable => + logError(s"Exception thrown in field <${fieldRefs(i).getFieldName}>") + throw ex } - i += 1 } mutableRow: InternalRow From 9331a5c44baa79998625829e9be624e8564c91ea Mon Sep 17 00:00:00 2001 From: moovlin Date: Thu, 9 Jul 2020 10:35:21 -0700 Subject: [PATCH 151/384] [SPARK-32035][DOCS][EXAMPLES] Fixed typos involving AWS Access, Secret, & Sessions tokens ### What changes were proposed in this pull request? I resolved some of the inconsistencies of AWS env variables. They're fixed in the documentation as well as in the examples. I grep-ed through the repo to try & find any more instances but nothing popped up. ### Why are the changes needed? As previously mentioned, there is a JIRA request, SPARK-32035, which encapsulates all the issues. But, in summary, the naming of items was inconsistent. ### Does this PR introduce _any_ user-facing change? Correct names: AWS_ACCESS_KEY_ID AWS_SECRET_ACCESS_KEY AWS_SESSION_TOKEN These are the same that AWS uses in their libraries. However, looking through the Spark documentation and comments, I see that these are not denoted correctly across the board: docs/cloud-integration.md 106:1. `spark-submit` reads the `AWS_ACCESS_KEY`, `AWS_SECRET_KEY` <-- both different 107:and `AWS_SESSION_TOKEN` environment variables and sets the associated authentication options docs/streaming-kinesis-integration.md 232:- Set up the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_KEY` with your AWS credentials. <-- secret key different external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py 34: $ export AWS_ACCESS_KEY_ID= 35: $ export AWS_SECRET_KEY= <-- different 48: Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY <-- secret key different core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 438: val keyId = System.getenv("AWS_ACCESS_KEY_ID") 439: val accessKey = System.getenv("AWS_SECRET_ACCESS_KEY") 448: val sessionToken = System.getenv("AWS_SESSION_TOKEN") external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala 53: * $ export AWS_ACCESS_KEY_ID= 54: * $ export AWS_SECRET_KEY= <-- different 65: * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY <-- secret key different external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java 59: * $ export AWS_ACCESS_KEY_ID=[your-access-key] 60: * $ export AWS_SECRET_KEY= <-- different 71: * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY <-- secret key different These were all fixed to match names listed under the "correct names" heading. ### How was this patch tested? I built the documentation using jekyll and verified that the changes were present & accurate. Closes #29058 from Moovlin/SPARK-32035. Authored-by: moovlin Signed-off-by: Dongjoon Hyun --- docs/cloud-integration.md | 2 +- docs/streaming-kinesis-integration.md | 2 +- .../spark/examples/streaming/JavaKinesisWordCountASL.java | 4 ++-- .../main/python/examples/streaming/kinesis_wordcount_asl.py | 4 ++-- .../apache/spark/examples/streaming/KinesisWordCountASL.scala | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 01f437f38ef17..749d026528017 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -103,7 +103,7 @@ for talking to cloud infrastructures, in which case this module may not be neede Spark jobs must authenticate with the object stores to access data within them. 1. When Spark is running in a cloud infrastructure, the credentials are usually automatically set up. -1. `spark-submit` reads the `AWS_ACCESS_KEY`, `AWS_SECRET_KEY` +1. `spark-submit` reads the `AWS_ACCESS_KEY_ID`, `AWS_SECRET_ACCESS_KEY` and `AWS_SESSION_TOKEN` environment variables and sets the associated authentication options for the `s3n` and `s3a` connectors to Amazon S3. 1. In a Hadoop cluster, settings may be set in the `core-site.xml` file. diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index db813c46949c2..c7959d4201151 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -229,7 +229,7 @@ To run the example, - Set up Kinesis stream (see earlier section) within AWS. Note the name of the Kinesis stream and the endpoint URL corresponding to the region where the stream was created. -- Set up the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_KEY` with your AWS credentials. +- Set up the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` with your AWS credentials. - In the Spark root directory, run the example as diff --git a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index 31ca2fe5c95ff..d704aeb507518 100644 --- a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -57,7 +57,7 @@ * Example: * # export AWS keys if necessary * $ export AWS_ACCESS_KEY_ID=[your-access-key] - * $ export AWS_SECRET_KEY= + * $ export AWS_SECRET_ACCESS_KEY= * * # run the example * $ SPARK_HOME/bin/run-example streaming.JavaKinesisWordCountASL myAppName mySparkStream \ @@ -68,7 +68,7 @@ * * This code uses the DefaultAWSCredentialsProviderChain to find credentials * in the following order: - * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY * Java System Properties - aws.accessKeyId and aws.secretKey * Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs * Instance profile credentials - delivered through the Amazon EC2 metadata service diff --git a/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py b/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py index 777a33270c415..5370b793897bb 100644 --- a/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py +++ b/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py @@ -32,7 +32,7 @@ Example: # export AWS keys if necessary $ export AWS_ACCESS_KEY_ID= - $ export AWS_SECRET_KEY= + $ export AWS_SECRET_ACCESS_KEY= # run the example $ bin/spark-submit --jars \ @@ -45,7 +45,7 @@ This code uses the DefaultAWSCredentialsProviderChain to find credentials in the following order: - Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY Java System Properties - aws.accessKeyId and aws.secretKey Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs Instance profile credentials - delivered through the Amazon EC2 metadata service diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index 32f4a6759474f..bbb6008c2dddf 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -51,7 +51,7 @@ import org.apache.spark.streaming.kinesis.KinesisInputDStream * Example: * # export AWS keys if necessary * $ export AWS_ACCESS_KEY_ID= - * $ export AWS_SECRET_KEY= + * $ export AWS_SECRET_ACCESS_KEY= * * # run the example * $ SPARK_HOME/bin/run-example streaming.KinesisWordCountASL myAppName mySparkStream \ @@ -62,7 +62,7 @@ import org.apache.spark.streaming.kinesis.KinesisInputDStream * * This code uses the DefaultAWSCredentialsProviderChain to find credentials * in the following order: - * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY * Java System Properties - aws.accessKeyId and aws.secretKey * Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs * Instance profile credentials - delivered through the Amazon EC2 metadata service From ac6406e7571027ba61193452d7a54f7895d2cbdc Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 9 Jul 2020 14:32:20 -0700 Subject: [PATCH 152/384] [SPARK-31831][SQL] HiveSessionImplSuite flakiness fix via mocking instances earlier than initializing HiveSessionImpl ### What changes were proposed in this pull request? This patch changes the HiveSessionImplSuite to mock instances "before" initializing HiveSessionImpl, to avoid possible classloader issue. ### Why are the changes needed? The failures of HiveSessionImplSuite always come from classloader issue. While I don't have clear idea what is happening, there's no part possibly dealing with classloader, except initializing HiveSessionImpl. We can move the mock initializations earlier than initialing HiveSessionImpl so that it can avoid possible classloader issue. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Verified with multiple triggers of Jenkins builds Closes #29039 from HeartSaVioR/hive-session-impl-suite-flakiness-fix. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../spark/sql/hive/thriftserver/HiveSessionImplSuite.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala index 05d540d782e31..afca4b49b3b4c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala @@ -34,6 +34,11 @@ class HiveSessionImplSuite extends SparkFunSuite { override def beforeAll() { super.beforeAll() + // mock the instance first - we observed weird classloader issue on creating mock, so + // would like to avoid any cases classloader gets switched + val sessionManager = mock(classOf[SessionManager]) + operationManager = mock(classOf[OperationManager]) + session = new HiveSessionImpl( ThriftserverShimUtils.testedProtocolVersions.head, "", @@ -41,9 +46,7 @@ class HiveSessionImplSuite extends SparkFunSuite { new HiveConf(), "" ) - val sessionManager = mock(classOf[SessionManager]) session.setSessionManager(sessionManager) - operationManager = mock(classOf[OperationManager]) session.setOperationManager(operationManager) when(operationManager.newGetCatalogsOperation(session)).thenAnswer( (_: InvocationOnMock) => { From 18aae21d96eed8c3b3d7b0dd0b305290ca0f211b Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 9 Jul 2020 18:27:07 -0700 Subject: [PATCH 153/384] [SPARK-31875][SQL] Provide a option to disable user supplied Hints ### What changes were proposed in this pull request? Introduce a new SQL config `spark.sql.optimizer.ignoreHints`. When this is set to true application of hints are disabled. This is similar to Oracle's OPTIMIZER_IGNORE_HINTS. This can be helpful to study the impact of performance difference when hints are applied vs when they are not. ### Why are the changes needed? Can be helpful to study the impact of performance difference when hints are applied vs when they are not. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? New tests added in ResolveHintsSuite. Closes #28683 from dilipbiswal/disable_hint. Authored-by: Dilip Biswal Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 2 + .../sql/catalyst/analysis/ResolveHints.scala | 11 ++++++ .../apache/spark/sql/internal/SQLConf.scala | 9 +++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 39 +++++++++++++++++++ 4 files changed, 61 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f92cf377bff12..761f3575f1b86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -200,6 +200,8 @@ class Analyzer( val postHocResolutionRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( + Batch("Disable Hints", Once, + new ResolveHints.DisableHints(conf)), Batch("Hints", fixedPoint, new ResolveHints.ResolveJoinStrategyHints(conf), new ResolveHints.ResolveCoalesceHints(conf)), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index 4cbff62e16cc1..120842b0c4a07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -278,4 +278,15 @@ object ResolveHints { h.child } } + + /** + * Removes all the hints when `spark.sql.optimizer.disableHints` is set. + * This is executed at the very beginning of the Analyzer to disable + * the hint functionality. + */ + class DisableHints(conf: SQLConf) extends RemoveAllHints(conf: SQLConf) { + override def apply(plan: LogicalPlan): LogicalPlan = { + if (conf.getConf(SQLConf.DISABLE_HINTS)) super.apply(plan) else plan + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 31dd943eeba2b..9be0497e46603 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2106,6 +2106,15 @@ object SQLConf { .booleanConf .createWithDefault(true) + val DISABLE_HINTS = + buildConf("spark.sql.optimizer.disableHints") + .internal() + .doc("When true, the optimizer will disable user-specified hints that are additional " + + "directives for better planning of a query.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + val NESTED_PREDICATE_PUSHDOWN_FILE_SOURCE_LIST = buildConf("spark.sql.optimizer.nestedPredicatePushdown.supportedFileSources") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index a219b91627b2b..989f304b1f07f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3521,6 +3521,45 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark |""".stripMargin), Row(1)) } } + + test("SPARK-31875: remove hints from plan when spark.sql.optimizer.disableHints = true") { + withSQLConf(SQLConf.DISABLE_HINTS.key -> "true") { + withTempView("t1", "t2") { + Seq[Integer](1, 2).toDF("c1").createOrReplaceTempView("t1") + Seq[Integer](1, 2).toDF("c1").createOrReplaceTempView("t2") + val repartitionHints = Seq( + "COALESCE(2)", + "REPARTITION(c1)", + "REPARTITION(c1, 2)", + "REPARTITION_BY_RANGE(c1, 2)", + "REPARTITION_BY_RANGE(c1)" + ) + val joinHints = Seq( + "BROADCASTJOIN (t1)", + "MAPJOIN(t1)", + "SHUFFLE_MERGE(t1)", + "MERGEJOIN(t1)", + "SHUFFLE_REPLICATE_NL(t1)" + ) + + repartitionHints.foreach { hintName => + val sqlText = s"SELECT /*+ $hintName */ * FROM t1" + val sqlTextWithoutHint = "SELECT * FROM t1" + val expectedPlan = sql(sqlTextWithoutHint) + val actualPlan = sql(sqlText) + comparePlans(actualPlan.queryExecution.analyzed, expectedPlan.queryExecution.analyzed) + } + + joinHints.foreach { hintName => + val sqlText = s"SELECT /*+ $hintName */ * FROM t1 INNER JOIN t2 ON t1.c1 = t2.c1" + val sqlTextWithoutHint = "SELECT * FROM t1 INNER JOIN t2 ON t1.c1 = t2.c1" + val expectedPlan = sql(sqlTextWithoutHint) + val actualPlan = sql(sqlText) + comparePlans(actualPlan.queryExecution.analyzed, expectedPlan.queryExecution.analyzed) + } + } + } + } } case class Foo(bar: Option[String]) From 01e9dd905028ef544881628fd823e3addaacbd6a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 9 Jul 2020 19:44:08 -0700 Subject: [PATCH 154/384] [SPARK-20680][SQL][FOLLOW-UP] Revert NullType.simpleString from 'unknown' to 'null' ### What changes were proposed in this pull request? This PR proposes to partially reverts the simple string in `NullType` at https://github.com/apache/spark/pull/28833: `NullType.simpleString` back from `unknown` to `null`. ### Why are the changes needed? - Technically speaking, it's orthogonal with the issue itself, SPARK-20680. - It needs some more discussion, see https://github.com/apache/spark/pull/28833#issuecomment-655277714 ### Does this PR introduce _any_ user-facing change? It reverts back the user-facing changes at https://github.com/apache/spark/pull/28833. The simple string of `NullType` is back to `null`. ### How was this patch tested? I just logically reverted. Jenkins should test it out. Closes #29041 from HyukjinKwon/SPARK-20680. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/types.py | 3 -- .../sql/connector/catalog/CatalogV2Util.scala | 2 +- .../org/apache/spark/sql/types/NullType.scala | 4 --- .../sql-functions/sql-expression-schema.md | 2 +- .../sql-tests/results/ansi/literals.sql.out | 2 +- .../sql-tests/results/inline-table.sql.out | 2 +- .../sql-tests/results/literals.sql.out | 2 +- .../sql-tests/results/misc-functions.sql.out | 2 +- .../results/postgreSQL/select.sql.out | 4 +-- .../sql-compatibility-functions.sql.out | 6 ++-- .../results/udf/udf-inline-table.sql.out | 2 +- .../spark/sql/FileBasedDataSourceSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 36 +++++++++---------- .../sql/hive/orc/HiveOrcSourceSuite.scala | 2 +- 14 files changed, 32 insertions(+), 39 deletions(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index ddd13ca3a01be..320a68dffe7a3 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -116,9 +116,6 @@ class NullType(DataType): __metaclass__ = DataTypeSingleton - def simpleString(self): - return 'unknown' - class AtomicType(DataType): """An internal type used to represent everything that is not diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index d130a13282cc8..1a3a7207c6ca9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -356,7 +356,7 @@ private[sql] object CatalogV2Util { } if (containsNullType(dt)) { throw new AnalysisException( - "Cannot create tables with unknown type.") + s"Cannot create tables with ${NullType.simpleString} type.") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala index 6c9a1d69ca681..14097a5280d50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala @@ -32,10 +32,6 @@ class NullType private() extends DataType { override def defaultSize: Int = 1 private[spark] override def asNullable: NullType = this - - // "null" is mainly used to represent a literal in Spark, - // it's better to avoid using it for data types. - override def simpleString: String = "unknown" } /** 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 c39adac4ac680..8898a11ec08fb 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 @@ -34,7 +34,7 @@ | org.apache.spark.sql.catalyst.expressions.Ascii | ascii | SELECT ascii('222') | struct | | org.apache.spark.sql.catalyst.expressions.Asin | asin | SELECT asin(0) | struct | | org.apache.spark.sql.catalyst.expressions.Asinh | asinh | SELECT asinh(0) | struct | -| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT assert_true(0 < 1) | struct | +| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT assert_true(0 < 1) | struct | | org.apache.spark.sql.catalyst.expressions.Atan | atan | SELECT atan(0) | struct | | org.apache.spark.sql.catalyst.expressions.Atan2 | atan2 | SELECT atan2(0, 0) | struct | | org.apache.spark.sql.catalyst.expressions.Atanh | atanh | SELECT atanh(0) | struct | diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 02747718c91df..f6720f6c5faa4 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -5,7 +5,7 @@ -- !query select null, Null, nUll -- !query schema -struct +struct -- !query output NULL NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out index 2dd6960682740..9943b93c431df 100644 --- a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -49,7 +49,7 @@ two 2 -- !query select * from values ("one", null), ("two", null) as data(a, b) -- !query schema -struct +struct -- !query output one NULL two NULL diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 02747718c91df..f6720f6c5faa4 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -5,7 +5,7 @@ -- !query select null, Null, nUll -- !query schema -struct +struct -- !query output NULL NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out index 8d34bf293ef2b..bd8ffb82ee129 100644 --- a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out @@ -7,7 +7,7 @@ select typeof(null) -- !query schema struct -- !query output -unknown +null -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out index 8b32bd6ce1995..1e59036b979b4 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out @@ -308,7 +308,7 @@ struct<1:int> -- !query select foo.* from (select null) as foo -- !query schema -struct +struct -- !query output NULL @@ -316,7 +316,7 @@ NULL -- !query select foo.* from (select 'xyzzy',1,null) as foo -- !query schema -struct +struct -- !query output xyzzy 1 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out index b905f9e038619..26a44a85841e0 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -5,7 +5,7 @@ -- !query SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null) -- !query schema -struct +struct -- !query output x y NULL @@ -21,7 +21,7 @@ NULL x -- !query SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null) -- !query schema -struct +struct -- !query output x y NULL @@ -29,7 +29,7 @@ x y NULL -- !query SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null) -- !query schema -struct +struct -- !query output y x NULL diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out index 0680a873fbf8f..d78d347bc9802 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out @@ -49,7 +49,7 @@ two 2 -- !query select udf(a), b from values ("one", null), ("two", null) as data(a, b) -- !query schema -struct +struct -- !query output one NULL two NULL diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index daa262d581cb0..231a8f2aa7ddd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -405,7 +405,7 @@ class FileBasedDataSourceSuite extends QueryTest "" } def errorMessage(format: String): String = { - s"$format data source does not support unknown data type." + s"$format data source does not support null data type." } withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> useV1List) { withTempDir { dir => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 774fb5b4b9ad5..6b1c30deec352 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2310,44 +2310,44 @@ class HiveDDLSuite } } - test("SPARK-20680: Spark-sql do not support for unknown column datatype") { + test("SPARK-20680: do not support for null column datatype") { withTable("t") { - withView("tabUnknownType") { + withView("tabNullType") { hiveClient.runSqlHive("CREATE TABLE t (t1 int)") hiveClient.runSqlHive("INSERT INTO t VALUES (3)") - hiveClient.runSqlHive("CREATE VIEW tabUnknownType AS SELECT NULL AS col FROM t") - checkAnswer(spark.table("tabUnknownType"), Row(null)) + hiveClient.runSqlHive("CREATE VIEW tabNullType AS SELECT NULL AS col FROM t") + checkAnswer(spark.table("tabNullType"), Row(null)) // No exception shows - val desc = spark.sql("DESC tabUnknownType").collect().toSeq + val desc = spark.sql("DESC tabNullType").collect().toSeq assert(desc.contains(Row("col", NullType.simpleString, null))) } } - // Forbid CTAS with unknown type + // Forbid CTAS with null type withTable("t1", "t2", "t3") { val e1 = intercept[AnalysisException] { spark.sql("CREATE TABLE t1 USING PARQUET AS SELECT null as null_col") }.getMessage - assert(e1.contains("Cannot create tables with unknown type")) + assert(e1.contains("Cannot create tables with null type")) val e2 = intercept[AnalysisException] { spark.sql("CREATE TABLE t2 AS SELECT null as null_col") }.getMessage - assert(e2.contains("Cannot create tables with unknown type")) + assert(e2.contains("Cannot create tables with null type")) val e3 = intercept[AnalysisException] { spark.sql("CREATE TABLE t3 STORED AS PARQUET AS SELECT null as null_col") }.getMessage - assert(e3.contains("Cannot create tables with unknown type")) + assert(e3.contains("Cannot create tables with null type")) } - // Forbid Replace table AS SELECT with unknown type + // Forbid Replace table AS SELECT with null type withTable("t") { val v2Source = classOf[FakeV2Provider].getName val e = intercept[AnalysisException] { spark.sql(s"CREATE OR REPLACE TABLE t USING $v2Source AS SELECT null as null_col") }.getMessage - assert(e.contains("Cannot create tables with unknown type")) + assert(e.contains("Cannot create tables with null type")) } // Forbid creating table with VOID type in Spark @@ -2355,19 +2355,19 @@ class HiveDDLSuite val e1 = intercept[AnalysisException] { spark.sql(s"CREATE TABLE t1 (v VOID) USING PARQUET") }.getMessage - assert(e1.contains("Cannot create tables with unknown type")) + assert(e1.contains("Cannot create tables with null type")) val e2 = intercept[AnalysisException] { spark.sql(s"CREATE TABLE t2 (v VOID) USING hive") }.getMessage - assert(e2.contains("Cannot create tables with unknown type")) + assert(e2.contains("Cannot create tables with null type")) val e3 = intercept[AnalysisException] { spark.sql(s"CREATE TABLE t3 (v VOID)") }.getMessage - assert(e3.contains("Cannot create tables with unknown type")) + assert(e3.contains("Cannot create tables with null type")) val e4 = intercept[AnalysisException] { spark.sql(s"CREATE TABLE t4 (v VOID) STORED AS PARQUET") }.getMessage - assert(e4.contains("Cannot create tables with unknown type")) + assert(e4.contains("Cannot create tables with null type")) } // Forbid Replace table with VOID type @@ -2376,7 +2376,7 @@ class HiveDDLSuite val e = intercept[AnalysisException] { spark.sql(s"CREATE OR REPLACE TABLE t (v VOID) USING $v2Source") }.getMessage - assert(e.contains("Cannot create tables with unknown type")) + assert(e.contains("Cannot create tables with null type")) } // Make sure spark.catalog.createTable with null type will fail @@ -2413,7 +2413,7 @@ class HiveDDLSuite schema = schema, options = Map("fileFormat" -> "parquet")) }.getMessage - assert(e.contains("Cannot create tables with unknown type")) + assert(e.contains("Cannot create tables with null type")) } } @@ -2426,7 +2426,7 @@ class HiveDDLSuite schema = schema, options = Map.empty[String, String]) }.getMessage - assert(e.contains("Cannot create tables with unknown type")) + assert(e.contains("Cannot create tables with null type")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index 61c48c6f9c115..91fd8a47339fc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -121,7 +121,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { msg = intercept[AnalysisException] { sql("select null").write.mode("overwrite").orc(orcDir) }.getMessage - assert(msg.contains("ORC data source does not support unknown data type.")) + assert(msg.contains("ORC data source does not support null data type.")) msg = intercept[AnalysisException] { spark.udf.register("testType", () => new IntervalData()) From 4609f1fdaba742fb9fa11050d9bf5a49898c1df5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 9 Jul 2020 19:45:16 -0700 Subject: [PATCH 155/384] [SPARK-32207][SQL] Support 'F'-suffixed Float Literals ### What changes were proposed in this pull request? In this PR, I suppose we support 'f'-suffixed float literal, e.g. `select 1.1f` ### Why are the changes needed? a very common feature across platforms, checked with pg, presto, hive, MySQL... ### Does this PR introduce _any_ user-facing change? yes, `select 1.1f` results float value 1.1 instead of throwing AnlaysisExceptiion`Can't extract value from 1: need struct type but got int;` ### How was this patch tested? add unit tests Closes #29022 from yaooqinn/SPARK-32207. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- docs/sql-ref-literals.md | 9 +++++ .../spark/sql/catalyst/parser/SqlBase.g4 | 6 +++ .../sql/catalyst/parser/AstBuilder.scala | 9 +++++ .../resources/sql-tests/inputs/literals.sql | 6 +++ .../sql-tests/results/ansi/literals.sql.out | 40 ++++++++++++++++++- .../sql-tests/results/literals.sql.out | 40 ++++++++++++++++++- 6 files changed, 108 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-literals.md b/docs/sql-ref-literals.md index b83f7f0a97c24..3dbed846d40b8 100644 --- a/docs/sql-ref-literals.md +++ b/docs/sql-ref-literals.md @@ -219,6 +219,11 @@ double literals: decimal_digits { D | exponent [ D ] } | digit [ ... ] { exponent [ D ] | [ exponent ] D } ``` +float literals: +```sql +decimal_digits { F | exponent [ F ] } | digit [ ... ] { exponent [ F ] | [ exponent ] F } +``` + While decimal_digits is defined as ```sql [ + | - ] { digit [ ... ] . [ digit [ ... ] ] | . digit [ ... ] } @@ -239,6 +244,10 @@ E [ + | - ] digit [ ... ] Case insensitive, indicates `DOUBLE`, which is an 8-byte double-precision floating point number. +* **F** + + Case insensitive, indicates `FLOAT`, which is a 4-byte single-precision floating point number. + * **BD** Case insensitive, indicates `DECIMAL`, with the total number of digits as precision and the number of digits to right of decimal point as scale. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index b383e037e1ed8..10e0d4e467271 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -988,6 +988,7 @@ number | MINUS? SMALLINT_LITERAL #smallIntLiteral | MINUS? TINYINT_LITERAL #tinyIntLiteral | MINUS? DOUBLE_LITERAL #doubleLiteral + | MINUS? FLOAT_LITERAL #floatLiteral | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; @@ -1775,6 +1776,11 @@ DECIMAL_VALUE : DECIMAL_DIGITS {isValidDecimal()}? ; +FLOAT_LITERAL + : DIGIT+ EXPONENT? 'F' + | DECIMAL_DIGITS EXPONENT? 'F' {isValidDecimal()}? + ; + DOUBLE_LITERAL : DIGIT+ EXPONENT? 'D' | DECIMAL_DIGITS EXPONENT? 'D' {isValidDecimal()}? 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 6b41a8b22fbee..f8261c293782d 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 @@ -2030,6 +2030,15 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Long.MinValue, Long.MaxValue, LongType.simpleString)(_.toLong) } + /** + * Create a Float Literal expression. + */ + override def visitFloatLiteral(ctx: FloatLiteralContext): Literal = { + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Float.MinValue, Float.MaxValue, FloatType.simpleString)(_.toFloat) + } + /** * Create a Double Literal expression. */ diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 108cfd766af2c..9f0eefc16a8cd 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -45,6 +45,11 @@ select 9223372036854775808, -9223372036854775809; select 1234567890123456789012345678901234567890; select 1234567890123456789012345678901234567890.0; +-- float +select 1F, 1.2F, .10f, 0.10f; +select -1F, -1.2F, -.10F, -0.10F; +select -3.4028235E39f; + -- double select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1; select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5; @@ -55,6 +60,7 @@ select 1E309, -1E309; -- decimal parsing select 0.3, -0.8, .5, -.18, 0.1111, .1111; +select 0.3 F, 0.4 D, 0.5 BD; -- super large scientific notation double literals should still be valid doubles select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index f6720f6c5faa4..ea74bb7175e96 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 50 +-- Number of queries: 54 -- !query @@ -164,6 +164,36 @@ decimal can only support precision up to 38 select 1234567890123456789012345678901234567890.0 +-- !query +select 1F, 1.2F, .10f, 0.10f +-- !query schema +struct<1.0:float,1.2:float,0.1:float,0.1:float> +-- !query output +1.0 1.2 0.1 0.1 + + +-- !query +select -1F, -1.2F, -.10F, -0.10F +-- !query schema +struct<-1.0:float,-1.2:float,-0.1:float,-0.1:float> +-- !query output +-1.0 -1.2 -0.1 -0.1 + + +-- !query +select -3.4028235E39f +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal -3.4028235E39 does not fit in range [-3.4028234663852886E+38, 3.4028234663852886E+38] for type float(line 1, pos 7) + +== SQL == +select -3.4028235E39f +-------^^^ + + -- !query select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query schema @@ -216,6 +246,14 @@ struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0. 0.3 -0.8 0.5 -0.18 0.1111 0.1111 +-- !query +select 0.3 F, 0.4 D, 0.5 BD +-- !query schema +struct +-- !query output +0.3 0.4 0.5 + + -- !query select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index f6720f6c5faa4..ea74bb7175e96 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 50 +-- Number of queries: 54 -- !query @@ -164,6 +164,36 @@ decimal can only support precision up to 38 select 1234567890123456789012345678901234567890.0 +-- !query +select 1F, 1.2F, .10f, 0.10f +-- !query schema +struct<1.0:float,1.2:float,0.1:float,0.1:float> +-- !query output +1.0 1.2 0.1 0.1 + + +-- !query +select -1F, -1.2F, -.10F, -0.10F +-- !query schema +struct<-1.0:float,-1.2:float,-0.1:float,-0.1:float> +-- !query output +-1.0 -1.2 -0.1 -0.1 + + +-- !query +select -3.4028235E39f +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal -3.4028235E39 does not fit in range [-3.4028234663852886E+38, 3.4028234663852886E+38] for type float(line 1, pos 7) + +== SQL == +select -3.4028235E39f +-------^^^ + + -- !query select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query schema @@ -216,6 +246,14 @@ struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0. 0.3 -0.8 0.5 -0.18 0.1111 0.1111 +-- !query +select 0.3 F, 0.4 D, 0.5 BD +-- !query schema +struct +-- !query output +0.3 0.4 0.5 + + -- !query select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d -- !query schema From e6e43cb2f989227e88d40ca75319c38b2f4c18fe Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 10 Jul 2020 13:12:25 +0900 Subject: [PATCH 156/384] [SPARK-32242][SQL] CliSuite flakiness fix via differentiating cli driver bootup timeout and query execution timeout ### What changes were proposed in this pull request? This patch tries to mitigate the flakiness of CliSuite, via below changes: 1. differentiate cli driver boot-up timeout (2 mins) and query execution timeout (parameter) Cli driver boot-up is determined by master and app ID message. Given spark-sql doesn't print the message if `-e` option is specified, the patch simply add 2 mins on timeout for the case to cover the boot-up timeout. 2. don't fail the test even spark-sql doesn't gracefully shut down in 1 min. 3. extend timeout for `path command` test in CliSuite ### Why are the changes needed? It took around 40 seconds for boot-up message (master: ... Application Id: ...) to be printed in stderr, while the overall timeout is 1 minute in many tests. This case the actual timeout for query execution is just 20 seconds, which may not be enough. Some of the tests also failed with `org.scalatest.exceptions.TestFailedException: spark-sql did not exit gracefully`, which I don't feel the test has to be failed. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Verified with multiple triggers of Jenkins builds Closes #29036 from HeartSaVioR/clisuite-flakiness-fix. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../sql/hive/thriftserver/CliSuite.scala | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 8546421a86927..2064a99137bf9 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -132,6 +132,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { } var next = 0 + val foundMasterAndApplicationIdMessage = Promise.apply[Unit]() val foundAllExpectedAnswers = Promise.apply[Unit]() val buffer = new ArrayBuffer[String]() val lock = new Object @@ -143,6 +144,10 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { log.info(newLine) buffer += newLine + if (line.startsWith("Spark master: ") && line.contains("Application Id: ")) { + foundMasterAndApplicationIdMessage.trySuccess(()) + } + // If we haven't found all expected answers and another expected answer comes up... if (next < expectedAnswers.size && line.contains(expectedAnswers(next))) { log.info(s"$source> found expected output line $next: '${expectedAnswers(next)}'") @@ -172,7 +177,18 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { new ProcessOutputCapturer(process.getErrorStream, captureOutput("stderr")).start() try { - ThreadUtils.awaitResult(foundAllExpectedAnswers.future, timeout) + val timeoutForQuery = if (!extraArgs.contains("-e")) { + // Wait for for cli driver to boot, up to two minutes + ThreadUtils.awaitResult(foundMasterAndApplicationIdMessage.future, 2.minutes) + log.info("Cli driver is booted. Waiting for expected answers.") + // Given timeout is applied after the cli driver is ready + timeout + } else { + // There's no boot message if -e option is provided, just extend timeout long enough + // so that the bootup duration is counted on the timeout + 2.minutes + timeout + } + ThreadUtils.awaitResult(foundAllExpectedAnswers.future, timeoutForQuery) log.info("Found all expected output.") } catch { case cause: Throwable => val message = @@ -194,7 +210,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { } finally { if (!process.waitFor(1, MINUTES)) { try { - fail("spark-sql did not exit gracefully.") + log.warn("spark-sql did not exit gracefully.") } finally { process.destroy() } @@ -447,7 +463,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { val jarFile = new File("../../sql/hive/src/test/resources/SPARK-21101-1.0.jar").getCanonicalPath val hiveContribJar = HiveTestJars.getHiveContribJar().getCanonicalPath runCliWithin( - 1.minute, + 2.minutes, Seq("--jars", s"$jarFile", "--conf", s"spark.hadoop.${ConfVars.HIVEAUXJARS}=$hiveContribJar"))( "CREATE TEMPORARY FUNCTION testjar AS" + From c8779d9dfc5bf7dbaa19a043d8312c3909499da6 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 10 Jul 2020 21:14:29 +0900 Subject: [PATCH 157/384] [SPARK-32256][SQL][TEST-HADOOP2.7] Force to initialize Hadoop VersionInfo in HiveExternalCatalog ### What changes were proposed in this pull request? Force to initialize Hadoop VersionInfo in HiveExternalCatalog to make sure Hive can get the Hadoop version when using the isolated classloader. ### Why are the changes needed? This is a regression in Spark 3.0.0 because we switched the default Hive execution version from 1.2.1 to 2.3.7. Spark allows the user to set `spark.sql.hive.metastore.jars` to specify jars to access Hive Metastore. These jars are loaded by the isolated classloader. Because we also share Hadoop classes with the isolated classloader, the user doesn't need to add Hadoop jars to `spark.sql.hive.metastore.jars`, which means when we are using the isolated classloader, hadoop-common jar is not available in this case. If Hadoop VersionInfo is not initialized before we switch to the isolated classloader, and we try to initialize it using the isolated classloader (the current thread context classloader), it will fail and report `Unknown` which causes Hive to throw the following exception: ``` java.lang.RuntimeException: Illegal Hadoop Version: Unknown (expected A.B.* format) at org.apache.hadoop.hive.shims.ShimLoader.getMajorVersion(ShimLoader.java:147) at org.apache.hadoop.hive.shims.ShimLoader.loadShims(ShimLoader.java:122) at org.apache.hadoop.hive.shims.ShimLoader.getHadoopShims(ShimLoader.java:88) at org.apache.hadoop.hive.metastore.ObjectStore.getDataSourceProps(ObjectStore.java:377) at org.apache.hadoop.hive.metastore.ObjectStore.setConf(ObjectStore.java:268) at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:76) at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:136) at org.apache.hadoop.hive.metastore.RawStoreProxy.(RawStoreProxy.java:58) at org.apache.hadoop.hive.metastore.RawStoreProxy.getProxy(RawStoreProxy.java:67) at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.newRawStore(HiveMetaStore.java:517) at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.getMS(HiveMetaStore.java:482) at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:544) at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:370) at org.apache.hadoop.hive.metastore.RetryingHMSHandler.(RetryingHMSHandler.java:78) at org.apache.hadoop.hive.metastore.RetryingHMSHandler.getProxy(RetryingHMSHandler.java:84) at org.apache.hadoop.hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:5762) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.(HiveMetaStoreClient.java:219) at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.(SessionHiveMetaStoreClient.java:67) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1548) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.(RetryingMetaStoreClient.java:86) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:132) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:104) at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3080) at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3108) at org.apache.hadoop.hive.ql.metadata.Hive.getAllFunctions(Hive.java:3349) at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:217) at org.apache.hadoop.hive.ql.metadata.Hive.registerAllFunctionsOnce(Hive.java:204) at org.apache.hadoop.hive.ql.metadata.Hive.(Hive.java:331) at org.apache.hadoop.hive.ql.metadata.Hive.get(Hive.java:292) at org.apache.hadoop.hive.ql.metadata.Hive.getInternal(Hive.java:262) at org.apache.hadoop.hive.ql.metadata.Hive.get(Hive.java:247) at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:543) at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:511) at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:175) at org.apache.spark.sql.hive.client.HiveClientImpl.(HiveClientImpl.scala:128) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:301) at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:431) at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:324) at org.apache.spark.sql.hive.HiveExternalCatalog.client$lzycompute(HiveExternalCatalog.scala:72) at org.apache.spark.sql.hive.HiveExternalCatalog.client(HiveExternalCatalog.scala:71) at org.apache.spark.sql.hive.client.HadoopVersionInfoSuite.$anonfun$new$1(HadoopVersionInfoSuite.scala:63) at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83) ``` Technically, This is indeed an issue of Hadoop VersionInfo which has been fixed: https://issues.apache.org/jira/browse/HADOOP-14067. But since we are still supporting old Hadoop versions, we should fix it. Why this issue starts to happen in Spark 3.0.0? In Spark 2.4.x, we use Hive 1.2.1 by default. It will trigger `VersionInfo` initialization in the static codes of `Hive` class. This will happen when we load `HiveClientImpl` class because `HiveClientImpl.clent` method refers to `Hive` class. At this moment, the thread context classloader is not using the isolcated classloader, so it can access hadoop-common jar on the classpath and initialize it correctly. In Spark 3.0.0, we use Hive 2.3.7. The static codes of `Hive` class are not accessing `VersionInfo` because of the change in https://issues.apache.org/jira/browse/HIVE-11657. Instead, accessing `VersionInfo` happens when creating a `Hive` object (See the above stack trace). This happens here https://github.com/apache/spark/blob/v3.0.0/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L260. But we switch to the isolated classloader before calling `HiveClientImpl.client` (See https://github.com/apache/spark/blob/v3.0.0/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L283). This is exactly what I mentioned above: `If Hadoop VersionInfo is not initialized before we switch to the isolated classloader, and we try to initialize it using the isolated classloader (the current thread context classloader), it will fail` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The new regression test added in this PR. Note that the new UT doesn't fail with the default profiles (-Phadoop-3.2) because it's already fixed at Hadoop 3.1. Please use the following to verify this. ``` build/sbt -Phadoop-2.7 -Phive "hive/testOnly *.HadoopVersionInfoSuite" ``` Closes #29059 from zsxwing/SPARK-32256. Authored-by: Shixiong Zhu Signed-off-by: HyukjinKwon --- project/SparkBuild.scala | 1 + .../spark/sql/hive/HiveExternalCatalog.scala | 4 ++ .../hive/client/HadoopVersionInfoSuite.scala | 72 +++++++++++++++++++ .../sql/hive/client/HiveClientBuilder.scala | 2 +- 4 files changed, 78 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 60c54dfc98a58..622fea8c54f13 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -476,6 +476,7 @@ object SparkParallelTestGrouping { "org.apache.spark.ml.classification.LogisticRegressionSuite", "org.apache.spark.ml.classification.LinearSVCSuite", "org.apache.spark.sql.SQLQueryTestSuite", + "org.apache.spark.sql.hive.client.HadoopVersionInfoSuite", "org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperationSuite", "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite", "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 2faf42028f3a2..571c25e356c08 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -61,6 +61,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat import HiveExternalCatalog._ import CatalogTableType._ + // SPARK-32256: Make sure `VersionInfo` is initialized before touching the isolated classloader. + // This is to ensure Hive can get the Hadoop version when using the isolated classloader. + org.apache.hadoop.util.VersionInfo.getVersion() + /** * A Hive client used to interact with the metastore. */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala new file mode 100644 index 0000000000000..65492abf38cc0 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.client + +import java.io.File +import java.net.URLClassLoader + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} +import org.apache.spark.util.Utils + +/** + * This test suite requires a clean JVM because it's testing the initialization of static codes in + * `org.apache.hadoop.util.VersionInfo`. + */ +class HadoopVersionInfoSuite extends SparkFunSuite { + override protected val enableAutoThreadAudit = false + + test("SPARK-32256: Hadoop VersionInfo should be preloaded") { + val ivyPath = + Utils.createTempDir(namePrefix = s"${classOf[HadoopVersionInfoSuite].getSimpleName}-ivy") + try { + val hadoopConf = new Configuration() + hadoopConf.set("test", "success") + hadoopConf.set("datanucleus.schema.autoCreateAll", "true") + hadoopConf.set("hive.metastore.schema.verification", "false") + + // Download jars for Hive 2.0 + val client = IsolatedClientLoader.forVersion( + hiveMetastoreVersion = "2.0", + hadoopVersion = "2.7.4", + sparkConf = new SparkConf(), + hadoopConf = hadoopConf, + config = HiveClientBuilder.buildConf(Map.empty), + ivyPath = Some(ivyPath.getCanonicalPath), + sharesHadoopClasses = true) + val jars = client.classLoader.getParent.asInstanceOf[URLClassLoader].getURLs + .map(u => new File(u.toURI)) + // Drop all Hadoop jars to use the existing Hadoop jars on the classpath + .filter(!_.getName.startsWith("org.apache.hadoop_hadoop-")) + + val sparkConf = new SparkConf() + sparkConf.set(HiveUtils.HIVE_METASTORE_VERSION, "2.0") + sparkConf.set( + HiveUtils.HIVE_METASTORE_JARS, + jars.map(_.getCanonicalPath).mkString(File.pathSeparator)) + HiveClientBuilder.buildConf(Map.empty).foreach { case (k, v) => + hadoopConf.set(k, v) + } + new HiveExternalCatalog(sparkConf, hadoopConf).client.getState + } finally { + Utils.deleteRecursively(ivyPath) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala index ab73f668c6ca6..2ad3afcb214b3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala @@ -33,7 +33,7 @@ private[client] object HiveClientBuilder { Some(new File(sys.props("java.io.tmpdir"), "hive-ivy-cache").getAbsolutePath)) } - private def buildConf(extraConf: Map[String, String]) = { + private[client] def buildConf(extraConf: Map[String, String]): Map[String, String] = { lazy val warehousePath = Utils.createTempDir() lazy val metastorePath = Utils.createTempDir() metastorePath.delete() From 578b90cdec37503c0d6db48e4c2ed4f3654faafc Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 10 Jul 2020 13:36:29 +0000 Subject: [PATCH 158/384] [SPARK-32091][CORE] Ignore timeout error when remove blocks on the lost executor ### What changes were proposed in this pull request? This PR adds the check to see whether the executor is lost (by asking the `CoarseGrainedSchedulerBackend`) after timeout error raised in `BlockManagerMasterEndponit` due to removing blocks(e.g. RDD, broadcast, shuffle). If the executor is lost, we will ignore the error. Otherwise, throw the error. ### Why are the changes needed? When removing blocks(e.g. RDD, broadcast, shuffle), `BlockManagerMaserEndpoint` will make RPC calls to each known `BlockManagerSlaveEndpoint` to remove the specific blocks. The PRC call sometimes could end in a timeout when the executor has been lost, but only notified the `BlockManagerMasterEndpoint` after the removing call has already happened. The timeout error could therefore fail the whole job. In this case, we actually could just ignore the error since those blocks on the lost executor could be considered as removed already. ### Does this PR introduce _any_ user-facing change? Yes. In case of users hits this issue, they will have the job executed successfully instead of throwing the exception. ### How was this patch tested? Added unit tests. Closes #28924 from Ngone51/ignore-timeout-error-for-inactive-executor. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../cluster/CoarseGrainedClusterMessage.scala | 2 + .../CoarseGrainedSchedulerBackend.scala | 3 + .../spark/storage/BlockManagerMaster.scala | 9 +- .../storage/BlockManagerMasterEndpoint.scala | 77 ++++++++++--- .../org/apache/spark/util/RpcUtils.scala | 10 ++ .../spark/storage/BlockManagerSuite.scala | 109 +++++++++++++++++- 6 files changed, 191 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 465c0d20de481..bb929c27b6a65 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -132,4 +132,6 @@ private[spark] object CoarseGrainedClusterMessages { // Used internally by executors to shut themselves down. case object Shutdown extends CoarseGrainedClusterMessage + // The message to check if `CoarseGrainedSchedulerBackend` thinks the executor is alive or not. + case class IsExecutorAlive(executorId: String) extends CoarseGrainedClusterMessage } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 67638a5f9593c..e4f4000d3574d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -285,6 +285,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp Option(delegationTokens.get()), rp) context.reply(reply) + + case IsExecutorAlive(executorId) => context.reply(isExecutorActive(executorId)) + case e => logError(s"Received unexpected ask ${e}") } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 3cfa5d2a25818..f571e428522c5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -142,7 +142,8 @@ class BlockManagerMaster( logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}", e) )(ThreadUtils.sameThread) if (blocking) { - timeout.awaitResult(future) + // the underlying Futures will timeout anyway, so it's safe to use infinite timeout here + RpcUtils.INFINITE_TIMEOUT.awaitResult(future) } } @@ -153,7 +154,8 @@ class BlockManagerMaster( logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}", e) )(ThreadUtils.sameThread) if (blocking) { - timeout.awaitResult(future) + // the underlying Futures will timeout anyway, so it's safe to use infinite timeout here + RpcUtils.INFINITE_TIMEOUT.awaitResult(future) } } @@ -166,7 +168,8 @@ class BlockManagerMaster( s" with removeFromMaster = $removeFromMaster - ${e.getMessage}", e) )(ThreadUtils.sameThread) if (blocking) { - timeout.awaitResult(future) + // the underlying Futures will timeout anyway, so it's safe to use infinite timeout here + RpcUtils.INFINITE_TIMEOUT.awaitResult(future) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index d936420a99276..f90216b973776 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -23,8 +23,9 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.{ExecutionContext, Future, TimeoutException} import scala.util.Random +import scala.util.control.NonFatal import com.google.common.cache.CacheBuilder @@ -32,8 +33,9 @@ import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.{config, Logging} import org.apache.spark.network.shuffle.ExternalBlockStoreClient -import org.apache.spark.rpc.{IsolatedRpcEndpoint, RpcCallContext, RpcEndpointRef, RpcEnv} +import org.apache.spark.rpc.{IsolatedRpcEndpoint, RpcCallContext, RpcEndpointAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.{CoarseGrainedClusterMessages, CoarseGrainedSchedulerBackend} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils} @@ -95,6 +97,9 @@ class BlockManagerMasterEndpoint( private val externalShuffleServiceRddFetchEnabled: Boolean = externalBlockStoreClient.isDefined private val externalShuffleServicePort: Int = StorageUtils.externalShuffleServicePort(conf) + private lazy val driverEndpoint = + RpcUtils.makeDriverRef(CoarseGrainedSchedulerBackend.ENDPOINT_NAME, conf, rpcEnv) + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RegisterBlockManager(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) => context.reply(register(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) @@ -168,6 +173,50 @@ class BlockManagerMasterEndpoint( stop() } + /** + * A function that used to handle the failures when removing blocks. In general, the failure + * should be considered as non-fatal since it won't cause any correctness issue. Therefore, + * this function would prefer to log the exception and return the default value. We only throw + * the exception when there's a TimeoutException from an active executor, which implies the + * unhealthy status of the executor while the driver still not be aware of it. + * @param blockType should be one of "RDD", "shuffle", "broadcast", "block", used for log + * @param blockId the string value of a certain block id, used for log + * @param bmId the BlockManagerId of the BlockManager, where we're trying to remove the block + * @param defaultValue the return value of a failure removal. e.g., 0 means no blocks are removed + * @tparam T the generic type for defaultValue, Int or Boolean. + * @return the defaultValue or throw exception if the executor is active but reply late. + */ + private def handleBlockRemovalFailure[T]( + blockType: String, + blockId: String, + bmId: BlockManagerId, + defaultValue: T): PartialFunction[Throwable, T] = { + case e: IOException => + logWarning(s"Error trying to remove $blockType $blockId" + + s" from block manager $bmId", e) + defaultValue + + case t: TimeoutException => + val executorId = bmId.executorId + val isAlive = try { + driverEndpoint.askSync[Boolean](CoarseGrainedClusterMessages.IsExecutorAlive(executorId)) + } catch { + // ignore the non-fatal error from driverEndpoint since the caller doesn't really + // care about the return result of removing blocks. And so we could avoid breaking + // down the whole application. + case NonFatal(e) => + logError(s"Fail to know the executor $executorId is alive or not.", e) + false + } + if (!isAlive) { + logWarning(s"Error trying to remove $blockType $blockId. " + + s"The executor $executorId may have been lost.", t) + defaultValue + } else { + throw t + } + } + private def removeRdd(rddId: Int): Future[Seq[Int]] = { // First remove the metadata for the given RDD, and then asynchronously remove the blocks // from the slaves. @@ -207,10 +256,8 @@ class BlockManagerMasterEndpoint( } val removeRddFromExecutorsFutures = blockManagerInfo.values.map { bmInfo => bmInfo.slaveEndpoint.ask[Int](removeMsg).recover { - case e: IOException => - logWarning(s"Error trying to remove RDD ${removeMsg.rddId} " + - s"from block manager ${bmInfo.blockManagerId}", e) - 0 // zero blocks were removed + // use 0 as default value means no blocks were removed + handleBlockRemovalFailure("RDD", rddId.toString, bmInfo.blockManagerId, 0) } }.toSeq @@ -235,7 +282,10 @@ class BlockManagerMasterEndpoint( val removeMsg = RemoveShuffle(shuffleId) Future.sequence( blockManagerInfo.values.map { bm => - bm.slaveEndpoint.ask[Boolean](removeMsg) + bm.slaveEndpoint.ask[Boolean](removeMsg).recover { + // use false as default value means no shuffle data were removed + handleBlockRemovalFailure("shuffle", shuffleId.toString, bm.blockManagerId, false) + } }.toSeq ) } @@ -252,10 +302,8 @@ class BlockManagerMasterEndpoint( } val futures = requiredBlockManagers.map { bm => bm.slaveEndpoint.ask[Int](removeMsg).recover { - case e: IOException => - logWarning(s"Error trying to remove broadcast $broadcastId from block manager " + - s"${bm.blockManagerId}", e) - 0 // zero blocks were removed + // use 0 as default value means no blocks were removed + handleBlockRemovalFailure("broadcast", broadcastId.toString, bm.blockManagerId, 0) } }.toSeq @@ -350,11 +398,14 @@ class BlockManagerMasterEndpoint( if (locations != null) { locations.foreach { blockManagerId: BlockManagerId => val blockManager = blockManagerInfo.get(blockManagerId) - if (blockManager.isDefined) { + blockManager.foreach { bm => // Remove the block from the slave's BlockManager. // Doesn't actually wait for a confirmation and the message might get lost. // If message loss becomes frequent, we should add retry logic here. - blockManager.get.slaveEndpoint.ask[Boolean](RemoveBlock(blockId)) + bm.slaveEndpoint.ask[Boolean](RemoveBlock(blockId)).recover { + // use false as default value means no blocks were removed + handleBlockRemovalFailure("block", blockId.toString, bm.blockManagerId, false) + } } } } diff --git a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala index 7272b375e5388..0e4debc595345 100644 --- a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.util +import scala.concurrent.duration._ + import org.apache.spark.SparkConf import org.apache.spark.internal.config import org.apache.spark.internal.config.Network._ @@ -54,6 +56,14 @@ private[spark] object RpcUtils { RpcTimeout(conf, Seq(RPC_LOOKUP_TIMEOUT.key, NETWORK_TIMEOUT.key), "120s") } + /** + * Infinite timeout is used internally, so there's no timeout configuration property that + * controls it. Therefore, we use "infinite" without any specific reason as its timeout + * configuration property. And its timeout property should never be accessed since infinite + * means we never timeout. + */ + val INFINITE_TIMEOUT = new RpcTimeout(Long.MaxValue.nanos, "infinite") + private val MAX_MESSAGE_SIZE_IN_MB = Int.MaxValue / 1024 / 1024 /** Returns the configured max message size for messages in bytes. */ diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index bfef8f1ab29d8..75e755f70ab0a 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Future +import scala.concurrent.{Future, TimeoutException} import scala.concurrent.duration._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -49,8 +49,9 @@ import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransport import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExecutorDiskUtils, ExternalBlockStoreClient} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} -import org.apache.spark.rpc.RpcEnv +import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEnv} import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerBlockUpdated} +import org.apache.spark.scheduler.cluster.{CoarseGrainedClusterMessages, CoarseGrainedSchedulerBackend} import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager} import org.apache.spark.shuffle.sort.SortShuffleManager @@ -93,6 +94,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE .set(MEMORY_STORAGE_FRACTION, 0.999) .set(Kryo.KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") .set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) + .set(Network.RPC_ASK_TIMEOUT, "5s") } private def makeBlockManager( @@ -137,8 +139,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf = new SparkConf(false) init(conf) - rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) + rpcEnv = RpcEnv.create("test", conf.get(config.DRIVER_HOST_ADDRESS), + conf.get(config.DRIVER_PORT), conf, securityMgr) conf.set(DRIVER_PORT, rpcEnv.address.port) + conf.set(DRIVER_HOST_ADDRESS, rpcEnv.address.host) // Mock SparkContext to reduce the memory usage of tests. It's fine since the only reason we // need to create a SparkContext is to initialize LiveListenerBus. @@ -177,6 +181,105 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE blockManager.stop() } + /** + * Setup driverEndpoint, executor-1(BlockManager), executor-2(BlockManager) to simulate + * the real cluster before the tests. Any requests from driver to executor-1 will be responded + * in time. However, any requests from driver to executor-2 will be timeouted, in order to test + * the specific handling of `TimeoutException`, which is raised at driver side. + * + * And, when `withLost` is true, we will not register the executor-2 to the driver. Therefore, + * it behaves like a lost executor in terms of driver's view. When `withLost` is false, we'll + * register the executor-2 normally. + */ + private def setupBlockManagerMasterWithBlocks(withLost: Boolean): Unit = { + // set up a simple DriverEndpoint which simply adds executorIds and + // checks whether a certain executorId has been added before. + val driverEndpoint = rpcEnv.setupEndpoint(CoarseGrainedSchedulerBackend.ENDPOINT_NAME, + new RpcEndpoint { + private val executorSet = mutable.HashSet[String]() + override val rpcEnv: RpcEnv = this.rpcEnv + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case CoarseGrainedClusterMessages.RegisterExecutor(executorId, _, _, _, _, _, _, _) => + executorSet += executorId + context.reply(true) + case CoarseGrainedClusterMessages.IsExecutorAlive(executorId) => + context.reply(executorSet.contains(executorId)) + } + } + ) + + def createAndRegisterBlockManager(timeout: Boolean): BlockManagerId = { + val id = if (timeout) "timeout" else "normal" + val bmRef = rpcEnv.setupEndpoint(s"bm-$id", new RpcEndpoint { + override val rpcEnv: RpcEnv = this.rpcEnv + private def reply[T](context: RpcCallContext, response: T): Unit = { + if (timeout) { + Thread.sleep(conf.getTimeAsMs(Network.RPC_ASK_TIMEOUT.key) + 1000) + } + context.reply(response) + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case RemoveRdd(_) => reply(context, 1) + case RemoveBroadcast(_, _) => reply(context, 1) + case RemoveShuffle(_) => reply(context, true) + } + }) + val bmId = BlockManagerId(s"exec-$id", "localhost", 1234, None) + master.registerBlockManager(bmId, Array.empty, 2000, 0, bmRef) + } + + // set up normal bm1 + val bm1Id = createAndRegisterBlockManager(false) + // set up bm2, which intentionally takes more time than RPC_ASK_TIMEOUT to + // remove rdd/broadcast/shuffle in order to raise timeout error + val bm2Id = createAndRegisterBlockManager(true) + + driverEndpoint.askSync[Boolean](CoarseGrainedClusterMessages.RegisterExecutor( + bm1Id.executorId, null, bm1Id.host, 1, Map.empty, Map.empty, + Map.empty, 0)) + + if (!withLost) { + driverEndpoint.askSync[Boolean](CoarseGrainedClusterMessages.RegisterExecutor( + bm2Id.executorId, null, bm1Id.host, 1, Map.empty, Map.empty, Map.empty, 0)) + } + + eventually(timeout(5.seconds)) { + // make sure both bm1 and bm2 are registered at driver side BlockManagerMaster + verify(master, times(2)) + .registerBlockManager(mc.any(), mc.any(), mc.any(), mc.any(), mc.any()) + assert(driverEndpoint.askSync[Boolean]( + CoarseGrainedClusterMessages.IsExecutorAlive(bm1Id.executorId))) + assert(driverEndpoint.askSync[Boolean]( + CoarseGrainedClusterMessages.IsExecutorAlive(bm2Id.executorId)) === !withLost) + } + + // update RDD block info for bm1 and bm2 (Broadcast and shuffle don't report block + // locations to BlockManagerMaster) + master.updateBlockInfo(bm1Id, RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100, 0) + master.updateBlockInfo(bm2Id, RDDBlockId(0, 1), StorageLevel.MEMORY_ONLY, 100, 0) + } + + test("SPARK-32091: count failures from active executors when remove rdd/broadcast/shuffle") { + setupBlockManagerMasterWithBlocks(false) + // fail because bm2 will timeout and it's not lost anymore + assert(intercept[Exception](master.removeRdd(0, true)) + .getCause.isInstanceOf[TimeoutException]) + assert(intercept[Exception](master.removeBroadcast(0, true, true)) + .getCause.isInstanceOf[TimeoutException]) + assert(intercept[Exception](master.removeShuffle(0, true)) + .getCause.isInstanceOf[TimeoutException]) + } + + test("SPARK-32091: ignore failures from lost executors when remove rdd/broadcast/shuffle") { + setupBlockManagerMasterWithBlocks(true) + // succeed because bm1 will remove rdd/broadcast successfully and bm2 will + // timeout but ignored as it's lost + master.removeRdd(0, true) + master.removeBroadcast(0, true, true) + master.removeShuffle(0, true) + } + test("StorageLevel object caching") { val level1 = StorageLevel(false, false, false, 3) // this should return the same object as level1 From 560fe1f54cb0ac8abbb070eae3510f138d434746 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 10 Jul 2020 09:03:16 -0700 Subject: [PATCH 159/384] [SPARK-32220][SQL] SHUFFLE_REPLICATE_NL Hint should not change Non-Cartesian Product join result ### What changes were proposed in this pull request? In current Join Hint strategies, if we use SHUFFLE_REPLICATE_NL hint, it will directly convert join to Cartesian Product Join and loss join condition making result not correct. For Example: ``` spark-sql> select * from test4 order by a asc; 1 2 Time taken: 1.063 seconds, Fetched 4 row(s)20/07/08 14:11:25 INFO SparkSQLCLIDriver: Time taken: 1.063 seconds, Fetched 4 row(s) spark-sql>select * from test5 order by a asc 1 2 2 2 Time taken: 1.18 seconds, Fetched 24 row(s)20/07/08 14:13:59 INFO SparkSQLCLIDriver: Time taken: 1.18 seconds, Fetched 24 row(s)spar spark-sql>select /*+ shuffle_replicate_nl(test4) */ * from test4 join test5 where test4.a = test5.a order by test4.a asc ; 1 2 1 2 1 2 2 2 Time taken: 0.351 seconds, Fetched 2 row(s) 20/07/08 14:18:16 INFO SparkSQLCLIDriver: Time taken: 0.351 seconds, Fetched 2 row(s) ``` ### Why are the changes needed? Fix wrong data result ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? Added UT Closes #29035 from AngersZhuuuu/SPARK-32220. Authored-by: angerszhu Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/SparkStrategies.scala | 4 +-- .../org/apache/spark/sql/JoinHintSuite.scala | 27 +++++++++++++++++++ 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 3f339347ab4db..583e5a2c5c57e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -159,7 +159,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // 4. Pick cartesian product if join type is inner like. // 5. Pick broadcast nested loop join as the final solution. It may OOM but we don't have // other choice. - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, hint) => + case p @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, hint) => def createBroadcastHashJoin(onlyLookingAtHint: Boolean) = { getBroadcastBuildSide(left, right, joinType, hint, onlyLookingAtHint, conf).map { buildSide => @@ -199,7 +199,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def createCartesianProduct() = { if (joinType.isInstanceOf[InnerLike]) { - Some(Seq(joins.CartesianProductExec(planLater(left), planLater(right), condition))) + Some(Seq(joins.CartesianProductExec(planLater(left), planLater(right), p.condition))) } else { None } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index 234978b9ce176..9f4c24b46a9b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -570,4 +570,31 @@ class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkP assert(joinHints == expectedHints) } } + + test("SPARK-32220: Non Cartesian Product Join Result Correct with SHUFFLE_REPLICATE_NL hint") { + withTempView("t1", "t2") { + Seq((1, "4"), (2, "2")).toDF("key", "value").createTempView("t1") + Seq((1, "1"), (2, "12.3"), (2, "123")).toDF("key", "value").createTempView("t2") + val df1 = sql("SELECT /*+ shuffle_replicate_nl(t1) */ * from t1 join t2 ON t1.key = t2.key") + val df2 = sql("SELECT * from t1 join t2 ON t1.key = t2.key") + assert(df1.collect().size == df2.collect().size) + + val df3 = sql("SELECT /*+ shuffle_replicate_nl(t1) */ * from t1 join t2") + val df4 = sql("SELECT * from t1 join t2") + assert(df3.collect().size == df4.collect().size) + + val df5 = sql("SELECT /*+ shuffle_replicate_nl(t1) */ * from t1 join t2 ON t1.key < t2.key") + val df6 = sql("SELECT * from t1 join t2 ON t1.key < t2.key") + assert(df5.collect().size == df6.collect().size) + + val df7 = sql("SELECT /*+ shuffle_replicate_nl(t1) */ * from t1 join t2 ON t1.key < 2") + val df8 = sql("SELECT * from t1 join t2 ON t1.key < 2") + assert(df7.collect().size == df8.collect().size) + + + val df9 = sql("SELECT /*+ shuffle_replicate_nl(t1) */ * from t1 join t2 ON t2.key < 2") + val df10 = sql("SELECT * from t1 join t2 ON t2.key < 2") + assert(df9.collect().size == df10.collect().size) + } + } } From 500877e7851ad79bad0511948ee6868d5d1c9b94 Mon Sep 17 00:00:00 2001 From: TJX2014 Date: Fri, 10 Jul 2020 11:06:52 -0700 Subject: [PATCH 160/384] [SPARK-32133][SQL] Forbid time field steps for date start/end in Sequence MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? 1.Add time field steps check for date start/end in Sequence at `org.apache.spark.sql.catalyst.expressions.Sequence.TemporalSequenceImpl` 2.Add a UT:`SPARK-32133: Sequence step must be a day interval if start and end values are dates` at `org.apache.spark.sql.catalyst.expressions.CollectionExpressionsSuite` ### Why are the changes needed? **Sequence time field steps for date start/end looks strange in spark as follows:** ``` scala> sql("select explode(sequence(cast('2011-03-01' as date), cast('2011-03-02' as date), interval 1 hour))").head(3) res0: Array[org.apache.spark.sql.Row] = _Array([2011-03-01], [2011-03-01], [2011-03-01])_ **<- strange result.** scala> sql("select explode(sequence(cast('2011-03-01' as date), cast('2011-03-02' as date), interval 1 day))").head(3) res1: Array[org.apache.spark.sql.Row] = Array([2011-03-01], [2011-03-02]) ``` **While this behavior in Prosto make sense:** ``` presto> select sequence(date('2011-03-01'),date('2011-03-02'),interval '1' hour); Query 20200624_122744_00002_pehix failed: sequence step must be a day interval if start and end values are dates presto> select sequence(date('2011-03-01'),date('2011-03-02'),interval '1' day); _col0 [2011-03-01, 2011-03-02] ``` ### Does this PR introduce _any_ user-facing change? Yes, after this patch, users will get informed `sequence step must be a day interval if start and end values are dates` when use time field steps for date start/end in Sequence. ### How was this patch tested? Unit test. Closes #28926 from TJX2014/master-SPARK-31982-sequence-cross-dst-follow-presto. Authored-by: TJX2014 Signed-off-by: Dongjoon Hyun --- .../expressions/collectionOperations.scala | 20 +++++++++++++++++++ .../CollectionExpressionsSuite.scala | 7 +++++++ 2 files changed, 27 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 026a2a677baec..74c9b12a109d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -2612,10 +2612,17 @@ object Sequence { val stepDays = step.days val stepMicros = step.microseconds + if (scale == MICROS_PER_DAY && stepMonths == 0 && stepDays == 0) { + throw new IllegalArgumentException( + "sequence step must be a day interval if start and end values are dates") + } + if (stepMonths == 0 && stepMicros == 0 && scale == MICROS_PER_DAY) { + // Adding pure days to date start/end backedSequenceImpl.eval(start, stop, fromLong(stepDays)) } else if (stepMonths == 0 && stepDays == 0 && scale == 1) { + // Adding pure microseconds to timestamp start/end backedSequenceImpl.eval(start, stop, fromLong(stepMicros)) } else { @@ -2674,11 +2681,24 @@ object Sequence { |${genSequenceLengthCode(ctx, startMicros, stopMicros, intervalInMicros, arrLength)} """.stripMargin + val check = if (scale == MICROS_PER_DAY) { + s""" + |if ($stepMonths == 0 && $stepDays == 0) { + | throw new IllegalArgumentException( + | "sequence step must be a day interval if start and end values are dates"); + |} + """.stripMargin + } else { + "" + } + s""" |final int $stepMonths = $step.months; |final int $stepDays = $step.days; |final long $stepMicros = $step.microseconds; | + |$check + | |if ($stepMonths == 0 && $stepMicros == 0 && ${scale}L == ${MICROS_PER_DAY}L) { | ${backedSequenceImpl.genCode(ctx, start, stop, stepDays, arr, elemType)}; | diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 3a0c02b29d92c..856c1fad9b204 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -933,6 +933,13 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper Literal(negateExact(stringToInterval("interval 1 month")))), EmptyRow, s"sequence boundaries: 0 to 2678400000000 by -${28 * MICROS_PER_DAY}") + + // SPARK-32133: Sequence step must be a day interval if start and end values are dates + checkExceptionInExpression[IllegalArgumentException](Sequence( + Cast(Literal("2011-03-01"), DateType), + Cast(Literal("2011-04-01"), DateType), + Option(Literal(stringToInterval("interval 1 hour")))), null, + "sequence step must be a day interval if start and end values are dates") } } From d7d5bdfd79bcca405ea10d218b4a2e7b1efb9083 Mon Sep 17 00:00:00 2001 From: Pavithraramachandran Date: Fri, 10 Jul 2020 13:55:20 -0700 Subject: [PATCH 161/384] [SPARK-32103][CORE] Support IPv6 host/port in core module ### What changes were proposed in this pull request? In IPv6 scenario, the current logic to split hostname and port is not correct. ### Why are the changes needed? to support IPV6 deployment scenario ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT and IPV6 spark deployment with yarn Closes #28931 from PavithraRamachandran/ipv6_issue. Authored-by: Pavithraramachandran Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/SparkStatusTracker.scala | 6 +- .../org/apache/spark/status/LiveEntity.scala | 4 +- .../org/apache/spark/status/storeTypes.scala | 3 +- .../scala/org/apache/spark/util/Utils.scala | 48 ++++++--- .../org/apache/spark/util/UtilsSuite.scala | 97 +++++++++++++++++++ .../spark/deploy/yarn/YarnRMClient.scala | 3 +- 6 files changed, 142 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala index 555c085d85a1e..37e673cd8c7e1 100644 --- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala +++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala @@ -21,6 +21,7 @@ import java.util.Arrays import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1.StageStatus +import org.apache.spark.util.Utils /** * Low-level status reporting APIs for monitoring job and stage progress. @@ -103,10 +104,7 @@ class SparkStatusTracker private[spark] (sc: SparkContext, store: AppStatusStore */ def getExecutorInfos: Array[SparkExecutorInfo] = { store.executorList(true).map { exec => - val (host, port) = exec.hostPort.split(":", 2) match { - case Array(h, p) => (h, p.toInt) - case Array(h) => (h, -1) - } + val (host, port) = Utils.parseHostPort(exec.hostPort) val cachedMem = exec.memoryMetrics.map { mem => mem.usedOnHeapStorageMemory + mem.usedOffHeapStorageMemory }.getOrElse(0L) diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 86cb4fe138773..81478214994b0 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -33,7 +33,7 @@ import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, TaskInfo} import org.apache.spark.status.api.v1 import org.apache.spark.storage.{RDDInfo, StorageLevel} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.AccumulatorContext +import org.apache.spark.util.{AccumulatorContext, Utils} import org.apache.spark.util.collection.OpenHashSet /** @@ -307,7 +307,7 @@ private[spark] class LiveExecutor(val executorId: String, _addTime: Long) extend // peak values for executor level metrics val peakExecutorMetrics = new ExecutorMetrics() - def hostname: String = if (host != null) host else hostPort.split(":")(0) + def hostname: String = if (host != null) host else Utils.parseHostPort(hostPort)._1 override protected def doUpdate(): Any = { val memoryMetrics = if (totalOnHeap >= 0) { diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index b40f7304b7ce2..5a164823297f9 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.status.KVUtils._ import org.apache.spark.status.api.v1._ import org.apache.spark.ui.scope._ +import org.apache.spark.util.Utils import org.apache.spark.util.kvstore.KVIndex private[spark] case class AppStatusStoreMetadata(version: Long) @@ -57,7 +58,7 @@ private[spark] class ExecutorSummaryWrapper(val info: ExecutorSummary) { private def active: Boolean = info.isActive @JsonIgnore @KVIndex("host") - val host: String = info.hostPort.split(":")(0) + val host: String = Utils.parseHostPort(info.hostPort)._1 } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 9636fe88c77c2..2b4b2508f9b7e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1026,13 +1026,27 @@ private[spark] object Utils extends Logging { customHostname.getOrElse(InetAddresses.toUriString(localIpAddress)) } + /** + * Checks if the host contains only valid hostname/ip without port + * NOTE: Incase of IPV6 ip it should be enclosed inside [] + */ def checkHost(host: String): Unit = { - assert(host != null && host.indexOf(':') == -1, s"Expected hostname (not IP) but got $host") + if (host != null && host.split(":").length > 2) { + assert(host.startsWith("[") && host.endsWith("]"), + s"Expected hostname or IPv6 IP enclosed in [] but got $host") + } else { + assert(host != null && host.indexOf(':') == -1, s"Expected hostname or IP but got $host") + } } def checkHostPort(hostPort: String): Unit = { - assert(hostPort != null && hostPort.indexOf(':') != -1, - s"Expected host and port but got $hostPort") + if (hostPort != null && hostPort.split(":").length > 2) { + assert(hostPort != null && hostPort.indexOf("]:") != -1, + s"Expected host and port but got $hostPort") + } else { + assert(hostPort != null && hostPort.indexOf(':') != -1, + s"Expected host and port but got $hostPort") + } } // Typically, this will be of order of number of nodes in cluster @@ -1046,18 +1060,30 @@ private[spark] object Utils extends Logging { return cached } - val indx: Int = hostPort.lastIndexOf(':') - // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... - // but then hadoop does not support ipv6 right now. - // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 - if (-1 == indx) { + def setDefaultPortValue: (String, Int) = { val retval = (hostPort, 0) hostPortParseResults.put(hostPort, retval) - return retval + retval + } + // checks if the hostport contains IPV6 ip and parses the host, port + if (hostPort != null && hostPort.split(":").length > 2) { + val indx: Int = hostPort.lastIndexOf("]:") + if (-1 == indx) { + return setDefaultPortValue + } + val port = hostPort.substring(indx + 2).trim() + val retval = (hostPort.substring(0, indx + 1).trim(), if (port.isEmpty) 0 else port.toInt) + hostPortParseResults.putIfAbsent(hostPort, retval) + } else { + val indx: Int = hostPort.lastIndexOf(':') + if (-1 == indx) { + return setDefaultPortValue + } + val port = hostPort.substring(indx + 1).trim() + val retval = (hostPort.substring(0, indx).trim(), if (port.isEmpty) 0 else port.toInt) + hostPortParseResults.putIfAbsent(hostPort, retval) } - val retval = (hostPort.substring(0, indx).trim(), hostPort.substring(indx + 1).trim().toInt) - hostPortParseResults.putIfAbsent(hostPort, retval) hostPortParseResults.get(hostPort) } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index c9c8ae6023877..7ec7c5afca1df 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1309,6 +1309,103 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.buildLocationMetadata(paths, 15) == "[path0, path1, path2]") assert(Utils.buildLocationMetadata(paths, 25) == "[path0, path1, path2, path3]") } + + test("checkHost supports both IPV4 and IPV6") { + // IPV4 ips + Utils.checkHost("0.0.0.0") + var e: AssertionError = intercept[AssertionError] { + Utils.checkHost("0.0.0.0:0") + } + assert(e.getMessage.contains("Expected hostname or IP but got 0.0.0.0:0")) + e = intercept[AssertionError] { + Utils.checkHost("0.0.0.0:") + } + assert(e.getMessage.contains("Expected hostname or IP but got 0.0.0.0:")) + // IPV6 ips + Utils.checkHost("[::1]") + e = intercept[AssertionError] { + Utils.checkHost("[::1]:0") + } + assert(e.getMessage.contains("Expected hostname or IPv6 IP enclosed in [] but got [::1]:0")) + e = intercept[AssertionError] { + Utils.checkHost("[::1]:") + } + assert(e.getMessage.contains("Expected hostname or IPv6 IP enclosed in [] but got [::1]:")) + // hostname + Utils.checkHost("localhost") + e = intercept[AssertionError] { + Utils.checkHost("localhost:0") + } + assert(e.getMessage.contains("Expected hostname or IP but got localhost:0")) + e = intercept[AssertionError] { + Utils.checkHost("localhost:") + } + assert(e.getMessage.contains("Expected hostname or IP but got localhost:")) + } + + test("checkHostPort support IPV6 and IPV4") { + // IPV4 ips + Utils.checkHostPort("0.0.0.0:0") + var e: AssertionError = intercept[AssertionError] { + Utils.checkHostPort("0.0.0.0") + } + assert(e.getMessage.contains("Expected host and port but got 0.0.0.0")) + + // IPV6 ips + Utils.checkHostPort("[::1]:0") + e = intercept[AssertionError] { + Utils.checkHostPort("[::1]") + } + assert(e.getMessage.contains("Expected host and port but got [::1]")) + + // hostname + Utils.checkHostPort("localhost:0") + e = intercept[AssertionError] { + Utils.checkHostPort("localhost") + } + assert(e.getMessage.contains("Expected host and port but got localhost")) + } + + test("parseHostPort support IPV6 and IPV4") { + // IPV4 ips + var hostnamePort = Utils.parseHostPort("0.0.0.0:80") + assert(hostnamePort._1.equals("0.0.0.0")) + assert(hostnamePort._2 === 80) + + hostnamePort = Utils.parseHostPort("0.0.0.0") + assert(hostnamePort._1.equals("0.0.0.0")) + assert(hostnamePort._2 === 0) + + hostnamePort = Utils.parseHostPort("0.0.0.0:") + assert(hostnamePort._1.equals("0.0.0.0")) + assert(hostnamePort._2 === 0) + + // IPV6 ips + hostnamePort = Utils.parseHostPort("[::1]:80") + assert(hostnamePort._1.equals("[::1]")) + assert(hostnamePort._2 === 80) + + hostnamePort = Utils.parseHostPort("[::1]") + assert(hostnamePort._1.equals("[::1]")) + assert(hostnamePort._2 === 0) + + hostnamePort = Utils.parseHostPort("[::1]:") + assert(hostnamePort._1.equals("[::1]")) + assert(hostnamePort._2 === 0) + + // hostname + hostnamePort = Utils.parseHostPort("localhost:80") + assert(hostnamePort._1.equals("localhost")) + assert(hostnamePort._2 === 80) + + hostnamePort = Utils.parseHostPort("localhost") + assert(hostnamePort._1.equals("localhost")) + assert(hostnamePort._2 === 0) + + hostnamePort = Utils.parseHostPort("localhost:") + assert(hostnamePort._1.equals("localhost")) + assert(hostnamePort._2 === 0) + } } private class SimpleExtension diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 7c67493c33160..2f272be60ba25 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -30,6 +30,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.util.Utils /** * Handles registering and unregistering the application with the YARN ResourceManager. @@ -107,7 +108,7 @@ private[spark] class YarnRMClient extends Logging { // so not all stable releases have it. val prefix = WebAppUtils.getHttpSchemePrefix(conf) val proxies = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf) - val hosts = proxies.asScala.map(_.split(":").head) + val hosts = proxies.asScala.map(proxy => Utils.parseHostPort(proxy)._1) val uriBases = proxies.asScala.map { proxy => prefix + proxy + proxyBase } val params = Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) From 84db660ebef4f9c543ab2709103c4542b407a829 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 10 Jul 2020 15:10:28 -0700 Subject: [PATCH 162/384] [SPARK-32251][SQL][DOCS][TESTS] Fix SQL keyword document ### What changes were proposed in this pull request? This PR improves the test to make sure all the SQL keywords are documented correctly. It fixes several issues: 1. some keywords are not documented 2. some keywords are not ANSI SQL keywords but documented as reserved/non-reserved. ### Why are the changes needed? To make sure the implementation matches the doc. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? new test Closes #29055 from cloud-fan/keyword. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- docs/sql-ref-ansi-compliance.md | 13 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/catalyst/SQLKeywordSuite.scala | 170 ++++++++++++++++++ .../parser/TableIdentifierParserSuite.scala | 127 +------------ 4 files changed, 182 insertions(+), 130 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index e5ca7e9d10d59..6488ad9cd34c9 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -127,7 +127,7 @@ By default `spark.sql.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. -|Keyword|Spark SQL
ANSI Mode|Spark SQL
Default Mode|SQL-2011| +|Keyword|Spark SQL
ANSI Mode|Spark SQL
Default Mode|SQL-2016| |-------|----------------------|-------------------------|--------| |ADD|non-reserved|non-reserved|non-reserved| |AFTER|non-reserved|non-reserved|non-reserved| @@ -149,7 +149,7 @@ Below is a list of all the keywords in Spark SQL. |BUCKETS|non-reserved|non-reserved|non-reserved| |BY|non-reserved|non-reserved|reserved| |CACHE|non-reserved|non-reserved|non-reserved| -|CASCADE|non-reserved|non-reserved|reserved| +|CASCADE|non-reserved|non-reserved|non-reserved| |CASE|reserved|non-reserved|reserved| |CAST|reserved|non-reserved|reserved| |CHANGE|non-reserved|non-reserved|non-reserved| @@ -193,7 +193,7 @@ Below is a list of all the keywords in Spark SQL. |DIRECTORY|non-reserved|non-reserved|non-reserved| |DISTINCT|reserved|non-reserved|reserved| |DISTRIBUTE|non-reserved|non-reserved|non-reserved| -|DIV|non-reserved|non-reserved|non-reserved| +|DIV|non-reserved|non-reserved|not a keyword| |DROP|non-reserved|non-reserved|reserved| |ELSE|reserved|non-reserved|reserved| |END|reserved|non-reserved|reserved| @@ -228,7 +228,7 @@ Below is a list of all the keywords in Spark SQL. |GROUPING|non-reserved|non-reserved|reserved| |HAVING|reserved|non-reserved|reserved| |HOUR|reserved|non-reserved|reserved| -|IF|non-reserved|non-reserved|reserved| +|IF|non-reserved|non-reserved|not a keyword| |IGNORE|non-reserved|non-reserved|non-reserved| |IMPORT|non-reserved|non-reserved|non-reserved| |IN|reserved|non-reserved|reserved| @@ -302,12 +302,14 @@ Below is a list of all the keywords in Spark SQL. |PROPERTIES|non-reserved|non-reserved|non-reserved| |PURGE|non-reserved|non-reserved|non-reserved| |QUERY|non-reserved|non-reserved|non-reserved| +|RANGE|non-reserved|non-reserved|reserved| |RECORDREADER|non-reserved|non-reserved|non-reserved| |RECORDWRITER|non-reserved|non-reserved|non-reserved| |RECOVER|non-reserved|non-reserved|non-reserved| |REDUCE|non-reserved|non-reserved|non-reserved| |REFERENCES|reserved|non-reserved|reserved| |REFRESH|non-reserved|non-reserved|non-reserved| +|REGEXP|non-reserved|non-reserved|not a keyword| |RENAME|non-reserved|non-reserved|non-reserved| |REPAIR|non-reserved|non-reserved|non-reserved| |REPLACE|non-reserved|non-reserved|non-reserved| @@ -323,6 +325,7 @@ Below is a list of all the keywords in Spark SQL. |ROW|non-reserved|non-reserved|reserved| |ROWS|non-reserved|non-reserved|reserved| |SCHEMA|non-reserved|non-reserved|non-reserved| +|SCHEMAS|non-reserved|non-reserved|not a keyword| |SECOND|reserved|non-reserved|reserved| |SELECT|reserved|non-reserved|reserved| |SEMI|non-reserved|strict-non-reserved|non-reserved| @@ -348,6 +351,7 @@ Below is a list of all the keywords in Spark SQL. |TABLES|non-reserved|non-reserved|non-reserved| |TABLESAMPLE|non-reserved|non-reserved|reserved| |TBLPROPERTIES|non-reserved|non-reserved|non-reserved| +|TEMP|non-reserved|non-reserved|not a keyword| |TEMPORARY|non-reserved|non-reserved|non-reserved| |TERMINATED|non-reserved|non-reserved|non-reserved| |THEN|reserved|non-reserved|reserved| @@ -360,6 +364,7 @@ Below is a list of all the keywords in Spark SQL. |TRIM|non-reserved|non-reserved|non-reserved| |TRUE|non-reserved|non-reserved|reserved| |TRUNCATE|non-reserved|non-reserved|reserved| +|TYPE|non-reserved|non-reserved|non-reserved| |UNARCHIVE|non-reserved|non-reserved|non-reserved| |UNBOUNDED|non-reserved|non-reserved|non-reserved| |UNCACHE|non-reserved|non-reserved|non-reserved| diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 10e0d4e467271..d29fa1319daa5 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -1532,6 +1532,7 @@ DIRECTORIES: 'DIRECTORIES'; DIRECTORY: 'DIRECTORY'; DISTINCT: 'DISTINCT'; DISTRIBUTE: 'DISTRIBUTE'; +DIV: 'DIV'; DROP: 'DROP'; ELSE: 'ELSE'; END: 'END'; @@ -1739,7 +1740,6 @@ MINUS: '-'; ASTERISK: '*'; SLASH: '/'; PERCENT: '%'; -DIV: 'DIV'; TILDE: '~'; AMPERSAND: '&'; PIPE: '|'; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala new file mode 100644 index 0000000000000..3d41d02b23df5 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import java.io.File +import java.nio.file.Files + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.fileToString + +trait SQLKeywordUtils extends SQLHelper { + + val sqlSyntaxDefs = { + val sqlBasePath = { + java.nio.file.Paths.get(sparkHome, "sql", "catalyst", "src", "main", "antlr4", "org", + "apache", "spark", "sql", "catalyst", "parser", "SqlBase.g4").toFile + } + fileToString(sqlBasePath).split("\n") + } + + // each element is an array of 4 string: the keyword name, reserve or not in Spark ANSI mode, + // Spark non-ANSI mode, and the SQL standard. + val keywordsInDoc: Array[Array[String]] = { + val docPath = { + java.nio.file.Paths.get(sparkHome, "docs", "sql-ref-ansi-compliance.md").toFile + } + fileToString(docPath).split("\n") + .dropWhile(!_.startsWith("|Keyword|")).drop(2).takeWhile(_.startsWith("|")) + .map(_.stripPrefix("|").split("\\|").map(_.trim)) + } + + private def parseAntlrGrammars[T](startTag: String, endTag: String) + (f: PartialFunction[String, Seq[T]]): Set[T] = { + val keywords = new mutable.ArrayBuffer[T] + val default = (_: String) => Nil + var startTagFound = false + var parseFinished = false + val lineIter = sqlSyntaxDefs.toIterator + while (!parseFinished && lineIter.hasNext) { + val line = lineIter.next() + if (line.trim.startsWith(startTag)) { + startTagFound = true + } else if (line.trim.startsWith(endTag)) { + parseFinished = true + } else if (startTagFound) { + f.applyOrElse(line, default).foreach { symbol => + keywords += symbol + } + } + } + assert(keywords.nonEmpty && startTagFound && parseFinished, "cannot extract keywords from " + + s"the `SqlBase.g4` file, so please check if the start/end tags (`$startTag` and `$endTag`) " + + "are placed correctly in the file.") + keywords.toSet + } + + // If a symbol does not have the same string with its literal (e.g., `SETMINUS: 'MINUS';`), + // we need to map a symbol to actual literal strings. + val symbolsToExpandIntoDifferentLiterals = { + val kwDef = """([A-Z_]+):(.+);""".r + val keywords = parseAntlrGrammars( + "//--SPARK-KEYWORD-LIST-START", "//--SPARK-KEYWORD-LIST-END") { + case kwDef(symbol, literalDef) => + val splitDefs = literalDef.split("""\|""") + val hasMultipleLiterals = splitDefs.length > 1 + // The case where a symbol has multiple literal definitions, + // e.g., `DATABASES: 'DATABASES' | 'SCHEMAS';`. + if (hasMultipleLiterals) { + // Filters out inappropriate entries, e.g., `!` in `NOT: 'NOT' | '!';` + val litDef = """([A-Z_]+)""".r + val literals = splitDefs.map(_.replaceAll("'", "").trim).toSeq.flatMap { + case litDef(lit) => Some(lit) + case _ => None + } + (symbol, literals) :: Nil + } else { + val literal = literalDef.replaceAll("'", "").trim + // The case where a symbol string and its literal string are different, + // e.g., `SETMINUS: 'MINUS';`. + if (symbol != literal) { + (symbol, literal :: Nil) :: Nil + } else { + Nil + } + } + } + keywords.toMap + } + + // All the SQL keywords defined in `SqlBase.g4` + val allCandidateKeywords: Set[String] = { + val kwDef = """([A-Z_]+):.+;""".r + parseAntlrGrammars( + "//--SPARK-KEYWORD-LIST-START", "//--SPARK-KEYWORD-LIST-END") { + // Parses a pattern, e.g., `AFTER: 'AFTER';` + case kwDef(symbol) => + if (symbolsToExpandIntoDifferentLiterals.contains(symbol)) { + symbolsToExpandIntoDifferentLiterals(symbol) + } else { + symbol :: Nil + } + } + } + + val nonReservedKeywordsInAnsiMode: Set[String] = { + val kwDef = """\s*[\|:]\s*([A-Z_]+)\s*""".r + parseAntlrGrammars("//--ANSI-NON-RESERVED-START", "//--ANSI-NON-RESERVED-END") { + // Parses a pattern, e.g., ` | AFTER` + case kwDef(symbol) => + if (symbolsToExpandIntoDifferentLiterals.contains(symbol)) { + symbolsToExpandIntoDifferentLiterals(symbol) + } else { + symbol :: Nil + } + } + } + + val reservedKeywordsInAnsiMode = allCandidateKeywords -- nonReservedKeywordsInAnsiMode +} + +class SQLKeywordSuite extends SparkFunSuite with SQLKeywordUtils { + test("all keywords are documented") { + val documentedKeywords = keywordsInDoc.map(_.head).toSet + if (allCandidateKeywords != documentedKeywords) { + val undocumented = (allCandidateKeywords -- documentedKeywords).toSeq.sorted + fail("Some keywords are not documented: " + undocumented.mkString(", ")) + } + } + + test("Spark keywords are documented correctly") { + val reservedKeywordsInDoc = keywordsInDoc.filter(_.apply(1) == "reserved").map(_.head).toSet + if (reservedKeywordsInAnsiMode != reservedKeywordsInDoc) { + val misImplemented = (reservedKeywordsInDoc -- reservedKeywordsInAnsiMode).toSeq.sorted + fail("Some keywords are documented as reserved but actually not: " + + misImplemented.mkString(", ")) + } + } + + test("SQL 2016 keywords are documented correctly") { + withTempDir { dir => + val tmpFile = new File(dir, "tmp") + val is = Thread.currentThread().getContextClassLoader + .getResourceAsStream("ansi-sql-2016-reserved-keywords.txt") + Files.copy(is, tmpFile.toPath) + val reservedKeywordsInSql2016 = Files.readAllLines(tmpFile.toPath) + .asScala.filterNot(_.startsWith("--")).map(_.trim).toSet + val documented = keywordsInDoc.filter(_.last == "reserved").map(_.head).toSet + assert((documented -- reservedKeywordsInSql2016).isEmpty) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index a721e17aef02d..f037ce7b9e793 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -16,19 +16,11 @@ */ package org.apache.spark.sql.catalyst.parser -import java.io.File -import java.nio.file.Files - -import scala.collection.JavaConverters._ -import scala.collection.mutable - import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.catalyst.util.fileToString +import org.apache.spark.sql.catalyst.{SQLKeywordUtils, TableIdentifier} import org.apache.spark.sql.internal.SQLConf -class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { +class TableIdentifierParserSuite extends SparkFunSuite with SQLKeywordUtils { import CatalystSqlParser._ // Add "$elem$", "$value$" & "$key$" @@ -292,121 +284,6 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "where", "with") - private val sqlSyntaxDefs = { - val sqlBasePath = { - java.nio.file.Paths.get(sparkHome, "sql", "catalyst", "src", "main", "antlr4", "org", - "apache", "spark", "sql", "catalyst", "parser", "SqlBase.g4").toFile - } - fileToString(sqlBasePath).split("\n") - } - - private def parseAntlrGrammars[T](startTag: String, endTag: String) - (f: PartialFunction[String, Seq[T]]): Set[T] = { - val keywords = new mutable.ArrayBuffer[T] - val default = (_: String) => Nil - var startTagFound = false - var parseFinished = false - val lineIter = sqlSyntaxDefs.toIterator - while (!parseFinished && lineIter.hasNext) { - val line = lineIter.next() - if (line.trim.startsWith(startTag)) { - startTagFound = true - } else if (line.trim.startsWith(endTag)) { - parseFinished = true - } else if (startTagFound) { - f.applyOrElse(line, default).foreach { symbol => - keywords += symbol - } - } - } - assert(keywords.nonEmpty && startTagFound && parseFinished, "cannot extract keywords from " + - s"the `SqlBase.g4` file, so please check if the start/end tags (`$startTag` and `$endTag`) " + - "are placed correctly in the file.") - keywords.toSet - } - - // If a symbol does not have the same string with its literal (e.g., `SETMINUS: 'MINUS';`), - // we need to map a symbol to actual literal strings. - val symbolsToExpandIntoDifferentLiterals = { - val kwDef = """([A-Z_]+):(.+);""".r - val keywords = parseAntlrGrammars( - "//--SPARK-KEYWORD-LIST-START", "//--SPARK-KEYWORD-LIST-END") { - case kwDef(symbol, literalDef) => - val splitDefs = literalDef.split("""\|""") - val hasMultipleLiterals = splitDefs.length > 1 - // The case where a symbol has multiple literal definitions, - // e.g., `DATABASES: 'DATABASES' | 'SCHEMAS';`. - if (hasMultipleLiterals) { - // Filters out inappropriate entries, e.g., `!` in `NOT: 'NOT' | '!';` - val litDef = """([A-Z_]+)""".r - val literals = splitDefs.map(_.replaceAll("'", "").trim).toSeq.flatMap { - case litDef(lit) => Some(lit) - case _ => None - } - (symbol, literals) :: Nil - } else { - val literal = literalDef.replaceAll("'", "").trim - // The case where a symbol string and its literal string are different, - // e.g., `SETMINUS: 'MINUS';`. - if (symbol != literal) { - (symbol, literal :: Nil) :: Nil - } else { - Nil - } - } - } - keywords.toMap - } - - // All the SQL keywords defined in `SqlBase.g4` - val allCandidateKeywords = { - val kwDef = """([A-Z_]+):.+;""".r - val keywords = parseAntlrGrammars( - "//--SPARK-KEYWORD-LIST-START", "//--SPARK-KEYWORD-LIST-END") { - // Parses a pattern, e.g., `AFTER: 'AFTER';` - case kwDef(symbol) => - if (symbolsToExpandIntoDifferentLiterals.contains(symbol)) { - symbolsToExpandIntoDifferentLiterals(symbol) - } else { - symbol :: Nil - } - } - keywords - } - - val nonReservedKeywordsInAnsiMode = { - val kwDef = """\s*[\|:]\s*([A-Z_]+)\s*""".r - parseAntlrGrammars("//--ANSI-NON-RESERVED-START", "//--ANSI-NON-RESERVED-END") { - // Parses a pattern, e.g., ` | AFTER` - case kwDef(symbol) => - if (symbolsToExpandIntoDifferentLiterals.contains(symbol)) { - symbolsToExpandIntoDifferentLiterals(symbol) - } else { - symbol :: Nil - } - } - } - - val reservedKeywordsInAnsiMode = allCandidateKeywords -- nonReservedKeywordsInAnsiMode - - test("check # of reserved keywords") { - val numReservedKeywords = 74 - assert(reservedKeywordsInAnsiMode.size == numReservedKeywords, - s"The expected number of reserved keywords is $numReservedKeywords, but " + - s"${reservedKeywordsInAnsiMode.size} found.") - } - - test("reserved keywords in Spark are also reserved in SQL 2016") { - withTempDir { dir => - val tmpFile = new File(dir, "tmp") - val is = Thread.currentThread().getContextClassLoader - .getResourceAsStream("ansi-sql-2016-reserved-keywords.txt") - Files.copy(is, tmpFile.toPath) - val reservedKeywordsInSql2016 = Files.readAllLines(tmpFile.toPath) - .asScala.filterNot(_.startsWith("--")).map(_.trim).toSet - assert((reservedKeywordsInAnsiMode -- reservedKeywordsInSql2016).isEmpty) - } - } test("table identifier") { // Regular names. From 0c9196e5493628d343ef67bb9e83d0c95ff3943a Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Sat, 11 Jul 2020 06:27:56 -0700 Subject: [PATCH 163/384] [SPARK-32238][SQL] Use Utils.getSimpleName to avoid hitting Malformed class name in ScalaUDF ### What changes were proposed in this pull request? This PR proposes to use `Utils.getSimpleName(function)` instead of `function.getClass.getSimpleName` to get the class name. ### Why are the changes needed? For some functions(see the demo below), using `function.getClass.getSimpleName` can hit "Malformed class name" error. ### Does this PR introduce _any_ user-facing change? Yes. For the demo, ```scala object MalformedClassObject extends Serializable { class MalformedNonPrimitiveFunction extends (String => Int) with Serializable { override def apply(v1: String): Int = v1.toInt / 0 } } OuterScopes.addOuterScope(MalformedClassObject) val f = new MalformedClassObject.MalformedNonPrimitiveFunction() Seq("20").toDF("col").select(udf(f).apply(Column("col"))).collect() ``` Before this PR, user can only see the error about "Malformed class name": ```scala An exception or error caused a run to abort: Malformed class name java.lang.InternalError: Malformed class name at java.lang.Class.getSimpleName(Class.java:1330) at org.apache.spark.sql.catalyst.expressions.ScalaUDF.udfErrorMessage$lzycompute(ScalaUDF.scala:1157) at org.apache.spark.sql.catalyst.expressions.ScalaUDF.udfErrorMessage(ScalaUDF.scala:1155) at org.apache.spark.sql.catalyst.expressions.ScalaUDF.doGenCode(ScalaUDF.scala:1077) at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:147) at scala.Option.getOrElse(Option.scala:189) at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:142) at org.apache.spark.sql.catalyst.expressions.Alias.genCode(namedExpressions.scala:160) at org.apache.spark.sql.execution.ProjectExec.$anonfun$doConsume$1(basicPhysicalOperators.scala:69) ... ``` After this PR, user can see the real root cause of the udf failure: ```scala org.apache.spark.SparkException: Failed to execute user defined function(UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction: (string) => int) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753) at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373) at org.apache.spark.rdd.RDD.iterator(RDD.scala:337) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:127) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:464) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:467) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.ArithmeticException: / by zero at org.apache.spark.sql.UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction.apply(UDFSuite.scala:677) at org.apache.spark.sql.UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction.apply(UDFSuite.scala:676) ... 17 more ``` ### How was this patch tested? Added a test. Closes #29050 from Ngone51/fix-malformed-udf. Authored-by: yi.wu Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/expressions/ScalaUDF.scala | 3 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 28 +++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 3d10b084a8db1..242a065a58e50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, DataType, UserDefinedType} +import org.apache.spark.util.Utils /** * User-defined function. @@ -1152,7 +1153,7 @@ case class ScalaUDF( private[this] val resultConverter = createToCatalystConverter(dataType) lazy val udfErrorMessage = { - val funcCls = function.getClass.getSimpleName + val funcCls = Utils.getSimpleName(function.getClass) val inputTypes = children.map(_.dataType.catalogString).mkString(", ") val outputType = dataType.catalogString s"Failed to execute user defined function($funcCls: ($inputTypes) => $outputType)" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 5c1fe265c15d0..8b7e9ecfe4e4d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql import java.math.BigDecimal +import org.apache.spark.SparkException import org.apache.spark.sql.api.java._ +import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.{QueryExecution, SimpleMode} import org.apache.spark.sql.execution.columnar.InMemoryRelation @@ -669,4 +671,30 @@ class UDFSuite extends QueryTest with SharedSparkSession { val df = Seq(Array(Some(TestData(50, "2")), None)).toDF("col") checkAnswer(df.select(myUdf(Column("col"))), Row(100) :: Nil) } + + object MalformedClassObject extends Serializable { + class MalformedNonPrimitiveFunction extends (String => Int) with Serializable { + override def apply(v1: String): Int = v1.toInt / 0 + } + + class MalformedPrimitiveFunction extends (Int => Int) with Serializable { + override def apply(v1: Int): Int = v1 / 0 + } + } + + test("SPARK-32238: Use Utils.getSimpleName to avoid hitting Malformed class name") { + OuterScopes.addOuterScope(MalformedClassObject) + val f1 = new MalformedClassObject.MalformedNonPrimitiveFunction() + val f2 = new MalformedClassObject.MalformedPrimitiveFunction() + + val e1 = intercept[SparkException] { + Seq("20").toDF("col").select(udf(f1).apply(Column("col"))).collect() + } + assert(e1.getMessage.contains("UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction")) + + val e2 = intercept[SparkException] { + Seq(20).toDF("col").select(udf(f2).apply(Column("col"))).collect() + } + assert(e2.getMessage.contains("UDFSuite$MalformedClassObject$MalformedPrimitiveFunction")) + } } From 1b3fc9a111b5eed34cd885bf2ce6dd9d58b49b77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Sat, 11 Jul 2020 22:55:26 +0900 Subject: [PATCH 164/384] [SPARK-32149][SHUFFLE] Improve file path name normalisation at block resolution within the external shuffle service MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Improving file path name normalisation by removing the approximate transformation from Spark and using the path normalization from the JDK. ### Why are the changes needed? In the external shuffle service during the block resolution the file paths (for disk persisted RDD and for shuffle blocks) are normalized by a custom Spark code which uses an OS dependent regexp. This is a redundant code of the package-private JDK counterpart. As the code not a perfect match even it could happen one method results in a bit different (but semantically equal) path. The reason of this redundant transformation is the interning of the normalized path to save some heap here which is only possible if both transformations results in the same string. Checking the JDK code I believe there is a better solution which is perfect match for the JDK code as it uses that package private method. Moreover based on some benchmarking even this new method seams to be more performant too. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? As we are reusing the JDK code for normalisation no test is needed. Even the existing test can be removed. But in a separate branch I have created a benchmark where the performance of the old and the new solution can be compared. It shows the new method is about 7-10 times better than old one. Closes #28967 from attilapiros/SPARK-32149. Authored-by: “attilapiros” Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../network/shuffle/ExecutorDiskUtils.java | 52 ++++--------------- .../shuffle/ExternalShuffleBlockResolver.java | 3 -- .../ExternalShuffleBlockResolverSuite.java | 27 ---------- 3 files changed, 10 insertions(+), 72 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java index 6549cac011feb..e5e61aae92d2f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java @@ -18,25 +18,11 @@ package org.apache.spark.network.shuffle; import java.io.File; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import com.google.common.annotations.VisibleForTesting; - -import org.apache.commons.lang3.SystemUtils; import org.apache.spark.network.util.JavaUtils; public class ExecutorDiskUtils { - private static final Pattern MULTIPLE_SEPARATORS; - static { - if (SystemUtils.IS_OS_WINDOWS) { - MULTIPLE_SEPARATORS = Pattern.compile("[/\\\\]+"); - } else { - MULTIPLE_SEPARATORS = Pattern.compile("/{2,}"); - } - } - /** * Hashes a filename into the corresponding local directory, in a manner consistent with * Spark's DiskBlockManager.getFile(). @@ -45,34 +31,16 @@ public static File getFile(String[] localDirs, int subDirsPerLocalDir, String fi int hash = JavaUtils.nonNegativeHash(filename); String localDir = localDirs[hash % localDirs.length]; int subDirId = (hash / localDirs.length) % subDirsPerLocalDir; - return new File(createNormalizedInternedPathname( - localDir, String.format("%02x", subDirId), filename)); - } - - /** - * This method is needed to avoid the situation when multiple File instances for the - * same pathname "foo/bar" are created, each with a separate copy of the "foo/bar" String. - * According to measurements, in some scenarios such duplicate strings may waste a lot - * of memory (~ 10% of the heap). To avoid that, we intern the pathname, and before that - * we make sure that it's in a normalized form (contains no "//", "///" etc.) Otherwise, - * the internal code in java.io.File would normalize it later, creating a new "foo/bar" - * String copy. Unfortunately, we cannot just reuse the normalization code that java.io.File - * uses, since it is in the package-private class java.io.FileSystem. - * - * On Windows, separator "\" is used instead of "/". - * - * "\\" is a legal character in path name on Unix-like OS, but illegal on Windows. - */ - @VisibleForTesting - static String createNormalizedInternedPathname(String dir1, String dir2, String fname) { - String pathname = dir1 + File.separator + dir2 + File.separator + fname; - Matcher m = MULTIPLE_SEPARATORS.matcher(pathname); - pathname = m.replaceAll(Matcher.quoteReplacement(File.separator)); - // A single trailing slash needs to be taken care of separately - if (pathname.length() > 1 && pathname.charAt(pathname.length() - 1) == File.separatorChar) { - pathname = pathname.substring(0, pathname.length() - 1); - } - return pathname.intern(); + final String notNormalizedPath = + localDir + File.separator + String.format("%02x", subDirId) + File.separator + filename; + // Interning the normalized path as according to measurements, in some scenarios such + // duplicate strings may waste a lot of memory (~ 10% of the heap). + // Unfortunately, we cannot just call the normalization code that java.io.File + // uses, since it is in the package-private class java.io.FileSystem. + // So we are creating a File just to get the normalized path back to intern it. + // Finally a new File is built and returned with this interned normalized path. + final String normalizedInternedPath = new File(notNormalizedPath).getPath().intern(); + return new File(normalizedInternedPath); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index ba1a17bf7e5ea..a6bcbb8850566 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -24,7 +24,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.Executors; -import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.commons.lang3.builder.ToStringBuilder; @@ -71,8 +70,6 @@ public class ExternalShuffleBlockResolver { private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; private static final StoreVersion CURRENT_VERSION = new StoreVersion(1, 0); - private static final Pattern MULTIPLE_SEPARATORS = Pattern.compile(File.separator + "{2,}"); - // Map containing all registered executors' metadata. @VisibleForTesting final ConcurrentMap executors; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 6515b6ca035f7..88bcf43c2371f 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.network.shuffle; -import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -25,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.io.CharStreams; -import org.apache.commons.lang3.SystemUtils; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -145,29 +143,4 @@ public void jsonSerializationOfExecutorRegistration() throws IOException { assertEquals(shuffleInfo, mapper.readValue(legacyShuffleJson, ExecutorShuffleInfo.class)); } - @Test - public void testNormalizeAndInternPathname() { - String sep = File.separator; - String expectedPathname = sep + "foo" + sep + "bar" + sep + "baz"; - assertPathsMatch("/foo", "bar", "baz", expectedPathname); - assertPathsMatch("//foo/", "bar/", "//baz", expectedPathname); - assertPathsMatch("/foo/", "/bar//", "/baz", expectedPathname); - assertPathsMatch("foo", "bar", "baz///", "foo" + sep + "bar" + sep + "baz"); - assertPathsMatch("/", "", "", sep); - assertPathsMatch("/", "/", "/", sep); - if (SystemUtils.IS_OS_WINDOWS) { - assertPathsMatch("/foo\\/", "bar", "baz", expectedPathname); - } else { - assertPathsMatch("/foo\\/", "bar", "baz", sep + "foo\\" + sep + "bar" + sep + "baz"); - } - } - - private void assertPathsMatch(String p1, String p2, String p3, String expectedPathname) { - String normPathname = - ExecutorDiskUtils.createNormalizedInternedPathname(p1, p2, p3); - assertEquals(expectedPathname, normPathname); - File file = new File(normPathname); - String returnedPath = file.getPath(); - assertEquals(normPathname, returnedPath); - } } From 22f9dfb25a547a5bc2fbc82c763117ea448e8b6a Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 12 Jul 2020 00:16:37 +0900 Subject: [PATCH 165/384] [SPARK-32173][SQL] Deduplicate code in FromUTCTimestamp and ToUTCTimestamp ### What changes were proposed in this pull request? - Extract common code from the `FromUTCTimestamp` and `ToUTCTimestamp` expressions to new trait `UTCTimestamp`. - Move `ToUTCTimestamp` closer to `FromUTCTimestamp` ### Why are the changes needed? Code deduplication improves maintainability. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running date-time test suites such as `DateFunctionsSuite` Closes #28998 from MaxGekk/dedup-from-to-utc-timestamp. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- .../expressions/datetimeExpressions.scala | 175 +++++++----------- 1 file changed, 72 insertions(+), 103 deletions(-) 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 551cbc3161cc1..1c379deb8e584 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 @@ -1176,41 +1176,15 @@ case class DateAddInterval( copy(timeZoneId = Option(timeZoneId)) } -/** - * This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function - * takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in UTC, and - * renders that timestamp as a timestamp in the given time zone. - * - * However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not - * timezone-agnostic. So in Spark this function just shift the timestamp value from UTC timezone to - * the given timezone. - * - * This function may return confusing result if the input is a string with timezone, e.g. - * '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp - * according to the timezone in the string, and finally display the result by converting the - * timestamp to string according to the session local timezone. - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(timestamp, timezone) - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1' would yield '2017-07-14 03:40:00.0'.", - examples = """ - Examples: - > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); - 2016-08-31 09:00:00 - """, - group = "datetime_funcs", - since = "1.5.0") -// scalastyle:on line.size.limit -case class FromUTCTimestamp(left: Expression, right: Expression) - extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { +sealed trait UTCTimestamp extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { + val func: (Long, String) => Long + val funcName: String override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) override def dataType: DataType = TimestampType - override def prettyName: String = "from_utc_timestamp" override def nullSafeEval(time: Any, timezone: Any): Any = { - DateTimeUtils.fromUTCTime(time.asInstanceOf[Long], - timezone.asInstanceOf[UTF8String].toString) + func(time.asInstanceOf[Long], timezone.asInstanceOf[UTF8String].toString) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -1229,24 +1203,90 @@ case class FromUTCTimestamp(left: Expression, right: Expression) val tzTerm = ctx.addMutableState(tzClass, "tz", v => s"""$v = $dtu.getZoneId("$escapedTz");""") val utcTerm = "java.time.ZoneOffset.UTC" + val (fromTz, toTz) = this match { + case _: FromUTCTimestamp => (utcTerm, tzTerm) + case _: ToUTCTimestamp => (tzTerm, utcTerm) + } val eval = left.genCode(ctx) ev.copy(code = code""" |${eval.code} |boolean ${ev.isNull} = ${eval.isNull}; |long ${ev.value} = 0; |if (!${ev.isNull}) { - | ${ev.value} = $dtu.convertTz(${eval.value}, $utcTerm, $tzTerm); + | ${ev.value} = $dtu.convertTz(${eval.value}, $fromTz, $toTz); |} """.stripMargin) } } else { defineCodeGen(ctx, ev, (timestamp, format) => { - s"""$dtu.fromUTCTime($timestamp, $format.toString())""" + s"""$dtu.$funcName($timestamp, $format.toString())""" }) } } } +/** + * This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function + * takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in UTC, and + * renders that timestamp as a timestamp in the given time zone. + * + * However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not + * timezone-agnostic. So in Spark this function just shift the timestamp value from UTC timezone to + * the given timezone. + * + * This function may return confusing result if the input is a string with timezone, e.g. + * '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp + * according to the timezone in the string, and finally display the result by converting the + * timestamp to string according to the session local timezone. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(timestamp, timezone) - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1' would yield '2017-07-14 03:40:00.0'.", + examples = """ + Examples: + > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); + 2016-08-31 09:00:00 + """, + group = "datetime_funcs", + since = "1.5.0") +// scalastyle:on line.size.limit +case class FromUTCTimestamp(left: Expression, right: Expression) extends UTCTimestamp { + override val func = DateTimeUtils.fromUTCTime + override val funcName: String = "fromUTCTime" + override val prettyName: String = "from_utc_timestamp" +} + +/** + * This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function + * takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in the given + * timezone, and renders that timestamp as a timestamp in UTC. + * + * However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not + * timezone-agnostic. So in Spark this function just shift the timestamp value from the given + * timezone to UTC timezone. + * + * This function may return confusing result if the input is a string with timezone, e.g. + * '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp + * according to the timezone in the string, and finally display the result by converting the + * timestamp to string according to the session local timezone. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(timestamp, timezone) - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in the given time zone, and renders that time as a timestamp in UTC. For example, 'GMT+1' would yield '2017-07-14 01:40:00.0'.", + examples = """ + Examples: + > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); + 2016-08-30 15:00:00 + """, + group = "datetime_funcs", + since = "1.5.0") +// scalastyle:on line.size.limit +case class ToUTCTimestamp(left: Expression, right: Expression) extends UTCTimestamp { + override val func = DateTimeUtils.toUTCTime + override val funcName: String = "toUTCTime" + override val prettyName: String = "to_utc_timestamp" +} + /** * Returns the date that is num_months after start_date. */ @@ -1349,77 +1389,6 @@ case class MonthsBetween( override def prettyName: String = "months_between" } -/** - * This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function - * takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in the given - * timezone, and renders that timestamp as a timestamp in UTC. - * - * However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not - * timezone-agnostic. So in Spark this function just shift the timestamp value from the given - * timezone to UTC timezone. - * - * This function may return confusing result if the input is a string with timezone, e.g. - * '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp - * according to the timezone in the string, and finally display the result by converting the - * timestamp to string according to the session local timezone. - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(timestamp, timezone) - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in the given time zone, and renders that time as a timestamp in UTC. For example, 'GMT+1' would yield '2017-07-14 01:40:00.0'.", - examples = """ - Examples: - > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); - 2016-08-30 15:00:00 - """, - group = "datetime_funcs", - since = "1.5.0") -// scalastyle:on line.size.limit -case class ToUTCTimestamp(left: Expression, right: Expression) - extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) - override def dataType: DataType = TimestampType - override def prettyName: String = "to_utc_timestamp" - - override def nullSafeEval(time: Any, timezone: Any): Any = { - DateTimeUtils.toUTCTime(time.asInstanceOf[Long], - timezone.asInstanceOf[UTF8String].toString) - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - if (right.foldable) { - val tz = right.eval().asInstanceOf[UTF8String] - if (tz == null) { - ev.copy(code = code""" - |boolean ${ev.isNull} = true; - |long ${ev.value} = 0; - """.stripMargin) - } else { - val tzClass = classOf[ZoneId].getName - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val escapedTz = StringEscapeUtils.escapeJava(tz.toString) - val tzTerm = ctx.addMutableState(tzClass, "tz", - v => s"""$v = $dtu.getZoneId("$escapedTz");""") - val utcTerm = "java.time.ZoneOffset.UTC" - val eval = left.genCode(ctx) - ev.copy(code = code""" - |${eval.code} - |boolean ${ev.isNull} = ${eval.isNull}; - |long ${ev.value} = 0; - |if (!${ev.isNull}) { - | ${ev.value} = $dtu.convertTz(${eval.value}, $tzTerm, $utcTerm); - |} - """.stripMargin) - } - } else { - defineCodeGen(ctx, ev, (timestamp, format) => { - s"""$dtu.toUTCTime($timestamp, $format.toString())""" - }) - } - } -} - /** * Parses a column to a date based on the given format. */ From 99b4b062555329d5da968ad5dbd9e2b22a193a55 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 11 Jul 2020 10:37:26 -0500 Subject: [PATCH 166/384] [SPARK-32232][ML][PYSPARK] Make sure ML has the same default solver values between Scala and Python # What changes were proposed in this pull request? current problems: ``` mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], seed=123) model = mlp.fit(df) path = tempfile.mkdtemp() model_path = path + "/mlp" model.save(model_path) model2 = MultilayerPerceptronClassificationModel.load(model_path) self.assertEqual(model2.getSolver(), "l-bfgs") # this fails because model2.getSolver() returns 'auto' model2.transform(df) # this fails with Exception pyspark.sql.utils.IllegalArgumentException: MultilayerPerceptronClassifier_dec859ed24ec parameter solver given invalid value auto. ``` FMClassifier/Regression and GeneralizedLinearRegression have the same problems. Here are the root cause of the problems: 1. In HasSolver, both Scala and Python default solver to 'auto' 2. On Scala side, mlp overrides the default of solver to 'l-bfgs', FMClassifier/Regression overrides the default of solver to 'adamW', and glr overrides the default of solver to 'irls' 3. On Scala side, mlp overrides the default of solver in MultilayerPerceptronClassificationParams, so both MultilayerPerceptronClassification and MultilayerPerceptronClassificationModel have 'l-bfgs' as default 4. On Python side, mlp overrides the default of solver in MultilayerPerceptronClassification, so it has default as 'l-bfgs', but MultilayerPerceptronClassificationModel doesn't override the default so it gets the default from HasSolver which is 'auto'. In theory, we don't care about the solver value or any other params values for MultilayerPerceptronClassificationModel, because we have the fitted model already. That's why on Python side, we never set default values for any of the XXXModel. 5. when calling getSolver on the loaded mlp model, it calls this line of code underneath: ``` def _transfer_params_from_java(self): """ Transforms the embedded params from the companion Java object. """ ...... # SPARK-14931: Only check set params back to avoid default params mismatch. if self._java_obj.isSet(java_param): value = _java2py(sc, self._java_obj.getOrDefault(java_param)) self._set(**{param.name: value}) ...... ``` that's why model2.getSolver() returns 'auto'. The code doesn't get the default Scala value (in this case 'l-bfgs') to set to Python param, so it takes the default value (in this case 'auto') on Python side. 6. when calling model2.transform(df), it calls this underneath: ``` def _transfer_params_to_java(self): """ Transforms the embedded params to the companion Java object. """ ...... if self.hasDefault(param): pair = self._make_java_param_pair(param, self._defaultParamMap[param]) pair_defaults.append(pair) ...... ``` Again, it gets the Python default solver which is 'auto', and this caused the Exception 7. Currently, on Scala side, for some of the algorithms, we set default values in the XXXParam, so both estimator and transformer get the default value. However, for some of the algorithms, we only set default in estimators, and the XXXModel doesn't get the default value. On Python side, we never set defaults for the XXXModel. This causes the default value inconsistency. 8. My proposed solution: set default params in XXXParam for both Scala and Python, so both the estimator and transformer have the same default value for both Scala and Python. I currently only changed solver in this PR. If everyone is OK with the fix, I will change all the other params as well. I hope my explanation makes sense to your folks :) ### Why are the changes needed? Fix bug ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? existing and new tests Closes #29060 from huaxingao/solver_parity. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../spark/ml/regression/FMRegressor.scala | 14 ++-- .../GeneralizedLinearRegression.scala | 9 +-- python/pyspark/ml/classification.py | 8 +-- python/pyspark/ml/regression.py | 16 +++-- python/pyspark/ml/tests/test_persistence.py | 65 ++++++++++++++++++- 5 files changed, 84 insertions(+), 28 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala index df4dac1e240e2..b9307ebb37ac2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala @@ -112,6 +112,10 @@ private[ml] trait FactorizationMachinesParams extends PredictorParams "The solver algorithm for optimization. Supported options: " + s"${supportedSolvers.mkString(", ")}. (Default adamW)", ParamValidators.inArray[String](supportedSolvers)) + + setDefault(factorSize -> 8, fitIntercept -> true, fitLinear -> true, regParam -> 0.0, + miniBatchFraction -> 1.0, initStd -> 0.01, maxIter -> 100, stepSize -> 1.0, tol -> 1E-6, + solver -> AdamW) } private[ml] trait FactorizationMachines extends FactorizationMachinesParams { @@ -308,7 +312,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setFactorSize(value: Int): this.type = set(factorSize, value) - setDefault(factorSize -> 8) /** * Set whether to fit intercept term. @@ -318,7 +321,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Set whether to fit linear term. @@ -328,7 +330,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setFitLinear(value: Boolean): this.type = set(fitLinear, value) - setDefault(fitLinear -> true) /** * Set the L2 regularization parameter. @@ -338,7 +339,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set the mini-batch fraction parameter. @@ -348,7 +348,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setMiniBatchFraction(value: Double): this.type = set(miniBatchFraction, value) - setDefault(miniBatchFraction -> 1.0) /** * Set the standard deviation of initial coefficients. @@ -358,7 +357,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setInitStd(value: Double): this.type = set(initStd, value) - setDefault(initStd -> 0.01) /** * Set the maximum number of iterations. @@ -368,7 +366,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the initial step size for the first step (like learning rate). @@ -378,7 +375,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setStepSize(value: Double): this.type = set(stepSize, value) - setDefault(stepSize -> 1.0) /** * Set the convergence tolerance of iterations. @@ -388,7 +384,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Set the solver algorithm used for optimization. @@ -399,7 +394,6 @@ class FMRegressor @Since("3.0.0") ( */ @Since("3.0.0") def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> AdamW) /** * Set the random seed for weight initialization. diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 8336df8e34ae0..f7dfda81d4e6f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -181,6 +181,9 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam s"${supportedSolvers.mkString(", ")}. (Default irls)", ParamValidators.inArray[String](supportedSolvers)) + setDefault(family -> Gaussian.name, variancePower -> 0.0, maxIter -> 25, tol -> 1E-6, + regParam -> 0.0, solver -> IRLS) + @Since("2.0.0") override def validateAndTransformSchema( schema: StructType, @@ -257,7 +260,6 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val */ @Since("2.0.0") def setFamily(value: String): this.type = set(family, value) - setDefault(family -> Gaussian.name) /** * Sets the value of param [[variancePower]]. @@ -268,7 +270,6 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val */ @Since("2.2.0") def setVariancePower(value: Double): this.type = set(variancePower, value) - setDefault(variancePower -> 0.0) /** * Sets the value of param [[linkPower]]. @@ -305,7 +306,6 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val */ @Since("2.0.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 25) /** * Sets the convergence tolerance of iterations. @@ -316,7 +316,6 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val */ @Since("2.0.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Sets the regularization parameter for L2 regularization. @@ -332,7 +331,6 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val */ @Since("2.0.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Sets the value of param [[weightCol]]. @@ -364,7 +362,6 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val */ @Since("2.0.0") def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> IRLS) /** * Sets the link prediction (linear predictor) column name. diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d70932a1bc6fc..cc8ce0567bd7f 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -2421,6 +2421,10 @@ class _MultilayerPerceptronParams(_ProbabilisticClassifierParams, HasSeed, HasMa initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", typeConverter=TypeConverters.toVector) + def __init__(self): + super(_MultilayerPerceptronParams, self).__init__() + self._setDefault(maxIter=100, tol=1E-6, blockSize=128, stepSize=0.03, solver="l-bfgs") + @since("1.6.0") def getLayers(self): """ @@ -2524,7 +2528,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(MultilayerPerceptronClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.MultilayerPerceptronClassifier", self.uid) - self._setDefault(maxIter=100, tol=1E-6, blockSize=128, stepSize=0.03, solver="l-bfgs") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -3120,9 +3123,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(FMClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.FMClassifier", self.uid) - self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, - tol=1e-6, solver="adamW") kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index b58255ea12afc..e82a35c8e78f1 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1891,6 +1891,11 @@ class _GeneralizedLinearRegressionParams(_PredictorParams, HasFitIntercept, HasM "or empty, we treat all instance offsets as 0.0", typeConverter=TypeConverters.toString) + def __init__(self): + super(_GeneralizedLinearRegressionParams, self).__init__() + self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls", + variancePower=0.0, aggregationDepth=2) + @since("2.0.0") def getFamily(self): """ @@ -2023,8 +2028,6 @@ def __init__(self, labelCol="label", featuresCol="features", predictionCol="pred super(GeneralizedLinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.GeneralizedLinearRegression", self.uid) - self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls", - variancePower=0.0, aggregationDepth=2) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2398,6 +2401,12 @@ class _FactorizationMachinesParams(_PredictorParams, HasMaxIter, HasStepSize, Ha solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) + def __init__(self): + super(_FactorizationMachinesParams, self).__init__() + self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, + tol=1e-6, solver="adamW") + @since("3.0.0") def getFactorSize(self): """ @@ -2489,9 +2498,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(FMRegressor, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.FMRegressor", self.uid) - self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, - tol=1e-6, solver="adamW") kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/tests/test_persistence.py b/python/pyspark/ml/tests/test_persistence.py index d4edcc26e17ac..2f6d451851b4b 100644 --- a/python/pyspark/ml/tests/test_persistence.py +++ b/python/pyspark/ml/tests/test_persistence.py @@ -21,19 +21,78 @@ import unittest from pyspark.ml import Transformer -from pyspark.ml.classification import DecisionTreeClassifier, LogisticRegression, OneVsRest, \ - OneVsRestModel +from pyspark.ml.classification import DecisionTreeClassifier, FMClassifier, \ + FMClassificationModel, LogisticRegression, MultilayerPerceptronClassifier, \ + MultilayerPerceptronClassificationModel, OneVsRest, OneVsRestModel from pyspark.ml.clustering import KMeans from pyspark.ml.feature import Binarizer, HashingTF, PCA from pyspark.ml.linalg import Vectors from pyspark.ml.param import Params from pyspark.ml.pipeline import Pipeline, PipelineModel -from pyspark.ml.regression import DecisionTreeRegressor, LinearRegression +from pyspark.ml.regression import DecisionTreeRegressor, GeneralizedLinearRegression, \ + GeneralizedLinearRegressionModel, \ + LinearRegression from pyspark.ml.util import DefaultParamsReadable, DefaultParamsWriter from pyspark.ml.wrapper import JavaParams from pyspark.testing.mlutils import MockUnaryTransformer, SparkSessionTestCase +class TestDefaultSolver(SparkSessionTestCase): + + def test_multilayer_load(self): + df = self.spark.createDataFrame([(0.0, Vectors.dense([0.0, 0.0])), + (1.0, Vectors.dense([0.0, 1.0])), + (1.0, Vectors.dense([1.0, 0.0])), + (0.0, Vectors.dense([1.0, 1.0]))], + ["label", "features"]) + + mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], seed=123) + model = mlp.fit(df) + self.assertEqual(model.getSolver(), "l-bfgs") + transformed1 = model.transform(df) + path = tempfile.mkdtemp() + model_path = path + "/mlp" + model.save(model_path) + model2 = MultilayerPerceptronClassificationModel.load(model_path) + self.assertEqual(model2.getSolver(), "l-bfgs") + transformed2 = model2.transform(df) + self.assertEqual(transformed1.take(4), transformed2.take(4)) + + def test_fm_load(self): + df = self.spark.createDataFrame([(1.0, Vectors.dense(1.0)), + (0.0, Vectors.sparse(1, [], []))], + ["label", "features"]) + fm = FMClassifier(factorSize=2, maxIter=50, stepSize=2.0) + model = fm.fit(df) + self.assertEqual(model.getSolver(), "adamW") + transformed1 = model.transform(df) + path = tempfile.mkdtemp() + model_path = path + "/fm" + model.save(model_path) + model2 = FMClassificationModel.load(model_path) + self.assertEqual(model2.getSolver(), "adamW") + transformed2 = model2.transform(df) + self.assertEqual(transformed1.take(2), transformed2.take(2)) + + def test_glr_load(self): + df = self.spark.createDataFrame([(1.0, Vectors.dense(0.0, 0.0)), + (1.0, Vectors.dense(1.0, 2.0)), + (2.0, Vectors.dense(0.0, 0.0)), + (2.0, Vectors.dense(1.0, 1.0))], + ["label", "features"]) + glr = GeneralizedLinearRegression(family="gaussian", link="identity", linkPredictionCol="p") + model = glr.fit(df) + self.assertEqual(model.getSolver(), "irls") + transformed1 = model.transform(df) + path = tempfile.mkdtemp() + model_path = path + "/glr" + model.save(model_path) + model2 = GeneralizedLinearRegressionModel.load(model_path) + self.assertEqual(model2.getSolver(), "irls") + transformed2 = model2.transform(df) + self.assertEqual(transformed1.take(4), transformed2.take(4)) + + class PersistenceTest(SparkSessionTestCase): def test_linear_regression(self): From 10a65ee9b42fd544e4698267e0cd16711ed92104 Mon Sep 17 00:00:00 2001 From: William Hyun Date: Sun, 12 Jul 2020 00:45:48 +0900 Subject: [PATCH 167/384] [SPARK-32150][BUILD] Upgrade to ZStd 1.4.5-4 ### What changes were proposed in this pull request? This PR aims to upgrade to ZStd 1.4.5-4. ### Why are the changes needed? ZStd 1.4.5-4 fixes the following. - https://github.com/luben/zstd-jni/commit/3d16e515253a0b6941810ab3ee4055201ca9ff54 - https://github.com/luben/zstd-jni/commit/3d51bdcb82c0003f943416cae7aba3f3001b1163 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #28969 from williamhyun/zstd2. Authored-by: William Hyun Signed-off-by: HyukjinKwon --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 2 +- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index f8a43488d0f7f..344806e447689 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -208,4 +208,4 @@ xmlenc/0.52//xmlenc-0.52.jar xz/1.5//xz-1.5.jar zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar zookeeper/3.4.14//zookeeper-3.4.14.jar -zstd-jni/1.4.5-2//zstd-jni-1.4.5-2.jar +zstd-jni/1.4.5-4//zstd-jni-1.4.5-4.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index a34970b3c9d1d..969249b963e7b 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -222,4 +222,4 @@ xmlenc/0.52//xmlenc-0.52.jar xz/1.5//xz-1.5.jar zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar zookeeper/3.4.14//zookeeper-3.4.14.jar -zstd-jni/1.4.5-2//zstd-jni-1.4.5-2.jar +zstd-jni/1.4.5-4//zstd-jni-1.4.5-4.jar 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 c8fade45739c0..e98e4676107ed 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -236,4 +236,4 @@ xbean-asm7-shaded/4.15//xbean-asm7-shaded-4.15.jar xz/1.5//xz-1.5.jar zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar zookeeper/3.4.14//zookeeper-3.4.14.jar -zstd-jni/1.4.5-2//zstd-jni-1.4.5-2.jar +zstd-jni/1.4.5-4//zstd-jni-1.4.5-4.jar diff --git a/pom.xml b/pom.xml index 08ca13bfe9d37..8b3ba305edda7 100644 --- a/pom.xml +++ b/pom.xml @@ -669,7 +669,7 @@ com.github.luben zstd-jni - 1.4.5-2 + 1.4.5-4 com.clearspring.analytics From b84ed4146d93b37adb2b83ca642c7978a1ac853e Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 11 Jul 2020 13:09:06 -0700 Subject: [PATCH 168/384] [SPARK-32245][INFRA] Run Spark tests in Github Actions ### What changes were proposed in this pull request? This PR aims to run the Spark tests in Github Actions. To briefly explain the main idea: - Reuse `dev/run-tests.py` with SBT build - Reuse the modules in `dev/sparktestsupport/modules.py` to test each module - Pass the modules to test into `dev/run-tests.py` directly via `TEST_ONLY_MODULES` environment variable. For example, `pyspark-sql,core,sql,hive`. - `dev/run-tests.py` _does not_ take the dependent modules into account but solely the specified modules to test. Another thing to note might be `SlowHiveTest` annotation. Running the tests in Hive modules takes too much so the slow tests are extracted and it runs as a separate job. It was extracted from the actual elapsed time in Jenkins: ![Screen Shot 2020-07-09 at 7 48 13 PM](https://user-images.githubusercontent.com/6477701/87050238-f6098e80-c238-11ea-9c4a-ab505af61381.png) So, Hive tests are separated into to jobs. One is slow test cases, and the other one is the other test cases. _Note that_ the current GitHub Actions build virtually copies what the default PR builder on Jenkins does (without other profiles such as JDK 11, Hadoop 2, etc.). The only exception is Kinesis https://github.com/apache/spark/pull/29057/files#diff-04eb107ee163a50b61281ca08f4e4c7bR23 ### Why are the changes needed? Last week and onwards, the Jenkins machines became very unstable for many reasons: - Apparently, the machines became extremely slow. Almost all tests can't pass. - One machine (worker 4) started to have the corrupt `.m2` which fails the build. - Documentation build fails time to time for an unknown reason in Jenkins machine specifically. This is disabled for now at https://github.com/apache/spark/pull/29017. - Almost all PRs are basically blocked by this instability currently. The advantages of using Github Actions: - To avoid depending on few persons who can access to the cluster. - To reduce the elapsed time in the build - we could split the tests (e.g., SQL, ML, CORE), and run them in parallel so the total build time will significantly reduce. - To control the environment more flexibly. - Other contributors can test and propose to fix Github Actions configurations so we can distribute this build management cost. Note that: - The current build in Jenkins takes _more than 7 hours_. With Github actions it takes _less than 2 hours_ - We can now control the environments especially for Python easily. - The test and build look more stable than the Jenkins'. ### Does this PR introduce _any_ user-facing change? No, dev-only change. ### How was this patch tested? Tested at https://github.com/HyukjinKwon/spark/pull/4 Closes #29057 from HyukjinKwon/migrate-to-github-actions. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 283 +++++++++++------- .../org/apache/spark/tags/SlowHiveTest.java | 30 ++ .../spark/deploy/master/MasterSuite.scala | 3 +- dev/run-pip-tests | 11 +- dev/run-tests.py | 89 ++++-- dev/sparktestsupport/modules.py | 93 ++++-- project/SparkBuild.scala | 11 + python/pyspark/sql/tests/test_arrow.py | 9 +- python/pyspark/sql/tests/test_types.py | 3 +- .../pyspark/streaming/tests/test_dstream.py | 15 +- python/run-tests.py | 2 +- .../execution/HiveCompatibilitySuite.scala | 2 + .../HiveExternalCatalogVersionsSuite.scala | 3 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 3 +- .../spark/sql/hive/client/VersionsSuite.scala | 3 +- .../execution/AggregationQuerySuite.scala | 2 + .../sql/hive/execution/HiveDDLSuite.scala | 3 + .../sql/hive/execution/HiveQuerySuite.scala | 2 + .../hive/execution/Hive_2_1_DDLSuite.scala | 3 +- .../sql/hive/execution/SQLQuerySuite.scala | 3 + 20 files changed, 400 insertions(+), 173 deletions(-) create mode 100644 common/tags/src/test/java/org/apache/spark/tags/SlowHiveTest.java diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 4282504cc3984..bf2feee9077d4 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1,156 +1,227 @@ name: master on: - push: - branches: - - master pull_request: branches: - master jobs: + # TODO(SPARK-32248): Recover JDK 11 builds + # Build: build Spark and run the tests for specified modules. build: - + name: "Build modules: ${{ matrix.modules }} ${{ matrix.comment }} (JDK ${{ matrix.java }}, ${{ matrix.hadoop }}, ${{ matrix.hive }})" runs-on: ubuntu-latest strategy: + fail-fast: false matrix: - java: [ '1.8', '11' ] - hadoop: [ 'hadoop-2.7', 'hadoop-3.2' ] - hive: [ 'hive-1.2', 'hive-2.3' ] - exclude: - - java: '11' - hive: 'hive-1.2' - - hadoop: 'hadoop-3.2' - hive: 'hive-1.2' - name: Build Spark - JDK${{ matrix.java }}/${{ matrix.hadoop }}/${{ matrix.hive }} - + java: + - 1.8 + hadoop: + - hadoop3.2 + hive: + - hive2.3 + # TODO(SPARK-32246): We don't test 'streaming-kinesis-asl' for now. + # Kinesis tests depends on external Amazon kinesis service. + # Note that the modules below are from sparktestsupport/modules.py. + modules: + - |- + core, unsafe, kvstore, avro, + network_common, network_shuffle, repl, launcher + examples, sketch, graphx + - |- + catalyst, hive-thriftserver + - |- + streaming, sql-kafka-0-10, streaming-kafka-0-10, + mllib-local, mllib, + yarn, mesos, kubernetes, hadoop-cloud, spark-ganglia-lgpl + - |- + pyspark-sql, pyspark-mllib, pyspark-resource + - |- + pyspark-core, pyspark-streaming, pyspark-ml + - |- + sparkr + # Here, we split Hive and SQL tests into some of slow ones and the rest of them. + included-tags: [""] + excluded-tags: [""] + comment: [""] + include: + # Hive tests + - modules: hive + java: 1.8 + hadoop: hadoop3.2 + hive: hive2.3 + included-tags: org.apache.spark.tags.SlowHiveTest + comment: "- slow tests" + - modules: hive + java: 1.8 + hadoop: hadoop3.2 + hive: hive2.3 + excluded-tags: org.apache.spark.tags.SlowHiveTest + comment: "- other tests" + # SQL tests + - modules: sql + java: 1.8 + hadoop: hadoop3.2 + hive: hive2.3 + included-tags: org.apache.spark.tags.ExtendedSQLTest + comment: "- slow tests" + - modules: sql + java: 1.8 + hadoop: hadoop3.2 + hive: hive2.3 + excluded-tags: org.apache.spark.tags.ExtendedSQLTest + comment: "- other tests" + env: + TEST_ONLY_MODULES: ${{ matrix.modules }} + TEST_ONLY_EXCLUDED_TAGS: ${{ matrix.excluded-tags }} + TEST_ONLY_INCLUDED_TAGS: ${{ matrix.included-tags }} + HADOOP_PROFILE: ${{ matrix.hadoop }} + HIVE_PROFILE: ${{ matrix.hive }} + # GitHub Actions' default miniconda to use in pip packaging test. + CONDA_PREFIX: /usr/share/miniconda steps: - - uses: actions/checkout@master - # We split caches because GitHub Action Cache has a 400MB-size limit. - - uses: actions/cache@v1 + - name: Checkout Spark repository + uses: actions/checkout@v2 + # Cache local repositories. Note that GitHub Actions cache has a 2G limit. + - name: Cache Scala, SBT, Maven and Zinc + uses: actions/cache@v1 with: path: build key: build-${{ hashFiles('**/pom.xml') }} restore-keys: | build- - - uses: actions/cache@v1 - with: - path: ~/.m2/repository/com - key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-com-${{ hashFiles('**/pom.xml') }} - restore-keys: | - ${{ matrix.java }}-${{ matrix.hadoop }}-maven-com- - - uses: actions/cache@v1 + - name: Cache Maven local repository + uses: actions/cache@v2 with: - path: ~/.m2/repository/org - key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-org-${{ hashFiles('**/pom.xml') }} - restore-keys: | - ${{ matrix.java }}-${{ matrix.hadoop }}-maven-org- - - uses: actions/cache@v1 - with: - path: ~/.m2/repository/net - key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-net-${{ hashFiles('**/pom.xml') }} + path: ~/.m2/repository + key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-${{ hashFiles('**/pom.xml') }} restore-keys: | - ${{ matrix.java }}-${{ matrix.hadoop }}-maven-net- - - uses: actions/cache@v1 + ${{ matrix.java }}-${{ matrix.hadoop }}-maven- + - name: Cache Ivy local repository + uses: actions/cache@v2 with: - path: ~/.m2/repository/io - key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-io-${{ hashFiles('**/pom.xml') }} + path: ~/.ivy2/cache + key: ${{ matrix.java }}-${{ matrix.hadoop }}-ivy-${{ hashFiles('**/pom.xml') }}-${{ hashFiles('**/plugins.sbt') }} restore-keys: | - ${{ matrix.java }}-${{ matrix.hadoop }}-maven-io- - - name: Set up JDK ${{ matrix.java }} + ${{ matrix.java }}-${{ matrix.hadoop }}-ivy- + - name: Install JDK ${{ matrix.java }} uses: actions/setup-java@v1 with: java-version: ${{ matrix.java }} - - name: Build with Maven - run: | - export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" - export MAVEN_CLI_OPTS="--no-transfer-progress" - mkdir -p ~/.m2 - ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -P${{ matrix.hive }} -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install - rm -rf ~/.m2/repository/org/apache/spark - - - lint: - runs-on: ubuntu-latest - name: Linters (Java/Scala/Python), licenses, dependencies - steps: - - uses: actions/checkout@master - - uses: actions/setup-java@v1 + # PySpark + - name: Install PyPy3 + # SQL component also has Python related tests, for example, IntegratedUDFTestUtils. + # Note that order of Python installations here matters because default python3 is + # overridden by pypy3. + uses: actions/setup-python@v2 + if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) with: - java-version: '11' - - uses: actions/setup-python@v1 + python-version: pypy3 + architecture: x64 + - name: Install Python 2.7 + uses: actions/setup-python@v2 + if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) with: - python-version: '3.x' - architecture: 'x64' - - name: Scala - run: ./dev/lint-scala - - name: Java - run: ./dev/lint-java - - name: Python - run: | - pip install flake8 sphinx numpy - ./dev/lint-python - - name: License - run: ./dev/check-license - - name: Dependencies - run: ./dev/test-dependencies.sh - - lintr: - runs-on: ubuntu-latest - name: Linter (R) - steps: - - uses: actions/checkout@master - - uses: actions/setup-java@v1 + python-version: 2.7 + architecture: x64 + - name: Install Python 3.6 + uses: actions/setup-python@v2 + if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) with: - java-version: '11' - - uses: r-lib/actions/setup-r@v1 + python-version: 3.6 + architecture: x64 + - name: Install Python packages + if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) + # PyArrow is not supported in PyPy yet, see ARROW-2651. + # TODO(SPARK-32247): scipy installation with PyPy fails for an unknown reason. + run: | + python3 -m pip install numpy pyarrow pandas scipy + python3 -m pip list + python2 -m pip install numpy pyarrow pandas scipy + python2 -m pip list + pypy3 -m pip install numpy pandas + pypy3 -m pip list + # SparkR + - name: Install R 3.6 + uses: r-lib/actions/setup-r@v1 + if: contains(matrix.modules, 'sparkr') with: - r-version: '3.6.2' - - name: Install lib + r-version: 3.6 + - name: Install R packages + if: contains(matrix.modules, 'sparkr') run: | sudo apt-get install -y libcurl4-openssl-dev - - name: install R packages + sudo Rscript -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'devtools', 'e1071', 'survival', 'arrow', 'roxygen2'), repos='https://cloud.r-project.org/')" + # Show installed packages in R. + sudo Rscript -e 'pkg_list <- as.data.frame(installed.packages()[, c(1,3:4)]); pkg_list[is.na(pkg_list$Priority), 1:2, drop = FALSE]' + # Run the tests. + - name: "Run tests: ${{ matrix.modules }}" run: | - sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" - sudo Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')" - - name: package and install SparkR - run: ./R/install-dev.sh - - name: lint-r - run: ./dev/lint-r + # Hive tests become flaky when running in parallel as it's too intensive. + if [[ "$TEST_ONLY_MODULES" == "hive" ]]; then export SERIAL_SBT_TESTS=1; fi + mkdir -p ~/.m2 + ./dev/run-tests --parallelism 2 + rm -rf ~/.m2/repository/org/apache/spark - docs: + # Static analysis, and documentation build + lint: + name: Linters, licenses, dependencies and documentation generation runs-on: ubuntu-latest - name: Generate documents steps: - - uses: actions/checkout@master - - uses: actions/cache@v1 + - name: Checkout Spark repository + uses: actions/checkout@v2 + - name: Cache Maven local repository + uses: actions/cache@v2 with: path: ~/.m2/repository key: docs-maven-repo-${{ hashFiles('**/pom.xml') }} restore-keys: | - docs-maven-repo- - - uses: actions/setup-java@v1 + docs-maven- + - name: Install JDK 1.8 + uses: actions/setup-java@v1 with: - java-version: '1.8' - - uses: actions/setup-python@v1 + java-version: 1.8 + - name: Install Python 3.6 + uses: actions/setup-python@v2 with: - python-version: '3.x' - architecture: 'x64' - - uses: actions/setup-ruby@v1 + python-version: 3.6 + architecture: x64 + - name: Install Python linter dependencies + run: | + pip3 install flake8 sphinx numpy + - name: Install R 3.6 + uses: r-lib/actions/setup-r@v1 with: - ruby-version: '2.7' - - uses: r-lib/actions/setup-r@v1 + r-version: 3.6 + - name: Install R linter dependencies and SparkR + run: | + sudo apt-get install -y libcurl4-openssl-dev + sudo Rscript -e "install.packages(c('devtools'), repos='https://cloud.r-project.org/')" + sudo Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')" + ./R/install-dev.sh + - name: Install Ruby 2.7 for documentation generation + uses: actions/setup-ruby@v1 with: - r-version: '3.6.2' - - name: Install lib and pandoc + ruby-version: 2.7 + - name: Install dependencies for documentation generation run: | sudo apt-get install -y libcurl4-openssl-dev pandoc - - name: Install packages - run: | pip install sphinx mkdocs numpy gem install jekyll jekyll-redirect-from rouge - sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" - - name: Run jekyll build + sudo Rscript -e "install.packages(c('devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2'), repos='https://cloud.r-project.org/')" + - name: Scala linter + run: ./dev/lint-scala + - name: Java linter + run: ./dev/lint-java + - name: Python linter + run: ./dev/lint-python + - name: R linter + run: ./dev/lint-r + - name: License test + run: ./dev/check-license + - name: Dependencies test + run: ./dev/test-dependencies.sh + - name: Run documentation build run: | cd docs jekyll build diff --git a/common/tags/src/test/java/org/apache/spark/tags/SlowHiveTest.java b/common/tags/src/test/java/org/apache/spark/tags/SlowHiveTest.java new file mode 100644 index 0000000000000..a7e6f352667d7 --- /dev/null +++ b/common/tags/src/test/java/org/apache/spark/tags/SlowHiveTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.tags; + +import org.scalatest.TagAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface SlowHiveTest { } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 0cf573c2490b3..91128af82b022 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -685,7 +685,8 @@ class MasterSuite extends SparkFunSuite } } - test("SPARK-27510: Master should avoid dead loop while launching executor failed in Worker") { + // TODO(SPARK-32250): Enable the test back. It is flaky in GitHub Actions. + ignore("SPARK-27510: Master should avoid dead loop while launching executor failed in Worker") { val master = makeAliveMaster() var worker: MockExecutorLaunchFailWorker = null try { diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 470f21e69d46a..9ebe69dbd476c 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -63,7 +63,7 @@ fi PYSPARK_VERSION=$(python3 -c "exec(open('python/pyspark/version.py').read());print(__version__)") PYSPARK_DIST="$FWDIR/python/dist/pyspark-$PYSPARK_VERSION.tar.gz" # The pip install options we use for all the pip commands -PIP_OPTIONS="--upgrade --no-cache-dir --force-reinstall " +PIP_OPTIONS="--user --upgrade --no-cache-dir --force-reinstall " # Test both regular user and edit/dev install modes. PIP_COMMANDS=("pip install $PIP_OPTIONS $PYSPARK_DIST" "pip install $PIP_OPTIONS -e python/") @@ -76,8 +76,12 @@ for python in "${PYTHON_EXECS[@]}"; do VIRTUALENV_PATH="$VIRTUALENV_BASE"/$python rm -rf "$VIRTUALENV_PATH" if [ -n "$USE_CONDA" ]; then + if [ -f "$CONDA_PREFIX/etc/profile.d/conda.sh" ]; then + # See also https://github.com/conda/conda/issues/7980 + source "$CONDA_PREFIX/etc/profile.d/conda.sh" + fi conda create -y -p "$VIRTUALENV_PATH" python=$python numpy pandas pip setuptools - source activate "$VIRTUALENV_PATH" + conda activate "$VIRTUALENV_PATH" || (echo "Falling back to 'source activate'" && source activate "$VIRTUALENV_PATH") else mkdir -p "$VIRTUALENV_PATH" virtualenv --python=$python "$VIRTUALENV_PATH" @@ -110,6 +114,7 @@ for python in "${PYTHON_EXECS[@]}"; do cd / echo "Run basic sanity check on pip installed version with spark-submit" + export PATH="$(python3 -m site --user-base)/bin:$PATH" spark-submit "$FWDIR"/dev/pip-sanity-check.py echo "Run basic sanity check with import based" python3 "$FWDIR"/dev/pip-sanity-check.py @@ -120,7 +125,7 @@ for python in "${PYTHON_EXECS[@]}"; do # conda / virtualenv environments need to be deactivated differently if [ -n "$USE_CONDA" ]; then - source deactivate + conda deactivate || (echo "Falling back to 'source deactivate'" && source deactivate) else deactivate fi diff --git a/dev/run-tests.py b/dev/run-tests.py index 223072cbe7bfb..03cc3230a65fd 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -112,10 +112,14 @@ def determine_modules_to_test(changed_modules): ['root'] >>> [x.name for x in determine_modules_to_test([modules.build])] ['root'] + >>> [x.name for x in determine_modules_to_test([modules.core])] + ['root'] + >>> [x.name for x in determine_modules_to_test([modules.launcher])] + ['root'] >>> [x.name for x in determine_modules_to_test([modules.graphx])] ['graphx', 'examples'] - >>> x = [x.name for x in determine_modules_to_test([modules.sql])] - >>> x # doctest: +NORMALIZE_WHITESPACE + >>> [x.name for x in determine_modules_to_test([modules.sql])] + ... # doctest: +NORMALIZE_WHITESPACE ['sql', 'avro', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver', 'pyspark-sql', 'repl', 'sparkr', 'pyspark-mllib', 'pyspark-ml'] """ @@ -416,7 +420,7 @@ def run_scala_tests_sbt(test_modules, test_profiles): exec_sbt(profiles_and_goals) -def run_scala_tests(build_tool, extra_profiles, test_modules, excluded_tags): +def run_scala_tests(build_tool, extra_profiles, test_modules, excluded_tags, included_tags): """Function to properly execute all tests passed in as a set from the `determine_test_suites` function""" set_title_and_block("Running Spark unit tests", "BLOCK_SPARK_UNIT_TESTS") @@ -426,6 +430,8 @@ def run_scala_tests(build_tool, extra_profiles, test_modules, excluded_tags): test_profiles = extra_profiles + \ list(set(itertools.chain.from_iterable(m.build_profile_flags for m in test_modules))) + if included_tags: + test_profiles += ['-Dtest.include.tags=' + ",".join(included_tags)] if excluded_tags: test_profiles += ['-Dtest.exclude.tags=' + ",".join(excluded_tags)] @@ -595,7 +601,23 @@ def main(): changed_modules = None changed_files = None - if test_env == "amplab_jenkins" and os.environ.get("AMP_JENKINS_PRB"): + should_only_test_modules = "TEST_ONLY_MODULES" in os.environ + included_tags = [] + if should_only_test_modules: + str_test_modules = [m.strip() for m in os.environ.get("TEST_ONLY_MODULES").split(",")] + test_modules = [m for m in modules.all_modules if m.name in str_test_modules] + # Directly uses test_modules as changed modules to apply tags and environments + # as if all specified test modules are changed. + changed_modules = test_modules + str_excluded_tags = os.environ.get("TEST_ONLY_EXCLUDED_TAGS", None) + str_included_tags = os.environ.get("TEST_ONLY_INCLUDED_TAGS", None) + excluded_tags = [] + if str_excluded_tags: + excluded_tags = [t.strip() for t in str_excluded_tags.split(",")] + included_tags = [] + if str_included_tags: + included_tags = [t.strip() for t in str_included_tags.split(",")] + elif test_env == "amplab_jenkins" and os.environ.get("AMP_JENKINS_PRB"): target_branch = os.environ["ghprbTargetBranch"] changed_files = identify_changed_files_from_git_commits("HEAD", target_branch=target_branch) changed_modules = determine_modules_for_files(changed_files) @@ -616,33 +638,34 @@ def main(): test_environ.update(m.environ) setup_test_environ(test_environ) - test_modules = determine_modules_to_test(changed_modules) - - # license checks - run_apache_rat_checks() - - # style checks - if not changed_files or any(f.endswith(".scala") - or f.endswith("scalastyle-config.xml") - for f in changed_files): - run_scala_style_checks(extra_profiles) should_run_java_style_checks = False - if not changed_files or any(f.endswith(".java") - or f.endswith("checkstyle.xml") - or f.endswith("checkstyle-suppressions.xml") - for f in changed_files): - # Run SBT Checkstyle after the build to prevent a side-effect to the build. - should_run_java_style_checks = True - if not changed_files or any(f.endswith("lint-python") - or f.endswith("tox.ini") - or f.endswith(".py") - for f in changed_files): - run_python_style_checks() - if not changed_files or any(f.endswith(".R") - or f.endswith("lint-r") - or f.endswith(".lintr") - for f in changed_files): - run_sparkr_style_checks() + if not should_only_test_modules: + test_modules = determine_modules_to_test(changed_modules) + + # license checks + run_apache_rat_checks() + + # style checks + if not changed_files or any(f.endswith(".scala") + or f.endswith("scalastyle-config.xml") + for f in changed_files): + run_scala_style_checks(extra_profiles) + if not changed_files or any(f.endswith(".java") + or f.endswith("checkstyle.xml") + or f.endswith("checkstyle-suppressions.xml") + for f in changed_files): + # Run SBT Checkstyle after the build to prevent a side-effect to the build. + should_run_java_style_checks = True + if not changed_files or any(f.endswith("lint-python") + or f.endswith("tox.ini") + or f.endswith(".py") + for f in changed_files): + run_python_style_checks() + if not changed_files or any(f.endswith(".R") + or f.endswith("lint-r") + or f.endswith(".lintr") + for f in changed_files): + run_sparkr_style_checks() # determine if docs were changed and if we're inside the amplab environment # note - the below commented out until *all* Jenkins workers can get `jekyll` installed @@ -664,7 +687,7 @@ def main(): build_spark_assembly_sbt(extra_profiles, should_run_java_style_checks) # run the test suites - run_scala_tests(build_tool, extra_profiles, test_modules, excluded_tags) + run_scala_tests(build_tool, extra_profiles, test_modules, excluded_tags, included_tags) modules_with_python_tests = [m for m in test_modules if m.python_test_goals] if modules_with_python_tests: @@ -679,6 +702,10 @@ def main(): def _test(): + if "TEST_ONLY_MODULES" in os.environ: + # TODO(SPARK-32252): Enable doctests back in Github Actions. + return + import doctest failure_count = doctest.testmod()[0] if failure_count: diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 85e6a1e9fadac..2e8b8b57a5b4f 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -100,9 +100,75 @@ def __hash__(self): ] ) +kvstore = Module( + name="kvstore", + dependencies=[tags], + source_file_regexes=[ + "common/kvstore/", + ], + sbt_test_goals=[ + "kvstore/test", + ], +) + +network_common = Module( + name="network-common", + dependencies=[tags], + source_file_regexes=[ + "common/network-common/", + ], + sbt_test_goals=[ + "network-common/test", + ], +) + +network_shuffle = Module( + name="network-shuffle", + dependencies=[tags], + source_file_regexes=[ + "common/network-shuffle/", + ], + sbt_test_goals=[ + "network-shuffle/test", + ], +) + +unsafe = Module( + name="unsafe", + dependencies=[tags], + source_file_regexes=[ + "common/unsafe", + ], + sbt_test_goals=[ + "unsafe/test", + ], +) + +launcher = Module( + name="launcher", + dependencies=[tags], + source_file_regexes=[ + "launcher/", + ], + sbt_test_goals=[ + "launcher/test", + ], +) + +core = Module( + name="core", + dependencies=[kvstore, network_common, network_shuffle, unsafe, launcher], + source_file_regexes=[ + "core/", + ], + sbt_test_goals=[ + "core/test", + ], +) + catalyst = Module( name="catalyst", - dependencies=[tags], + dependencies=[tags, core], source_file_regexes=[ "sql/catalyst/", ], @@ -111,7 +177,6 @@ def __hash__(self): ], ) - sql = Module( name="sql", dependencies=[catalyst], @@ -123,7 +188,6 @@ def __hash__(self): ], ) - hive = Module( name="hive", dependencies=[sql], @@ -142,7 +206,6 @@ def __hash__(self): ] ) - repl = Module( name="repl", dependencies=[hive], @@ -154,7 +217,6 @@ def __hash__(self): ], ) - hive_thriftserver = Module( name="hive-thriftserver", dependencies=[hive], @@ -192,7 +254,6 @@ def __hash__(self): ] ) - sketch = Module( name="sketch", dependencies=[tags], @@ -204,10 +265,9 @@ def __hash__(self): ] ) - graphx = Module( name="graphx", - dependencies=[tags], + dependencies=[tags, core], source_file_regexes=[ "graphx/", ], @@ -216,10 +276,9 @@ def __hash__(self): ] ) - streaming = Module( name="streaming", - dependencies=[tags], + dependencies=[tags, core], source_file_regexes=[ "streaming", ], @@ -235,7 +294,7 @@ def __hash__(self): # fail other PRs. streaming_kinesis_asl = Module( name="streaming-kinesis-asl", - dependencies=[tags], + dependencies=[tags, core], source_file_regexes=[ "external/kinesis-asl/", "external/kinesis-asl-assembly/", @@ -254,21 +313,23 @@ def __hash__(self): streaming_kafka_0_10 = Module( name="streaming-kafka-0-10", - dependencies=[streaming], + dependencies=[streaming, core], source_file_regexes=[ # The ending "/" is necessary otherwise it will include "sql-kafka" codes "external/kafka-0-10/", "external/kafka-0-10-assembly", + "external/kafka-0-10-token-provider", ], sbt_test_goals=[ "streaming-kafka-0-10/test", + "token-provider-kafka-0-10/test" ] ) mllib_local = Module( name="mllib-local", - dependencies=[tags], + dependencies=[tags, core], source_file_regexes=[ "mllib-local", ], @@ -302,10 +363,9 @@ def __hash__(self): ] ) - pyspark_core = Module( name="pyspark-core", - dependencies=[], + dependencies=[core], source_file_regexes=[ "python/(?!pyspark/(ml|mllib|sql|streaming))" ], @@ -339,7 +399,6 @@ def __hash__(self): ] ) - pyspark_sql = Module( name="pyspark-sql", dependencies=[pyspark_core, hive, avro], @@ -593,7 +652,7 @@ def __hash__(self): # No other modules should directly depend on this module. root = Module( name="root", - dependencies=[build], # Changes to build should trigger all tests. + dependencies=[build, core], # Changes to build should trigger all tests. source_file_regexes=[], # In order to run all of the tests, enable every test profile: build_profile_flags=list(set( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 622fea8c54f13..5184fd85e422f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1016,9 +1016,20 @@ object TestSettings { sys.props.get("test.exclude.tags").map { tags => Seq("--exclude-categories=" + tags) }.getOrElse(Nil): _*), + // Include tags defined in a system property + testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, + sys.props.get("test.include.tags").map { tags => + tags.split(",").flatMap { tag => Seq("-n", tag) }.toSeq + }.getOrElse(Nil): _*), + testOptions in Test += Tests.Argument(TestFrameworks.JUnit, + sys.props.get("test.include.tags").map { tags => + Seq("--include-categories=" + tags) + }.getOrElse(Nil): _*), // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), + // Required to detect Junit tests for each project, see also https://github.com/sbt/junit-interface/issues/35 + crossPaths := false, // Enable Junit testing. libraryDependencies += "com.novocode" % "junit-interface" % "0.11" % "test", // `parallelExecutionInTest` controls whether test suites belonging to the same SBT project diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index 913b43b6ddb5a..a96354e3ecb58 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -21,6 +21,9 @@ import time import unittest import warnings +import sys +if sys.version >= '3': + basestring = unicode = str from pyspark import SparkContext, SparkConf from pyspark.sql import Row, SparkSession @@ -435,12 +438,12 @@ def test_createDateFrame_with_category_type(self): assert_frame_equal(result_spark, result_arrow) # ensure original category elements are string - self.assertIsInstance(category_first_element, str) + self.assertIsInstance(category_first_element, basestring) # spark data frame and arrow execution mode enabled data frame type must match pandas self.assertEqual(spark_type, 'string') self.assertEqual(arrow_type, 'string') - self.assertIsInstance(arrow_first_category_element, str) - self.assertIsInstance(spark_first_category_element, str) + self.assertIsInstance(arrow_first_category_element, basestring) + self.assertIsInstance(spark_first_category_element, basestring) def test_createDataFrame_with_float_index(self): # SPARK-32098: float index should not produce duplicated or truncated Spark DataFrame diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 81402f52af3b3..016cafd669019 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -725,7 +725,8 @@ def assertCollectSuccess(typecode, value): if sys.version_info[0] < 3: all_types = set(['c', 'b', 'B', 'u', 'h', 'H', 'i', 'I', 'l', 'L', 'f', 'd']) else: - all_types = set(array.typecodes) + # PyPy seems not having array.typecodes. + all_types = set(['b', 'B', 'u', 'h', 'H', 'i', 'I', 'l', 'L', 'q', 'Q', 'f', 'd']) unsupported_types = all_types - set(supported_types) # test unsupported types for t in unsupported_types: diff --git a/python/pyspark/streaming/tests/test_dstream.py b/python/pyspark/streaming/tests/test_dstream.py index 7ecdf6b0b12db..89edb23070c69 100644 --- a/python/pyspark/streaming/tests/test_dstream.py +++ b/python/pyspark/streaming/tests/test_dstream.py @@ -30,8 +30,9 @@ @unittest.skipIf( - "pypy" in platform.python_implementation().lower() and "COVERAGE_PROCESS_START" in os.environ, - "PyPy implementation causes to hang DStream tests forever when Coverage report is used.") + "pypy" in platform.python_implementation().lower(), + "The tests fail in PyPy3 implementation for an unknown reason. " + "With PyPy, it causes to hang DStream tests forever when Coverage report is used.") class BasicOperationTests(PySparkStreamingTestCase): def test_map(self): @@ -394,8 +395,9 @@ def failed_func(i): @unittest.skipIf( - "pypy" in platform.python_implementation().lower() and "COVERAGE_PROCESS_START" in os.environ, - "PyPy implementation causes to hang DStream tests forever when Coverage report is used.") + "pypy" in platform.python_implementation().lower(), + "The tests fail in PyPy3 implementation for an unknown reason. " + "With PyPy, it causes to hang DStream tests forever when Coverage report is used.") class WindowFunctionTests(PySparkStreamingTestCase): timeout = 15 @@ -474,8 +476,9 @@ def func(dstream): @unittest.skipIf( - "pypy" in platform.python_implementation().lower() and "COVERAGE_PROCESS_START" in os.environ, - "PyPy implementation causes to hang DStream tests forever when Coverage report is used.") + "pypy" in platform.python_implementation().lower(), + "The tests fail in PyPy3 implementation for an unknown reason. " + "With PyPy, it causes to hang DStream tests forever when Coverage report is used.") class CheckpointTests(unittest.TestCase): setupCalled = False diff --git a/python/run-tests.py b/python/run-tests.py index b677a5134ec93..42510c7642264 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -161,7 +161,7 @@ def run_individual_python_test(target_dir, test_name, pyspark_python): def get_default_python_executables(): - python_execs = [x for x in ["python3.6", "python2.7", "pypy"] if which(x)] + python_execs = [x for x in ["python3.6", "python2.7", "pypy3", "pypy"] if which(x)] if "python3.6" not in python_execs: p = which("python3") diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 82af7dceb27f2..d1ed55a514e7c 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -26,10 +26,12 @@ import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy +import org.apache.spark.tags.SlowHiveTest /** * Runs the test cases that are included in the hive distribution. */ +@SlowHiveTest class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath private lazy val hiveQueryDir = TestHive.getHiveFile( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 8be3d26bfc93a..aa96fa035c4f0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.tags.ExtendedHiveTest +import org.apache.spark.tags.{ExtendedHiveTest, SlowHiveTest} import org.apache.spark.util.Utils /** @@ -46,6 +46,7 @@ import org.apache.spark.util.Utils * expected version under this local directory, e.g. `/tmp/spark-test/spark-2.0.3`, we will skip the * downloading for this spark version. */ +@SlowHiveTest @ExtendedHiveTest class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { private val isTestAtLeastJava9 = SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 8b97489e2d818..3a7e92ee1c00b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -38,12 +38,13 @@ import org.apache.spark.sql.hive.test.{HiveTestJars, TestHiveContext} import org.apache.spark.sql.internal.SQLConf.SHUFFLE_PARTITIONS import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.types.{DecimalType, StructType} -import org.apache.spark.tags.ExtendedHiveTest +import org.apache.spark.tags.{ExtendedHiveTest, SlowHiveTest} import org.apache.spark.util.{ResetSystemProperties, Utils} /** * This suite tests spark-submit with applications using HiveContext. */ +@SlowHiveTest @ExtendedHiveTest class HiveSparkSubmitSuite extends SparkSubmitTestUtils diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 8642a5ff16812..c5c92ddad9014 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.hive.test.TestHiveVersion import org.apache.spark.sql.types.IntegerType import org.apache.spark.sql.types.StructType -import org.apache.spark.tags.ExtendedHiveTest +import org.apache.spark.tags.{ExtendedHiveTest, SlowHiveTest} import org.apache.spark.util.{MutableURLClassLoader, Utils} /** @@ -51,6 +51,7 @@ import org.apache.spark.util.{MutableURLClassLoader, Utils} * is not fully tested. */ // TODO: Refactor this to `HiveClientSuite` and make it a subclass of `HiveVersionSuite` +@SlowHiveTest @ExtendedHiveTest class VersionsSuite extends SparkFunSuite with Logging { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index fac981267f4d7..1dd2ad3837cc3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ +import org.apache.spark.tags.SlowHiveTest import org.apache.spark.unsafe.UnsafeAlignedOffset @@ -1054,6 +1055,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te class HashAggregationQuerySuite extends AggregationQuerySuite +@SlowHiveTest class HashAggregationQueryWithControlledFallbackSuite extends AggregationQuerySuite { override protected def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 6b1c30deec352..f95251a669715 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -45,9 +45,11 @@ import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ +import org.apache.spark.tags.SlowHiveTest import org.apache.spark.util.Utils // TODO(gatorsmile): combine HiveCatalogedDDLSuite and HiveDDLSuite +@SlowHiveTest class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeAndAfterEach { override def afterEach(): Unit = { try { @@ -405,6 +407,7 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA } } +@SlowHiveTest class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { import testImplicits._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index e5628c33b5ec8..cea7c5686054a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.hive.test.{HiveTestJars, TestHive} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.tags.SlowHiveTest case class TestData(a: Int, b: String) @@ -46,6 +47,7 @@ case class TestData(a: Int, b: String) * A set of test cases expressed in Hive QL that are not covered by the tests * included in the hive distribution. */ +@SlowHiveTest class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala index b20ef035594da..6f37e39a532d6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala @@ -27,13 +27,14 @@ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.types._ -import org.apache.spark.tags.ExtendedHiveTest +import org.apache.spark.tags.{ExtendedHiveTest, SlowHiveTest} import org.apache.spark.util.Utils /** * A separate set of DDL tests that uses Hive 2.1 libraries, which behave a little differently * from the built-in ones. */ +@SlowHiveTest @ExtendedHiveTest class Hive_2_1_DDLSuite extends SparkFunSuite with TestHiveSingleton with BeforeAndAfterEach with BeforeAndAfterAll { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 2fe6a59a27c1b..920f6385f8e19 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ +import org.apache.spark.tags.SlowHiveTest import org.apache.spark.util.Utils case class Nested1(f1: Nested2) @@ -2559,6 +2560,8 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } } +@SlowHiveTest class SQLQuerySuite extends SQLQuerySuiteBase with DisableAdaptiveExecutionSuite +@SlowHiveTest class SQLQuerySuiteAE extends SQLQuerySuiteBase with EnableAdaptiveExecutionSuite From ceaa3924cba4fea9689d0de1e917d273cdd7a3ac Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 11 Jul 2020 13:46:59 -0700 Subject: [PATCH 169/384] [SPARK-32200][WEBUI] Redirect to the history page when accessed to /history on the HistoryServer without appliation id ### What changes were proposed in this pull request? This PR proposes to change the HistoryServer to redirect to the history page when we access to /history without application id. ### Why are the changes needed? In the current master, status code 400 will be returned when we access to /history. So I wonder it's better to redirect to the history page for the better UX. ### Does this PR introduce _any_ user-facing change? Yes. In the current master, if we access to /history without application id, we will see like the following page. ![history-400](https://user-images.githubusercontent.com/4736016/86649650-e9105380-c01c-11ea-93bb-78fd8d2e6f7b.png) After this change applied, we will be redirected to the history page. ### How was this patch tested? New test added. Closes #29016 from sarutak/history-redirect. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../apache/spark/deploy/history/HistoryServer.scala | 4 +--- .../spark/deploy/history/HistoryServerSuite.scala | 13 +++++++++++++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index aa9e9a6dd4887..ca21a8056d1b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -76,9 +76,7 @@ class HistoryServer( // attempt ID (separated by a slash). val parts = Option(req.getPathInfo()).getOrElse("").split("/") if (parts.length < 2) { - res.sendError(HttpServletResponse.SC_BAD_REQUEST, - s"Unexpected path info in request (URI = ${req.getRequestURI()}") - return + res.sendRedirect("/") } val appId = parts(1) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 6ede98d55f094..c856cc1897660 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -643,6 +643,19 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val actualContentType = conn.getContentType assert(actualContentType === expectedContentType) } + + test("Redirect to the root page when accessed to /history/") { + val port = server.boundPort + val url = new URL(s"http://localhost:$port/history/") + val conn = url.openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod("GET") + conn.setUseCaches(false) + conn.setDefaultUseCaches(false) + conn.setInstanceFollowRedirects(false) + conn.connect() + assert(conn.getResponseCode === 302) + assert(conn.getHeaderField("Location") === s"http://localhost:$port/") + } } object HistoryServerSuite { From 3ad4863673fc46080dda963be3055a3e554cfbc7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 11 Jul 2020 14:34:02 -0700 Subject: [PATCH 170/384] [SPARK-29292][SPARK-30010][CORE] Let core compile for Scala 2.13 ### What changes were proposed in this pull request? The purpose of this PR is to partly resolve SPARK-29292, and fully resolve SPARK-30010, which should allow Spark to compile vs Scala 2.13 in Spark Core and up through GraphX (not SQL, Streaming, etc). Note that we are not trying to determine here whether this makes Spark work on 2.13 yet, just compile, as a prerequisite for assessing test outcomes. However, of course, we need to ensure that the change does not break 2.12. The changes are, in the main, adding .toSeq and .toMap calls where mutable collections / maps are returned as Seq / Map, which are immutable by default in Scala 2.13. The theory is that it should be a no-op for Scala 2.12 (these return themselves), and required for 2.13. There are a few non-trivial changes highlighted below. In particular, to get Core to compile, we need to resolve SPARK-30010 which removes a deprecated SparkConf method ### Why are the changes needed? Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1. ### Does this PR introduce _any_ user-facing change? Yes, removal of the deprecated SparkConf.setAll overload, which isn't legal in Scala 2.13 anymore. ### How was this patch tested? Existing tests. (2.13 was not _tested_; this is about getting it to compile without breaking 2.12) Closes #28971 from srowen/SPARK-29292.1. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../apache/spark/ExecutorAllocationManager.scala | 6 +++--- .../scala/org/apache/spark/MapOutputTracker.scala | 2 +- .../main/scala/org/apache/spark/SparkConf.scala | 9 --------- .../scala/org/apache/spark/TaskContextImpl.scala | 6 +++--- .../scala/org/apache/spark/api/java/JavaRDD.scala | 2 +- .../org/apache/spark/api/java/JavaRDDLike.scala | 9 +++++---- .../apache/spark/api/java/JavaSparkContext.scala | 6 +++--- .../org/apache/spark/api/python/PythonRDD.scala | 2 +- .../org/apache/spark/api/python/PythonUtils.scala | 2 +- .../apache/spark/deploy/FaultToleranceTest.scala | 6 +++--- .../org/apache/spark/deploy/PythonRunner.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../deploy/master/ZooKeeperPersistenceEngine.scala | 2 +- .../spark/deploy/rest/RestSubmissionClient.scala | 2 +- .../apache/spark/deploy/worker/CommandUtils.scala | 2 +- .../apache/spark/deploy/worker/DriverRunner.scala | 4 ++-- .../spark/deploy/worker/ExecutorRunner.scala | 2 +- .../executor/CoarseGrainedExecutorBackend.scala | 8 ++++---- .../scala/org/apache/spark/executor/Executor.scala | 10 ++++++---- .../org/apache/spark/executor/TaskMetrics.scala | 4 ++-- .../org/apache/spark/metrics/MetricsSystem.scala | 2 +- .../org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../apache/spark/rdd/ParallelCollectionRDD.scala | 9 ++++----- .../main/scala/org/apache/spark/rdd/PipedRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/UnionRDD.scala | 2 +- .../apache/spark/resource/ResourceProfile.scala | 5 +++-- .../apache/spark/scheduler/LiveListenerBus.scala | 2 +- .../org/apache/spark/scheduler/SplitInfo.scala | 4 ++-- .../spark/scheduler/StatsReportListener.scala | 10 +++++----- .../org/apache/spark/scheduler/TaskResult.scala | 2 +- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 4 ++-- .../apache/spark/storage/BlockInfoManager.scala | 2 +- .../apache/spark/storage/DiskBlockManager.scala | 2 +- .../storage/ShuffleBlockFetcherIterator.scala | 10 +++++----- .../main/scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../src/main/scala/org/apache/spark/ui/WebUI.scala | 6 +++--- .../org/apache/spark/ui/jobs/AllJobsPage.scala | 8 ++++---- .../scala/org/apache/spark/ui/jobs/JobPage.scala | 14 +++++++------- .../apache/spark/ui/scope/RDDOperationGraph.scala | 6 +++--- .../scala/org/apache/spark/util/JsonProtocol.scala | 3 ++- .../main/scala/org/apache/spark/util/Utils.scala | 2 +- .../spark/util/collection/ExternalSorter.scala | 2 +- .../scala/org/apache/spark/AccumulatorSuite.scala | 2 +- .../org/apache/spark/ContextCleanerSuite.scala | 8 ++++---- .../org/apache/spark/HeartbeatReceiverSuite.scala | 4 +++- .../apache/spark/InternalAccumulatorSuite.scala | 2 +- .../test/scala/org/apache/spark/ShuffleSuite.scala | 2 +- .../org/apache/spark/deploy/IvyTestUtils.scala | 2 +- .../spark/rdd/ParallelCollectionSplitSuite.scala | 2 +- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 4 ++-- .../scheduler/EventLoggingListenerSuite.scala | 2 +- .../scheduler/ExecutorResourceInfoSuite.scala | 12 ++++++------ .../spark/scheduler/SparkListenerSuite.scala | 2 +- .../spark/scheduler/TaskResultGetterSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../storage/ShuffleBlockFetcherIteratorSuite.scala | 2 +- pom.xml | 2 +- .../sql/execution/streaming/StreamProgress.scala | 2 +- 58 files changed, 119 insertions(+), 122 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index e0ac2b3e0f4b8..620a6fe2f9d72 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -550,7 +550,7 @@ private[spark] class ExecutorAllocationManager( } else { // We don't want to change our target number of executors, because we already did that // when the task backlog decreased. - client.killExecutors(executorIdsToBeRemoved, adjustTargetNumExecutors = false, + client.killExecutors(executorIdsToBeRemoved.toSeq, adjustTargetNumExecutors = false, countFailures = false, force = false) } @@ -563,9 +563,9 @@ private[spark] class ExecutorAllocationManager( // reset the newExecutorTotal to the existing number of executors if (testing || executorsRemoved.nonEmpty) { - executorMonitor.executorsKilled(executorsRemoved) + executorMonitor.executorsKilled(executorsRemoved.toSeq) logInfo(s"Executors ${executorsRemoved.mkString(",")} removed due to idle timeout.") - executorsRemoved + executorsRemoved.toSeq } else { logWarning(s"Unable to reach the cluster manager to kill executor/s " + s"${executorIdsToBeRemoved.mkString(",")} or no executor eligible to kill!") diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 18cd5de4cfada..32251df6f4bbe 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -972,6 +972,6 @@ private[spark] object MapOutputTracker extends Logging { } } - splitsByAddress.iterator + splitsByAddress.mapValues(_.toSeq).iterator } } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 40915e3904f7e..dbd89d646ae54 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -173,15 +173,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria this } - /** - * Set multiple parameters together - */ - @deprecated("Use setAll(Iterable) instead", "3.0.0") - def setAll(settings: Traversable[(String, String)]): SparkConf = { - settings.foreach { case (k, v) => set(k, v) } - this - } - /** Set a parameter if it isn't already configured */ def setIfMissing(key: String, value: String): SparkConf = { if (settings.putIfAbsent(key, value) == null) { diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 08a58a029528b..db4b74bb89f0c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -111,7 +111,7 @@ private[spark] class TaskContextImpl( if (failed) return failed = true failure = error - invokeListeners(onFailureCallbacks, "TaskFailureListener", Option(error)) { + invokeListeners(onFailureCallbacks.toSeq, "TaskFailureListener", Option(error)) { _.onTaskFailure(this, error) } } @@ -120,7 +120,7 @@ private[spark] class TaskContextImpl( private[spark] override def markTaskCompleted(error: Option[Throwable]): Unit = synchronized { if (completed) return completed = true - invokeListeners(onCompleteCallbacks, "TaskCompletionListener", error) { + invokeListeners(onCompleteCallbacks.toSeq, "TaskCompletionListener", error) { _.onTaskCompletion(this) } } @@ -142,7 +142,7 @@ private[spark] class TaskContextImpl( } } if (errorMsgs.nonEmpty) { - throw new TaskCompletionListenerException(errorMsgs, error) + throw new TaskCompletionListenerException(errorMsgs.toSeq, error) } } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index e4140f659d979..15cb01a173287 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -256,7 +256,7 @@ object JavaRDD { } catch { case eof: EOFException => // No-op } - JavaRDD.fromRDD(sc.parallelize(objs, parallelism)) + JavaRDD.fromRDD(sc.parallelize(objs.toSeq, parallelism)) } finally { din.close() } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 1ca5262742665..89b33945dfb08 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -265,14 +265,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD created by piping elements to a forked external process. */ def pipe(command: JList[String]): JavaRDD[String] = { - rdd.pipe(command.asScala) + rdd.pipe(command.asScala.toSeq) } /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: JList[String], env: JMap[String, String]): JavaRDD[String] = { - rdd.pipe(command.asScala, env.asScala) + rdd.pipe(command.asScala.toSeq, env.asScala) } /** @@ -282,7 +282,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { env: JMap[String, String], separateWorkingDir: Boolean, bufferSize: Int): JavaRDD[String] = { - rdd.pipe(command.asScala, env.asScala, null, null, separateWorkingDir, bufferSize) + rdd.pipe(command.asScala.toSeq, env.asScala, null, null, separateWorkingDir, bufferSize) } /** @@ -293,7 +293,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { separateWorkingDir: Boolean, bufferSize: Int, encoding: String): JavaRDD[String] = { - rdd.pipe(command.asScala, env.asScala, null, null, separateWorkingDir, bufferSize, encoding) + rdd.pipe(command.asScala.toSeq, env.asScala, null, null, separateWorkingDir, bufferSize, + encoding) } /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 149def29b8fbd..347f59fe8f77e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -133,7 +133,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag - sc.parallelize(list.asScala, numSlices) + sc.parallelize(list.asScala.toSeq, numSlices) } /** Get an RDD that has no partitions or elements. */ @@ -152,7 +152,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { : JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[V] = fakeClassTag - JavaPairRDD.fromRDD(sc.parallelize(list.asScala, numSlices)) + JavaPairRDD.fromRDD(sc.parallelize(list.asScala.toSeq, numSlices)) } /** Distribute a local Scala collection to form an RDD. */ @@ -161,7 +161,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { /** Distribute a local Scala collection to form an RDD. */ def parallelizeDoubles(list: java.util.List[java.lang.Double], numSlices: Int): JavaDoubleRDD = - JavaDoubleRDD.fromRDD(sc.parallelize(list.asScala.map(_.doubleValue()), numSlices)) + JavaDoubleRDD.fromRDD(sc.parallelize(list.asScala.map(_.doubleValue()).toSeq, numSlices)) /** Distribute a local Scala collection to form an RDD. */ def parallelizeDoubles(list: java.util.List[java.lang.Double]): JavaDoubleRDD = diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 726cff6703dcb..86a1ac31c0845 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -163,7 +163,7 @@ private[spark] object PythonRDD extends Logging { type ByteArray = Array[Byte] type UnrolledPartition = Array[ByteArray] val allPartitions: Array[UnrolledPartition] = - sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions.asScala) + sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions.asScala.toSeq) val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*) serveIterator(flattenedPartition.iterator, s"serve RDD ${rdd.id} with partitions ${partitions.asScala.mkString(",")}") diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 490b48719b6be..527d0d6d3a48d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -54,7 +54,7 @@ private[spark] object PythonUtils { * Convert list of T into seq of T (for calling API with varargs) */ def toSeq[T](vs: JList[T]): Seq[T] = { - vs.asScala + vs.asScala.toSeq } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 6ff68b694f8f3..ab389f99b11a7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -205,7 +205,7 @@ private object FaultToleranceTest extends App with Logging { private def addWorkers(num: Int): Unit = { logInfo(s">>>>> ADD WORKERS $num <<<<<") - val masterUrls = getMasterUrls(masters) + val masterUrls = getMasterUrls(masters.toSeq) (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) } } @@ -216,7 +216,7 @@ private object FaultToleranceTest extends App with Logging { // Counter-hack: Because of a hack in SparkEnv#create() that changes this // property, we need to reset it. System.setProperty(config.DRIVER_PORT.key, "0") - sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome) + sc = new SparkContext(getMasterUrls(masters.toSeq), "fault-tolerance", containerSparkHome) } private def getMasterUrls(masters: Seq[TestMasterInfo]): String = { @@ -279,7 +279,7 @@ private object FaultToleranceTest extends App with Logging { var liveWorkerIPs: Seq[String] = List() def stateValid(): Boolean = { - (workers.map(_.ip) -- liveWorkerIPs).isEmpty && + workers.map(_.ip).forall(liveWorkerIPs.contains) && numAlive == 1 && numStandby == masters.size - 1 && numLiveApps >= 1 } diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 574ce60b19b4e..7ad92da4e055a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -69,7 +69,7 @@ object PythonRunner { pathElements ++= formattedPyFiles pathElements += PythonUtils.sparkPythonPath pathElements += sys.env.getOrElse("PYTHONPATH", "") - val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) + val pythonPath = PythonUtils.mergePythonPaths(pathElements.toSeq: _*) // Launch Python process val builder = new ProcessBuilder((Seq(pythonExec, formattedPythonFile) ++ otherArgs).asJava) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 1271a3dbfc3f6..6d38a1d281464 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -820,7 +820,7 @@ private[spark] class SparkSubmit extends Logging { } sparkConf.set(SUBMIT_PYTHON_FILES, formattedPyFiles.split(",").toSeq) - (childArgs, childClasspath, sparkConf, childMainClass) + (childArgs.toSeq, childClasspath.toSeq, sparkConf, childMainClass) } private def renameResourcesToLocalFS(resources: String, localResources: String): String = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 8eae445b439d9..ded816b992db8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -52,7 +52,7 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer override def read[T: ClassTag](prefix: String): Seq[T] = { zk.getChildren.forPath(workingDir).asScala - .filter(_.startsWith(prefix)).flatMap(deserializeFromFile[T]) + .filter(_.startsWith(prefix)).flatMap(deserializeFromFile[T]).toSeq } override def close(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 1648ba516d9b6..89ef051e58549 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -422,7 +422,7 @@ private[spark] object RestSubmissionClient { private[rest] def filterSystemEnvironment(env: Map[String, String]): Map[String, String] = { env.filterKeys { k => (k.startsWith("SPARK_") && !BLACKLISTED_SPARK_ENV_VARS.contains(k)) || k.startsWith("MESOS_") - } + }.toMap } private[spark] def supportsRestClient(master: String): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index f7423f1fc3f1c..8240bd6d2f438 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -61,7 +61,7 @@ object CommandUtils extends Logging { // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows val cmd = new WorkerCommandBuilder(sparkHome, memory, command).buildCommand() - cmd.asScala ++ Seq(command.mainClass) ++ command.arguments + (cmd.asScala ++ Seq(command.mainClass) ++ command.arguments).toSeq } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 53ec7b3a88f35..4f9c497fc3d76 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -201,7 +201,7 @@ private[deploy] class DriverRunner( CommandUtils.redirectStream(process.getInputStream, stdout) val stderr = new File(baseDir, "stderr") - val redactedCommand = Utils.redactCommandLineArgs(conf, builder.command.asScala) + val redactedCommand = Utils.redactCommandLineArgs(conf, builder.command.asScala.toSeq) .mkString("\"", "\" \"", "\"") val header = "Launch Command: %s\n%s\n\n".format(redactedCommand, "=" * 40) Files.append(header, stderr, StandardCharsets.UTF_8) @@ -262,6 +262,6 @@ private[deploy] trait ProcessBuilderLike { private[deploy] object ProcessBuilderLike { def apply(processBuilder: ProcessBuilder): ProcessBuilderLike = new ProcessBuilderLike { override def start(): Process = processBuilder.start() - override def command: Seq[String] = processBuilder.command().asScala + override def command: Seq[String] = processBuilder.command().asScala.toSeq } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2a5528bbe89cb..e4fcae13a2f89 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -158,7 +158,7 @@ private[deploy] class ExecutorRunner( val builder = CommandUtils.buildProcessBuilder(subsCommand, new SecurityManager(conf), memory, sparkHome.getAbsolutePath, substituteVariables) val command = builder.command() - val redactedCommand = Utils.redactCommandLineArgs(conf, command.asScala) + val redactedCommand = Utils.redactCommandLineArgs(conf, command.asScala.toSeq) .mkString("\"", "\" \"", "\"") logInfo(s"Launch command: $redactedCommand") diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 6625457749f6a..e072d7919450e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -140,13 +140,13 @@ private[spark] class CoarseGrainedExecutorBackend( def extractLogUrls: Map[String, String] = { val prefix = "SPARK_LOG_URL_" sys.env.filterKeys(_.startsWith(prefix)) - .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)) + .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)).toMap } def extractAttributes: Map[String, String] = { val prefix = "SPARK_EXECUTOR_ATTRIBUTE_" sys.env.filterKeys(_.startsWith(prefix)) - .map(e => (e._1.substring(prefix.length).toUpperCase(Locale.ROOT), e._2)) + .map(e => (e._1.substring(prefix.length).toUpperCase(Locale.ROOT), e._2)).toMap } override def receive: PartialFunction[Any, Unit] = { @@ -304,8 +304,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, Arguments, SparkEnv, ResourceProfile) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env, resourceProfile) => new CoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, - arguments.resourcesFileOpt, resourceProfile) + arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath.toSeq, + env, arguments.resourcesFileOpt, resourceProfile) } run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn) System.exit(0) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c8b1afeebac0d..bc0f0c0a7b705 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -606,7 +606,8 @@ private[spark] class Executor( // Here and below, put task metric peaks in a WrappedArray to expose them as a Seq // without requiring a copy. val metricPeaks = WrappedArray.make(metricsPoller.getTaskMetricPeaks(taskId)) - val serializedTK = ser.serialize(TaskKilled(t.reason, accUpdates, accums, metricPeaks)) + val serializedTK = ser.serialize( + TaskKilled(t.reason, accUpdates, accums, metricPeaks.toSeq)) execBackend.statusUpdate(taskId, TaskState.KILLED, serializedTK) case _: InterruptedException | NonFatal(_) if @@ -616,7 +617,8 @@ private[spark] class Executor( val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) val metricPeaks = WrappedArray.make(metricsPoller.getTaskMetricPeaks(taskId)) - val serializedTK = ser.serialize(TaskKilled(killReason, accUpdates, accums, metricPeaks)) + val serializedTK = ser.serialize( + TaskKilled(killReason, accUpdates, accums, metricPeaks.toSeq)) execBackend.statusUpdate(taskId, TaskState.KILLED, serializedTK) case t: Throwable if hasFetchFailure && !Utils.isFatalError(t) => @@ -661,13 +663,13 @@ private[spark] class Executor( val serializedTaskEndReason = { try { val ef = new ExceptionFailure(t, accUpdates).withAccums(accums) - .withMetricPeaks(metricPeaks) + .withMetricPeaks(metricPeaks.toSeq) ser.serialize(ef) } catch { case _: NotSerializableException => // t is not serializable so just send the stacktrace val ef = new ExceptionFailure(t, accUpdates, false).withAccums(accums) - .withMetricPeaks(metricPeaks) + .withMetricPeaks(metricPeaks.toSeq) ser.serialize(ef) } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 1470a23884bb0..43742a4d46cbb 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -123,7 +123,7 @@ class TaskMetrics private[spark] () extends Serializable { def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = { // This is called on driver. All accumulator updates have a fixed value. So it's safe to use // `asScala` which accesses the internal values using `java.util.Iterator`. - _updatedBlockStatuses.value.asScala + _updatedBlockStatuses.value.asScala.toSeq } // Setters and increment-ers @@ -199,7 +199,7 @@ class TaskMetrics private[spark] () extends Serializable { */ private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { if (tempShuffleReadMetrics.nonEmpty) { - shuffleReadMetrics.setMergeValues(tempShuffleReadMetrics) + shuffleReadMetrics.setMergeValues(tempShuffleReadMetrics.toSeq) } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 57dcbe501c6dd..48f816f649d36 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -156,7 +156,7 @@ private[spark] class MetricsSystem private ( } def getSourcesByName(sourceName: String): Seq[Source] = - sources.filter(_.sourceName == sourceName) + sources.filter(_.sourceName == sourceName).toSeq def registerSource(source: Source): Unit = { sources += source diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 1e39e10856877..f280c220a2c8d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -934,7 +934,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) for (pair <- it if pair._1 == key) { buf += pair._2 } - buf + buf.toSeq } : Seq[V] val res = self.context.runJob(self, process, Array(index)) res(0) diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 9f8019b80a4dd..324cba5b4de42 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -133,12 +133,11 @@ private object ParallelCollectionRDD { // If the range is inclusive, use inclusive range for the last slice if (r.isInclusive && index == numSlices - 1) { new Range.Inclusive(r.start + start * r.step, r.end, r.step) - } - else { - new Range(r.start + start * r.step, r.start + end * r.step, r.step) + } else { + new Range.Inclusive(r.start + start * r.step, r.start + (end - 1) * r.step, r.step) } }.toSeq.asInstanceOf[Seq[Seq[T]]] - case nr: NumericRange[_] => + case nr: NumericRange[T] => // For ranges of Long, Double, BigInteger, etc val slices = new ArrayBuffer[Seq[T]](numSlices) var r = nr @@ -147,7 +146,7 @@ private object ParallelCollectionRDD { slices += r.take(sliceSize).asInstanceOf[Seq[T]] r = r.drop(sliceSize) } - slices + slices.toSeq case _ => val array = seq.toArray // To prevent O(n^2) operations for List etc positions(array.length, numSlices).map { case (start, end) => diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 3b11e82dab196..5dd8cb8440be6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -238,7 +238,7 @@ private object PipedRDD { while(tok.hasMoreElements) { buf += tok.nextToken() } - buf + buf.toSeq } val STDIN_WRITER_THREAD_PREFIX = "stdin writer for" diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 63fa3c2487c33..0a93023443704 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -98,7 +98,7 @@ class UnionRDD[T: ClassTag]( deps += new RangeDependency(rdd, 0, pos, rdd.partitions.length) pos += rdd.partitions.length } - deps + deps.toSeq } override def compute(s: Partition, context: TaskContext): Iterator[T] = { diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 1dbdc3d81e44d..f56ea69f6cec5 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -319,12 +319,13 @@ object ResourceProfile extends Logging { private[spark] def getCustomTaskResources( rp: ResourceProfile): Map[String, TaskResourceRequest] = { - rp.taskResources.filterKeys(k => !k.equals(ResourceProfile.CPUS)) + rp.taskResources.filterKeys(k => !k.equals(ResourceProfile.CPUS)).toMap } private[spark] def getCustomExecutorResources( rp: ResourceProfile): Map[String, ExecutorResourceRequest] = { - rp.executorResources.filterKeys(k => !ResourceProfile.allSupportedExecutorResources.contains(k)) + rp.executorResources. + filterKeys(k => !ResourceProfile.allSupportedExecutorResources.contains(k)).toMap } /* diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 95b0096cade38..f13f1eaeeaa43 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -232,7 +232,7 @@ private[spark] class LiveListenerBus(conf: SparkConf) { // For testing only. private[spark] def findListenersByClass[T <: SparkListenerInterface : ClassTag](): Seq[T] = { - queues.asScala.flatMap { queue => queue.findListenersByClass[T]() } + queues.asScala.flatMap { queue => queue.findListenersByClass[T]() }.toSeq } // For testing only. diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index bc1431835e258..6112d8ef051e0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -69,7 +69,7 @@ object SplitInfo { for (host <- mapredSplit.getLocations) { retval += new SplitInfo(inputFormatClazz, host, path, length, mapredSplit) } - retval + retval.toSeq } def toSplitInfo(inputFormatClazz: Class[_], path: String, @@ -79,6 +79,6 @@ object SplitInfo { for (host <- mapreduceSplit.getLocations) { retval += new SplitInfo(inputFormatClazz, host, path, length, mapreduceSplit) } - retval + retval.toSeq } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala index ca48775e77f27..be881481bf4ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala @@ -47,19 +47,19 @@ class StatsReportListener extends SparkListener with Logging { override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { implicit val sc = stageCompleted this.logInfo(s"Finished stage: ${getStatusDetail(stageCompleted.stageInfo)}") - showMillisDistribution("task runtime:", (info, _) => info.duration, taskInfoMetrics) + showMillisDistribution("task runtime:", (info, _) => info.duration, taskInfoMetrics.toSeq) // Shuffle write showBytesDistribution("shuffle bytes written:", - (_, metric) => metric.shuffleWriteMetrics.bytesWritten, taskInfoMetrics) + (_, metric) => metric.shuffleWriteMetrics.bytesWritten, taskInfoMetrics.toSeq) // Fetch & I/O showMillisDistribution("fetch wait time:", - (_, metric) => metric.shuffleReadMetrics.fetchWaitTime, taskInfoMetrics) + (_, metric) => metric.shuffleReadMetrics.fetchWaitTime, taskInfoMetrics.toSeq) showBytesDistribution("remote bytes read:", - (_, metric) => metric.shuffleReadMetrics.remoteBytesRead, taskInfoMetrics) + (_, metric) => metric.shuffleReadMetrics.remoteBytesRead, taskInfoMetrics.toSeq) showBytesDistribution("task result size:", - (_, metric) => metric.resultSize, taskInfoMetrics) + (_, metric) => metric.resultSize, taskInfoMetrics.toSeq) // Runtime breakdown val runtimePcts = taskInfoMetrics.map { case (info, metrics) => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index b6df216d537e4..11d969e1aba90 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -71,7 +71,7 @@ private[spark] class DirectTaskResult[T]( for (i <- 0 until numUpdates) { _accumUpdates += in.readObject.asInstanceOf[AccumulatorV2[_, _]] } - accumUpdates = _accumUpdates + accumUpdates = _accumUpdates.toSeq } val numMetrics = in.readInt diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2c37fec271766..45cb5e534220c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -137,7 +137,7 @@ private[spark] class TaskSchedulerImpl( private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]] def runningTasksByExecutors: Map[String, Int] = synchronized { - executorIdToRunningTaskIds.toMap.mapValues(_.size) + executorIdToRunningTaskIds.toMap.mapValues(_.size).toMap } // The set of executors we have on each host; this is used to compute hostsAlive, which @@ -719,7 +719,7 @@ private[spark] class TaskSchedulerImpl( if (tasks.nonEmpty) { hasLaunchedTask = true } - return tasks + return tasks.map(_.toSeq) } private def createUnschedulableTaskSetAbortTimer( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 219a0e799cc73..95d901f292971 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -367,7 +367,7 @@ private[storage] class BlockInfoManager extends Logging { notifyAll() - blocksWithReleasedLocks + blocksWithReleasedLocks.toSeq } /** Returns the number of locks held by the given task. Used only for testing. */ diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index bf76eef443e81..5db4965b67347 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -98,7 +98,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea } }.filter(_ != null).flatMap { dir => val files = dir.listFiles() - if (files != null) files else Seq.empty + if (files != null) files.toSeq else Seq.empty } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 5efbc0703f729..a2843da0561e0 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -368,25 +368,25 @@ final class ShuffleBlockFetcherIterator( collectedRemoteRequests: ArrayBuffer[FetchRequest]): Unit = { val iterator = blockInfos.iterator var curRequestSize = 0L - var curBlocks = new ArrayBuffer[FetchBlockInfo] + var curBlocks = Seq.empty[FetchBlockInfo] while (iterator.hasNext) { val (blockId, size, mapIndex) = iterator.next() assertPositiveBlockSize(blockId, size) - curBlocks += FetchBlockInfo(blockId, size, mapIndex) + curBlocks = curBlocks ++ Seq(FetchBlockInfo(blockId, size, mapIndex)) curRequestSize += size // For batch fetch, the actual block in flight should count for merged block. val mayExceedsMaxBlocks = !doBatchFetch && curBlocks.size >= maxBlocksInFlightPerAddress if (curRequestSize >= targetRemoteRequestSize || mayExceedsMaxBlocks) { curBlocks = createFetchRequests(curBlocks, address, isLast = false, - collectedRemoteRequests).to[ArrayBuffer] + collectedRemoteRequests) curRequestSize = curBlocks.map(_.size).sum } } // Add in the final request if (curBlocks.nonEmpty) { curBlocks = createFetchRequests(curBlocks, address, isLast = true, - collectedRemoteRequests).to[ArrayBuffer] + collectedRemoteRequests) curRequestSize = curBlocks.map(_.size).sum } } @@ -928,7 +928,7 @@ object ShuffleBlockFetcherIterator { } else { blocks } - result + result.toSeq } /** diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 087a22d6c6140..a070cc9c7b39d 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -443,7 +443,7 @@ private[spark] object UIUtils extends Logging { case None => {getHeaderContent(x._1)} } - } + }.toSeq } {headerRow} diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 9faa3dcf2cdf2..a4e87704927c6 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -58,11 +58,11 @@ private[spark] abstract class WebUI( private val className = Utils.getFormattedClassName(this) def getBasePath: String = basePath - def getTabs: Seq[WebUITab] = tabs - def getHandlers: Seq[ServletContextHandler] = handlers + def getTabs: Seq[WebUITab] = tabs.toSeq + def getHandlers: Seq[ServletContextHandler] = handlers.toSeq def getDelegatingHandlers: Seq[DelegatingServletContextHandler] = { - handlers.map(new DelegatingServletContextHandler(_)) + handlers.map(new DelegatingServletContextHandler(_)).toSeq } /** Attaches a tab to this UI, along with all of its attached pages. */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 066512d159d00..4e76ea289ede6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -259,11 +259,11 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We } val activeJobsTable = - jobsTable(request, "active", "activeJob", activeJobs, killEnabled = parent.killEnabled) + jobsTable(request, "active", "activeJob", activeJobs.toSeq, killEnabled = parent.killEnabled) val completedJobsTable = - jobsTable(request, "completed", "completedJob", completedJobs, killEnabled = false) + jobsTable(request, "completed", "completedJob", completedJobs.toSeq, killEnabled = false) val failedJobsTable = - jobsTable(request, "failed", "failedJob", failedJobs, killEnabled = false) + jobsTable(request, "failed", "failedJob", failedJobs.toSeq, killEnabled = false) val shouldShowActiveJobs = activeJobs.nonEmpty val shouldShowCompletedJobs = completedJobs.nonEmpty @@ -330,7 +330,7 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We var content = summary - content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs, + content ++= makeTimeline((activeJobs ++ completedJobs ++ failedJobs).toSeq, store.executorList(false), startTime) if (shouldShowActiveJobs) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 542dc39eee4f0..bba5e3dda6c47 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -288,20 +288,20 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP } val activeStagesTable = - new StageTableBase(store, request, activeStages, "active", "activeStage", parent.basePath, - basePath, parent.isFairScheduler, + new StageTableBase(store, request, activeStages.toSeq, "active", "activeStage", + parent.basePath, basePath, parent.isFairScheduler, killEnabled = parent.killEnabled, isFailedStage = false) val pendingOrSkippedStagesTable = - new StageTableBase(store, request, pendingOrSkippedStages, pendingOrSkippedTableId, + new StageTableBase(store, request, pendingOrSkippedStages.toSeq, pendingOrSkippedTableId, "pendingStage", parent.basePath, basePath, parent.isFairScheduler, killEnabled = false, isFailedStage = false) val completedStagesTable = - new StageTableBase(store, request, completedStages, "completed", "completedStage", + new StageTableBase(store, request, completedStages.toSeq, "completed", "completedStage", parent.basePath, basePath, parent.isFairScheduler, killEnabled = false, isFailedStage = false) val failedStagesTable = - new StageTableBase(store, request, failedStages, "failed", "failedStage", parent.basePath, - basePath, parent.isFairScheduler, + new StageTableBase(store, request, failedStages.toSeq, "failed", "failedStage", + parent.basePath, basePath, parent.isFairScheduler, killEnabled = false, isFailedStage = true) val shouldShowActiveStages = activeStages.nonEmpty @@ -391,7 +391,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP var content = summary val appStartTime = store.applicationInfo().attempts.head.startTime.getTime() - content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, + content ++= makeTimeline((activeStages ++ completedStages ++ failedStages).toSeq, store.executorList(false), appStartTime) val operationGraphContent = store.asOption(store.operationGraphForJob(jobId)) match { diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index 842ee7aaf49bf..f8d9279c2404f 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -81,11 +81,11 @@ private[spark] class RDDOperationCluster( /** Return all the nodes which are cached. */ def getCachedNodes: Seq[RDDOperationNode] = { - _childNodes.filter(_.cached) ++ _childClusters.flatMap(_.getCachedNodes) + (_childNodes.filter(_.cached) ++ _childClusters.flatMap(_.getCachedNodes)).toSeq } def getBarrierClusters: Seq[RDDOperationCluster] = { - _childClusters.filter(_.barrier) ++ _childClusters.flatMap(_.getBarrierClusters) + (_childClusters.filter(_.barrier) ++ _childClusters.flatMap(_.getBarrierClusters)).toSeq } def canEqual(other: Any): Boolean = other.isInstanceOf[RDDOperationCluster] @@ -210,7 +210,7 @@ private[spark] object RDDOperationGraph extends Logging { } } - RDDOperationGraph(internalEdges, outgoingEdges, incomingEdges, rootCluster) + RDDOperationGraph(internalEdges.toSeq, outgoingEdges.toSeq, incomingEdges.toSeq, rootCluster) } /** diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index f3372501f471b..170ba2b9c6acc 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -1214,7 +1214,8 @@ private[spark] object JsonProtocol { case Some(id) => id.extract[Int] case None => ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID } - new ExecutorInfo(executorHost, totalCores, logUrls, attributes, resources, resourceProfileId) + new ExecutorInfo(executorHost, totalCores, logUrls, attributes.toMap, resources.toMap, + resourceProfileId) } def blockUpdatedInfoFromJson(json: JValue): BlockUpdatedInfo = { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 2b4b2508f9b7e..35d60bb514405 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1742,7 +1742,7 @@ private[spark] object Utils extends Logging { if (inWord || inDoubleQuote || inSingleQuote) { endWord() } - buf + buf.toSeq } /* Calculates 'x' modulo 'mod', takes to consideration sign of x, diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index cc97bbfa7201f..dc39170ecf382 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -659,7 +659,7 @@ private[spark] class ExternalSorter[K, V, C]( } } else { // Merge spilled and in-memory data - merge(spills, destructiveIterator( + merge(spills.toSeq, destructiveIterator( collection.partitionedDestructiveSortedIterator(comparator))) } } diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index a75cf3f0381df..d701cb65460af 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -157,7 +157,7 @@ private class SaveInfoListener extends SparkListener { def getCompletedStageInfos: Seq[StageInfo] = completedStageInfos.toArray.toSeq def getCompletedTaskInfos: Seq[TaskInfo] = completedTaskInfos.values.flatten.toSeq def getCompletedTaskInfos(stageId: StageId, stageAttemptId: StageAttemptId): Seq[TaskInfo] = - completedTaskInfos.getOrElse((stageId, stageAttemptId), Seq.empty[TaskInfo]) + completedTaskInfos.getOrElse((stageId, stageAttemptId), Seq.empty[TaskInfo]).toSeq /** * If `jobCompletionCallback` is set, block until the next call has finished. diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 92ed24408384f..7a95ea0fa3218 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -291,14 +291,14 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val shuffleIds = 0 until sc.newShuffleId val broadcastIds = broadcastBuffer.map(_.id) - val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds.toSeq) runGC() intercept[Exception] { preGCTester.assertCleanup()(timeout(1.second)) } // Test that GC triggers the cleanup of all variables after the dereferencing them - val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds.toSeq) broadcastBuffer.clear() rddBuffer.clear() runGC() @@ -331,14 +331,14 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val shuffleIds = 0 until sc.newShuffleId val broadcastIds = broadcastBuffer.map(_.id) - val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds.toSeq) runGC() intercept[Exception] { preGCTester.assertCleanup()(timeout(1.second)) } // Test that GC triggers the cleanup of all variables after the dereferencing them - val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds.toSeq) broadcastBuffer.clear() rddBuffer.clear() runGC() diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 312691302b064..a2e70b23a3e5d 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -261,7 +261,7 @@ class HeartbeatReceiverSuite // We may receive undesired SparkListenerExecutorAdded from LocalSchedulerBackend, // so exclude it from the map. See SPARK-10800. heartbeatReceiver.invokePrivate(_executorLastSeen()). - filterKeys(_ != SparkContext.DRIVER_IDENTIFIER) + filterKeys(_ != SparkContext.DRIVER_IDENTIFIER).toMap } } @@ -287,6 +287,8 @@ private class FakeSchedulerBackend( resourceProfileManager: ResourceProfileManager) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + def this() = this(null, null, null, null) + protected override def doRequestTotalExecutors( resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { clusterManagerEndpoint.ask[Boolean]( diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 5399d868f46f1..f2b81e5153ae4 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -220,7 +220,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { super.registerAccumulatorForCleanup(a) } - def accumsRegisteredForCleanup: Seq[Long] = accumsRegistered.toArray + def accumsRegisteredForCleanup: Seq[Long] = accumsRegistered.toSeq } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 9e39271bdf9ee..3d6690cb85348 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -182,7 +182,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) val pairs2: RDD[MutablePair[Int, String]] = sc.parallelize(data2, 2) val results = new CoGroupedRDD[Int](Seq(pairs1, pairs2), new HashPartitioner(2)) - .map(p => (p._1, p._2.map(_.toArray))) + .map(p => (p._1, p._2.map(_.toSeq))) .collectAsMap() assert(results(1)(0).length === 3) diff --git a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala index 42b8cde650390..b986be03e965c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala @@ -317,7 +317,7 @@ private[deploy] object IvyTestUtils { val rFiles = createRFiles(root, className, artifact.groupId) allFiles.append(rFiles: _*) } - val jarFile = packJar(jarPath, artifact, allFiles, useIvyLayout, withR) + val jarFile = packJar(jarPath, artifact, allFiles.toSeq, useIvyLayout, withR) assert(jarFile.exists(), "Problem creating Jar file") val descriptor = createDescriptor(tempPath, artifact, dependencies, useIvyLayout) assert(descriptor.exists(), "Problem creating Pom file") diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index 10f4bbcf7f48b..879107350bb52 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -140,7 +140,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { assert(slices(i).isInstanceOf[Range]) val range = slices(i).asInstanceOf[Range] assert(range.start === i * (N / 40), "slice " + i + " start") - assert(range.end === (i + 1) * (N / 40), "slice " + i + " end") + assert(range.last === (i + 1) * (N / 40) - 1, "slice " + i + " end") assert(range.step === 1, "slice " + i + " step") } } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 18154d861a731..1f4e784723b43 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -656,7 +656,7 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { } test("top with predefined ordering") { - val nums = Array.range(1, 100000) + val nums = Seq.range(1, 100000) val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) val topK = ints.top(5) assert(topK.size === 5) @@ -1098,7 +1098,7 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { override def getPartitions: Array[Partition] = Array(new Partition { override def index: Int = 0 }) - override def getDependencies: Seq[Dependency[_]] = mutableDependencies + override def getDependencies: Seq[Dependency[_]] = mutableDependencies.toSeq def addDependency(dep: Dependency[_]): Unit = { mutableDependencies += dep } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 7c23e4449f461..915035e9eb71c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -325,7 +325,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L) def max(a: Array[Long], b: Array[Long]): Array[Long] = - (a, b).zipped.map(Math.max) + (a, b).zipped.map(Math.max).toArray // calculated metric peaks per stage per executor // metrics sent during stage 0 for each executor diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala index 388d4e25a06cf..e392ff53e02c9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -26,7 +26,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { test("Track Executor Resource information") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) assert(info.availableAddrs.sorted sameElements Seq("0", "1", "2", "3")) assert(info.assignedAddrs.isEmpty) @@ -43,7 +43,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { test("Don't allow acquire address that is not available") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) // Acquire some addresses. info.acquire(Seq("0", "1")) assert(!info.availableAddrs.contains("1")) @@ -56,7 +56,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { test("Don't allow acquire address that doesn't exist") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) assert(!info.availableAddrs.contains("4")) // Acquire an address that doesn't exist val e = intercept[SparkException] { @@ -67,7 +67,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { test("Don't allow release address that is not assigned") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) // Acquire addresses info.acquire(Array("0", "1")) assert(!info.assignedAddrs.contains("2")) @@ -80,7 +80,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { test("Don't allow release address that doesn't exist") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) assert(!info.assignedAddrs.contains("4")) // Release an address that doesn't exist val e = intercept[SparkException] { @@ -93,7 +93,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val slotSeq = Seq(10, 9, 8, 7, 6, 5, 4, 3, 2, 1) val addresses = ArrayBuffer("0", "1", "2", "3") slotSeq.foreach { slots => - val info = new ExecutorResourceInfo(GPU, addresses, slots) + val info = new ExecutorResourceInfo(GPU, addresses.toSeq, slots) for (_ <- 0 until slots) { addresses.foreach(addr => info.acquire(Seq(addr))) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index d4e8d63b54e5f..270b2c606ad0c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -621,7 +621,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match } override def onStageCompleted(stage: SparkListenerStageCompleted): Unit = { - stageInfos(stage.stageInfo) = taskInfoMetrics + stageInfos(stage.stageInfo) = taskInfoMetrics.toSeq taskInfoMetrics = mutable.Buffer.empty } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 2efe6da5e986f..ea44a2d948ca9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -103,7 +103,7 @@ private class MyTaskResultGetter(env: SparkEnv, scheduler: TaskSchedulerImpl) // DirectTaskResults that we receive from the executors private val _taskResults = new ArrayBuffer[DirectTaskResult[_]] - def taskResults: Seq[DirectTaskResult[_]] = _taskResults + def taskResults: Seq[DirectTaskResult[_]] = _taskResults.toSeq override def enqueueSuccessfulTask(tsm: TaskSetManager, tid: Long, data: ByteBuffer): Unit = { // work on a copy since the super class still needs to use the buffer diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index e4aad58d25064..759e68219c2d0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1670,7 +1670,7 @@ class TaskSetManagerSuite for (i <- 0 to 99) { locations += Seq(TaskLocation("host" + i)) } - val taskSet = FakeTask.createTaskSet(100, locations: _*) + val taskSet = FakeTask.createTaskSet(100, locations.toSeq: _*) val clock = new ManualClock // make sure we only do one rack resolution call, for the entire batch of hosts, as this // can be expensive. The FakeTaskScheduler calls rack resolution more than the real one diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 43917a5b83bb0..bf1379ceb89a8 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -1047,7 +1047,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer() ) - val transfer = createMockTransfer(blocks.mapValues(_ => createMockManagedBuffer(0))) + val transfer = createMockTransfer(blocks.mapValues(_ => createMockManagedBuffer(0)).toMap) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)) diff --git a/pom.xml b/pom.xml index 8b3ba305edda7..279d76d42ce12 100644 --- a/pom.xml +++ b/pom.xml @@ -3159,7 +3159,7 @@ scala-2.13 - 2.13.1 + 2.13.3 2.13 diff --git a/sql/core/src/main/scala-2.13/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala-2.13/org/apache/spark/sql/execution/streaming/StreamProgress.scala index 0aa29640899c6..6aa1b46cbb94a 100644 --- a/sql/core/src/main/scala-2.13/org/apache/spark/sql/execution/streaming/StreamProgress.scala +++ b/sql/core/src/main/scala-2.13/org/apache/spark/sql/execution/streaming/StreamProgress.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkD */ class StreamProgress( val baseMap: immutable.Map[SparkDataStream, OffsetV2] = - new immutable.HashMap[SparkDataStream, OffsetV2]) + new immutable.HashMap[SparkDataStream, OffsetV2]) extends scala.collection.immutable.Map[SparkDataStream, OffsetV2] { // Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation. From 09789ff725f1718a55df9595e736c3cf13746f25 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 11 Jul 2020 14:48:23 -0700 Subject: [PATCH 171/384] [SPARK-31226][CORE][TESTS] SizeBasedCoalesce logic will lose partition ### What changes were proposed in this pull request? When last partition's splitFile's split size is larger then maxSize, this partition will be lost Origin logic error like below as 1, 2, 3, 4, 5 ```scala // 5. since index = partition.size now, jump out of the loop , then the last partition is lost since we won't call updatePartition() again. while (index < partitions.size) { // 1. we assume that when index = partitions.length -1(the last partition) val partition = partitions(index) val fileSplit = partition.asInstanceOf[HadoopPartition].inputSplit.value.asInstanceOf[FileSplit] val splitSize = fileSplit.getLength // 2. if this partition's splitSize > maxSize if (currentSum + splitSize < maxSize) { addPartition(partition, splitSize) index += 1 if (index == partitions.size) { updateGroups } } else { // 3. if currentGroup.partitions.size >0, this situation is possiable if (currentGroup.partitions.size == 0) { addPartition(partition, splitSize) index += 1 } else { // 4. then will just call updateGroups() here first, and index won't update in group updateGroups } } } groups.toArray } } ``` ### Why are the changes needed? Fix bug ### Does this PR introduce any user-facing change? NO ### How was this patch tested? Manual code review. Closes #27988 from AngersZhuuuu/SPARK-31226. Authored-by: angerszhu Signed-off-by: Dongjoon Hyun --- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 1f4e784723b43..79f9c1396c87b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -1298,19 +1298,15 @@ class SizeBasedCoalescer(val maxSize: Int) extends PartitionCoalescer with Seria val splitSize = fileSplit.getLength if (currentSum + splitSize < maxSize) { addPartition(partition, splitSize) - index += 1 - if (index == partitions.size) { - updateGroups - } } else { - if (currentGroup.partitions.size == 0) { - addPartition(partition, splitSize) - index += 1 - } else { - updateGroups + if (currentGroup.partitions.nonEmpty) { + updateGroups() } + addPartition(partition, splitSize) } + index += 1 } + updateGroups() groups.toArray } } From 98504e92a719f60c097b8542e275ee3415438221 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 11 Jul 2020 15:58:40 -0700 Subject: [PATCH 172/384] [SPARK-29358][SQL] Make unionByName optionally fill missing columns with nulls ### What changes were proposed in this pull request? This patch proposes to make `unionByName` optionally fill missing columns with nulls. ### Why are the changes needed? Currently, `unionByName` throws exception if detecting different column names between two Datasets. It is strict requirement and sometimes users require more flexible usage that two Datasets with different subset of columns can be union by name resolution. ### Does this PR introduce _any_ user-facing change? Yes. Adding overloading `Dataset.unionByName` with a boolean parameter that allows different set of column names between two Datasets. Missing columns at each side, will be filled with null values. ### How was this patch tested? Unit test. Closes #28996 from viirya/SPARK-29358. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/Dataset.scala | 65 +++++++++++++++++-- .../sql/DataFrameSetOperationsSuite.scala | 31 +++++++++ 2 files changed, 91 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 6f97121d88ede..3472b9fdec9d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2030,7 +2030,47 @@ class Dataset[T] private[sql]( * @group typedrel * @since 2.3.0 */ - def unionByName(other: Dataset[T]): Dataset[T] = withSetOperator { + def unionByName(other: Dataset[T]): Dataset[T] = unionByName(other, false) + + /** + * Returns a new Dataset containing union of rows in this Dataset and another Dataset. + * + * The difference between this function and [[union]] is that this function + * resolves columns by name (not by position). + * + * When the parameter `allowMissingColumns` is true, this function allows different set + * of column names between two Datasets. Missing columns at each side, will be filled with + * null values. The missing columns at left Dataset will be added at the end in the schema + * of the union result: + * + * {{{ + * val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2") + * val df2 = Seq((4, 5, 6)).toDF("col1", "col0", "col3") + * df1.unionByName(df2, true).show + * + * // output: "col3" is missing at left df1 and added at the end of schema. + * // +----+----+----+----+ + * // |col0|col1|col2|col3| + * // +----+----+----+----+ + * // | 1| 2| 3|null| + * // | 5| 4|null| 6| + * // +----+----+----+----+ + * + * df2.unionByName(df1, true).show + * + * // output: "col2" is missing at left df2 and added at the end of schema. + * // +----+----+----+----+ + * // |col1|col0|col3|col2| + * // +----+----+----+----+ + * // | 4| 5| 6|null| + * // | 2| 1|null| 3| + * // +----+----+----+----+ + * }}} + * + * @group typedrel + * @since 3.1.0 + */ + def unionByName(other: Dataset[T], allowMissingColumns: Boolean): Dataset[T] = withSetOperator { // Check column name duplication val resolver = sparkSession.sessionState.analyzer.resolver val leftOutputAttrs = logicalPlan.output @@ -2048,9 +2088,13 @@ class Dataset[T] private[sql]( // Builds a project list for `other` based on `logicalPlan` output names val rightProjectList = leftOutputAttrs.map { lattr => rightOutputAttrs.find { rattr => resolver(lattr.name, rattr.name) }.getOrElse { - throw new AnalysisException( - s"""Cannot resolve column name "${lattr.name}" among """ + - s"""(${rightOutputAttrs.map(_.name).mkString(", ")})""") + if (allowMissingColumns) { + Alias(Literal(null, lattr.dataType), lattr.name)() + } else { + throw new AnalysisException( + s"""Cannot resolve column name "${lattr.name}" among """ + + s"""(${rightOutputAttrs.map(_.name).mkString(", ")})""") + } } } @@ -2058,9 +2102,20 @@ class Dataset[T] private[sql]( val notFoundAttrs = rightOutputAttrs.diff(rightProjectList) val rightChild = Project(rightProjectList ++ notFoundAttrs, other.logicalPlan) + // Builds a project for `logicalPlan` based on `other` output names, if allowing + // missing columns. + val leftChild = if (allowMissingColumns) { + val missingAttrs = notFoundAttrs.map { attr => + Alias(Literal(null, attr.dataType), attr.name)() + } + Project(leftOutputAttrs ++ missingAttrs, logicalPlan) + } else { + logicalPlan + } + // This breaks caching, but it's usually ok because it addresses a very specific use case: // using union to union many files or partitions. - CombineUnions(Union(logicalPlan, rightChild)) + CombineUnions(Union(leftChild, rightChild)) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala index bd3f48078374d..11d7907c5a193 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala @@ -506,4 +506,35 @@ class DataFrameSetOperationsSuite extends QueryTest with SharedSparkSession { check(lit(2).cast("int"), $"c" === 2, Seq(Row(1, 1, 2, 0), Row(1, 1, 2, 1))) check(lit(2).cast("int"), $"c" =!= 2, Seq()) } + + test("SPARK-29358: Make unionByName optionally fill missing columns with nulls") { + var df1 = Seq(1, 2, 3).toDF("a") + var df2 = Seq(3, 1, 2).toDF("b") + val df3 = Seq(2, 3, 1).toDF("c") + val unionDf = df1.unionByName(df2.unionByName(df3, true), true) + checkAnswer(unionDf, + Row(1, null, null) :: Row(2, null, null) :: Row(3, null, null) :: // df1 + Row(null, 3, null) :: Row(null, 1, null) :: Row(null, 2, null) :: // df2 + Row(null, null, 2) :: Row(null, null, 3) :: Row(null, null, 1) :: Nil // df3 + ) + + df1 = Seq((1, 2)).toDF("a", "c") + df2 = Seq((3, 4, 5)).toDF("a", "b", "c") + checkAnswer(df1.unionByName(df2, true), + Row(1, 2, null) :: Row(3, 5, 4) :: Nil) + checkAnswer(df2.unionByName(df1, true), + Row(3, 4, 5) :: Row(1, null, 2) :: Nil) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + df2 = Seq((3, 4, 5)).toDF("a", "B", "C") + val union1 = df1.unionByName(df2, true) + val union2 = df2.unionByName(df1, true) + + checkAnswer(union1, Row(1, 2, null, null) :: Row(3, null, 4, 5) :: Nil) + checkAnswer(union2, Row(3, 4, 5, null) :: Row(1, null, null, 2) :: Nil) + + assert(union1.schema.fieldNames === Array("a", "c", "B", "C")) + assert(union2.schema.fieldNames === Array("a", "B", "C", "c")) + } + } } From 004aea8155d3b768684ab20cd29b3a820a204f22 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Sun, 12 Jul 2020 09:45:18 +0900 Subject: [PATCH 173/384] [SPARK-32154][SQL] Use ExpressionEncoder for the return type of ScalaUDF to convert to catalyst type ### What changes were proposed in this pull request? This PR proposes to use `ExpressionEncoder` for the return type of ScalaUDF to convert to the catalyst type, instead of using `CatalystTypeConverters`. Note, this change only takes effect for typed Scala UDF since its the only case where we know the type tag of the raw type. ### Why are the changes needed? Users now could register a UDF with `Instant`/`LocalDate` as return types even with `spark.sql.datetime.java8API.enabled=false`. However, the UDF can not really be used. For example, if we try: ```scala scala> sql("set spark.sql.datetime.java8API.enabled=false") scala> spark.udf.register("buildDate", udf{ d: String => java.time.LocalDate.parse(d) }) scala> Seq("2020-07-02").toDF("d").selectExpr("CAST(buildDate(d) AS STRING)").show ``` Then, we will hit the error: ```scala java.lang.ClassCastException: java.time.LocalDate cannot be cast to java.sql.Date at org.apache.spark.sql.catalyst.CatalystTypeConverters$DateConverter$.toCatalystImpl(CatalystTypeConverters.scala:304) at org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:107) at org.apache.spark.sql.catalyst.CatalystTypeConverters$.$anonfun$createToCatalystConverter$2(CatalystTypeConverters.scala:425) at org.apache.spark.sql.catalyst.expressions.ScalaUDF.eval(ScalaUDF.scala:1169) ... ``` as it actually requires enabling `spark.sql.datetime.java8API.enabled` when using the UDF. And I think this could make users get confused. This happens because when registering the UDF, Spark actually uses `ExpressionEncoder` to ser/deser types. However, when using UDF, Spark uses `CatalystTypeConverters`, which is under control of `spark.sql.datetime.java8API.enabled`, to ser/deser types. Therefore, Spark would fail to convert the Java8 date time types. If we could also use `ExpressionEncoder` to ser/deser types for the return type, similar to what we do for the input parameter types, then, UDF could support Instant/LocalDate, event other combined complex types as well. ### Does this PR introduce _any_ user-facing change? Yes. Before this PR, if users run the demo above, they would hit the error. After this PR, the demo will run successfully. ### How was this patch tested? Updated 2 tests and added a new one for combined types of `Instant` and `LocalDate`. Closes #28979 from Ngone51/udf-return-encoder. Authored-by: yi.wu Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/analysis/Analyzer.scala | 7 +- .../catalyst/encoders/ExpressionEncoder.scala | 8 +- .../sql/catalyst/expressions/ScalaUDF.scala | 26 +++- .../apache/spark/sql/UDFRegistration.scala | 120 +++++++++++------- .../sql/expressions/UserDefinedFunction.scala | 2 + .../org/apache/spark/sql/functions.scala | 60 +++++---- .../scala/org/apache/spark/sql/UDFSuite.scala | 106 ++++++++++++++-- 7 files changed, 235 insertions(+), 94 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 761f3575f1b86..d669509d18455 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2819,13 +2819,12 @@ class Analyzer( case p => p transformExpressionsUp { - case udf @ ScalaUDF(_, _, inputs, _, _, _, _) - if udf.inputPrimitives.contains(true) => + case udf: ScalaUDF if udf.inputPrimitives.contains(true) => // Otherwise, add special handling of null for fields that can't accept null. // The result of operations like this, when passed null, is generally to return null. - assert(udf.inputPrimitives.length == inputs.length) + assert(udf.inputPrimitives.length == udf.children.length) - val inputPrimitivesPair = udf.inputPrimitives.zip(inputs) + val inputPrimitivesPair = udf.inputPrimitives.zip(udf.children) val inputNullCheck = inputPrimitivesPair.collect { case (isPrimitive, input) if isPrimitive && input.nullable => IsNull(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index f08416fcaba8a..3d5c1855f6975 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -24,6 +24,7 @@ import scala.reflect.runtime.universe.{typeTag, TypeTag} import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.{InternalRow, JavaTypeInference, ScalaReflection} +import org.apache.spark.sql.catalyst.ScalaReflection.Schema import org.apache.spark.sql.catalyst.analysis.{Analyzer, GetColumnByOrdinal, SimpleAnalyzer, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.{Deserializer, Serializer} import org.apache.spark.sql.catalyst.expressions._ @@ -32,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, Initial import org.apache.spark.sql.catalyst.optimizer.{ReassignLambdaVariableID, SimplifyCasts} import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LeafNode, LocalRelation} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ObjectType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{DataType, ObjectType, StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -305,6 +306,11 @@ case class ExpressionEncoder[T]( StructField(s.name, s.dataType, s.nullable) }) + def dataTypeAndNullable: Schema = { + val dataType = if (isSerializedAsStruct) schema else schema.head.dataType + Schema(dataType, objSerializer.nullable) + } + /** * Returns true if the type `T` is serialized as a struct by `objSerializer`. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 242a065a58e50..44ee06ae011af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -37,6 +37,8 @@ import org.apache.spark.util.Utils * @param inputEncoders ExpressionEncoder for each input parameters. For a input parameter which * serialized as struct will use encoder instead of CatalystTypeConverters to * convert internal value to Scala value. + * @param outputEncoder ExpressionEncoder for the return type of function. It's only defined when + * this is a typed Scala UDF. * @param udfName The user-specified name of this UDF. * @param nullable True if the UDF can return null value. * @param udfDeterministic True if the UDF is deterministic. Deterministic UDF returns same result @@ -47,6 +49,7 @@ case class ScalaUDF( dataType: DataType, children: Seq[Expression], inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, + outputEncoder: Option[ExpressionEncoder[_]] = None, udfName: Option[String] = None, nullable: Boolean = true, udfDeterministic: Boolean = true) @@ -63,7 +66,7 @@ case class ScalaUDF( * Nil(has same effect with all false) and analyzer will skip null-handling * on them. */ - def inputPrimitives: Seq[Boolean] = { + lazy val inputPrimitives: Seq[Boolean] = { inputEncoders.map { encoderOpt => // It's possible that some of the inputs don't have a specific encoder(e.g. `Any`) if (encoderOpt.isDefined) { @@ -103,6 +106,23 @@ case class ScalaUDF( } } + /** + * Create the converter which converts the scala data type to the catalyst data type for + * the return data type of udf function. We'd use `ExpressionEncoder` to create the + * converter for typed ScalaUDF only, since its the only case where we know the type tag + * of the return data type of udf function. + */ + private def catalystConverter: Any => Any = outputEncoder.map { enc => + val toRow = enc.createSerializer().asInstanceOf[Any => Any] + if (enc.isSerializedAsStruct) { + value: Any => + if (value == null) null else toRow(value).asInstanceOf[InternalRow] + } else { + value: Any => + if (value == null) null else toRow(value).asInstanceOf[InternalRow].get(0, dataType) + } + }.getOrElse(createToCatalystConverter(dataType)) + /** * Create the converter which converts the catalyst data type to the scala data type. * We use `CatalystTypeConverters` to create the converter for: @@ -1072,7 +1092,7 @@ case class ScalaUDF( val (converters, useEncoders): (Array[Any => Any], Array[Boolean]) = (children.zipWithIndex.map { case (c, i) => scalaConverter(i, c.dataType) - }.toArray :+ (createToCatalystConverter(dataType), false)).unzip + }.toArray :+ (catalystConverter, false)).unzip val convertersTerm = ctx.addReferenceObj("converters", converters, s"$converterClassName[]") val errorMsgTerm = ctx.addReferenceObj("errMsg", udfErrorMessage) val resultTerm = ctx.freshName("result") @@ -1150,7 +1170,7 @@ case class ScalaUDF( """.stripMargin) } - private[this] val resultConverter = createToCatalystConverter(dataType) + private[this] val resultConverter = catalystConverter lazy val udfErrorMessage = { val funcCls = Utils.getSimpleName(function.getClass) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index ced4af46c3f30..0f6ae9c5d44e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -133,9 +133,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | * @since 1.3.0 | */ |def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { - | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + | val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + | val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) | val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = $inputEncoders - | val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + | val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) | val finalUdf = if (nullable) udf else udf.asNonNullable() | def builder(e: Seq[Expression]) = if (e.length == $x) { | finalUdf.createScalaUDF(e) @@ -179,9 +180,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 0) { finalUdf.createScalaUDF(e) @@ -199,9 +201,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 1) { finalUdf.createScalaUDF(e) @@ -219,9 +222,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 2) { finalUdf.createScalaUDF(e) @@ -239,9 +243,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 3) { finalUdf.createScalaUDF(e) @@ -259,9 +264,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 4) { finalUdf.createScalaUDF(e) @@ -279,9 +285,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 5) { finalUdf.createScalaUDF(e) @@ -299,9 +306,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 6) { finalUdf.createScalaUDF(e) @@ -319,9 +327,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 7) { finalUdf.createScalaUDF(e) @@ -339,9 +348,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 8) { finalUdf.createScalaUDF(e) @@ -359,9 +369,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 9) { finalUdf.createScalaUDF(e) @@ -379,9 +390,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 10) { finalUdf.createScalaUDF(e) @@ -399,9 +411,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 11) { finalUdf.createScalaUDF(e) @@ -419,9 +432,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 12) { finalUdf.createScalaUDF(e) @@ -439,9 +453,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 13) { finalUdf.createScalaUDF(e) @@ -459,9 +474,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 14) { finalUdf.createScalaUDF(e) @@ -479,9 +495,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 15) { finalUdf.createScalaUDF(e) @@ -499,9 +516,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 16) { finalUdf.createScalaUDF(e) @@ -519,9 +537,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 17) { finalUdf.createScalaUDF(e) @@ -539,9 +558,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 18) { finalUdf.createScalaUDF(e) @@ -559,9 +579,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Try(ExpressionEncoder[A19]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 19) { finalUdf.createScalaUDF(e) @@ -579,9 +600,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Try(ExpressionEncoder[A19]()).toOption :: Try(ExpressionEncoder[A20]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 20) { finalUdf.createScalaUDF(e) @@ -599,9 +621,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Try(ExpressionEncoder[A19]()).toOption :: Try(ExpressionEncoder[A20]()).toOption :: Try(ExpressionEncoder[A21]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 21) { finalUdf.createScalaUDF(e) @@ -619,9 +642,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Try(ExpressionEncoder[A19]()).toOption :: Try(ExpressionEncoder[A20]()).toOption :: Try(ExpressionEncoder[A21]()).toOption :: Try(ExpressionEncoder[A22]()).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders, outputEncoder).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 22) { finalUdf.createScalaUDF(e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 6a20a46756f85..0cef33509a175 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -94,6 +94,7 @@ private[spark] case class SparkUserDefinedFunction( f: AnyRef, dataType: DataType, inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, + outputEncoder: Option[ExpressionEncoder[_]] = None, name: Option[String] = None, nullable: Boolean = true, deterministic: Boolean = true) extends UserDefinedFunction { @@ -109,6 +110,7 @@ private[spark] case class SparkUserDefinedFunction( dataType, exprs, inputEncoders, + outputEncoder, udfName = name, nullable = nullable, udfDeterministic = deterministic) 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 239b705a473d0..f9e6e2f7b359c 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 @@ -4415,9 +4415,10 @@ object functions { | * @since 1.3.0 | */ |def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { - | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + | val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + | val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) | val inputEncoders = $inputEncoders - | val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + | val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) | if (nullable) udf else udf.asNonNullable() |}""".stripMargin) } @@ -4521,9 +4522,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4537,9 +4539,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4553,9 +4556,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4569,9 +4573,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4585,9 +4590,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4601,9 +4607,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4617,9 +4624,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4633,9 +4641,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4649,9 +4658,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4665,9 +4675,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } @@ -4681,9 +4692,10 @@ object functions { * @since 1.3.0 */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { - val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val outputEncoder = Try(ExpressionEncoder[RT]()).toOption + val ScalaReflection.Schema(dataType, nullable) = outputEncoder.map(_.dataTypeAndNullable).getOrElse(ScalaReflection.schemaFor[RT]) val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders, outputEncoder) if (nullable) udf else udf.asNonNullable() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 8b7e9ecfe4e4d..05a33f9aa17bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -18,15 +18,20 @@ package org.apache.spark.sql import java.math.BigDecimal +import java.sql.Timestamp +import java.time.{Instant, LocalDate} +import java.time.format.DateTimeFormatter import org.apache.spark.SparkException import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{QueryExecution, SimpleMode} import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, ExplainCommand} import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand +import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions.{lit, struct, udf} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -35,6 +40,8 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.QueryExecutionListener private case class FunctionResult(f1: String, f2: String) +private case class LocalDateInstantType(date: LocalDate, instant: Instant) +private case class TimestampInstantType(t: Timestamp, instant: Instant) class UDFSuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -506,23 +513,94 @@ class UDFSuite extends QueryTest with SharedSparkSession { } test("Using java.time.Instant in UDF") { - withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { - val expected = java.time.Instant.parse("2019-02-27T00:00:00Z") - val plusSec = udf((i: java.time.Instant) => i.plusSeconds(1)) - val df = spark.sql("SELECT TIMESTAMP '2019-02-26 23:59:59Z' as t") - .select(plusSec('t)) - assert(df.collect().toSeq === Seq(Row(expected))) - } + val dtf = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss") + val expected = java.time.Instant.parse("2019-02-27T00:00:00Z") + .atZone(DateTimeUtils.getZoneId(conf.sessionLocalTimeZone)) + .toLocalDateTime + .format(dtf) + val plusSec = udf((i: java.time.Instant) => i.plusSeconds(1)) + val df = spark.sql("SELECT TIMESTAMP '2019-02-26 23:59:59Z' as t") + .select(plusSec('t).cast(StringType)) + checkAnswer(df, Row(expected) :: Nil) } test("Using java.time.LocalDate in UDF") { - withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { - val expected = java.time.LocalDate.parse("2019-02-27") - val plusDay = udf((i: java.time.LocalDate) => i.plusDays(1)) - val df = spark.sql("SELECT DATE '2019-02-26' as d") - .select(plusDay('d)) - assert(df.collect().toSeq === Seq(Row(expected))) - } + val expected = java.time.LocalDate.parse("2019-02-27").toString + val plusDay = udf((i: java.time.LocalDate) => i.plusDays(1)) + val df = spark.sql("SELECT DATE '2019-02-26' as d") + .select(plusDay('d).cast(StringType)) + checkAnswer(df, Row(expected) :: Nil) + } + + test("Using combined types of Instant/LocalDate in UDF") { + val dtf = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss") + val date = LocalDate.parse("2019-02-26") + val instant = Instant.parse("2019-02-26T23:59:59Z") + val expectedDate = date.toString + val expectedInstant = + instant.atZone(DateTimeUtils.getZoneId(conf.sessionLocalTimeZone)) + .toLocalDateTime + .format(dtf) + val df = Seq((date, instant)).toDF("d", "i") + + // test normal case + spark.udf.register("buildLocalDateInstantType", + udf((d: LocalDate, i: Instant) => LocalDateInstantType(d, i))) + checkAnswer(df.selectExpr(s"buildLocalDateInstantType(d, i) as di") + .select('di.cast(StringType)), + Row(s"[$expectedDate, $expectedInstant]") :: Nil) + + // test null cases + spark.udf.register("buildLocalDateInstantType", + udf((d: LocalDate, i: Instant) => LocalDateInstantType(null, null))) + checkAnswer(df.selectExpr("buildLocalDateInstantType(d, i) as di"), + Row(Row(null, null))) + + spark.udf.register("buildLocalDateInstantType", + udf((d: LocalDate, i: Instant) => null.asInstanceOf[LocalDateInstantType])) + checkAnswer(df.selectExpr("buildLocalDateInstantType(d, i) as di"), + Row(null)) + } + + test("Using combined types of Instant/Timestamp in UDF") { + val dtf = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss") + val timestamp = Timestamp.valueOf("2019-02-26 23:59:59") + val instant = Instant.parse("2019-02-26T23:59:59Z") + val expectedTimestamp = timestamp.toLocalDateTime.format(dtf) + val expectedInstant = + instant.atZone(DateTimeUtils.getZoneId(conf.sessionLocalTimeZone)) + .toLocalDateTime + .format(dtf) + val df = Seq((timestamp, instant)).toDF("t", "i") + + // test normal case + spark.udf.register("buildTimestampInstantType", + udf((t: Timestamp, i: Instant) => TimestampInstantType(t, i))) + checkAnswer(df.selectExpr("buildTimestampInstantType(t, i) as ti") + .select('ti.cast(StringType)), + Row(s"[$expectedTimestamp, $expectedInstant]")) + + // test null cases + spark.udf.register("buildTimestampInstantType", + udf((t: Timestamp, i: Instant) => TimestampInstantType(null, null))) + checkAnswer(df.selectExpr("buildTimestampInstantType(t, i) as ti"), + Row(Row(null, null))) + + spark.udf.register("buildTimestampInstantType", + udf((t: Timestamp, i: Instant) => null.asInstanceOf[TimestampInstantType])) + checkAnswer(df.selectExpr("buildTimestampInstantType(t, i) as ti"), + Row(null)) + } + + test("SPARK-32154: return null with or without explicit type") { + // without explicit type + val udf1 = udf((i: String) => null) + assert(udf1.asInstanceOf[SparkUserDefinedFunction] .dataType === NullType) + checkAnswer(Seq("1").toDF("a").select(udf1('a)), Row(null) :: Nil) + // with explicit type + val udf2 = udf((i: String) => null.asInstanceOf[String]) + assert(udf2.asInstanceOf[SparkUserDefinedFunction].dataType === StringType) + checkAnswer(Seq("1").toDF("a").select(udf1('a)), Row(null) :: Nil) } test("SPARK-28321 0-args Java UDF should not be called only once") { From 6ae400ccbe79376ec9f7c25a56b691cef8f6c10f Mon Sep 17 00:00:00 2001 From: Michael Chirico Date: Sun, 12 Jul 2020 09:53:27 -0500 Subject: [PATCH 174/384] [MINOR][SQL][DOCS] consistency in argument naming for time functions ### What changes were proposed in this pull request? Rename documented argument `format` as `fmt`, to match the same argument name in several other SQL date/time functions, to wit, `date_format`, `date_trunc`, `trunc`, `to_date`, and `to_timestamp` all use `fmt`. Also `format_string` and `printf` use the same abbreviation in their argument `strfmt`. ### Why are the changes needed? Consistency -- I was trying to scour the documentation for functions with arguments using Java string formatting, it would have been nice to rely on searching for `fmt` instead of my more manual approach. ### Does this PR introduce _any_ user-facing change? In the documentation only ### How was this patch tested? No tests Closes #29007 from MichaelChirico/sql-doc-format-fmt. Authored-by: Michael Chirico Signed-off-by: Sean Owen --- .../expressions/datetimeExpressions.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) 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 1c379deb8e584..3d9612018aaf5 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 @@ -686,13 +686,13 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(timeExp[, format]) - Returns the UNIX timestamp of the given time.", + usage = "_FUNC_(timeExp[, fmt]) - Returns the UNIX timestamp of the given time.", arguments = """ Arguments: * timeExp - A date/timestamp or string which is returned as a UNIX timestamp. - * format - Date/time format pattern to follow. Ignored if `timeExp` is not a string. - Default value is "yyyy-MM-dd HH:mm:ss". See Datetime Patterns - for valid date and time format patterns. + * fmt - Date/time format pattern to follow. Ignored if `timeExp` is not a string. + Default value is "yyyy-MM-dd HH:mm:ss". See Datetime Patterns + for valid date and time format patterns. """, examples = """ Examples: @@ -734,13 +734,13 @@ case class ToUnixTimestamp( * second parameter. */ @ExpressionDescription( - usage = "_FUNC_([timeExp[, format]]) - Returns the UNIX timestamp of current or specified time.", + usage = "_FUNC_([timeExp[, fmt]]) - Returns the UNIX timestamp of current or specified time.", arguments = """ Arguments: * timeExp - A date/timestamp or string. If not provided, this defaults to current time. - * format - Date/time format pattern to follow. Ignored if `timeExp` is not a string. - Default value is "yyyy-MM-dd HH:mm:ss". See Datetime Patterns - for valid date and time format patterns. + * fmt - Date/time format pattern to follow. Ignored if `timeExp` is not a string. + Default value is "yyyy-MM-dd HH:mm:ss". See Datetime Patterns + for valid date and time format patterns. """, examples = """ Examples: @@ -891,16 +891,16 @@ abstract class UnixTime extends ToTimestamp { * Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string * representing the timestamp of that moment in the current system time zone in the given * format. If the format is missing, using format like "1970-01-01 00:00:00". - * Note that hive Language Manual says it returns 0 if fail, but in fact it returns null. + * Note that Hive Language Manual says it returns 0 if fail, but in fact it returns null. */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(unix_time, format) - Returns `unix_time` in the specified `format`.", + usage = "_FUNC_(unix_time, fmt) - Returns `unix_time` in the specified `fmt`.", arguments = """ Arguments: * unix_time - UNIX Timestamp to be converted to the provided format. - * format - Date/time format pattern to follow. See Datetime Patterns - for valid date and time format patterns. + * fmt - Date/time format pattern to follow. See Datetime Patterns + for valid date and time format patterns. """, examples = """ Examples: @@ -2140,4 +2140,3 @@ case class SubtractDates(left: Expression, right: Expression) }) } } - From c56c84af473547c9e9cab7ef6422f2b550084b59 Mon Sep 17 00:00:00 2001 From: Chuliang Xiao Date: Sun, 12 Jul 2020 09:01:41 -0700 Subject: [PATCH 175/384] [MINOR][DOCS] Fix typo in PySpark example in ml-datasource.md ### What changes were proposed in this pull request? This PR changes `true` to `True` in the python code. ### Why are the changes needed? The previous example is a syntax error. ### Does this PR introduce _any_ user-facing change? Yes, but this is doc-only typo fix. ### How was this patch tested? Manually run the example. Closes #29073 from ChuliangXiao/patch-1. Authored-by: Chuliang Xiao Signed-off-by: Dongjoon Hyun --- docs/ml-datasource.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ml-datasource.md b/docs/ml-datasource.md index 0f2f5f482ec50..8e9c947b75f38 100644 --- a/docs/ml-datasource.md +++ b/docs/ml-datasource.md @@ -86,7 +86,7 @@ Will output: In PySpark we provide Spark SQL data source API for loading image data as a DataFrame. {% highlight python %} ->>> df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens") +>>> df = spark.read.format("image").option("dropInvalid", True).load("data/mllib/images/origin/kittens") >>> df.select("image.origin", "image.width", "image.height").show(truncate=False) +-----------------------------------------------------------------------+-----+------+ |origin |width|height| From c4b0639f830cb5184328473db65e17b3fd0e74fc Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sun, 12 Jul 2020 09:44:27 -0700 Subject: [PATCH 176/384] [SPARK-32270][SQL] Use TextFileFormat in CSV's schema inference with a different encoding ### What changes were proposed in this pull request? This PR proposes to use text datasource in CSV's schema inference. This shares the same reasons of SPARK-18362, SPARK-19885 and SPARK-19918 - we're currently using Hadoop RDD when the encoding is different, which is unnecessary. This PR completes SPARK-18362, and address the comment at https://github.com/apache/spark/pull/15813#discussion_r90751405. We should better keep the code paths consistent with existing CSV and JSON datasources as well, but this CSV schema inference with the encoding specified is different from UTF-8 alone. There can be another story that this PR might lead to a bug fix: Spark session configurations, say Hadoop configurations, are not respected during CSV schema inference when the encoding is different (but it has to be set to Spark context for schema inference when the encoding is different). ### Why are the changes needed? For consistency, potentially better performance, and fixing a potentially very corner case bug. ### Does this PR introduce _any_ user-facing change? Virtually no. ### How was this patch tested? Existing tests should cover. Closes #29063 from HyukjinKwon/SPARK-32270. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../datasources/csv/CSVDataSource.scala | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala index 375cec597166c..cdac9d9c93925 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala @@ -150,21 +150,23 @@ object TextInputCSVDataSource extends CSVDataSource { inputPaths: Seq[FileStatus], options: CSVOptions): Dataset[String] = { val paths = inputPaths.map(_.getPath.toString) + val df = sparkSession.baseRelationToDataFrame( + DataSource.apply( + sparkSession, + paths = paths, + className = classOf[TextFileFormat].getName, + options = options.parameters + ).resolveRelation(checkFilesExist = false)) + .select("value").as[String](Encoders.STRING) + if (Charset.forName(options.charset) == StandardCharsets.UTF_8) { - sparkSession.baseRelationToDataFrame( - DataSource.apply( - sparkSession, - paths = paths, - className = classOf[TextFileFormat].getName, - options = options.parameters - ).resolveRelation(checkFilesExist = false)) - .select("value").as[String](Encoders.STRING) + df } else { val charset = options.charset - val rdd = sparkSession.sparkContext - .hadoopFile[LongWritable, Text, TextInputFormat](paths.mkString(",")) - .mapPartitions(_.map(pair => new String(pair._2.getBytes, 0, pair._2.getLength, charset))) - sparkSession.createDataset(rdd)(Encoders.STRING) + sparkSession.createDataset(df.queryExecution.toRdd.map { row => + val bytes = row.getBinary(0) + new String(bytes, 0, bytes.length, charset) + })(Encoders.STRING) } } } From ad90cbff42cce91ee106378f51552e438593c68d Mon Sep 17 00:00:00 2001 From: Frank Yin Date: Mon, 13 Jul 2020 05:04:47 +0900 Subject: [PATCH 177/384] [SPARK-31831][SQL][TESTS] Use subclasses for mock in HiveSessionImplSuite ### What changes were proposed in this pull request? Fix flaky test org.apache.spark.sql.hive.thriftserver.HiveSessionImplSuite by using subclasses to avoid classloader issue. ### Why are the changes needed? It causes build instability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? It is a fix for a flaky test, but need to run multiple times against Jenkins. Closes #29069 from frankyin-factual/hive-tests. Authored-by: Frank Yin Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../thriftserver/HiveSessionImplSuite.scala | 86 ++++++++++++++----- 1 file changed, 63 insertions(+), 23 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala index afca4b49b3b4c..42d86e98a7273 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala @@ -16,28 +16,30 @@ */ package org.apache.spark.sql.hive.thriftserver +import java.lang.reflect.InvocationTargetException +import java.nio.ByteBuffer +import java.util.UUID + import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.hadoop.hive.conf.HiveConf import org.apache.hive.service.cli.OperationHandle -import org.apache.hive.service.cli.operation.{GetCatalogsOperation, OperationManager} -import org.apache.hive.service.cli.session.{HiveSessionImpl, SessionManager} -import org.mockito.Mockito.{mock, verify, when} -import org.mockito.invocation.InvocationOnMock +import org.apache.hive.service.cli.operation.{GetCatalogsOperation, Operation, OperationManager} +import org.apache.hive.service.cli.session.{HiveSession, HiveSessionImpl, SessionManager} +import org.apache.hive.service.rpc.thrift.{THandleIdentifier, TOperationHandle, TOperationType} import org.apache.spark.SparkFunSuite class HiveSessionImplSuite extends SparkFunSuite { private var session: HiveSessionImpl = _ - private var operationManager: OperationManager = _ + private var operationManager: OperationManagerMock = _ override def beforeAll() { super.beforeAll() - // mock the instance first - we observed weird classloader issue on creating mock, so - // would like to avoid any cases classloader gets switched - val sessionManager = mock(classOf[SessionManager]) - operationManager = mock(classOf[OperationManager]) + val sessionManager = new SessionManager(null) + operationManager = new OperationManagerMock() session = new HiveSessionImpl( ThriftserverShimUtils.testedProtocolVersions.head, @@ -48,13 +50,6 @@ class HiveSessionImplSuite extends SparkFunSuite { ) session.setSessionManager(sessionManager) session.setOperationManager(operationManager) - when(operationManager.newGetCatalogsOperation(session)).thenAnswer( - (_: InvocationOnMock) => { - val operation = mock(classOf[GetCatalogsOperation]) - when(operation.getHandle).thenReturn(mock(classOf[OperationHandle])) - operation - } - ) session.open(Map.empty[String, String].asJava) } @@ -63,14 +58,59 @@ class HiveSessionImplSuite extends SparkFunSuite { val operationHandle1 = session.getCatalogs val operationHandle2 = session.getCatalogs - when(operationManager.closeOperation(operationHandle1)) - .thenThrow(classOf[NullPointerException]) - when(operationManager.closeOperation(operationHandle2)) - .thenThrow(classOf[NullPointerException]) - session.close() - verify(operationManager).closeOperation(operationHandle1) - verify(operationManager).closeOperation(operationHandle2) + assert(operationManager.getCalledHandles.contains(operationHandle1)) + assert(operationManager.getCalledHandles.contains(operationHandle2)) } } + +class GetCatalogsOperationMock(parentSession: HiveSession) + extends GetCatalogsOperation(parentSession) { + + override def runInternal(): Unit = {} + + override def getHandle: OperationHandle = { + val uuid: UUID = UUID.randomUUID() + val tHandleIdentifier: THandleIdentifier = new THandleIdentifier() + tHandleIdentifier.setGuid(getByteBufferFromUUID(uuid)) + tHandleIdentifier.setSecret(getByteBufferFromUUID(uuid)) + val tOperationHandle: TOperationHandle = new TOperationHandle() + tOperationHandle.setOperationId(tHandleIdentifier) + tOperationHandle.setOperationType(TOperationType.GET_TYPE_INFO) + tOperationHandle.setHasResultSetIsSet(false) + new OperationHandle(tOperationHandle) + } + + private def getByteBufferFromUUID(uuid: UUID): Array[Byte] = { + val bb: ByteBuffer = ByteBuffer.wrap(new Array[Byte](16)) + bb.putLong(uuid.getMostSignificantBits) + bb.putLong(uuid.getLeastSignificantBits) + bb.array + } +} + +class OperationManagerMock extends OperationManager { + private val calledHandles: mutable.Set[OperationHandle] = new mutable.HashSet[OperationHandle]() + + override def newGetCatalogsOperation(parentSession: HiveSession): GetCatalogsOperation = { + val operation = new GetCatalogsOperationMock(parentSession) + try { + val m = classOf[OperationManager].getDeclaredMethod("addOperation", classOf[Operation]) + m.setAccessible(true) + m.invoke(this, operation) + } catch { + case e@(_: NoSuchMethodException | _: IllegalAccessException | + _: InvocationTargetException) => + throw new RuntimeException(e) + } + operation + } + + override def closeOperation(opHandle: OperationHandle): Unit = { + calledHandles.add(opHandle) + throw new RuntimeException + } + + def getCalledHandles: mutable.Set[OperationHandle] = calledHandles +} From bc3d4bacb598d57ad9d43ff8c313ff9b8132b572 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 12 Jul 2020 14:50:47 -0700 Subject: [PATCH 178/384] [SPARK-32245][INFRA][FOLLOWUP] Reenable Github Actions on commit ### What changes were proposed in this pull request? This PR reenables GitHub Action on every commit as a next step. ### Why are the changes needed? We carefully enabled GitHub Action on every PRs, and it looks good so far. As we saw at https://github.com/apache/spark/pull/29072, GitHub Action is already triggered at every commits on every PRs. Enabling GitHub Action on `master` branch commit doesn't make a big difference. And, we need to start to test at every commit as a next step. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual. Closes #29076 from dongjoon-hyun/reenable_gha_commit. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index bf2feee9077d4..7bb5481a561b9 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1,6 +1,9 @@ name: master on: + push: + branches: + - master pull_request: branches: - master From b6229df16c02d9edcd53bc16ee12b199aaa0ee38 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 12 Jul 2020 15:34:43 -0700 Subject: [PATCH 179/384] [SPARK-32258][SQL] NormalizeFloatingNumbers directly normalizes IF/CaseWhen/Coalesce child expressions ### What changes were proposed in this pull request? This patch proposes to let `NormalizeFloatingNumbers` rule directly normalizes on certain children expressions. It could simplify expression tree. ### Why are the changes needed? Currently NormalizeFloatingNumbers rule treats some expressions as black box but we can optimize it a bit by normalizing directly the inner children expressions. Also see https://github.com/apache/spark/pull/28962#discussion_r448526240. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit tests. Closes #29061 from viirya/SPARK-32258. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../optimizer/NormalizeFloatingNumbers.scala | 11 ++++- .../NormalizeFloatingPointNumbersSuite.scala | 41 +++++++++++++++++-- 2 files changed, 48 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala index 8d5dbc7dc90eb..98c78c6312222 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, If, IsNull, KnownFloatingPointNormalized, LambdaFunction, Literal, NamedLambdaVariable, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CaseWhen, Coalesce, CreateArray, CreateMap, CreateNamedStruct, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, If, IsNull, KnownFloatingPointNormalized, LambdaFunction, Literal, NamedLambdaVariable, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, Window} @@ -116,6 +116,15 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { case CreateMap(children, useStringTypeWhenEmpty) => CreateMap(children.map(normalize), useStringTypeWhenEmpty) + case If(cond, trueValue, falseValue) => + If(cond, normalize(trueValue), normalize(falseValue)) + + case CaseWhen(branches, elseVale) => + CaseWhen(branches.map(br => (br._1, normalize(br._2))), elseVale.map(normalize)) + + case Coalesce(children) => + Coalesce(children.map(normalize)) + case _ if expr.dataType == FloatType || expr.dataType == DoubleType => KnownFloatingPointNormalized(NormalizeNaNAndZero(expr)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala index f5af416602c9d..3f6bdd206535b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{And, IsNull, KnownFloatingPointNormalized} +import org.apache.spark.sql.catalyst.expressions.{CaseWhen, If, IsNull, KnownFloatingPointNormalized} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -85,8 +85,43 @@ class NormalizeFloatingPointNumbersSuite extends PlanTest { val optimized = Optimize.execute(query) val doubleOptimized = Optimize.execute(optimized) val joinCond = IsNull(a) === IsNull(b) && - KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(a, 0.0))) === - KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(b, 0.0))) + coalesce(KnownFloatingPointNormalized(NormalizeNaNAndZero(a)), + KnownFloatingPointNormalized(NormalizeNaNAndZero(0.0))) === + coalesce(KnownFloatingPointNormalized(NormalizeNaNAndZero(b)), + KnownFloatingPointNormalized(NormalizeNaNAndZero(0.0))) + val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) + + comparePlans(doubleOptimized, correctAnswer) + } + + test("SPARK-32258: normalize the children of If") { + val cond = If(a > 0.1D, a, a + 0.2D) === b + val query = testRelation1.join(testRelation2, condition = Some(cond)) + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + + val joinCond = If(a > 0.1D, + KnownFloatingPointNormalized(NormalizeNaNAndZero(a)), + KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.2D))) === + KnownFloatingPointNormalized(NormalizeNaNAndZero(b)) + val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) + + comparePlans(doubleOptimized, correctAnswer) + } + + test("SPARK-32258: normalize the children of CaseWhen") { + val cond = CaseWhen( + Seq((a > 0.1D, a), (a > 0.2D, a + 0.2D)), + Some(a + 0.3D)) === b + val query = testRelation1.join(testRelation2, condition = Some(cond)) + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + + val joinCond = CaseWhen( + Seq((a > 0.1D, KnownFloatingPointNormalized(NormalizeNaNAndZero(a))), + (a > 0.2D, KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.2D)))), + Some(KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.3D)))) === + KnownFloatingPointNormalized(NormalizeNaNAndZero(b)) val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) comparePlans(doubleOptimized, correctAnswer) From 6d499647b36c45ff43e190af754a670321c6b274 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 13 Jul 2020 08:58:25 +0000 Subject: [PATCH 180/384] [SPARK-32105][SQL] Refactor current ScriptTransformationExec code MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? * Renamed hive transform scrip class `hive/execution/ScriptTransformationExec` to `hive/execution/HiveScriptTransformationExec` (don't rename file) * Extract class `BaseScriptTransformationExec ` about common code used across `SparkScriptTransformationExec(next pr add this)` and `HiveScriptTransformationExec` * Extract class `BaseScriptTransformationWriterThread` of writing data thread across `SparkScriptTransformationWriterThread(added next for support transform in sql/core )` and `HiveScriptTransformationWriterThread` , * `HiveScriptTransformationWriterThread` additionally supports Hive serde format * Rename current `Script` strategies in hive module to `HiveScript`, in next pr will add `SparkScript` strategies for support transform in sql/core. Todo List; - Support transform in sql/core base on `BaseScriptTransformationExec`, which would run script operator in SQL mode (without Hive). The output of script would be read as a string and column values are extracted by using a delimiter (default : tab character) - For Hive, by default only serde's must be used, and without hive we can run without serde - Cleanup past hacks that are observed (and people suggest / report), such as - [Solve string value error about Date/Timestamp in ScriptTransform](https://issues.apache.org/jira/browse/SPARK-31947) - [support use transform with aggregation](https://issues.apache.org/jira/browse/SPARK-28227) - [support array/map as transform's input](https://issues.apache.org/jira/browse/SPARK-22435) - Use code-gen projection to serialize rows to output stream() ### Why are the changes needed? Support run transform in SQL mode without hive ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Added UT Closes #27983 from AngersZhuuuu/follow_spark_15694. Authored-by: angerszhu Signed-off-by: Wenchen Fan --- .../BaseScriptTransformationExec.scala | 202 ++++++++ .../sql/hive/HiveSessionStateBuilder.scala | 3 +- .../spark/sql/hive/HiveStrategies.scala | 5 +- .../execution/ScriptTransformationExec.scala | 432 +++++++----------- ...la => HiveScriptTransformationSuite.scala} | 18 +- 5 files changed, 380 insertions(+), 280 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala rename sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/{ScriptTransformationSuite.scala => HiveScriptTransformationSuite.scala} (94%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala new file mode 100644 index 0000000000000..22bf6df58b040 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import java.io.OutputStream +import java.nio.charset.StandardCharsets +import java.util.concurrent.TimeUnit + +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeSet, UnsafeProjection} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DataType +import org.apache.spark.util.{CircularBuffer, SerializableConfiguration, Utils} + +trait BaseScriptTransformationExec extends UnaryExecNode { + + override def producedAttributes: AttributeSet = outputSet -- inputSet + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def doExecute(): RDD[InternalRow] = { + val broadcastedHadoopConf = + new SerializableConfiguration(sqlContext.sessionState.newHadoopConf()) + + child.execute().mapPartitions { iter => + if (iter.hasNext) { + val proj = UnsafeProjection.create(schema) + processIterator(iter, broadcastedHadoopConf.value).map(proj) + } else { + // If the input iterator has no rows then do not launch the external script. + Iterator.empty + } + } + } + + def processIterator( + inputIterator: Iterator[InternalRow], + hadoopConf: Configuration): Iterator[InternalRow] + + protected def checkFailureAndPropagate( + writerThread: BaseScriptTransformationWriterThread, + cause: Throwable = null, + proc: Process, + stderrBuffer: CircularBuffer): Unit = { + if (writerThread.exception.isDefined) { + throw writerThread.exception.get + } + + // There can be a lag between reader read EOF and the process termination. + // If the script fails to startup, this kind of error may be missed. + // So explicitly waiting for the process termination. + val timeout = conf.getConf(SQLConf.SCRIPT_TRANSFORMATION_EXIT_TIMEOUT) + val exitRes = proc.waitFor(timeout, TimeUnit.SECONDS) + if (!exitRes) { + log.warn(s"Transformation script process exits timeout in $timeout seconds") + } + + if (!proc.isAlive) { + val exitCode = proc.exitValue() + if (exitCode != 0) { + logError(stderrBuffer.toString) // log the stderr circular buffer + throw new SparkException(s"Subprocess exited with status $exitCode. " + + s"Error: ${stderrBuffer.toString}", cause) + } + } + } +} + +abstract class BaseScriptTransformationWriterThread( + iter: Iterator[InternalRow], + inputSchema: Seq[DataType], + ioSchema: BaseScriptTransformIOSchema, + outputStream: OutputStream, + proc: Process, + stderrBuffer: CircularBuffer, + taskContext: TaskContext, + conf: Configuration) extends Thread with Logging { + + setDaemon(true) + + @volatile protected var _exception: Throwable = null + + /** Contains the exception thrown while writing the parent iterator to the external process. */ + def exception: Option[Throwable] = Option(_exception) + + protected def processRows(): Unit + + protected def processRowsWithoutSerde(): Unit = { + val len = inputSchema.length + iter.foreach { row => + val data = if (len == 0) { + ioSchema.inputRowFormatMap("TOK_TABLEROWFORMATLINES") + } else { + val sb = new StringBuilder + sb.append(row.get(0, inputSchema(0))) + var i = 1 + while (i < len) { + sb.append(ioSchema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + sb.append(row.get(i, inputSchema(i))) + i += 1 + } + sb.append(ioSchema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")) + sb.toString() + } + outputStream.write(data.getBytes(StandardCharsets.UTF_8)) + } + } + + override def run(): Unit = Utils.logUncaughtExceptions { + TaskContext.setTaskContext(taskContext) + + // We can't use Utils.tryWithSafeFinally here because we also need a `catch` block, so + // let's use a variable to record whether the `finally` block was hit due to an exception + var threwException: Boolean = true + try { + processRows() + threwException = false + } catch { + // SPARK-25158 Exception should not be thrown again, otherwise it will be captured by + // SparkUncaughtExceptionHandler, then Executor will exit because of this Uncaught Exception, + // so pass the exception to `ScriptTransformationExec` is enough. + case t: Throwable => + // An error occurred while writing input, so kill the child process. According to the + // Javadoc this call will not throw an exception: + _exception = t + proc.destroy() + logError("Thread-ScriptTransformation-Feed exit cause by: ", t) + } finally { + try { + Utils.tryLogNonFatalError(outputStream.close()) + if (proc.waitFor() != 0) { + logError(stderrBuffer.toString) // log the stderr circular buffer + } + } catch { + case NonFatal(exceptionFromFinallyBlock) => + if (!threwException) { + throw exceptionFromFinallyBlock + } else { + log.error("Exception in finally block", exceptionFromFinallyBlock) + } + } + } + } +} + +/** + * The wrapper class of input and output schema properties + */ +abstract class BaseScriptTransformIOSchema extends Serializable { + import ScriptIOSchema._ + + def inputRowFormat: Seq[(String, String)] + + def outputRowFormat: Seq[(String, String)] + + def inputSerdeClass: Option[String] + + def outputSerdeClass: Option[String] + + def inputSerdeProps: Seq[(String, String)] + + def outputSerdeProps: Seq[(String, String)] + + def recordReaderClass: Option[String] + + def recordWriterClass: Option[String] + + def schemaLess: Boolean + + val inputRowFormatMap = inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + val outputRowFormatMap = outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) +} + +object ScriptIOSchema { + val defaultFormat = Map( + ("TOK_TABLEROWFORMATFIELD", "\t"), + ("TOK_TABLEROWFORMATLINES", "\n") + ) +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index e25610757a69b..78ec2b8e2047e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -111,7 +111,8 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session override val sparkSession: SparkSession = session override def extraPlanningStrategies: Seq[Strategy] = - super.extraPlanningStrategies ++ customPlanningStrategies ++ Seq(HiveTableScans, Scripts) + super.extraPlanningStrategies ++ customPlanningStrategies ++ + Seq(HiveTableScans, HiveScripts) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 2b1eb05e22cc7..dae68df08f32e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.hive.execution._ +import org.apache.spark.sql.hive.execution.{HiveScriptIOSchema, HiveScriptTransformationExec} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -240,11 +241,11 @@ private[hive] trait HiveStrategies { val sparkSession: SparkSession - object Scripts extends Strategy { + object HiveScripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ScriptTransformation(input, script, output, child, ioschema) => val hiveIoSchema = HiveScriptIOSchema(ioschema) - ScriptTransformationExec(input, script, output, planLater(child), hiveIoSchema) :: Nil + HiveScriptTransformationExec(input, script, output, planLater(child), hiveIoSchema) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala index c7183fd7385a6..96fe646d39fde 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive.execution import java.io._ import java.nio.charset.StandardCharsets import java.util.Properties -import java.util.concurrent.TimeUnit import javax.annotation.Nullable import scala.collection.JavaConverters._ @@ -33,19 +32,15 @@ import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.io.Writable -import org.apache.spark.{SparkException, TaskContext} -import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD +import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema -import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive.HiveInspectors import org.apache.spark.sql.hive.HiveShim._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.DataType -import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfiguration, Utils} +import org.apache.spark.util.{CircularBuffer, RedirectThread, Utils} /** * Transforms the input by forking and running the specified script. @@ -54,301 +49,211 @@ import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfig * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ -case class ScriptTransformationExec( +case class HiveScriptTransformationExec( input: Seq[Expression], script: String, output: Seq[Attribute], child: SparkPlan, ioschema: HiveScriptIOSchema) - extends UnaryExecNode { - - override def producedAttributes: AttributeSet = outputSet -- inputSet - - override def outputPartitioning: Partitioning = child.outputPartitioning - - protected override def doExecute(): RDD[InternalRow] = { - def processIterator(inputIterator: Iterator[InternalRow], hadoopConf: Configuration) - : Iterator[InternalRow] = { - val cmd = List("/bin/bash", "-c", script) - val builder = new ProcessBuilder(cmd.asJava) - - val proc = builder.start() - val inputStream = proc.getInputStream - val outputStream = proc.getOutputStream - val errorStream = proc.getErrorStream - - // In order to avoid deadlocks, we need to consume the error output of the child process. - // To avoid issues caused by large error output, we use a circular buffer to limit the amount - // of error output that we retain. See SPARK-7862 for more discussion of the deadlock / hang - // that motivates this. - val stderrBuffer = new CircularBuffer(2048) - new RedirectThread( - errorStream, - stderrBuffer, - "Thread-ScriptTransformation-STDERR-Consumer").start() - - val outputProjection = new InterpretedProjection(input, child.output) - - // This nullability is a performance optimization in order to avoid an Option.foreach() call - // inside of a loop - @Nullable val (inputSerde, inputSoi) = ioschema.initInputSerDe(input).getOrElse((null, null)) - - // This new thread will consume the ScriptTransformation's input rows and write them to the - // external process. That process's output will be read by this current thread. - val writerThread = new ScriptTransformationWriterThread( - inputIterator.map(outputProjection), - input.map(_.dataType), - inputSerde, - inputSoi, - ioschema, - outputStream, - proc, - stderrBuffer, - TaskContext.get(), - hadoopConf - ) - - // This nullability is a performance optimization in order to avoid an Option.foreach() call - // inside of a loop - @Nullable val (outputSerde, outputSoi) = { - ioschema.initOutputSerDe(output).getOrElse((null, null)) - } - - val reader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8)) - val outputIterator: Iterator[InternalRow] = new Iterator[InternalRow] with HiveInspectors { - var curLine: String = null - val scriptOutputStream = new DataInputStream(inputStream) - - @Nullable val scriptOutputReader = - ioschema.recordReader(scriptOutputStream, hadoopConf).orNull - - var scriptOutputWritable: Writable = null - val reusedWritableObject: Writable = if (null != outputSerde) { - outputSerde.getSerializedClass().getConstructor().newInstance() - } else { - null - } - val mutableRow = new SpecificInternalRow(output.map(_.dataType)) + extends BaseScriptTransformationExec { + + override def processIterator( + inputIterator: Iterator[InternalRow], + hadoopConf: Configuration): Iterator[InternalRow] = { + val cmd = List("/bin/bash", "-c", script) + val builder = new ProcessBuilder(cmd.asJava) + + val proc = builder.start() + val inputStream = proc.getInputStream + val outputStream = proc.getOutputStream + val errorStream = proc.getErrorStream + + // In order to avoid deadlocks, we need to consume the error output of the child process. + // To avoid issues caused by large error output, we use a circular buffer to limit the amount + // of error output that we retain. See SPARK-7862 for more discussion of the deadlock / hang + // that motivates this. + val stderrBuffer = new CircularBuffer(2048) + new RedirectThread( + errorStream, + stderrBuffer, + "Thread-ScriptTransformation-STDERR-Consumer").start() + + val outputProjection = new InterpretedProjection(input, child.output) + + // This nullability is a performance optimization in order to avoid an Option.foreach() call + // inside of a loop + @Nullable val (inputSerde, inputSoi) = ioschema.initInputSerDe(input).getOrElse((null, null)) + + // This new thread will consume the ScriptTransformation's input rows and write them to the + // external process. That process's output will be read by this current thread. + val writerThread = new HiveScriptTransformationWriterThread( + inputIterator.map(outputProjection), + input.map(_.dataType), + inputSerde, + inputSoi, + ioschema, + outputStream, + proc, + stderrBuffer, + TaskContext.get(), + hadoopConf + ) + + // This nullability is a performance optimization in order to avoid an Option.foreach() call + // inside of a loop + @Nullable val (outputSerde, outputSoi) = { + ioschema.initOutputSerDe(output).getOrElse((null, null)) + } - @transient - lazy val unwrappers = outputSoi.getAllStructFieldRefs.asScala.map(unwrapperFor) + val reader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8)) + val outputIterator: Iterator[InternalRow] = new Iterator[InternalRow] with HiveInspectors { + var curLine: String = null + val scriptOutputStream = new DataInputStream(inputStream) - private def checkFailureAndPropagate(cause: Throwable = null): Unit = { - if (writerThread.exception.isDefined) { - throw writerThread.exception.get - } + @Nullable val scriptOutputReader = + ioschema.recordReader(scriptOutputStream, hadoopConf).orNull - // There can be a lag between reader read EOF and the process termination. - // If the script fails to startup, this kind of error may be missed. - // So explicitly waiting for the process termination. - val timeout = conf.getConf(SQLConf.SCRIPT_TRANSFORMATION_EXIT_TIMEOUT) - val exitRes = proc.waitFor(timeout, TimeUnit.SECONDS) - if (!exitRes) { - log.warn(s"Transformation script process exits timeout in $timeout seconds") - } + var scriptOutputWritable: Writable = null + val reusedWritableObject: Writable = if (null != outputSerde) { + outputSerde.getSerializedClass().getConstructor().newInstance() + } else { + null + } + val mutableRow = new SpecificInternalRow(output.map(_.dataType)) - if (!proc.isAlive) { - val exitCode = proc.exitValue() - if (exitCode != 0) { - logError(stderrBuffer.toString) // log the stderr circular buffer - throw new SparkException(s"Subprocess exited with status $exitCode. " + - s"Error: ${stderrBuffer.toString}", cause) - } - } - } + @transient + lazy val unwrappers = outputSoi.getAllStructFieldRefs.asScala.map(unwrapperFor) - override def hasNext: Boolean = { - try { - if (outputSerde == null) { + override def hasNext: Boolean = { + try { + if (outputSerde == null) { + if (curLine == null) { + curLine = reader.readLine() if (curLine == null) { - curLine = reader.readLine() - if (curLine == null) { - checkFailureAndPropagate() - return false - } + checkFailureAndPropagate(writerThread, null, proc, stderrBuffer) + return false } - } else if (scriptOutputWritable == null) { - scriptOutputWritable = reusedWritableObject + } + } else if (scriptOutputWritable == null) { + scriptOutputWritable = reusedWritableObject - if (scriptOutputReader != null) { - if (scriptOutputReader.next(scriptOutputWritable) <= 0) { - checkFailureAndPropagate() + if (scriptOutputReader != null) { + if (scriptOutputReader.next(scriptOutputWritable) <= 0) { + checkFailureAndPropagate(writerThread, null, proc, stderrBuffer) + return false + } + } else { + try { + scriptOutputWritable.readFields(scriptOutputStream) + } catch { + case _: EOFException => + // This means that the stdout of `proc` (ie. TRANSFORM process) has exhausted. + // Ideally the proc should *not* be alive at this point but + // there can be a lag between EOF being written out and the process + // being terminated. So explicitly waiting for the process to be done. + checkFailureAndPropagate(writerThread, null, proc, stderrBuffer) return false - } - } else { - try { - scriptOutputWritable.readFields(scriptOutputStream) - } catch { - case _: EOFException => - // This means that the stdout of `proc` (ie. TRANSFORM process) has exhausted. - // Ideally the proc should *not* be alive at this point but - // there can be a lag between EOF being written out and the process - // being terminated. So explicitly waiting for the process to be done. - checkFailureAndPropagate() - return false - } } } + } - true - } catch { - case NonFatal(e) => - // If this exception is due to abrupt / unclean termination of `proc`, - // then detect it and propagate a better exception message for end users - checkFailureAndPropagate(e) + true + } catch { + case NonFatal(e) => + // If this exception is due to abrupt / unclean termination of `proc`, + // then detect it and propagate a better exception message for end users + checkFailureAndPropagate(writerThread, e, proc, stderrBuffer) - throw e - } + throw e } + } - override def next(): InternalRow = { - if (!hasNext) { - throw new NoSuchElementException + override def next(): InternalRow = { + if (!hasNext) { + throw new NoSuchElementException + } + if (outputSerde == null) { + val prevLine = curLine + curLine = reader.readLine() + if (!ioschema.schemaLess) { + new GenericInternalRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + .map(CatalystTypeConverters.convertToCatalyst)) + } else { + new GenericInternalRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + .map(CatalystTypeConverters.convertToCatalyst)) } - if (outputSerde == null) { - val prevLine = curLine - curLine = reader.readLine() - if (!ioschema.schemaLess) { - new GenericInternalRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) - .map(CatalystTypeConverters.convertToCatalyst)) + } else { + val raw = outputSerde.deserialize(scriptOutputWritable) + scriptOutputWritable = null + val dataList = outputSoi.getStructFieldsDataAsList(raw) + var i = 0 + while (i < dataList.size()) { + if (dataList.get(i) == null) { + mutableRow.setNullAt(i) } else { - new GenericInternalRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) - .map(CatalystTypeConverters.convertToCatalyst)) - } - } else { - val raw = outputSerde.deserialize(scriptOutputWritable) - scriptOutputWritable = null - val dataList = outputSoi.getStructFieldsDataAsList(raw) - var i = 0 - while (i < dataList.size()) { - if (dataList.get(i) == null) { - mutableRow.setNullAt(i) - } else { - unwrappers(i)(dataList.get(i), mutableRow, i) - } - i += 1 + unwrappers(i)(dataList.get(i), mutableRow, i) } - mutableRow + i += 1 } + mutableRow } } - - writerThread.start() - - outputIterator } - val broadcastedHadoopConf = - new SerializableConfiguration(sqlContext.sessionState.newHadoopConf()) + writerThread.start() - child.execute().mapPartitions { iter => - if (iter.hasNext) { - val proj = UnsafeProjection.create(schema) - processIterator(iter, broadcastedHadoopConf.value).map(proj) - } else { - // If the input iterator has no rows then do not launch the external script. - Iterator.empty - } - } + outputIterator } } -private class ScriptTransformationWriterThread( +private class HiveScriptTransformationWriterThread( iter: Iterator[InternalRow], inputSchema: Seq[DataType], @Nullable inputSerde: AbstractSerDe, @Nullable inputSoi: StructObjectInspector, - ioschema: HiveScriptIOSchema, + ioSchema: HiveScriptIOSchema, outputStream: OutputStream, proc: Process, stderrBuffer: CircularBuffer, taskContext: TaskContext, - conf: Configuration - ) extends Thread("Thread-ScriptTransformation-Feed") with HiveInspectors with Logging { - - setDaemon(true) - - @volatile private var _exception: Throwable = null - - /** Contains the exception thrown while writing the parent iterator to the external process. */ - def exception: Option[Throwable] = Option(_exception) - - override def run(): Unit = Utils.logUncaughtExceptions { - TaskContext.setTaskContext(taskContext) - + conf: Configuration) + extends BaseScriptTransformationWriterThread( + iter, + inputSchema, + ioSchema, + outputStream, + proc, + stderrBuffer, + taskContext, + conf) with HiveInspectors { + + override def processRows(): Unit = { val dataOutputStream = new DataOutputStream(outputStream) - @Nullable val scriptInputWriter = ioschema.recordWriter(dataOutputStream, conf).orNull - - // We can't use Utils.tryWithSafeFinally here because we also need a `catch` block, so - // let's use a variable to record whether the `finally` block was hit due to an exception - var threwException: Boolean = true - val len = inputSchema.length - try { - if (inputSerde == null) { - iter.foreach { row => - val data = if (len == 0) { - ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES") - } else { - val sb = new StringBuilder - sb.append(row.get(0, inputSchema(0))) - var i = 1 - while (i < len) { - sb.append(ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD")) - sb.append(row.get(i, inputSchema(i))) - i += 1 - } - sb.append(ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")) - sb.toString() - } - outputStream.write(data.getBytes(StandardCharsets.UTF_8)) + @Nullable val scriptInputWriter = ioSchema.recordWriter(dataOutputStream, conf).orNull + + if (inputSerde == null) { + processRowsWithoutSerde() + } else { + // Convert Spark InternalRows to hive data via `HiveInspectors.wrapperFor`. + val hiveData = new Array[Any](inputSchema.length) + val fieldOIs = inputSoi.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray + val wrappers = fieldOIs.zip(inputSchema).map { case (f, dt) => wrapperFor(f, dt) } + + iter.foreach { row => + var i = 0 + while (i < fieldOIs.length) { + hiveData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, inputSchema(i))) + i += 1 } - } else { - // Convert Spark InternalRows to hive data via `HiveInspectors.wrapperFor`. - val hiveData = new Array[Any](inputSchema.length) - val fieldOIs = inputSoi.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray - val wrappers = fieldOIs.zip(inputSchema).map { case (f, dt) => wrapperFor(f, dt) } - iter.foreach { row => - var i = 0 - while (i < fieldOIs.length) { - hiveData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, inputSchema(i))) - i += 1 - } - - val writable = inputSerde.serialize(hiveData, inputSoi) - if (scriptInputWriter != null) { - scriptInputWriter.write(writable) - } else { - prepareWritable(writable, ioschema.outputSerdeProps).write(dataOutputStream) - } - } - } - threwException = false - } catch { - // SPARK-25158 Exception should not be thrown again, otherwise it will be captured by - // SparkUncaughtExceptionHandler, then Executor will exit because of this Uncaught Exception, - // so pass the exception to `ScriptTransformationExec` is enough. - case t: Throwable => - // An error occurred while writing input, so kill the child process. According to the - // Javadoc this call will not throw an exception: - _exception = t - proc.destroy() - logError("Thread-ScriptTransformation-Feed exit cause by: ", t) - } finally { - try { - Utils.tryLogNonFatalError(outputStream.close()) - if (proc.waitFor() != 0) { - logError(stderrBuffer.toString) // log the stderr circular buffer + val writable = inputSerde.serialize(hiveData, inputSoi) + if (scriptInputWriter != null) { + scriptInputWriter.write(writable) + } else { + prepareWritable(writable, ioSchema.outputSerdeProps).write(dataOutputStream) } - } catch { - case NonFatal(exceptionFromFinallyBlock) => - if (!threwException) { - throw exceptionFromFinallyBlock - } else { - log.error("Exception in finally block", exceptionFromFinallyBlock) - } } } } @@ -382,16 +287,7 @@ case class HiveScriptIOSchema ( recordReaderClass: Option[String], recordWriterClass: Option[String], schemaLess: Boolean) - extends HiveInspectors { - - private val defaultFormat = Map( - ("TOK_TABLEROWFORMATFIELD", "\t"), - ("TOK_TABLEROWFORMATLINES", "\n") - ) - - val inputRowFormatMap = inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) - val outputRowFormatMap = outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) - + extends BaseScriptTransformIOSchema with HiveInspectors { def initInputSerDe(input: Seq[Expression]): Option[(AbstractSerDe, StructObjectInspector)] = { inputSerdeClass.map { serdeClass => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala similarity index 94% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala index b97eb869a9e54..35252fc47f49f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StringType -class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with TestHiveSingleton +class HiveScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { import spark.implicits._ @@ -83,7 +83,7 @@ class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with Tes val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformationExec( + (child: SparkPlan) => new HiveScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), @@ -100,7 +100,7 @@ class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with Tes val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformationExec( + (child: SparkPlan) => new HiveScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), @@ -118,7 +118,7 @@ class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with Tes val e = intercept[TestFailedException] { checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformationExec( + (child: SparkPlan) => new HiveScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), @@ -139,7 +139,7 @@ class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with Tes val e = intercept[TestFailedException] { checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformationExec( + (child: SparkPlan) => new HiveScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), @@ -160,7 +160,7 @@ class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with Tes val e = intercept[SparkException] { val plan = - new ScriptTransformationExec( + new HiveScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "some_non_existent_command", output = Seq(AttributeReference("a", StringType)()), @@ -181,7 +181,7 @@ class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with Tes checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformationExec( + (child: SparkPlan) => new HiveScriptTransformationExec( input = Seq(rowsDf.col("name").expr), script = "cat", output = Seq(AttributeReference("name", StringType)()), @@ -234,7 +234,7 @@ class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with Tes val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") val e = intercept[SparkException] { val plan = - new ScriptTransformationExec( + new HiveScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "some_non_existent_command", output = Seq(AttributeReference("a", StringType)()), @@ -252,7 +252,7 @@ class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with Tes val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") val e = intercept[SparkException] { val plan = - new ScriptTransformationExec( + new HiveScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "some_non_existent_command", output = Seq(AttributeReference("a", StringType)()), From 5521afbd227ecd0adf1a914698738d4ebe1bac8c Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 13 Jul 2020 08:23:25 -0700 Subject: [PATCH 181/384] [SPARK-32220][SQL][FOLLOW-UP] SHUFFLE_REPLICATE_NL Hint should not change Non-Cartesian Product join result ### What changes were proposed in this pull request? follow comment https://github.com/apache/spark/pull/29035#discussion_r453468999 Explain for pr ### Why are the changes needed? add comment ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Not need Closes #29084 from AngersZhuuuu/follow-spark-32220. Authored-by: angerszhu Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/SparkStrategies.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 583e5a2c5c57e..7b5d8f15962d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -159,7 +159,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // 4. Pick cartesian product if join type is inner like. // 5. Pick broadcast nested loop join as the final solution. It may OOM but we don't have // other choice. - case p @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, hint) => + case j @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, nonEquiCond, left, right, hint) => def createBroadcastHashJoin(onlyLookingAtHint: Boolean) = { getBroadcastBuildSide(left, right, joinType, hint, onlyLookingAtHint, conf).map { buildSide => @@ -168,7 +168,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { rightKeys, joinType, buildSide, - condition, + nonEquiCond, planLater(left), planLater(right))) } @@ -182,7 +182,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { rightKeys, joinType, buildSide, - condition, + nonEquiCond, planLater(left), planLater(right))) } @@ -191,7 +191,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def createSortMergeJoin() = { if (RowOrdering.isOrderable(leftKeys)) { Some(Seq(joins.SortMergeJoinExec( - leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)))) + leftKeys, rightKeys, joinType, nonEquiCond, planLater(left), planLater(right)))) } else { None } @@ -199,7 +199,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def createCartesianProduct() = { if (joinType.isInstanceOf[InnerLike]) { - Some(Seq(joins.CartesianProductExec(planLater(left), planLater(right), p.condition))) + // `CartesianProductExec` can't implicitly evaluate equal join condition, here we should + // pass the original condition which includes both equal and non-equal conditions. + Some(Seq(joins.CartesianProductExec(planLater(left), planLater(right), j.condition))) } else { None } @@ -220,7 +222,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // This join could be very slow or OOM val buildSide = getSmallerSide(left, right) Seq(joins.BroadcastNestedLoopJoinExec( - planLater(left), planLater(right), buildSide, joinType, condition)) + planLater(left), planLater(right), buildSide, joinType, nonEquiCond)) } } From 27ef3629dd96d5ee3368cdb258561ff96e907880 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 13 Jul 2020 08:31:39 -0700 Subject: [PATCH 182/384] [SPARK-32292][SPARK-32252][INFRA] Run the relevant tests only in GitHub Actions ### What changes were proposed in this pull request? This PR mainly proposes to run only relevant tests just like Jenkins PR builder does. Currently, GitHub Actions always run full tests which wastes the resources. In addition, this PR also fixes 3 more issues very closely related together while I am here. 1. The main idea here is: It reuses the existing logic embedded in `dev/run-tests.py` which Jenkins PR builder use in order to run only the related test cases. 2. While I am here, I fixed SPARK-32292 too to run the doc tests. It was because other references were not available when it is cloned via `checkoutv2`. With `fetch-depth: 0`, the history is available. 3. In addition, it fixes the `dev/run-tests.py` to match with `python/run-tests.py` in terms of its options. Environment variables such as `TEST_ONLY_XXX` were moved as proper options. For example, ```bash dev/run-tests.py --modules sql,core ``` which is consistent with `python/run-tests.py`, for example, ```bash python/run-tests.py --modules pyspark-core,pyspark-ml ``` 4. Lastly, also fixed the formatting issue in module specification in the matrix: ```diff - network_common, network_shuffle, repl, launcher + network-common, network-shuffle, repl, launcher, ``` which incorrectly runs build/test the modules. ### Why are the changes needed? By running only related tests, we can hugely save the resources and avoid unrelated flaky tests, etc. Also, now it runs the doctest of `dev/run-tests.py` properly, the usages are similar between `dev/run-tests.py` and `python/run-tests.py`, and run `network-common`, `network-shuffle`, `launcher` and `examples` modules too. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually tested in my own forked Spark: https://github.com/HyukjinKwon/spark/pull/7 https://github.com/HyukjinKwon/spark/pull/8 https://github.com/HyukjinKwon/spark/pull/9 https://github.com/HyukjinKwon/spark/pull/10 https://github.com/HyukjinKwon/spark/pull/11 https://github.com/HyukjinKwon/spark/pull/12 Closes #29086 from HyukjinKwon/SPARK-32292. Authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 16 +++-- dev/run-tests.py | 109 +++++++++++++++++++++++++++-------- 2 files changed, 96 insertions(+), 29 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 7bb5481a561b9..d6458bf44fd36 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -29,7 +29,7 @@ jobs: modules: - |- core, unsafe, kvstore, avro, - network_common, network_shuffle, repl, launcher + network-common, network-shuffle, repl, launcher, examples, sketch, graphx - |- catalyst, hive-thriftserver @@ -75,16 +75,20 @@ jobs: excluded-tags: org.apache.spark.tags.ExtendedSQLTest comment: "- other tests" env: - TEST_ONLY_MODULES: ${{ matrix.modules }} - TEST_ONLY_EXCLUDED_TAGS: ${{ matrix.excluded-tags }} - TEST_ONLY_INCLUDED_TAGS: ${{ matrix.included-tags }} + MODULES_TO_TEST: ${{ matrix.modules }} + EXCLUDED_TAGS: ${{ matrix.excluded-tags }} + INCLUDED_TAGS: ${{ matrix.included-tags }} HADOOP_PROFILE: ${{ matrix.hadoop }} HIVE_PROFILE: ${{ matrix.hive }} # GitHub Actions' default miniconda to use in pip packaging test. CONDA_PREFIX: /usr/share/miniconda + GITHUB_PREV_SHA: ${{ github.event.before }} steps: - name: Checkout Spark repository uses: actions/checkout@v2 + # In order to fetch changed files + with: + fetch-depth: 0 # Cache local repositories. Note that GitHub Actions cache has a 2G limit. - name: Cache Scala, SBT, Maven and Zinc uses: actions/cache@v1 @@ -161,9 +165,9 @@ jobs: - name: "Run tests: ${{ matrix.modules }}" run: | # Hive tests become flaky when running in parallel as it's too intensive. - if [[ "$TEST_ONLY_MODULES" == "hive" ]]; then export SERIAL_SBT_TESTS=1; fi + if [[ "$MODULES_TO_TEST" == "hive" ]]; then export SERIAL_SBT_TESTS=1; fi mkdir -p ~/.m2 - ./dev/run-tests --parallelism 2 + ./dev/run-tests --parallelism 2 --modules "$MODULES_TO_TEST" --included-tags "$INCLUDED_TAGS" --excluded-tags "$EXCLUDED_TAGS" rm -rf ~/.m2/repository/org/apache/spark # Static analysis, and documentation build diff --git a/dev/run-tests.py b/dev/run-tests.py index 03cc3230a65fd..065a27c0e853b 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -101,12 +101,14 @@ def setup_test_environ(environ): os.environ[k] = v -def determine_modules_to_test(changed_modules): +def determine_modules_to_test(changed_modules, deduplicated=True): """ Given a set of modules that have changed, compute the transitive closure of those modules' dependent modules in order to determine the set of modules that should be tested. Returns a topologically-sorted list of modules (ties are broken by sorting on module names). + If ``deduplicated`` is disabled, the modules are returned without tacking the deduplication + by dependencies into account. >>> [x.name for x in determine_modules_to_test([modules.root])] ['root'] @@ -122,11 +124,29 @@ def determine_modules_to_test(changed_modules): ... # doctest: +NORMALIZE_WHITESPACE ['sql', 'avro', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver', 'pyspark-sql', 'repl', 'sparkr', 'pyspark-mllib', 'pyspark-ml'] + >>> sorted([x.name for x in determine_modules_to_test( + ... [modules.sparkr, modules.sql], deduplicated=False)]) + ... # doctest: +NORMALIZE_WHITESPACE + ['avro', 'examples', 'hive', 'hive-thriftserver', 'mllib', 'pyspark-ml', + 'pyspark-mllib', 'pyspark-sql', 'repl', 'sparkr', 'sql', 'sql-kafka-0-10'] + >>> sorted([x.name for x in determine_modules_to_test( + ... [modules.sql, modules.core], deduplicated=False)]) + ... # doctest: +NORMALIZE_WHITESPACE + ['avro', 'catalyst', 'core', 'examples', 'graphx', 'hive', 'hive-thriftserver', + 'mllib', 'mllib-local', 'pyspark-core', 'pyspark-ml', 'pyspark-mllib', + 'pyspark-resource', 'pyspark-sql', 'pyspark-streaming', 'repl', 'root', + 'sparkr', 'sql', 'sql-kafka-0-10', 'streaming', 'streaming-kafka-0-10', + 'streaming-kinesis-asl'] """ modules_to_test = set() for module in changed_modules: - modules_to_test = modules_to_test.union(determine_modules_to_test(module.dependent_modules)) + modules_to_test = modules_to_test.union( + determine_modules_to_test(module.dependent_modules, deduplicated)) modules_to_test = modules_to_test.union(set(changed_modules)) + + if not deduplicated: + return modules_to_test + # If we need to run all of the tests, then we should short-circuit and return 'root' if modules.root in modules_to_test: return [modules.root] @@ -539,6 +559,24 @@ def parse_opts(): "-p", "--parallelism", type=int, default=8, help="The number of suites to test in parallel (default %(default)d)" ) + parser.add_argument( + "-m", "--modules", type=str, + default=None, + help="A comma-separated list of modules to test " + "(default: %s)" % ",".join(sorted([m.name for m in modules.all_modules])) + ) + parser.add_argument( + "-e", "--excluded-tags", type=str, + default=None, + help="A comma-separated list of tags to exclude in the tests, " + "e.g., org.apache.spark.tags.ExtendedHiveTest " + ) + parser.add_argument( + "-i", "--included-tags", type=str, + default=None, + help="A comma-separated list of tags to include in the tests, " + "e.g., org.apache.spark.tags.ExtendedHiveTest " + ) args, unknown = parser.parse_known_args() if unknown: @@ -589,43 +627,74 @@ def main(): # /home/jenkins/anaconda2/envs/py36/bin os.environ["PATH"] = "/home/anaconda/envs/py36/bin:" + os.environ.get("PATH") else: - # else we're running locally and can use local settings + # else we're running locally or Github Actions. build_tool = "sbt" hadoop_version = os.environ.get("HADOOP_PROFILE", "hadoop2.7") hive_version = os.environ.get("HIVE_PROFILE", "hive2.3") - test_env = "local" + if "GITHUB_ACTIONS" in os.environ: + test_env = "github_actions" + else: + test_env = "local" print("[info] Using build tool", build_tool, "with Hadoop profile", hadoop_version, "and Hive profile", hive_version, "under environment", test_env) extra_profiles = get_hadoop_profiles(hadoop_version) + get_hive_profiles(hive_version) changed_modules = None + test_modules = None changed_files = None - should_only_test_modules = "TEST_ONLY_MODULES" in os.environ + should_only_test_modules = opts.modules is not None included_tags = [] + excluded_tags = [] if should_only_test_modules: - str_test_modules = [m.strip() for m in os.environ.get("TEST_ONLY_MODULES").split(",")] + str_test_modules = [m.strip() for m in opts.modules.split(",")] test_modules = [m for m in modules.all_modules if m.name in str_test_modules] - # Directly uses test_modules as changed modules to apply tags and environments - # as if all specified test modules are changed. + + # If we're running the tests in Github Actions, attempt to detect and test + # only the affected modules. + if test_env == "github_actions": + if os.environ["GITHUB_BASE_REF"] != "": + # Pull requests + changed_files = identify_changed_files_from_git_commits( + os.environ["GITHUB_SHA"], target_branch=os.environ["GITHUB_BASE_REF"]) + else: + # Build for each commit. + changed_files = identify_changed_files_from_git_commits( + os.environ["GITHUB_SHA"], target_ref=os.environ["GITHUB_PREV_SHA"]) + + modules_to_test = determine_modules_to_test( + determine_modules_for_files(changed_files), deduplicated=False) + + if modules.root not in modules_to_test: + # If root module is not found, only test the intersected modules. + # If root module is found, just run the modules as specified initially. + test_modules = list(set(modules_to_test).intersection(test_modules)) + changed_modules = test_modules - str_excluded_tags = os.environ.get("TEST_ONLY_EXCLUDED_TAGS", None) - str_included_tags = os.environ.get("TEST_ONLY_INCLUDED_TAGS", None) - excluded_tags = [] - if str_excluded_tags: - excluded_tags = [t.strip() for t in str_excluded_tags.split(",")] - included_tags = [] - if str_included_tags: - included_tags = [t.strip() for t in str_included_tags.split(",")] + if len(changed_modules) == 0: + print("[info] There are no modules to test, exiting without testing.") + return + + # If we're running the tests in AMPLab Jenkins, calculate the diff from the targeted branch, and + # detect modules to test. elif test_env == "amplab_jenkins" and os.environ.get("AMP_JENKINS_PRB"): target_branch = os.environ["ghprbTargetBranch"] changed_files = identify_changed_files_from_git_commits("HEAD", target_branch=target_branch) changed_modules = determine_modules_for_files(changed_files) + test_modules = determine_modules_to_test(changed_modules) excluded_tags = determine_tags_to_exclude(changed_modules) + # If there is no changed module found, tests all. if not changed_modules: changed_modules = [modules.root] - excluded_tags = [] + if not test_modules: + test_modules = determine_modules_to_test(changed_modules) + + if opts.excluded_tags: + excluded_tags.extend([t.strip() for t in opts.excluded_tags.split(",")]) + if opts.included_tags: + included_tags.extend([t.strip() for t in opts.included_tags.split(",")]) + print("[info] Found the following changed modules:", ", ".join(x.name for x in changed_modules)) @@ -640,8 +709,6 @@ def main(): should_run_java_style_checks = False if not should_only_test_modules: - test_modules = determine_modules_to_test(changed_modules) - # license checks run_apache_rat_checks() @@ -702,10 +769,6 @@ def main(): def _test(): - if "TEST_ONLY_MODULES" in os.environ: - # TODO(SPARK-32252): Enable doctests back in Github Actions. - return - import doctest failure_count = doctest.testmod()[0] if failure_count: From 90ac9f975bbb73e2f020a6c310e00fe1e71b6258 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 13 Jul 2020 14:05:33 -0700 Subject: [PATCH 183/384] [SPARK-32004][ALL] Drop references to slave ### What changes were proposed in this pull request? This change replaces the world slave with alternatives matching the context. ### Why are the changes needed? There is no need to call things slave, we might as well use better clearer names. ### Does this PR introduce _any_ user-facing change? Yes, the ouput JSON does change. To allow backwards compatibility this is an additive change. The shell scripts for starting & stopping workers are renamed, and for backwards compatibility old scripts are added to call through to the new ones while printing a deprecation message to stderr. ### How was this patch tested? Existing tests. Closes #28864 from holdenk/SPARK-32004-drop-references-to-slave. Lead-authored-by: Holden Karau Co-authored-by: Holden Karau Signed-off-by: Holden Karau --- conf/{slaves.template => workers.template} | 0 .../org/apache/spark/HeartbeatReceiver.scala | 9 +- .../scala/org/apache/spark/SparkContext.scala | 16 +-- .../spark/api/java/JavaSparkContext.scala | 6 +- .../apache/spark/deploy/JsonProtocol.scala | 15 ++- .../spark/internal/config/package.scala | 5 +- .../org/apache/spark/rdd/HadoopRDD.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 8 +- .../spark/scheduler/ExecutorLossReason.scala | 4 +- .../spark/scheduler/TaskScheduler.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 9 +- .../CoarseGrainedSchedulerBackend.scala | 12 +- .../cluster/StandaloneSchedulerBackend.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 20 +-- .../spark/storage/BlockManagerMaster.scala | 22 ++-- .../storage/BlockManagerMasterEndpoint.scala | 73 ++++++----- .../spark/storage/BlockManagerMessages.scala | 26 ++-- ...cala => BlockManagerStorageEndpoint.scala} | 6 +- .../org/apache/spark/CheckpointSuite.scala | 2 +- .../apache/spark/ContextCleanerSuite.scala | 10 +- .../org/apache/spark/DistributedSuite.scala | 20 +-- .../spark/ExternalShuffleServiceSuite.scala | 12 +- .../apache/spark/MapOutputTrackerSuite.scala | 26 ++-- .../spark/broadcast/BroadcastSuite.scala | 42 +++--- .../spark/deploy/JsonProtocolSuite.scala | 6 +- .../StandaloneDynamicAllocationSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 8 +- .../scheduler/TaskSchedulerImplSuite.scala | 6 +- .../spark/scheduler/TaskSetManagerSuite.scala | 16 +-- .../spark/storage/BlockManagerInfoSuite.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 44 ++++--- docs/configuration.md | 2 +- docs/job-scheduling.md | 2 +- docs/running-on-mesos.md | 16 +-- docs/spark-standalone.md | 16 +-- docs/streaming-programming-guide.md | 2 +- .../spark/deploy/mesos/ui/DriverPage.scala | 4 +- .../deploy/mesos/ui/MesosClusterPage.scala | 6 +- .../spark/executor/MesosExecutorBackend.scala | 12 +- .../cluster/mesos/MesosClusterScheduler.scala | 30 ++--- .../MesosCoarseGrainedSchedulerBackend.scala | 93 +++++++------- .../MesosFineGrainedSchedulerBackend.scala | 115 ++++++++--------- .../cluster/mesos/MesosScheduler.scala | 28 ++++ .../mesos/MesosSchedulerBackendUtil.scala | 20 ++- .../cluster/mesos/MesosSchedulerUtils.scala | 18 +-- .../mesos/MesosClusterSchedulerSuite.scala | 6 +- ...osCoarseGrainedSchedulerBackendSuite.scala | 12 +- .../spark/scheduler/cluster/mesos/Utils.scala | 11 +- .../cluster/YarnSchedulerBackend.scala | 7 +- sbin/decommission-slave.sh | 40 +----- sbin/decommission-worker.sh | 57 +++++++++ sbin/slaves.sh | 86 +------------ sbin/spark-daemons.sh | 4 +- sbin/start-all.sh | 4 +- sbin/start-slave.sh | 75 +---------- sbin/start-slaves.sh | 29 +---- sbin/start-worker.sh | 92 ++++++++++++++ sbin/start-workers.sh | 46 +++++++ sbin/stop-all.sh | 6 +- sbin/stop-slave.sh | 27 +--- sbin/stop-slaves.sh | 11 +- sbin/stop-worker.sh | 44 +++++++ sbin/stop-workers.sh | 28 ++++ sbin/workers.sh | 120 ++++++++++++++++++ .../sql/hive/thriftserver/SparkSQLEnv.scala | 2 +- .../apache/spark/sql/hive/TableReader.scala | 4 +- .../api/java/JavaStreamingContext.scala | 6 +- .../streaming/scheduler/ReceiverTracker.scala | 2 +- .../spark/streaming/util/RawTextHelper.scala | 2 +- 69 files changed, 863 insertions(+), 657 deletions(-) rename conf/{slaves.template => workers.template} (100%) rename core/src/main/scala/org/apache/spark/storage/{BlockManagerSlaveEndpoint.scala => BlockManagerStorageEndpoint.scala} (94%) create mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosScheduler.scala mode change 100644 => 100755 sbin/decommission-slave.sh create mode 100755 sbin/decommission-worker.sh create mode 100755 sbin/start-worker.sh create mode 100755 sbin/start-workers.sh create mode 100755 sbin/stop-worker.sh create mode 100755 sbin/stop-workers.sh create mode 100755 sbin/workers.sh diff --git a/conf/slaves.template b/conf/workers.template similarity index 100% rename from conf/slaves.template rename to conf/workers.template diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 2ac72e66d6f32..c99698f99d904 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -80,7 +80,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new HashMap[String, Long] - private val executorTimeoutMs = sc.conf.get(config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT) + private val executorTimeoutMs = sc.conf.get(config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT) private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL) @@ -88,10 +88,10 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) require(checkTimeoutIntervalMs <= executorTimeoutMs, s"${Network.NETWORK_TIMEOUT_INTERVAL.key} should be less than or " + - s"equal to ${config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT.key}.") + s"equal to ${config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT.key}.") require(executorHeartbeatIntervalMs <= executorTimeoutMs, s"${config.EXECUTOR_HEARTBEAT_INTERVAL.key} should be less than or " + - s"equal to ${config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT.key}") + s"equal to ${config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT.key}") private var timeoutCheckingTask: ScheduledFuture[_] = null @@ -218,7 +218,8 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) sc.schedulerBackend match { case backend: CoarseGrainedSchedulerBackend => backend.driverEndpoint.send(RemoveExecutor(executorId, - SlaveLost(s"Executor heartbeat timed out after ${now - lastSeenMs} ms"))) + ExecutorProcessLost( + s"Executor heartbeat timed out after ${now - lastSeenMs} ms"))) // LocalSchedulerBackend is used locally and only has one single executor case _: LocalSchedulerBackend => diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5078ef43adf68..06abc0541a9a9 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1732,7 +1732,7 @@ class SparkContext(config: SparkConf) extends Logging { def version: String = SPARK_VERSION /** - * Return a map from the slave to the max memory available for caching and the remaining + * Return a map from the block manager to the max memory available for caching and the remaining * memory available for caching. */ def getExecutorMemoryStatus: Map[String, (Long, Long)] = { @@ -2830,14 +2830,14 @@ object SparkContext extends Logging { scheduler.initialize(backend) (backend, scheduler) - case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - checkResourcesPerTask(coresPerSlave.toInt) - // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. - val memoryPerSlaveInt = memoryPerSlave.toInt - if (sc.executorMemory > memoryPerSlaveInt) { + case LOCAL_CLUSTER_REGEX(numWorkers, coresPerWorker, memoryPerWorker) => + checkResourcesPerTask(coresPerWorker.toInt) + // Check to make sure memory requested <= memoryPerWorker. Otherwise Spark will just hang. + val memoryPerWorkerInt = memoryPerWorker.toInt + if (sc.executorMemory > memoryPerWorkerInt) { throw new SparkException( "Asked to launch cluster with %d MiB RAM / worker but requested %d MiB/worker".format( - memoryPerSlaveInt, sc.executorMemory)) + memoryPerWorkerInt, sc.executorMemory)) } // For host local mode setting the default of SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED @@ -2850,7 +2850,7 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc) val localCluster = new LocalSparkCluster( - numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt, sc.conf) + numWorkers.toInt, coresPerWorker.toInt, memoryPerWorkerInt, sc.conf) val masterUrls = localCluster.start() val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 347f59fe8f77e..39eb1ee731d50 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -74,7 +74,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI - * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jarFile JAR file to send to the cluster. This can be a path on the local file system * or an HDFS, HTTP, HTTPS, or FTP URL. */ @@ -84,7 +84,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI - * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. */ @@ -94,7 +94,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI - * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. * @param environment Environment variables to set on worker nodes diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 6c3276c5c790a..17733d99cd5bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -90,11 +90,12 @@ private[deploy] object JsonProtocol { * `name` the description of the application * `cores` total cores granted to the application * `user` name of the user who submitted the application - * `memoryperslave` minimal memory in MB required to each executor - * `resourcesperslave` minimal resources required to each executor + * `memoryperexecutor` minimal memory in MB required to each executor + * `resourcesperexecutor` minimal resources required to each executor * `submitdate` time in Date that the application is submitted * `state` state of the application, see [[ApplicationState]] * `duration` time in milliseconds that the application has been running + * For compatibility also returns the deprecated `memoryperslave` & `resourcesperslave` fields. */ def writeApplicationInfo(obj: ApplicationInfo): JObject = { ("id" -> obj.id) ~ @@ -102,7 +103,10 @@ private[deploy] object JsonProtocol { ("name" -> obj.desc.name) ~ ("cores" -> obj.coresGranted) ~ ("user" -> obj.desc.user) ~ + ("memoryperexecutor" -> obj.desc.memoryPerExecutorMB) ~ ("memoryperslave" -> obj.desc.memoryPerExecutorMB) ~ + ("resourcesperexecutor" -> obj.desc.resourceReqsPerExecutor + .toList.map(writeResourceRequirement)) ~ ("resourcesperslave" -> obj.desc.resourceReqsPerExecutor .toList.map(writeResourceRequirement)) ~ ("submitdate" -> obj.submitDate.toString) ~ @@ -117,14 +121,17 @@ private[deploy] object JsonProtocol { * @return a Json object containing the following fields: * `name` the description of the application * `cores` max cores that can be allocated to the application, 0 means unlimited - * `memoryperslave` minimal memory in MB required to each executor - * `resourcesperslave` minimal resources required to each executor + * `memoryperexecutor` minimal memory in MB required to each executor + * `resourcesperexecutor` minimal resources required to each executor * `user` name of the user who submitted the application * `command` the command string used to submit the application + * For compatibility also returns the deprecated `memoryperslave` & `resourcesperslave` fields. */ def writeApplicationDescription(obj: ApplicationDescription): JObject = { ("name" -> obj.name) ~ ("cores" -> obj.maxCores.getOrElse(0)) ~ + ("memoryperexecutor" -> obj.memoryPerExecutorMB) ~ + ("resourcesperexecutor" -> obj.resourceReqsPerExecutor.toList.map(writeResourceRequirement)) ~ ("memoryperslave" -> obj.memoryPerExecutorMB) ~ ("resourcesperslave" -> obj.resourceReqsPerExecutor.toList.map(writeResourceRequirement)) ~ ("user" -> obj.user) ~ diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index ee437c696b47e..ca75a19af7bf6 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -459,9 +459,10 @@ package object config { .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("60s") - private[spark] val STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT = - ConfigBuilder("spark.storage.blockManagerSlaveTimeoutMs") + private[spark] val STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT = + ConfigBuilder("spark.storage.blockManagerHeartbeatTimeoutMs") .version("0.7.0") + .withAlternative("spark.storage.blockManagerSlaveTimeoutMs") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString(Network.NETWORK_TIMEOUT.defaultValueString) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 9742d12cfe01e..d5f21112c0c9e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -81,7 +81,7 @@ private[spark] class HadoopPartition(rddId: Int, override val index: Int, s: Inp * @param sc The SparkContext to associate the RDD with. * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variable references an instance of JobConf, then that JobConf will be used for the Hadoop job. - * Otherwise, a new JobConf will be created on each slave using the enclosed Configuration. + * Otherwise, a new JobConf will be created on each executor using the enclosed Configuration. * @param initLocalJobConfFuncOpt Optional closure used to initialize any JobConf that HadoopRDD * creates. * @param inputFormatClass Storage format of the data to be read. @@ -140,7 +140,7 @@ class HadoopRDD[K, V]( private val ignoreEmptySplits = sparkContext.conf.get(HADOOP_RDD_IGNORE_EMPTY_SPLITS) - // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. + // Returns a JobConf that will be used on executors to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { val conf: Configuration = broadcastedConf.value.value if (shouldCloneJobConf) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 37f9e0bb483c2..cb024d0852d06 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1912,9 +1912,9 @@ private[spark] class DAGScheduler( * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * * We will also assume that we've lost all shuffle blocks associated with the executor if the - * executor serves its own blocks (i.e., we're not using external shuffle), the entire slave - * is lost (likely including the shuffle service), or a FetchFailed occurred, in which case we - * presume all shuffle data related to this executor to be lost. + * executor serves its own blocks (i.e., we're not using external shuffle), the entire executor + * process is lost (likely including the shuffle service), or a FetchFailed occurred, in which + * case we presume all shuffle data related to this executor to be lost. * * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. @@ -2273,7 +2273,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case ExecutorLost(execId, reason) => val workerLost = reason match { - case SlaveLost(_, true) => true + case ExecutorProcessLost(_, true) => true case _ => false } dagScheduler.handleExecutorLost(execId, workerLost) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index ee31093ec0652..4141ed799a4e0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import org.apache.spark.executor.ExecutorExitCode /** - * Represents an explanation for an executor or whole slave failing or exiting. + * Represents an explanation for an executor or whole process failing or exiting. */ private[spark] class ExecutorLossReason(val message: String) extends Serializable { @@ -56,7 +56,7 @@ private [spark] object LossReasonPending extends ExecutorLossReason("Pending los * @param workerLost whether the worker is confirmed lost too (i.e. including shuffle service) */ private[spark] -case class SlaveLost(_message: String = "Slave lost", workerLost: Boolean = false) +case class ExecutorProcessLost(_message: String = "Worker lost", workerLost: Boolean = false) extends ExecutorLossReason(_message) /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index e9e638a3645ac..08f9f3c256e69 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -45,7 +45,7 @@ private[spark] trait TaskScheduler { // Invoked after system has successfully initialized (typically in spark context). // Yarn uses this to bootstrap allocation of resources based on preferred locations, - // wait for slave registrations, etc. + // wait for executor registrations, etc. def postStartHook(): Unit = { } // Disconnect from the cluster. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 45cb5e534220c..12bd93286d736 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -526,14 +526,14 @@ private[spark] class TaskSchedulerImpl( } /** - * Called by cluster manager to offer resources on slaves. We respond by asking our active task + * Called by cluster manager to offer resources on workers. We respond by asking our active task * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so * that tasks are balanced across the cluster. */ def resourceOffers( offers: IndexedSeq[WorkerOffer], isAllFreeResources: Boolean = true): Seq[Seq[TaskDescription]] = synchronized { - // Mark each slave as alive and remember its hostname + // Mark each worker as alive and remember its hostname // Also track if new executor is added var newExecAvail = false for (o <- offers) { @@ -765,7 +765,8 @@ private[spark] class TaskSchedulerImpl( }) if (executorIdToRunningTaskIds.contains(execId)) { reason = Some( - SlaveLost(s"Task $tid was lost, so marking the executor as lost as well.")) + ExecutorProcessLost( + s"Task $tid was lost, so marking the executor as lost as well.")) removeExecutor(execId, reason.get) failedExecutor = Some(execId) } @@ -936,7 +937,7 @@ private[spark] class TaskSchedulerImpl( case None => // We may get multiple executorLost() calls with different loss reasons. For example, - // one may be triggered by a dropped connection from the slave while another may be a + // one may be triggered by a dropped connection from the worker while another may be a // report of executor termination from Mesos. We produce log messages for both so we // eventually report the termination reason. logError(s"Lost an executor $executorId (already removed): $reason") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index e4f4000d3574d..6b9b4d6fe57e0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -316,9 +316,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def onDisconnected(remoteAddress: RpcAddress): Unit = { addressToExecutorId .get(remoteAddress) - .foreach(removeExecutor(_, SlaveLost("Remote RPC client disassociated. Likely due to " + - "containers exceeding thresholds, or network issues. Check driver logs for WARN " + - "messages."))) + .foreach(removeExecutor(_, + ExecutorProcessLost("Remote RPC client disassociated. Likely due to " + + "containers exceeding thresholds, or network issues. Check driver logs for WARN " + + "messages."))) } // Make fake resource offers on just one executor @@ -382,7 +383,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } } - // Remove a disconnected slave from the cluster + // Remove a disconnected executor from the cluster private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { logDebug(s"Asked to remove executor $executorId with reason $reason") executorDataMap.get(executorId) match { @@ -556,7 +557,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Remove all the lingering executors that should be removed but not yet. The reason might be // because (1) disconnected event is not yet received; (2) executors die silently. executors.foreach { eid => - removeExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered.")) + removeExecutor(eid, + ExecutorProcessLost("Stale executor after cluster manager re-registered.")) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 42c46464d79e1..ec1299a924b5c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -168,7 +168,7 @@ private[spark] class StandaloneSchedulerBackend( fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit = { val reason: ExecutorLossReason = exitStatus match { case Some(code) => ExecutorExited(code, exitCausedByApp = true, message) - case None => SlaveLost(message, workerLost = workerLost) + case None => ExecutorProcessLost(message, workerLost = workerLost) } logInfo("Executor %s removed: %s".format(fullId, message)) removeExecutor(fullId.split("/")(1), reason) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e0478ad09601d..6eec288015380 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -226,9 +226,9 @@ private[spark] class BlockManager( private val maxFailuresBeforeLocationRefresh = conf.get(config.BLOCK_FAILURES_BEFORE_LOCATION_REFRESH) - private val slaveEndpoint = rpcEnv.setupEndpoint( + private val storageEndpoint = rpcEnv.setupEndpoint( "BlockManagerEndpoint" + BlockManager.ID_GENERATOR.next, - new BlockManagerSlaveEndpoint(rpcEnv, this, mapOutputTracker)) + new BlockManagerStorageEndpoint(rpcEnv, this, mapOutputTracker)) // Pending re-registration action being executed asynchronously or null if none is pending. // Accesses should synchronize on asyncReregisterLock. @@ -465,7 +465,7 @@ private[spark] class BlockManager( diskBlockManager.localDirsString, maxOnHeapMemory, maxOffHeapMemory, - slaveEndpoint) + storageEndpoint) blockManagerId = if (idFromMaster != null) idFromMaster else id @@ -543,8 +543,8 @@ private[spark] class BlockManager( * an executor crash. * * This function deliberately fails silently if the master returns false (indicating that - * the slave needs to re-register). The error condition will be detected again by the next - * heart beat attempt or new block registration and another try to re-register all blocks + * the storage endpoint needs to re-register). The error condition will be detected again by the + * next heart beat attempt or new block registration and another try to re-register all blocks * will be made then. */ private def reportAllBlocks(): Unit = { @@ -568,7 +568,7 @@ private[spark] class BlockManager( // TODO: We might need to rate limit re-registering. logInfo(s"BlockManager $blockManagerId re-registering with master") master.registerBlockManager(blockManagerId, diskBlockManager.localDirsString, maxOnHeapMemory, - maxOffHeapMemory, slaveEndpoint) + maxOffHeapMemory, storageEndpoint) reportAllBlocks() } @@ -718,7 +718,7 @@ private[spark] class BlockManager( * * droppedMemorySize exists to account for when the block is dropped from memory to disk (so * it is still valid). This ensures that update in master will compensate for the increase in - * memory on slave. + * memory on the storage endpoint. */ private def reportBlockStatus( blockId: BlockId, @@ -736,7 +736,7 @@ private[spark] class BlockManager( /** * Actually send a UpdateBlockInfo message. Returns the master's response, * which will be true if the block was successfully recorded and false if - * the slave needs to re-register. + * the storage endpoint needs to re-register. */ private def tryToReportBlockStatus( blockId: BlockId, @@ -934,7 +934,7 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") // Because all the remote blocks are registered in driver, it is not necessary to ask - // all the slave executors to get block status. + // all the storage endpoints to get block status. val locationsAndStatusOption = master.getLocationsAndStatus(blockId, blockManagerId.host) if (locationsAndStatusOption.isEmpty) { logDebug(s"Block $blockId is unknown by block manager master") @@ -1960,7 +1960,7 @@ private[spark] class BlockManager( } remoteBlockTempFileManager.stop() diskBlockManager.stop() - rpcEnv.stop(slaveEndpoint) + rpcEnv.stop(storageEndpoint) blockInfoManager.clear() memoryStore.clear() futureExecutionContext.shutdownNow() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index f571e428522c5..93492cc6d7db6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -71,10 +71,10 @@ class BlockManagerMaster( localDirs: Array[String], maxOnHeapMemSize: Long, maxOffHeapMemSize: Long, - slaveEndpoint: RpcEndpointRef): BlockManagerId = { + storageEndpoint: RpcEndpointRef): BlockManagerId = { logInfo(s"Registering BlockManager $id") val updatedId = driverEndpoint.askSync[BlockManagerId]( - RegisterBlockManager(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) + RegisterBlockManager(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, storageEndpoint)) logInfo(s"Registered BlockManager $updatedId") updatedId } @@ -128,7 +128,7 @@ class BlockManagerMaster( } /** - * Remove a block from the slaves that have it. This can only be used to remove + * Remove a block from the storage endpoints that have it. This can only be used to remove * blocks that the driver knows about. */ def removeBlock(blockId: BlockId): Unit = { @@ -193,14 +193,14 @@ class BlockManagerMaster( * Return the block's status on all block managers, if any. NOTE: This is a * potentially expensive operation and should only be used for testing. * - * If askSlaves is true, this invokes the master to query each block manager for the most - * updated block statuses. This is useful when the master is not informed of the given block + * If askStorageEndpoints is true, this invokes the master to query each block manager for the + * most updated block statuses. This is useful when the master is not informed of the given block * by all block managers. */ def getBlockStatus( blockId: BlockId, - askSlaves: Boolean = true): Map[BlockManagerId, BlockStatus] = { - val msg = GetBlockStatus(blockId, askSlaves) + askStorageEndpoints: Boolean = true): Map[BlockManagerId, BlockStatus] = { + val msg = GetBlockStatus(blockId, askStorageEndpoints) /* * To avoid potential deadlocks, the use of Futures is necessary, because the master endpoint * should not block on waiting for a block manager, which can in turn be waiting for the @@ -229,14 +229,14 @@ class BlockManagerMaster( * Return a list of ids of existing blocks such that the ids match the given filter. NOTE: This * is a potentially expensive operation and should only be used for testing. * - * If askSlaves is true, this invokes the master to query each block manager for the most - * updated block statuses. This is useful when the master is not informed of the given block + * If askStorageEndpoints is true, this invokes the master to query each block manager for the + * most updated block statuses. This is useful when the master is not informed of the given block * by all block managers. */ def getMatchingBlockIds( filter: BlockId => Boolean, - askSlaves: Boolean): Seq[BlockId] = { - val msg = GetMatchingBlockIds(filter, askSlaves) + askStorageEndpoints: Boolean): Seq[BlockId] = { + val msg = GetMatchingBlockIds(filter, askStorageEndpoints) val future = driverEndpoint.askSync[Future[Seq[BlockId]]](msg) timeout.awaitResult(future) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index f90216b973776..2a4817797a87c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils} /** * BlockManagerMasterEndpoint is an [[IsolatedRpcEndpoint]] on the master node to track statuses - * of all slaves' block managers. + * of all the storage endpoints' block managers. */ private[spark] class BlockManagerMasterEndpoint( @@ -101,8 +101,8 @@ class BlockManagerMasterEndpoint( RpcUtils.makeDriverRef(CoarseGrainedSchedulerBackend.ENDPOINT_NAME, conf, rpcEnv) override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterBlockManager(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) => - context.reply(register(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) + case RegisterBlockManager(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, endpoint) => + context.reply(register(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, endpoint)) case _updateBlockInfo @ UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => @@ -135,14 +135,14 @@ class BlockManagerMasterEndpoint( case GetStorageStatus => context.reply(storageStatus) - case GetBlockStatus(blockId, askSlaves) => - context.reply(blockStatus(blockId, askSlaves)) + case GetBlockStatus(blockId, askStorageEndpoints) => + context.reply(blockStatus(blockId, askStorageEndpoints)) case IsExecutorAlive(executorId) => context.reply(blockManagerIdByExecutor.contains(executorId)) - case GetMatchingBlockIds(filter, askSlaves) => - context.reply(getMatchingBlockIds(filter, askSlaves)) + case GetMatchingBlockIds(filter, askStorageEndpoints) => + context.reply(getMatchingBlockIds(filter, askStorageEndpoints)) case RemoveRdd(rddId) => context.reply(removeRdd(rddId)) @@ -219,14 +219,14 @@ class BlockManagerMasterEndpoint( private def removeRdd(rddId: Int): Future[Seq[Int]] = { // First remove the metadata for the given RDD, and then asynchronously remove the blocks - // from the slaves. + // from the storage endpoints. - // The message sent to the slaves to remove the RDD + // The message sent to the storage endpoints to remove the RDD val removeMsg = RemoveRdd(rddId) // Find all blocks for the given RDD, remove the block from both blockLocations and // the blockManagerInfo that is tracking the blocks and create the futures which asynchronously - // remove the blocks from slaves and gives back the number of removed blocks + // remove the blocks from storage endpoints and gives back the number of removed blocks val blocks = blockLocations.asScala.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) val blocksToDeleteByShuffleService = new mutable.HashMap[BlockManagerId, mutable.HashSet[RDDBlockId]] @@ -255,7 +255,7 @@ class BlockManagerMasterEndpoint( } } val removeRddFromExecutorsFutures = blockManagerInfo.values.map { bmInfo => - bmInfo.slaveEndpoint.ask[Int](removeMsg).recover { + bmInfo.storageEndpoint.ask[Int](removeMsg).recover { // use 0 as default value means no blocks were removed handleBlockRemovalFailure("RDD", rddId.toString, bmInfo.blockManagerId, 0) } @@ -276,13 +276,12 @@ class BlockManagerMasterEndpoint( Future.sequence(removeRddFromExecutorsFutures ++ removeRddBlockViaExtShuffleServiceFutures) } - private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { // Nothing to do in the BlockManagerMasterEndpoint data structures val removeMsg = RemoveShuffle(shuffleId) Future.sequence( blockManagerInfo.values.map { bm => - bm.slaveEndpoint.ask[Boolean](removeMsg).recover { + bm.storageEndpoint.ask[Boolean](removeMsg).recover { // use false as default value means no shuffle data were removed handleBlockRemovalFailure("shuffle", shuffleId.toString, bm.blockManagerId, false) } @@ -301,7 +300,7 @@ class BlockManagerMasterEndpoint( removeFromDriver || !info.blockManagerId.isDriver } val futures = requiredBlockManagers.map { bm => - bm.slaveEndpoint.ask[Int](removeMsg).recover { + bm.storageEndpoint.ask[Int](removeMsg).recover { // use 0 as default value means no blocks were removed handleBlockRemovalFailure("broadcast", broadcastId.toString, bm.blockManagerId, 0) } @@ -343,7 +342,7 @@ class BlockManagerMasterEndpoint( blockManagerInfo.get(candidateBMId).foreach { bm => val remainingLocations = locations.toSeq.filter(bm => bm != candidateBMId) val replicateMsg = ReplicateBlock(blockId, remainingLocations, maxReplicas) - bm.slaveEndpoint.ask[Boolean](replicateMsg) + bm.storageEndpoint.ask[Boolean](replicateMsg) } } } @@ -361,14 +360,14 @@ class BlockManagerMasterEndpoint( /** * Decommission the given Seq of blockmanagers * - Adds these block managers to decommissioningBlockManagerSet Set - * - Sends the DecommissionBlockManager message to each of the [[BlockManagerSlaveEndpoint]] + * - Sends the DecommissionBlockManager message to each of the [[BlockManagerReplicaEndpoint]] */ def decommissionBlockManagers(blockManagerIds: Seq[BlockManagerId]): Future[Seq[Unit]] = { val newBlockManagersToDecommission = blockManagerIds.toSet.diff(decommissioningBlockManagerSet) val futures = newBlockManagersToDecommission.map { blockManagerId => decommissioningBlockManagerSet.add(blockManagerId) val info = blockManagerInfo(blockManagerId) - info.slaveEndpoint.ask[Unit](DecommissionBlockManager) + info.storageEndpoint.ask[Unit](DecommissionBlockManager) } Future.sequence{ futures.toSeq } } @@ -391,7 +390,7 @@ class BlockManagerMasterEndpoint( }.toSeq } - // Remove a block from the slaves that have it. This can only be used to remove + // Remove a block from the workers that have it. This can only be used to remove // blocks that the master knows about. private def removeBlockFromWorkers(blockId: BlockId): Unit = { val locations = blockLocations.get(blockId) @@ -399,10 +398,10 @@ class BlockManagerMasterEndpoint( locations.foreach { blockManagerId: BlockManagerId => val blockManager = blockManagerInfo.get(blockManagerId) blockManager.foreach { bm => - // Remove the block from the slave's BlockManager. + // Remove the block from the BlockManager. // Doesn't actually wait for a confirmation and the message might get lost. // If message loss becomes frequent, we should add retry logic here. - bm.slaveEndpoint.ask[Boolean](RemoveBlock(blockId)).recover { + bm.storageEndpoint.ask[Boolean](RemoveBlock(blockId)).recover { // use false as default value means no blocks were removed handleBlockRemovalFailure("block", blockId.toString, bm.blockManagerId, false) } @@ -429,13 +428,13 @@ class BlockManagerMasterEndpoint( * Return the block's status for all block managers, if any. NOTE: This is a * potentially expensive operation and should only be used for testing. * - * If askSlaves is true, the master queries each block manager for the most updated block - * statuses. This is useful when the master is not informed of the given block by all block + * If askStorageEndpoints is true, the master queries each block manager for the most updated + * block statuses. This is useful when the master is not informed of the given block by all block * managers. */ private def blockStatus( blockId: BlockId, - askSlaves: Boolean): Map[BlockManagerId, Future[Option[BlockStatus]]] = { + askStorageEndpoints: Boolean): Map[BlockManagerId, Future[Option[BlockStatus]]] = { val getBlockStatus = GetBlockStatus(blockId) /* * Rather than blocking on the block status query, master endpoint should simply return @@ -444,8 +443,8 @@ class BlockManagerMasterEndpoint( */ blockManagerInfo.values.map { info => val blockStatusFuture = - if (askSlaves) { - info.slaveEndpoint.ask[Option[BlockStatus]](getBlockStatus) + if (askStorageEndpoints) { + info.storageEndpoint.ask[Option[BlockStatus]](getBlockStatus) } else { Future { info.getStatus(blockId) } } @@ -457,19 +456,19 @@ class BlockManagerMasterEndpoint( * Return the ids of blocks present in all the block managers that match the given filter. * NOTE: This is a potentially expensive operation and should only be used for testing. * - * If askSlaves is true, the master queries each block manager for the most updated block - * statuses. This is useful when the master is not informed of the given block by all block + * If askStorageEndpoints is true, the master queries each block manager for the most updated + * block statuses. This is useful when the master is not informed of the given block by all block * managers. */ private def getMatchingBlockIds( filter: BlockId => Boolean, - askSlaves: Boolean): Future[Seq[BlockId]] = { + askStorageEndpoints: Boolean): Future[Seq[BlockId]] = { val getMatchingBlockIds = GetMatchingBlockIds(filter) Future.sequence( blockManagerInfo.values.map { info => val future = - if (askSlaves) { - info.slaveEndpoint.ask[Seq[BlockId]](getMatchingBlockIds) + if (askStorageEndpoints) { + info.storageEndpoint.ask[Seq[BlockId]](getMatchingBlockIds) } else { Future { info.blocks.asScala.keys.filter(filter).toSeq } } @@ -492,7 +491,7 @@ class BlockManagerMasterEndpoint( localDirs: Array[String], maxOnHeapMemSize: Long, maxOffHeapMemSize: Long, - slaveEndpoint: RpcEndpointRef): BlockManagerId = { + storageEndpoint: RpcEndpointRef): BlockManagerId = { // the dummy id is not expected to contain the topology information. // we get that info here and respond back with a more fleshed out block manager id val id = BlockManagerId( @@ -527,7 +526,7 @@ class BlockManagerMasterEndpoint( } blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), - maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint, externalShuffleServiceBlockStatus) + maxOnHeapMemSize, maxOffHeapMemSize, storageEndpoint, externalShuffleServiceBlockStatus) } listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxOnHeapMemSize + maxOffHeapMemSize, Some(maxOnHeapMemSize), Some(maxOffHeapMemSize))) @@ -581,7 +580,7 @@ class BlockManagerMasterEndpoint( } } - // Remove the block from master tracking if it has been removed on all slaves. + // Remove the block from master tracking if it has been removed on all endpoints. if (locations.size == 0) { blockLocations.remove(blockId) } @@ -642,14 +641,14 @@ class BlockManagerMasterEndpoint( } /** - * Returns an [[RpcEndpointRef]] of the [[BlockManagerSlaveEndpoint]] for sending RPC messages. + * Returns an [[RpcEndpointRef]] of the [[BlockManagerReplicaEndpoint]] for sending RPC messages. */ private def getExecutorEndpointRef(executorId: String): Option[RpcEndpointRef] = { for ( blockManagerId <- blockManagerIdByExecutor.get(executorId); info <- blockManagerInfo.get(blockManagerId) ) yield { - info.slaveEndpoint + info.storageEndpoint } } @@ -673,7 +672,7 @@ private[spark] class BlockManagerInfo( timeMs: Long, val maxOnHeapMem: Long, val maxOffHeapMem: Long, - val slaveEndpoint: RpcEndpointRef, + val storageEndpoint: RpcEndpointRef, val externalShuffleServiceBlockStatus: Option[JHashMap[BlockId, BlockStatus]]) extends Logging { @@ -707,7 +706,7 @@ private[spark] class BlockManagerInfo( var originalLevel: StorageLevel = StorageLevel.NONE if (blockExists) { - // The block exists on the slave already. + // The block exists on the storage endpoint already. val blockStatus: BlockStatus = _blocks.get(blockId) originalLevel = blockStatus.storageLevel originalMemSize = blockStatus.memSize diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 7d4f2fff5c34c..bbc076cea9ba8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -24,37 +24,37 @@ import org.apache.spark.util.Utils private[spark] object BlockManagerMessages { ////////////////////////////////////////////////////////////////////////////////// - // Messages from the master to slaves. + // Messages from the master to storage endpoints. ////////////////////////////////////////////////////////////////////////////////// - sealed trait ToBlockManagerSlave + sealed trait ToBlockManagerMasterStorageEndpoint - // Remove a block from the slaves that have it. This can only be used to remove + // Remove a block from the storage endpoints that have it. This can only be used to remove // blocks that the master knows about. - case class RemoveBlock(blockId: BlockId) extends ToBlockManagerSlave + case class RemoveBlock(blockId: BlockId) extends ToBlockManagerMasterStorageEndpoint // Replicate blocks that were lost due to executor failure case class ReplicateBlock(blockId: BlockId, replicas: Seq[BlockManagerId], maxReplicas: Int) - extends ToBlockManagerSlave + extends ToBlockManagerMasterStorageEndpoint - case object DecommissionBlockManager extends ToBlockManagerSlave + case object DecommissionBlockManager extends ToBlockManagerMasterStorageEndpoint // Remove all blocks belonging to a specific RDD. - case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + case class RemoveRdd(rddId: Int) extends ToBlockManagerMasterStorageEndpoint // Remove all blocks belonging to a specific shuffle. - case class RemoveShuffle(shuffleId: Int) extends ToBlockManagerSlave + case class RemoveShuffle(shuffleId: Int) extends ToBlockManagerMasterStorageEndpoint // Remove all blocks belonging to a specific broadcast. case class RemoveBroadcast(broadcastId: Long, removeFromDriver: Boolean = true) - extends ToBlockManagerSlave + extends ToBlockManagerMasterStorageEndpoint /** * Driver to Executor message to trigger a thread dump. */ - case object TriggerThreadDump extends ToBlockManagerSlave + case object TriggerThreadDump extends ToBlockManagerMasterStorageEndpoint ////////////////////////////////////////////////////////////////////////////////// - // Messages from slaves to the master. + // Messages from storage endpoints to the master. ////////////////////////////////////////////////////////////////////////////////// sealed trait ToBlockManagerMaster @@ -132,10 +132,10 @@ private[spark] object BlockManagerMessages { case class GetReplicateInfoForRDDBlocks(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - case class GetBlockStatus(blockId: BlockId, askSlaves: Boolean = true) + case class GetBlockStatus(blockId: BlockId, askStorageEndpoints: Boolean = true) extends ToBlockManagerMaster - case class GetMatchingBlockIds(filter: BlockId => Boolean, askSlaves: Boolean = true) + case class GetMatchingBlockIds(filter: BlockId => Boolean, askStorageEndpoints: Boolean = true) extends ToBlockManagerMaster case class BlockManagerHeartbeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerStorageEndpoint.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerStorageEndpoint.scala index a3a7149103491..a69bebc23c661 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerStorageEndpoint.scala @@ -27,17 +27,17 @@ import org.apache.spark.util.{ThreadUtils, Utils} /** * An RpcEndpoint to take commands from the master to execute options. For example, - * this is used to remove blocks from the slave's BlockManager. + * this is used to remove blocks from the storage endpoint's BlockManager. */ private[storage] -class BlockManagerSlaveEndpoint( +class BlockManagerStorageEndpoint( override val rpcEnv: RpcEnv, blockManager: BlockManager, mapOutputTracker: MapOutputTracker) extends IsolatedRpcEndpoint with Logging { private val asyncThreadPool = - ThreadUtils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool", 100) + ThreadUtils.newDaemonCachedThreadPool("block-manager-storage-async-thread-pool", 100) private implicit val asyncExecutionContext = ExecutionContext.fromExecutorService(asyncThreadPool) // Operations that involve removing blocks may be slow and should be done asynchronously diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index a69381d18e3b6..21090e98ea285 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -194,7 +194,7 @@ trait RDDCheckpointTester { self: SparkFunSuite => /** * Serialize and deserialize an object. This is useful to verify the objects * contents after deserialization (e.g., the contents of an RDD split after - * it is sent to a slave along with a task) + * it is sent to an executor along with a task) */ protected def serializeDeserialize[T](obj: T): T = { val bytes = Utils.serialize(obj) diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 7a95ea0fa3218..81530a8fda84d 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -309,7 +309,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { assert(sc.env.blockManager.master.getMatchingBlockIds({ case BroadcastBlockId(`taskClosureBroadcastId`, _) => true case _ => false - }, askSlaves = true).isEmpty) + }, askStorageEndpoints = true).isEmpty) } test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { @@ -349,7 +349,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { assert(sc.env.blockManager.master.getMatchingBlockIds({ case BroadcastBlockId(`taskClosureBroadcastId`, _) => true case _ => false - }, askSlaves = true).isEmpty) + }, askStorageEndpoints = true).isEmpty) } } @@ -528,7 +528,7 @@ class CleanerTester( blockManager.master.getMatchingBlockIds( _ match { case RDDBlockId(`rddId`, _) => true case _ => false - }, askSlaves = true) + }, askStorageEndpoints = true) } private def getShuffleBlocks(shuffleId: Int): Seq[BlockId] = { @@ -536,14 +536,14 @@ class CleanerTester( case ShuffleBlockId(`shuffleId`, _, _) => true case ShuffleIndexBlockId(`shuffleId`, _, _) => true case _ => false - }, askSlaves = true) + }, askStorageEndpoints = true) } private def getBroadcastBlocks(broadcastId: Long): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { case BroadcastBlockId(`broadcastId`, _) => true case _ => false - }, askSlaves = true) + }, askStorageEndpoints = true) } private def blockManager = sc.env.blockManager diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 4d157b9607000..27862806c0840 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -45,11 +45,11 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex // this test will hang. Correct behavior is that executors don't crash but fail tasks // and the scheduler throws a SparkException. - // numSlaves must be less than numPartitions - val numSlaves = 3 + // numWorkers must be less than numPartitions + val numWorkers = 3 val numPartitions = 10 - sc = new SparkContext("local-cluster[%s,1,1024]".format(numSlaves), "test") + sc = new SparkContext("local-cluster[%s,1,1024]".format(numWorkers), "test") val data = sc.parallelize(1 to 100, numPartitions). map(x => throw new NotSerializableExn(new NotSerializableClass)) intercept[SparkException] { @@ -69,10 +69,10 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex ) masterStrings.foreach { - case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - assert(numSlaves.toInt == 2) - assert(coresPerSlave.toInt == 1) - assert(memoryPerSlave.toInt == 1024) + case LOCAL_CLUSTER_REGEX(numWorkers, coresPerWorker, memoryPerWorker) => + assert(numWorkers.toInt == 2) + assert(coresPerWorker.toInt == 1) + assert(memoryPerWorker.toInt == 1024) } } @@ -227,7 +227,8 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(data.count() === size) assert(data.count() === size) // ensure only a subset of partitions were cached - val rddBlocks = sc.env.blockManager.master.getMatchingBlockIds(_.isRDD, askSlaves = true) + val rddBlocks = sc.env.blockManager.master.getMatchingBlockIds(_.isRDD, + askStorageEndpoints = true) assert(rddBlocks.size === 0, s"expected no RDD blocks, found ${rddBlocks.size}") } @@ -244,7 +245,8 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(data.count() === size) assert(data.count() === size) // ensure only a subset of partitions were cached - val rddBlocks = sc.env.blockManager.master.getMatchingBlockIds(_.isRDD, askSlaves = true) + val rddBlocks = sc.env.blockManager.master.getMatchingBlockIds(_.isRDD, + askStorageEndpoints = true) assert(rddBlocks.size > 0, "no RDD blocks found") assert(rddBlocks.size < numPartitions, s"too many RDD blocks found, expected <$numPartitions") } diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index c217419f4092e..65391db405a55 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -72,12 +72,12 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi sc.env.blockManager.externalShuffleServiceEnabled should equal(true) sc.env.blockManager.blockStoreClient.getClass should equal(classOf[ExternalBlockStoreClient]) - // In a slow machine, one slave may register hundreds of milliseconds ahead of the other one. - // If we don't wait for all slaves, it's possible that only one executor runs all jobs. Then + // In a slow machine, one executor may register hundreds of milliseconds ahead of the other one. + // If we don't wait for all executors, it's possible that only one executor runs all jobs. Then // all shuffle blocks will be in this executor, ShuffleBlockFetcherIterator will directly fetch // local blocks from the local BlockManager and won't send requests to ExternalShuffleService. // In this case, we won't receive FetchFailed. And it will make this test fail. - // Therefore, we should wait until all slaves are up + // Therefore, we should wait until all executors are up TestUtils.waitUntilExecutorsUp(sc, 2, 60000) val rdd = sc.parallelize(0 until 1000, 10) @@ -109,12 +109,12 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi sc.env.blockManager.hostLocalDirManager.isDefined should equal(true) sc.env.blockManager.blockStoreClient.getClass should equal(classOf[ExternalBlockStoreClient]) - // In a slow machine, one slave may register hundreds of milliseconds ahead of the other one. - // If we don't wait for all slaves, it's possible that only one executor runs all jobs. Then + // In a slow machine, one executor may register hundreds of milliseconds ahead of the other one. + // If we don't wait for all executors, it's possible that only one executor runs all jobs. Then // all shuffle blocks will be in this executor, ShuffleBlockFetcherIterator will directly fetch // local blocks from the local BlockManager and won't send requests to ExternalShuffleService. // In this case, we won't receive FetchFailed. And it will make this test fail. - // Therefore, we should wait until all slaves are up + // Therefore, we should wait until all executors are up TestUtils.waitUntilExecutorsUp(sc, 2, 60000) val rdd = sc.parallelize(0 until 1000, 10) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 630ffd9baa06e..b5b68f639ffc9 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -136,21 +136,21 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) - val slaveRpcEnv = createRpcEnv("spark-slave", hostname, 0, new SecurityManager(conf)) - val slaveTracker = new MapOutputTrackerWorker(conf) - slaveTracker.trackerEndpoint = - slaveRpcEnv.setupEndpointRef(rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) + val mapWorkerRpcEnv = createRpcEnv("spark-worker", hostname, 0, new SecurityManager(conf)) + val mapWorkerTracker = new MapOutputTrackerWorker(conf) + mapWorkerTracker.trackerEndpoint = + mapWorkerRpcEnv.setupEndpointRef(rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) masterTracker.registerShuffle(10, 1) - slaveTracker.updateEpoch(masterTracker.getEpoch) + mapWorkerTracker.updateEpoch(masterTracker.getEpoch) // This is expected to fail because no outputs have been registered for the shuffle. - intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } + intercept[FetchFailedException] { mapWorkerTracker.getMapSizesByExecutorId(10, 0) } val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( BlockManagerId("a", "hostA", 1000), Array(1000L), 5)) - slaveTracker.updateEpoch(masterTracker.getEpoch) - assert(slaveTracker.getMapSizesByExecutorId(10, 0).toSeq === + mapWorkerTracker.updateEpoch(masterTracker.getEpoch) + assert(mapWorkerTracker.getMapSizesByExecutorId(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000, 0))))) assert(0 == masterTracker.getNumCachedSerializedBroadcast) @@ -158,17 +158,17 @@ class MapOutputTrackerSuite extends SparkFunSuite { val masterTrackerEpochBeforeLossOfMapOutput = masterTracker.getEpoch masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) assert(masterTracker.getEpoch > masterTrackerEpochBeforeLossOfMapOutput) - slaveTracker.updateEpoch(masterTracker.getEpoch) - intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } + mapWorkerTracker.updateEpoch(masterTracker.getEpoch) + intercept[FetchFailedException] { mapWorkerTracker.getMapSizesByExecutorId(10, 0) } // failure should be cached - intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } + intercept[FetchFailedException] { mapWorkerTracker.getMapSizesByExecutorId(10, 0) } assert(0 == masterTracker.getNumCachedSerializedBroadcast) masterTracker.stop() - slaveTracker.stop() + mapWorkerTracker.stop() rpcEnv.shutdown() - slaveRpcEnv.shutdown() + mapWorkerRpcEnv.shutdown() } test("remote fetch below max RPC message size") { diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index a6776ee077894..5e8b25f425166 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -68,14 +68,14 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio } encryptionTest("Accessing TorrentBroadcast variables in a local cluster") { conf => - val numSlaves = 4 + val numWorkers = 4 conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") conf.set(config.BROADCAST_COMPRESS, true) - sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", conf) + sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numWorkers), "test", conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) - val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) - assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + val results = sc.parallelize(1 to numWorkers).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to numWorkers).map(x => (x, 10)).toSet) } test("TorrentBroadcast's blockifyObject and unblockifyObject are inverses") { @@ -99,12 +99,12 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio } test("Test Lazy Broadcast variables with TorrentBroadcast") { - val numSlaves = 2 - sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test") - val rdd = sc.parallelize(1 to numSlaves) + val numWorkers = 2 + sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numWorkers), "test") + val rdd = sc.parallelize(1 to numWorkers) val results = new DummyBroadcastClass(rdd).doSomething() - assert(results.toSet === (1 to numSlaves).map(x => (x, false)).toSet) + assert(results.toSet === (1 to numWorkers).map(x => (x, false)).toSet) } test("Unpersisting TorrentBroadcast on executors only in local mode") { @@ -196,27 +196,27 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio */ private def testUnpersistTorrentBroadcast(distributed: Boolean, removeFromDriver: Boolean): Unit = { - val numSlaves = if (distributed) 2 else 0 + val numWorkers = if (distributed) 2 else 0 // Verify that blocks are persisted only on the driver def afterCreation(broadcastId: Long, bmm: BlockManagerMaster): Unit = { var blockId = BroadcastBlockId(broadcastId) - var statuses = bmm.getBlockStatus(blockId, askSlaves = true) + var statuses = bmm.getBlockStatus(blockId, askStorageEndpoints = true) assert(statuses.size === 1) blockId = BroadcastBlockId(broadcastId, "piece0") - statuses = bmm.getBlockStatus(blockId, askSlaves = true) + statuses = bmm.getBlockStatus(blockId, askStorageEndpoints = true) assert(statuses.size === 1) } // Verify that blocks are persisted in both the executors and the driver def afterUsingBroadcast(broadcastId: Long, bmm: BlockManagerMaster): Unit = { var blockId = BroadcastBlockId(broadcastId) - val statuses = bmm.getBlockStatus(blockId, askSlaves = true) - assert(statuses.size === numSlaves + 1) + val statuses = bmm.getBlockStatus(blockId, askStorageEndpoints = true) + assert(statuses.size === numWorkers + 1) blockId = BroadcastBlockId(broadcastId, "piece0") - assert(statuses.size === numSlaves + 1) + assert(statuses.size === numWorkers + 1) } // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver @@ -224,16 +224,16 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio def afterUnpersist(broadcastId: Long, bmm: BlockManagerMaster): Unit = { var blockId = BroadcastBlockId(broadcastId) var expectedNumBlocks = if (removeFromDriver) 0 else 1 - var statuses = bmm.getBlockStatus(blockId, askSlaves = true) + var statuses = bmm.getBlockStatus(blockId, askStorageEndpoints = true) assert(statuses.size === expectedNumBlocks) blockId = BroadcastBlockId(broadcastId, "piece0") expectedNumBlocks = if (removeFromDriver) 0 else 1 - statuses = bmm.getBlockStatus(blockId, askSlaves = true) + statuses = bmm.getBlockStatus(blockId, askStorageEndpoints = true) assert(statuses.size === expectedNumBlocks) } - testUnpersistBroadcast(distributed, numSlaves, afterCreation, + testUnpersistBroadcast(distributed, numWorkers, afterCreation, afterUsingBroadcast, afterUnpersist, removeFromDriver) } @@ -248,7 +248,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio */ private def testUnpersistBroadcast( distributed: Boolean, - numSlaves: Int, // used only when distributed = true + numWorkers: Int, // used only when distributed = true afterCreation: (Long, BlockManagerMaster) => Unit, afterUsingBroadcast: (Long, BlockManagerMaster) => Unit, afterUnpersist: (Long, BlockManagerMaster) => Unit, @@ -256,10 +256,10 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio sc = if (distributed) { val _sc = - new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test") + new SparkContext("local-cluster[%d, 1, 1024]".format(numWorkers), "test") // Wait until all salves are up try { - TestUtils.waitUntilExecutorsUp(_sc, numSlaves, 60000) + TestUtils.waitUntilExecutorsUp(_sc, numWorkers, 60000) _sc } catch { case e: Throwable => @@ -278,7 +278,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio // Use broadcast variable on all executors val partitions = 10 - assert(partitions > numSlaves) + assert(partitions > numWorkers) val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) afterUsingBroadcast(broadcast.id, blockManagerMaster) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index eeccf56cbf02e..354e6eb2138d9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -106,6 +106,9 @@ object JsonConstants { """ |{"id":"id","starttime":3,"name":"name", |"cores":0,"user":"%s", + |"memoryperexecutor":1234, + |"resourcesperexecutor":[{"name":"gpu", + |"amount":3},{"name":"fpga","amount":3}], |"memoryperslave":1234, |"resourcesperslave":[{"name":"gpu", |"amount":3},{"name":"fpga","amount":3}], @@ -132,7 +135,8 @@ object JsonConstants { val appDescJsonStr = """ - |{"name":"name","cores":4,"memoryperslave":1234,"resourcesperslave":[], + |{"name":"name","cores":4,"memoryperexecutor":1234,"resourcesperexecutor":[], + |"memoryperslave":1234,"resourcesperslave":[], |"user":"%s","command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),List())"} """.format(System.getProperty("user.name", "")).stripMargin diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 57cbda3c0620d..c7c3ad27675fa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -545,7 +545,7 @@ class StandaloneDynamicAllocationSuite // will not timeout anything related to executors. .set(config.Network.NETWORK_TIMEOUT.key, "2h") .set(config.EXECUTOR_HEARTBEAT_INTERVAL.key, "1h") - .set(config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT.key, "1h") + .set(config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT.key, "1h") } /** Make a master to which our application will send executor requests. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 51d20d3428915..7013832757e38 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -474,7 +474,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assertDataStructuresEmpty() } - test("All shuffle files on the slave should be cleaned up when slave lost") { + test("All shuffle files on the storage endpoint should be cleaned up when it is lost") { // reset the test context with the right shuffle service config afterEach() val conf = new SparkConf() @@ -779,9 +779,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } private val shuffleFileLossTests = Seq( - ("slave lost with shuffle service", SlaveLost("", false), true, false), - ("worker lost with shuffle service", SlaveLost("", true), true, true), - ("worker lost without shuffle service", SlaveLost("", true), false, true), + ("executor process lost with shuffle service", ExecutorProcessLost("", false), true, false), + ("worker lost with shuffle service", ExecutorProcessLost("", true), true, true), + ("worker lost without shuffle service", ExecutorProcessLost("", true), false, true), ("executor failure with shuffle service", ExecutorKilled, true, false), ("executor failure without shuffle service", ExecutorKilled, false, true)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index a75bae56229b4..e43be60e956be 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -641,7 +641,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions2.length) // provide the actual loss reason for executor0 - taskScheduler.executorLost("executor0", SlaveLost("oops")) + taskScheduler.executorLost("executor0", ExecutorProcessLost("oops")) // executor0's tasks should have failed now that the loss reason is known, so offering more // resources should make them be scheduled on the new executor. @@ -1141,7 +1141,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // Now we fail our second executor. The other task can still run on executor1, so make an offer // on that executor, and make sure that the other task (not the failed one) is assigned there. - taskScheduler.executorLost("executor1", SlaveLost("oops")) + taskScheduler.executorLost("executor1", ExecutorProcessLost("oops")) val nextTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(new WorkerOffer("executor0", "host0", 1))).flatten // Note: Its OK if some future change makes this already realize the taskset has become @@ -1273,7 +1273,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(1 === taskDescriptions.length) // mark executor0 as dead - taskScheduler.executorLost("executor0", SlaveLost()) + taskScheduler.executorLost("executor0", ExecutorProcessLost()) assert(!taskScheduler.isExecutorAlive("executor0")) assert(!taskScheduler.hasExecutorsAliveOnHost("host0")) assert(taskScheduler.getExecutorsAliveOnHost("host0").isEmpty) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 759e68219c2d0..95c8197abbf0b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -415,7 +415,7 @@ class TaskSetManagerSuite // Now mark host2 as dead sched.removeExecutor("exec2") - manager.executorLost("exec2", "host2", SlaveLost()) + manager.executorLost("exec2", "host2", ExecutorProcessLost()) // nothing should be chosen assert(manager.resourceOffer("exec1", "host1", ANY)._1 === None) @@ -598,10 +598,10 @@ class TaskSetManagerSuite Array(PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY))) // test if the valid locality is recomputed when the executor is lost sched.removeExecutor("execC") - manager.executorLost("execC", "host2", SlaveLost()) + manager.executorLost("execC", "host2", ExecutorProcessLost()) assert(manager.myLocalityLevels.sameElements(Array(NODE_LOCAL, NO_PREF, ANY))) sched.removeExecutor("execD") - manager.executorLost("execD", "host1", SlaveLost()) + manager.executorLost("execD", "host1", ExecutorProcessLost()) assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY))) } @@ -814,7 +814,7 @@ class TaskSetManagerSuite assert(resubmittedTasks === 0) assert(manager.runningTasks === 1) - manager.executorLost("execB", "host2", new SlaveLost()) + manager.executorLost("execB", "host2", new ExecutorProcessLost()) assert(manager.runningTasks === 0) assert(resubmittedTasks === 0) } @@ -923,7 +923,7 @@ class TaskSetManagerSuite // Make sure schedBackend.killTask(2, "exec3", true, "another attempt succeeded") gets called assert(killTaskCalled) // Host 3 Losts, there's only task 2.0 on it, which killed by task 2.1 - manager.executorLost("exec3", "host3", SlaveLost()) + manager.executorLost("exec3", "host3", ExecutorProcessLost()) // Check the resubmittedTasks assert(resubmittedTasks === 0) } @@ -1044,8 +1044,8 @@ class TaskSetManagerSuite assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") - manager.executorLost("execA", "host1", SlaveLost()) - manager.executorLost("execB.2", "host2", SlaveLost()) + manager.executorLost("execA", "host1", ExecutorProcessLost()) + manager.executorLost("execB.2", "host2", ExecutorProcessLost()) clock.advance(LOCALITY_WAIT_MS * 4) sched.addExecutor("execC", "host3") manager.executorAdded() @@ -1569,7 +1569,7 @@ class TaskSetManagerSuite assert(resubmittedTasks.isEmpty) // Host 2 Losts, meaning we lost the map output task4 - manager.executorLost("exec2", "host2", SlaveLost()) + manager.executorLost("exec2", "host2", ExecutorProcessLost()) // Make sure that task with index 2 is re-submitted assert(resubmittedTasks.contains(2)) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala index 01e3d6a46e709..3f5ffaa732f25 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -33,7 +33,7 @@ class BlockManagerInfoSuite extends SparkFunSuite { timeMs = 300, maxOnHeapMem = 10000, maxOffHeapMem = 20000, - slaveEndpoint = null, + storageEndpoint = null, if (svcEnabled) Some(new JHashMap[BlockId, BlockStatus]) else None) test(s"$testName externalShuffleServiceEnabled=$svcEnabled") { f(svcEnabled, bmInfo) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 75e755f70ab0a..dc1c7cd52d466 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1374,12 +1374,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.master.getLocations("list1").size === 0) assert(store.master.getLocations("list2").size === 1) assert(store.master.getLocations("list3").size === 1) - assert(store.master.getBlockStatus("list1", askSlaves = false).size === 0) - assert(store.master.getBlockStatus("list2", askSlaves = false).size === 1) - assert(store.master.getBlockStatus("list3", askSlaves = false).size === 1) - assert(store.master.getBlockStatus("list1", askSlaves = true).size === 0) - assert(store.master.getBlockStatus("list2", askSlaves = true).size === 1) - assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) + assert(store.master.getBlockStatus("list1", askStorageEndpoints = false).size === 0) + assert(store.master.getBlockStatus("list2", askStorageEndpoints = false).size === 1) + assert(store.master.getBlockStatus("list3", askStorageEndpoints = false).size === 1) + assert(store.master.getBlockStatus("list1", askStorageEndpoints = true).size === 0) + assert(store.master.getBlockStatus("list2", askStorageEndpoints = true).size === 1) + assert(store.master.getBlockStatus("list3", askStorageEndpoints = true).size === 1) // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. store.putIterator( @@ -1390,17 +1390,17 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE "list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) // getLocations should return nothing because the master is not informed - // getBlockStatus without asking slaves should have the same result - // getBlockStatus with asking slaves, however, should return the actual block statuses + // getBlockStatus without asking storage endpoints should have the same result + // getBlockStatus with asking storage endpoints, however, should return the actual statuses assert(store.master.getLocations("list4").size === 0) assert(store.master.getLocations("list5").size === 0) assert(store.master.getLocations("list6").size === 0) - assert(store.master.getBlockStatus("list4", askSlaves = false).size === 0) - assert(store.master.getBlockStatus("list5", askSlaves = false).size === 0) - assert(store.master.getBlockStatus("list6", askSlaves = false).size === 0) - assert(store.master.getBlockStatus("list4", askSlaves = true).size === 0) - assert(store.master.getBlockStatus("list5", askSlaves = true).size === 1) - assert(store.master.getBlockStatus("list6", askSlaves = true).size === 1) + assert(store.master.getBlockStatus("list4", askStorageEndpoints = false).size === 0) + assert(store.master.getBlockStatus("list5", askStorageEndpoints = false).size === 0) + assert(store.master.getBlockStatus("list6", askStorageEndpoints = false).size === 0) + assert(store.master.getBlockStatus("list4", askStorageEndpoints = true).size === 0) + assert(store.master.getBlockStatus("list5", askStorageEndpoints = true).size === 1) + assert(store.master.getBlockStatus("list6", askStorageEndpoints = true).size === 1) } test("get matching blocks") { @@ -1416,9 +1416,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE "list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations - assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size + assert(store.master.getMatchingBlockIds( + _.toString.contains("list"), askStorageEndpoints = false).size === 3) - assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size + assert(store.master.getMatchingBlockIds( + _.toString.contains("list1"), askStorageEndpoints = false).size === 1) // insert some more blocks @@ -1430,9 +1432,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE "newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations - assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size + assert( + store.master.getMatchingBlockIds( + _.toString.contains("newlist"), askStorageEndpoints = false).size === 1) - assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size + assert( + store.master.getMatchingBlockIds( + _.toString.contains("newlist"), askStorageEndpoints = true).size === 3) val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) @@ -1443,7 +1449,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val matchedBlockIds = store.master.getMatchingBlockIds(_ match { case RDDBlockId(1, _) => true case _ => false - }, askSlaves = true) + }, askStorageEndpoints = true) assert(matchedBlockIds.toSet === Set(RDDBlockId(1, 0), RDDBlockId(1, 1))) } diff --git a/docs/configuration.md b/docs/configuration.md index 706c2552b1d17..42f706b296d30 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1890,7 +1890,7 @@ Apart from these, the following properties are also available, and may be useful diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index eaacfa49c657c..5c19c77f37a81 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -95,7 +95,7 @@ varies across cluster managers: In standalone mode, simply start your workers with `spark.shuffle.service.enabled` set to `true`. In Mesos coarse-grained mode, run `$SPARK_HOME/sbin/start-mesos-shuffle-service.sh` on all -slave nodes with `spark.shuffle.service.enabled` set to `true`. For instance, you may do so +worker nodes with `spark.shuffle.service.enabled` set to `true`. For instance, you may do so through Marathon. In YARN mode, follow the instructions [here](running-on-yarn.html#configuring-the-external-shuffle-service). diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 6f6ae1c0ff264..578ab90fedfca 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -91,7 +91,7 @@ but Mesos can be run without ZooKeeper using a single master as well. ## Verification To verify that the Mesos cluster is ready for Spark, navigate to the Mesos master webui at port -`:5050` Confirm that all expected machines are present in the slaves tab. +`:5050` Confirm that all expected machines are present in the agents tab. # Connecting Spark to Mesos @@ -99,7 +99,7 @@ To verify that the Mesos cluster is ready for Spark, navigate to the Mesos maste To use Mesos from Spark, you need a Spark binary package available in a place accessible by Mesos, and a Spark driver program configured to connect to Mesos. -Alternatively, you can also install Spark in the same location in all the Mesos slaves, and configure +Alternatively, you can also install Spark in the same location in all the Mesos agents, and configure `spark.mesos.executor.home` (defaults to SPARK_HOME) to point to that location. ## Authenticating to Mesos @@ -138,7 +138,7 @@ Then submit happens as described in Client mode or Cluster mode below ## Uploading Spark Package -When Mesos runs a task on a Mesos slave for the first time, that slave must have a Spark binary +When Mesos runs a task on a Mesos agent for the first time, that agent must have a Spark binary package for running the Spark Mesos executor backend. The Spark package can be hosted at any Hadoop-accessible URI, including HTTP via `http://`, [Amazon Simple Storage Service](http://aws.amazon.com/s3) via `s3n://`, or HDFS via `hdfs://`. @@ -237,7 +237,7 @@ For example: {% endhighlight %} -Note that jars or python files that are passed to spark-submit should be URIs reachable by Mesos slaves, as the Spark driver doesn't automatically upload local jars. +Note that jars or python files that are passed to spark-submit should be URIs reachable by Mesos agents, as the Spark driver doesn't automatically upload local jars. # Mesos Run Modes @@ -360,7 +360,7 @@ see [Dynamic Resource Allocation](job-scheduling.html#dynamic-resource-allocatio The External Shuffle Service to use is the Mesos Shuffle Service. It provides shuffle data cleanup functionality on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's -termination. To launch it, run `$SPARK_HOME/sbin/start-mesos-shuffle-service.sh` on all slave nodes, with `spark.shuffle.service.enabled` set to `true`. +termination. To launch it, run `$SPARK_HOME/sbin/start-mesos-shuffle-service.sh` on all agent nodes, with `spark.shuffle.service.enabled` set to `true`. This can also be achieved through Marathon, using a unique host constraint, and the following command: `./bin/spark-class org.apache.spark.deploy.mesos.MesosExternalShuffleService`. @@ -840,17 +840,17 @@ See the [configuration page](configuration.html) for information on Spark config A few places to look during debugging: - Mesos master on port `:5050` - - Slaves should appear in the slaves tab + - Agents should appear in the agents tab - Spark applications should appear in the frameworks tab - Tasks should appear in the details of a framework - Check the stdout and stderr of the sandbox of failed tasks - Mesos logs - - Master and slave logs are both in `/var/log/mesos` by default + - Master and agent logs are both in `/var/log/mesos` by default And common pitfalls: - Spark assembly not reachable/accessible - - Slaves must be able to download the Spark binary package from the `http://`, `hdfs://` or `s3n://` URL you gave + - Agents must be able to download the Spark binary package from the `http://`, `hdfs://` or `s3n://` URL you gave - Firewall blocking communications - Check for messages about failed connections - Temporarily disable firewalls for debugging and then poke appropriate holes diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index f3c479ba26547..4344893fd3584 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -44,7 +44,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by Similarly, you can start one or more workers and connect them to the master via: - ./sbin/start-slave.sh + ./sbin/start-worker.sh Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default). You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS). @@ -90,9 +90,9 @@ Finally, the following configuration options can be passed to the master and wor # Cluster Launch Scripts -To launch a Spark standalone cluster with the launch scripts, you should create a file called conf/slaves in your Spark directory, +To launch a Spark standalone cluster with the launch scripts, you should create a file called conf/workers in your Spark directory, which must contain the hostnames of all the machines where you intend to start Spark workers, one per line. -If conf/slaves does not exist, the launch scripts defaults to a single machine (localhost), which is useful for testing. +If conf/workers does not exist, the launch scripts defaults to a single machine (localhost), which is useful for testing. Note, the master machine accesses each of the worker machines via ssh. By default, ssh is run in parallel and requires password-less (using a private key) access to be setup. If you do not have a password-less setup, you can set the environment variable SPARK_SSH_FOREGROUND and serially provide a password for each worker. @@ -100,12 +100,12 @@ If you do not have a password-less setup, you can set the environment variable S Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/sbin`: - `sbin/start-master.sh` - Starts a master instance on the machine the script is executed on. -- `sbin/start-slaves.sh` - Starts a worker instance on each machine specified in the `conf/slaves` file. -- `sbin/start-slave.sh` - Starts a worker instance on the machine the script is executed on. +- `sbin/start-workers.sh` - Starts a worker instance on each machine specified in the `conf/workers` file. +- `sbin/start-worker.sh` - Starts a worker instance on the machine the script is executed on. - `sbin/start-all.sh` - Starts both a master and a number of workers as described above. - `sbin/stop-master.sh` - Stops the master that was started via the `sbin/start-master.sh` script. -- `sbin/stop-slave.sh` - Stops all worker instances on the machine the script is executed on. -- `sbin/stop-slaves.sh` - Stops all worker instances on the machines specified in the `conf/slaves` file. +- `sbin/stop-worker.sh` - Stops all worker instances on the machine the script is executed on. +- `sbin/stop-workers.sh` - Stops all worker instances on the machines specified in the `conf/workers` file. - `sbin/stop-all.sh` - Stops both the master and the workers as described above. Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine. @@ -457,7 +457,7 @@ worker during one single schedule iteration. Spark's standalone mode offers a web-based user interface to monitor the cluster. The master and each worker has its own web UI that shows cluster and job statistics. By default, you can access the web UI for the master at port 8080. The port can be changed either in the configuration file or via command-line options. -In addition, detailed log output for each job is also written to the work directory of each slave node (`SPARK_HOME/work` by default). You will see two files for each job, `stdout` and `stderr`, with all output it wrote to its console. +In addition, detailed log output for each job is also written to the work directory of each worker node (`SPARK_HOME/work` by default). You will see two files for each job, `stdout` and `stderr`, with all output it wrote to its console. # Running Alongside Hadoop diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index ac4aa9255ae68..587deebcc5de4 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -2216,7 +2216,7 @@ In specific cases where the amount of data that needs to be retained for the str ### Task Launching Overheads {:.no_toc} If the number of tasks launched per second is high (say, 50 or more per second), then the overhead -of sending out tasks to the slaves may be significant and will make it hard to achieve sub-second +of sending out tasks to the executors may be significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: * **Execution mode**: Running Spark in Standalone mode or coarse-grained Mesos mode leads to diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala index 8dc123e93fe16..b8c64a28c72cd 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala @@ -92,8 +92,8 @@ private[ui] class DriverPage(parent: MesosClusterUI) extends WebUIPage("driver") private def launchedRow(submissionState: Option[MesosClusterSubmissionState]): Seq[Node] = { submissionState.map { state => - - + + diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala index 173a9b86e7de6..772906397546c 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -41,7 +41,7 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( val queuedHeaders = driverHeader ++ submissionHeader val driverHeaders = driverHeader ++ historyHeader ++ submissionHeader ++ - Seq("Start Date", "Mesos Slave ID", "State") ++ sandboxHeader + Seq("Start Date", "Mesos Agent ID", "State") ++ sandboxHeader val retryHeaders = Seq("Driver ID", "Submit Date", "Description") ++ Seq("Last Failed Status", "Next Retry Time", "Attempt Count") val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) @@ -81,7 +81,7 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( val sandboxCol = if (proxy.isDefined) { val clusterSchedulerId = parent.scheduler.getSchedulerState().frameworkId - val sandBoxUri = s"${proxy.get}/#/agents/${state.slaveId.getValue}/frameworks/" + + val sandBoxUri = s"${proxy.get}/#/agents/${state.agentId.getValue}/frameworks/" + s"${clusterSchedulerId}/executors/${id}/browse" Sandbox } else { @@ -103,7 +103,7 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( - + diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 47243e83d1335..b023cf1fa4bb2 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -26,18 +26,16 @@ import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} import org.apache.mesos.protobuf.ByteString import org.apache.spark.{SparkConf, SparkEnv, TaskState} -import org.apache.spark.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config.EXECUTOR_ID import org.apache.spark.resource.ResourceInformation import org.apache.spark.scheduler.TaskDescription -import org.apache.spark.scheduler.cluster.mesos.MesosSchedulerUtils +import org.apache.spark.scheduler.cluster.mesos.MesosSchedulerBackendUtil import org.apache.spark.util.Utils private[spark] class MesosExecutorBackend extends MesosExecutor - with MesosSchedulerUtils // TODO: fix with ExecutorBackend with Logging { @@ -48,7 +46,7 @@ private[spark] class MesosExecutorBackend val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() driver.sendStatusUpdate(MesosTaskStatus.newBuilder() .setTaskId(mesosTaskId) - .setState(taskStateToMesos(state)) + .setState(MesosSchedulerBackendUtil.taskStateToMesos(state)) .setData(ByteString.copyFrom(data)) .build()) } @@ -57,7 +55,7 @@ private[spark] class MesosExecutorBackend driver: ExecutorDriver, executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, - slaveInfo: SlaveInfo): Unit = { + agentInfo: SlaveInfo): Unit = { // Get num cores for this task from ExecutorInfo, created in MesosSchedulerBackend. val cpusPerTask = executorInfo.getResourcesList.asScala @@ -78,11 +76,11 @@ private[spark] class MesosExecutorBackend val conf = new SparkConf(loadDefaults = true).setAll(properties) conf.set(EXECUTOR_ID, executorId) val env = SparkEnv.createExecutorEnv( - conf, executorId, slaveInfo.getHostname, cpusPerTask, None, isLocal = false) + conf, executorId, agentInfo.getHostname, cpusPerTask, None, isLocal = false) executor = new Executor( executorId, - slaveInfo.getHostname, + agentInfo.getHostname, env, resources = Map.empty[String, ResourceInformation]) } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 289b109a42747..edcdb923ee55d 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -25,7 +25,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.mesos.{Scheduler, SchedulerDriver} -import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} +import org.apache.mesos.Protos.{SlaveID => AgentID, TaskState => MesosTaskState, _} import org.apache.mesos.Protos.Environment.Variable import org.apache.mesos.Protos.TaskStatus.Reason @@ -41,7 +41,7 @@ import org.apache.spark.util.Utils * @param driverDescription Submitted driver description from * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] * @param taskId Mesos TaskID generated for the task - * @param slaveId Slave ID that the task is assigned to + * @param agentId Agent ID that the task is assigned to * @param mesosTaskStatus The last known task status update. * @param startDate The date the task was launched * @param finishDate The date the task finished @@ -50,7 +50,7 @@ import org.apache.spark.util.Utils private[spark] class MesosClusterSubmissionState( val driverDescription: MesosDriverDescription, val taskId: TaskID, - val slaveId: SlaveID, + val agentId: AgentID, var mesosTaskStatus: Option[TaskStatus], var startDate: Date, var finishDate: Option[Date], @@ -59,7 +59,7 @@ private[spark] class MesosClusterSubmissionState( def copy(): MesosClusterSubmissionState = { new MesosClusterSubmissionState( - driverDescription, taskId, slaveId, mesosTaskStatus, startDate, finishDate, frameworkId) + driverDescription, taskId, agentId, mesosTaskStatus, startDate, finishDate, frameworkId) } } @@ -113,7 +113,7 @@ private[spark] class MesosDriverState( * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode * as Mesos tasks in a Mesos cluster. * All drivers are launched asynchronously by the framework, which will eventually be launched - * by one of the slaves in the cluster. The results of the driver will be stored in slave's task + * by one of the agents in the cluster. The results of the driver will be stored in agent's task * sandbox which is accessible by visiting the Mesos UI. * This scheduler supports recovery by persisting all its state and performs task reconciliation * on recover, which gets all the latest state for all the drivers from Mesos master. @@ -121,7 +121,7 @@ private[spark] class MesosDriverState( private[spark] class MesosClusterScheduler( engineFactory: MesosClusterPersistenceEngineFactory, conf: SparkConf) - extends Scheduler with MesosSchedulerUtils { + extends Scheduler with MesosSchedulerUtils with MesosScheduler { var frameworkUrl: String = _ private val metricsSystem = MetricsSystem.createMetricsSystem(MetricsSystemInstances.MESOS_CLUSTER, conf, @@ -139,10 +139,10 @@ private[spark] class MesosClusterScheduler( private var frameworkId: String = null // Holds all the launched drivers and current launch state, keyed by submission id. private val launchedDrivers = new mutable.HashMap[String, MesosClusterSubmissionState]() - // Holds a map of driver id to expected slave id that is passed to Mesos for reconciliation. + // Holds a map of driver id to expected agent id that is passed to Mesos for reconciliation. // All drivers that are loaded after failover are added here, as we need get the latest // state of the tasks from Mesos. Keyed by task Id. - private val pendingRecover = new mutable.HashMap[String, SlaveID]() + private val pendingRecover = new mutable.HashMap[String, AgentID]() // Stores all the submitted drivers that hasn't been launched, keyed by submission id private val queuedDrivers = new ArrayBuffer[MesosDriverDescription]() // All supervised drivers that are waiting to retry after termination, keyed by submission id @@ -277,7 +277,7 @@ private[spark] class MesosClusterScheduler( stateLock.synchronized { launchedDriversState.fetchAll[MesosClusterSubmissionState]().foreach { state => launchedDrivers(state.driverDescription.submissionId) = state - pendingRecover(state.taskId.getValue) = state.slaveId + pendingRecover(state.taskId.getValue) = state.agentId } queuedDriversState.fetchAll[MesosDriverDescription]().foreach(d => queuedDrivers += d) // There is potential timing issue where a queued driver might have been launched @@ -348,10 +348,10 @@ private[spark] class MesosClusterScheduler( if (!pendingRecover.isEmpty) { // Start task reconciliation if we need to recover. val statuses = pendingRecover.collect { - case (taskId, slaveId) => + case (taskId, agentId) => val newStatus = TaskStatus.newBuilder() .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(slaveId) + .setSlaveId(agentId) .setState(MesosTaskState.TASK_STAGING) .build() launchedDrivers.get(getSubmissionIdFromTaskId(taskId)) @@ -657,7 +657,7 @@ private[spark] class MesosClusterScheduler( finishedDrivers += new MesosClusterSubmissionState( submission, TaskID.newBuilder().setValue(submission.submissionId).build(), - SlaveID.newBuilder().setValue("").build(), + AgentID.newBuilder().setValue("").build(), None, null, None, @@ -731,7 +731,7 @@ private[spark] class MesosClusterScheduler( override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = { logInfo(s"Framework re-registered with master ${masterInfo.getId}") } - override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {} + override def agentLost(driver: SchedulerDriver, agentId: AgentID): Unit = {} override def error(driver: SchedulerDriver, error: String): Unit = { logError("Error received: " + error) markErr() @@ -815,13 +815,13 @@ private[spark] class MesosClusterScheduler( override def frameworkMessage( driver: SchedulerDriver, executorId: ExecutorID, - slaveId: SlaveID, + agentId: AgentID, message: Array[Byte]): Unit = {} override def executorLost( driver: SchedulerDriver, executorId: ExecutorID, - slaveId: SlaveID, + agentId: AgentID, status: Int): Unit = {} private def removeFromQueuedDrivers(subId: String): Boolean = { diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 0b447025c8a7a..5e7a29ac6d344 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -27,7 +27,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.Future -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} +import org.apache.mesos.Protos.{SlaveID => AgentID, TaskInfo => MesosTaskInfo, _} import org.apache.mesos.SchedulerDriver import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException, TaskState} @@ -40,7 +40,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalBlockStoreClient import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.{RpcEndpointAddress, RpcEndpointRef} -import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} +import org.apache.spark.scheduler.{ExecutorProcessLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -60,10 +60,11 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( master: String, securityManager: SecurityManager) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with org.apache.mesos.Scheduler with MesosSchedulerUtils { + with MesosScheduler + with MesosSchedulerUtils { - // Blacklist a slave after this many failures - private val MAX_SLAVE_FAILURES = 2 + // Blacklist a agent after this many failures + private val MAX_AGENT_FAILURES = 2 private val maxCoresOption = conf.get(config.CORES_MAX) @@ -116,10 +117,10 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // executor limit private var launchingExecutors = false - // SlaveID -> Slave - // This map accumulates entries for the duration of the job. Slaves are never deleted, because + // AgentID -> Agent + // This map accumulates entries for the duration of the job. Agents are never deleted, because // we need to maintain e.g. failure state and connection state. - private val slaves = new mutable.HashMap[String, Slave] + private val agents = new mutable.HashMap[String, Agent] /** * The total number of executors we aim to have. Undefined when not using dynamic allocation. @@ -147,7 +148,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( private val extraCoresPerExecutor = conf.get(EXTRA_CORES_PER_EXECUTOR) // Offer constraints - private val slaveOfferConstraints = + private val agentOfferConstraints = parseConstraintString(sc.conf.get(CONSTRAINTS)) // Reject offers with mismatched constraints in seconds @@ -354,7 +355,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } /** - * Method called by Mesos to offer resources on slaves. We respond by launching an executor, + * Method called by Mesos to offer resources on agents. We respond by launching an executor, * unless we've already launched more than we wanted to. */ override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]): Unit = { @@ -384,7 +385,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( val (matchedOffers, unmatchedOffers) = offers.asScala.partition { offer => val offerAttributes = toAttributeMap(offer.getAttributesList) - matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) + matchesAttributeRequirements(agentOfferConstraints, offerAttributes) } declineUnmatchedOffers(d, unmatchedOffers) @@ -441,7 +442,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( val ports = getRangeResource(task.getResourcesList, "ports").mkString(",") logDebug(s"Launching Mesos task: ${taskId.getValue} with mem: $mem cpu: $cpus" + - s" ports: $ports" + s" on slave with slave id: ${task.getSlaveId.getValue} ") + s" ports: $ports" + s" on agent with agent id: ${task.getSlaveId.getValue} ") } driver.launchTasks( @@ -495,18 +496,18 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( var launchTasks = true - // TODO(mgummelt): combine offers for a single slave + // TODO(mgummelt): combine offers for a single agent // // round-robin create executors on the available offers while (launchTasks) { launchTasks = false for (offer <- offers) { - val slaveId = offer.getSlaveId.getValue + val agentId = offer.getSlaveId.getValue val offerId = offer.getId.getValue val resources = remainingResources(offerId) - if (canLaunchTask(slaveId, offer.getHostname, resources)) { + if (canLaunchTask(agentId, offer.getHostname, resources)) { // Create a task launchTasks = true val taskId = newMesosTaskId() @@ -517,7 +518,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( val taskCPUs = executorCores(offerCPUs) val taskMemory = executorMemory(sc) - slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId) + agents.getOrElseUpdate(agentId, new Agent(offer.getHostname)).taskIDs.add(taskId) val (resourcesLeft, resourcesToUse) = partitionTaskResources(resources, taskCPUs, taskMemory, taskGPUs) @@ -540,8 +541,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( gpusByTaskId(taskId) = taskGPUs } } else { - logDebug(s"Cannot launch a task for offer with id: $offerId on slave " + - s"with id: $slaveId. Requirements were not met for this offer.") + logDebug(s"Cannot launch a task for offer with id: $offerId on agent " + + s"with id: $agentId. Requirements were not met for this offer.") } } } @@ -573,7 +574,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( cpuResourcesToUse ++ memResourcesToUse ++ portResourcesToUse ++ gpuResourcesToUse) } - private def canLaunchTask(slaveId: String, offerHostname: String, + private def canLaunchTask(agentId: String, offerHostname: String, resources: JList[Resource]): Boolean = { val offerMem = getResource(resources, "mem") val offerCPUs = getResource(resources, "cpus").toInt @@ -587,7 +588,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( cpus + totalCoresAcquired <= maxCores && mem <= offerMem && numExecutors < executorLimit && - slaves.get(slaveId).map(_.taskFailures).getOrElse(0) < MAX_SLAVE_FAILURES && + agents.get(agentId).map(_.taskFailures).getOrElse(0) < MAX_AGENT_FAILURES && meetsPortRequirements && satisfiesLocality(offerHostname) } @@ -606,7 +607,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } // Check the locality information - val currentHosts = slaves.values.filter(_.taskIDs.nonEmpty).map(_.hostname).toSet + val currentHosts = agents.values.filter(_.taskIDs.nonEmpty).map(_.hostname).toSet val allDesiredHosts = hostToLocalTaskCount.map { case (k, v) => k }.toSet // Try to match locality for hosts which do not have executors yet, to potentially @@ -622,13 +623,13 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus): Unit = { val taskId = status.getTaskId.getValue - val slaveId = status.getSlaveId.getValue + val agentId = status.getSlaveId.getValue val state = mesosToTaskState(status.getState) logInfo(s"Mesos task $taskId is now ${status.getState}") stateLock.synchronized { - val slave = slaves(slaveId) + val agent = agents(agentId) // If the shuffle service is enabled, have the driver register with each one of the // shuffle services. This allows the shuffle services to clean up state associated with @@ -636,23 +637,23 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // this through Mesos, since the shuffle services are set up independently. if (state.equals(TaskState.RUNNING) && shuffleServiceEnabled && - !slave.shuffleRegistered) { + !agent.shuffleRegistered) { assume(mesosExternalShuffleClient.isDefined, "External shuffle client was not instantiated even though shuffle service is enabled.") // TODO: Remove this and allow the MesosExternalShuffleService to detect // framework termination when new Mesos Framework HTTP API is available. val externalShufflePort = conf.get(config.SHUFFLE_SERVICE_PORT) - logDebug(s"Connecting to shuffle service on slave $slaveId, " + - s"host ${slave.hostname}, port $externalShufflePort for app ${conf.getAppId}") + logDebug(s"Connecting to shuffle service on agent $agentId, " + + s"host ${agent.hostname}, port $externalShufflePort for app ${conf.getAppId}") mesosExternalShuffleClient.get .registerDriverWithShuffleService( - slave.hostname, + agent.hostname, externalShufflePort, - sc.conf.get(config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT), + sc.conf.get(config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT), sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)) - slave.shuffleRegistered = true + agent.shuffleRegistered = true } if (TaskState.isFinished(state)) { @@ -666,16 +667,16 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( totalGpusAcquired -= gpus gpusByTaskId -= taskId } - // If it was a failure, mark the slave as failed for blacklisting purposes + // If it was a failure, mark the agent as failed for blacklisting purposes if (TaskState.isFailed(state)) { - slave.taskFailures += 1 + agent.taskFailures += 1 - if (slave.taskFailures >= MAX_SLAVE_FAILURES) { - logInfo(s"Blacklisting Mesos slave $slaveId due to too many failures; " + + if (agent.taskFailures >= MAX_AGENT_FAILURES) { + logInfo(s"Blacklisting Mesos agent $agentId due to too many failures; " + "is Spark installed on it?") } } - executorTerminated(d, slaveId, taskId, s"Executor finished with state $state") + executorTerminated(d, agentId, taskId, s"Executor finished with state $state") // In case we'd rejected everything before but have now lost a node d.reviveOffers() } @@ -708,7 +709,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // See SPARK-12330 val startTime = System.nanoTime() - // slaveIdsWithExecutors has no memory barrier, so this is eventually consistent + // agentIdsWithExecutors has no memory barrier, so this is eventually consistent while (numExecutors() > 0 && System.nanoTime() - startTime < shutdownTimeoutMS * 1000L * 1000L) { Thread.sleep(100) @@ -729,15 +730,15 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } override def frameworkMessage( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]): Unit = {} + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: AgentID, b: Array[Byte]): Unit = {} /** - * Called when a slave is lost or a Mesos task finished. Updates local view on + * Called when a agent is lost or a Mesos task finished. Updates local view on * what tasks are running. It also notifies the driver that an executor was removed. */ private def executorTerminated( d: org.apache.mesos.SchedulerDriver, - slaveId: String, + agentId: String, taskId: String, reason: String): Unit = { stateLock.synchronized { @@ -745,18 +746,18 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // removeExecutor() internally will send a message to the driver endpoint but // the driver endpoint is not available now, otherwise an exception will be thrown. if (!stopCalled) { - removeExecutor(taskId, SlaveLost(reason)) + removeExecutor(taskId, ExecutorProcessLost(reason)) } - slaves(slaveId).taskIDs.remove(taskId) + agents(agentId).taskIDs.remove(taskId) } } - override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = { - logInfo(s"Mesos slave lost: ${slaveId.getValue}") + override def agentLost(d: org.apache.mesos.SchedulerDriver, agentId: AgentID): Unit = { + logInfo(s"Mesos agent lost: ${agentId.getValue}") } override def executorLost( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: AgentID, status: Int): Unit = { logInfo("Mesos executor lost: %s".format(e.getValue)) } @@ -770,7 +771,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( resourceProfileToTotalExecs: Map[ResourceProfile, Int] ): Future[Boolean] = Future.successful { // We don't truly know if we can fulfill the full amount of executors - // since at coarse grain it depends on the amount of slaves available. + // since at coarse grain it depends on the amount of agents available. val numExecs = resourceProfileToTotalExecs.getOrElse(defaultProfile, 0) logInfo("Capping the total amount of executors to " + numExecs) executorLimitOption = Some(numExecs) @@ -800,11 +801,11 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } private def numExecutors(): Int = { - slaves.values.map(_.taskIDs.size).sum + agents.values.map(_.taskIDs.size).sum } } -private class Slave(val hostname: String) { +private class Agent(val hostname: String) { val taskIDs = new mutable.HashSet[String]() var taskFailures = 0 var shuffleRegistered = false diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala index f1e3fcab7e6af..586c2bdd67cfa 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala @@ -23,7 +23,8 @@ import java.util.{ArrayList => JArrayList, Collections, List => JList} import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, HashSet} -import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _} +import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, SlaveID => AgentID, + TaskInfo => MesosTaskInfo, _} import org.apache.mesos.SchedulerDriver import org.apache.mesos.protobuf.ByteString @@ -46,12 +47,12 @@ private[spark] class MesosFineGrainedSchedulerBackend( sc: SparkContext, master: String) extends SchedulerBackend - with org.apache.mesos.Scheduler + with MesosScheduler with MesosSchedulerUtils { - // Stores the slave ids that has launched a Mesos executor. - val slaveIdToExecutorInfo = new HashMap[String, MesosExecutorInfo] - val taskIdToSlaveId = new HashMap[Long, String] + // Stores the agent ids that has launched a Mesos executor. + val agentIdToExecutorInfo = new HashMap[String, MesosExecutorInfo] + val taskIdToAgentId = new HashMap[Long, String] // An ExecutorInfo for our tasks var execArgs: Array[Byte] = null @@ -64,7 +65,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( private[mesos] val mesosExecutorCores = sc.conf.get(mesosConfig.EXECUTOR_CORES) // Offer constraints - private[this] val slaveOfferConstraints = + private[this] val agentOfferConstraints = parseConstraintString(sc.conf.get(mesosConfig.CONSTRAINTS)) // reject offers with mismatched constraints in seconds @@ -217,7 +218,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( val builder = new StringBuilder tasks.asScala.foreach { t => builder.append("Task id: ").append(t.getTaskId.getValue).append("\n") - .append("Slave id: ").append(t.getSlaveId.getValue).append("\n") + .append("Agent id: ").append(t.getSlaveId.getValue).append("\n") .append("Task resources: ").append(t.getResourcesList).append("\n") .append("Executor resources: ").append(t.getExecutor.getResourcesList) .append("---------------------------------------------\n") @@ -226,7 +227,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( } /** - * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets + * Method called by Mesos to offer resources on agents. We respond by asking our active task sets * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ @@ -237,7 +238,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( offers.asScala.partition { o => val offerAttributes = toAttributeMap(o.getAttributesList) val meetsConstraints = - matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) + matchesAttributeRequirements(agentOfferConstraints, offerAttributes) // add some debug messaging if (!meetsConstraints) { @@ -259,7 +260,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( val (usableOffers, unUsableOffers) = offersMatchingConstraints.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") - val slaveId = o.getSlaveId.getValue + val agentId = o.getSlaveId.getValue val offerAttributes = toAttributeMap(o.getAttributesList) // check offers for @@ -269,7 +270,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( val meetsCPURequirements = cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK) val meetsRequirements = (meetsMemoryRequirements && meetsCPURequirements) || - (slaveIdToExecutorInfo.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK) + (agentIdToExecutorInfo.contains(agentId) && cpus >= scheduler.CPUS_PER_TASK) val debugstr = if (meetsRequirements) "Accepting" else "Declining" logDebug(s"$debugstr offer: ${o.getId.getValue} with attributes: " + s"$offerAttributes mem: $mem cpu: $cpus") @@ -281,10 +282,10 @@ private[spark] class MesosFineGrainedSchedulerBackend( unUsableOffers.foreach(o => d.declineOffer(o.getId)) val workerOffers = usableOffers.map { o => - val cpus = if (slaveIdToExecutorInfo.contains(o.getSlaveId.getValue)) { + val cpus = if (agentIdToExecutorInfo.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { - // If the Mesos executor has not been started on this slave yet, set aside a few + // If the Mesos executor has not been started on this agent yet, set aside a few // cores for the Mesos executor by offering fewer cores to the Spark executor (getResource(o.getResourcesList, "cpus") - mesosExecutorCores).toInt } @@ -294,51 +295,51 @@ private[spark] class MesosFineGrainedSchedulerBackend( cpus) }.toIndexedSeq - val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap - val slaveIdToWorkerOffer = workerOffers.map(o => o.executorId -> o).toMap - val slaveIdToResources = new HashMap[String, JList[Resource]]() + val agentIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap + val agentIdToWorkerOffer = workerOffers.map(o => o.executorId -> o).toMap + val agentIdToResources = new HashMap[String, JList[Resource]]() usableOffers.foreach { o => - slaveIdToResources(o.getSlaveId.getValue) = o.getResourcesList + agentIdToResources(o.getSlaveId.getValue) = o.getResourcesList } val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] - val slavesIdsOfAcceptedOffers = HashSet[String]() + val agentsIdsOfAcceptedOffers = HashSet[String]() // Call into the TaskSchedulerImpl val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) acceptedOffers .foreach { offer => offer.foreach { taskDesc => - val slaveId = taskDesc.executorId - slavesIdsOfAcceptedOffers += slaveId - taskIdToSlaveId(taskDesc.taskId) = slaveId + val agentId = taskDesc.executorId + agentsIdsOfAcceptedOffers += agentId + taskIdToAgentId(taskDesc.taskId) = agentId val (mesosTask, remainingResources) = createMesosTask( taskDesc, - slaveIdToResources(slaveId), - slaveId) - mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) + agentIdToResources(agentId), + agentId) + mesosTasks.getOrElseUpdate(agentId, new JArrayList[MesosTaskInfo]) .add(mesosTask) - slaveIdToResources(slaveId) = remainingResources + agentIdToResources(agentId) = remainingResources } } // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? - mesosTasks.foreach { case (slaveId, tasks) => - slaveIdToWorkerOffer.get(slaveId).foreach(o => - listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId, + mesosTasks.foreach { case (agentId, tasks) => + agentIdToWorkerOffer.get(agentId).foreach(o => + listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), agentId, // TODO: Add support for log urls for Mesos new ExecutorInfo(o.host, o.cores, Map.empty, Map.empty))) ) - logTrace(s"Launching Mesos tasks on slave '$slaveId', tasks:\n${getTasksSummary(tasks)}") - d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) + logTrace(s"Launching Mesos tasks on agent '$agentId', tasks:\n${getTasksSummary(tasks)}") + d.launchTasks(Collections.singleton(agentIdToOffer(agentId).getId), tasks, filters) } // Decline offers that weren't used // NOTE: This logic assumes that we only get a single offer for each host in a given batch - for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { + for (o <- usableOffers if !agentsIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { d.declineOffer(o.getId) } } @@ -348,19 +349,19 @@ private[spark] class MesosFineGrainedSchedulerBackend( def createMesosTask( task: TaskDescription, resources: JList[Resource], - slaveId: String): (MesosTaskInfo, JList[Resource]) = { + agentId: String): (MesosTaskInfo, JList[Resource]) = { val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build() - val (executorInfo, remainingResources) = if (slaveIdToExecutorInfo.contains(slaveId)) { - (slaveIdToExecutorInfo(slaveId), resources) + val (executorInfo, remainingResources) = if (agentIdToExecutorInfo.contains(agentId)) { + (agentIdToExecutorInfo(agentId), resources) } else { - createExecutorInfo(resources, slaveId) + createExecutorInfo(resources, agentId) } - slaveIdToExecutorInfo(slaveId) = executorInfo + agentIdToExecutorInfo(agentId) = executorInfo val (finalResources, cpuResources) = partitionResources(remainingResources, "cpus", scheduler.CPUS_PER_TASK) val taskInfo = MesosTaskInfo.newBuilder() .setTaskId(taskId) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) + .setSlaveId(AgentID.newBuilder().setValue(agentId).build()) .setExecutor(executorInfo) .setName(task.name) .addAllResources(cpuResources.asJava) @@ -375,12 +376,12 @@ private[spark] class MesosFineGrainedSchedulerBackend( val state = mesosToTaskState(status.getState) synchronized { if (TaskState.isFailed(mesosToTaskState(status.getState)) - && taskIdToSlaveId.contains(tid)) { - // We lost the executor on this slave, so remember that it's gone - removeExecutor(taskIdToSlaveId(tid), "Lost executor") + && taskIdToAgentId.contains(tid)) { + // We lost the executor on this agent, so remember that it's gone + removeExecutor(taskIdToAgentId(tid), "Lost executor") } if (TaskState.isFinished(state)) { - taskIdToSlaveId.remove(tid) + taskIdToAgentId.remove(tid) } } scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) @@ -406,39 +407,39 @@ private[spark] class MesosFineGrainedSchedulerBackend( } override def frameworkMessage( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]): Unit = {} + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: AgentID, b: Array[Byte]): Unit = {} /** - * Remove executor associated with slaveId in a thread safe manner. + * Remove executor associated with agentId in a thread safe manner. */ - private def removeExecutor(slaveId: String, reason: String) = { + private def removeExecutor(agentId: String, reason: String) = { synchronized { - listenerBus.post(SparkListenerExecutorRemoved(System.currentTimeMillis(), slaveId, reason)) - slaveIdToExecutorInfo -= slaveId + listenerBus.post(SparkListenerExecutorRemoved(System.currentTimeMillis(), agentId, reason)) + agentIdToExecutorInfo -= agentId } } - private def recordSlaveLost( - d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason): Unit = { + private def recordAgentLost( + d: org.apache.mesos.SchedulerDriver, agentId: AgentID, reason: ExecutorLossReason): Unit = { inClassLoader() { - logInfo("Mesos slave lost: " + slaveId.getValue) - removeExecutor(slaveId.getValue, reason.toString) - scheduler.executorLost(slaveId.getValue, reason) + logInfo("Mesos agent lost: " + agentId.getValue) + removeExecutor(agentId.getValue, reason.toString) + scheduler.executorLost(agentId.getValue, reason) } } - override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = { - recordSlaveLost(d, slaveId, SlaveLost()) + override def agentLost(d: org.apache.mesos.SchedulerDriver, agentId: AgentID): Unit = { + recordAgentLost(d, agentId, ExecutorProcessLost()) } override def executorLost( d: org.apache.mesos.SchedulerDriver, executorId: ExecutorID, - slaveId: SlaveID, + agentId: AgentID, status: Int): Unit = { - logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue, - slaveId.getValue)) - recordSlaveLost(d, slaveId, ExecutorExited(status, exitCausedByApp = true)) + logInfo("Executor lost: %s, marking agent %s as lost".format(executorId.getValue, + agentId.getValue)) + recordAgentLost(d, agentId, ExecutorExited(status, exitCausedByApp = true)) } override def killTask( diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosScheduler.scala new file mode 100644 index 0000000000000..f55b9efb3e64b --- /dev/null +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosScheduler.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.mesos + +import org.apache.mesos.Protos.{SlaveID => AgentID} + +trait MesosScheduler extends org.apache.mesos.Scheduler { + override def slaveLost(d: org.apache.mesos.SchedulerDriver, agentId: AgentID): Unit = { + agentLost(d, agentId) + } + + def agentLost(d: org.apache.mesos.SchedulerDriver, agentId: AgentID): Unit +} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala index 7b2f6a2535eda..981b8e9df1747 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala @@ -17,22 +17,23 @@ package org.apache.spark.scheduler.cluster.mesos -import org.apache.mesos.Protos.{ContainerInfo, Environment, Image, NetworkInfo, Parameter, Secret, Volume} +import org.apache.mesos.Protos.{ContainerInfo, Environment, Image, NetworkInfo, Parameter, Secret, + TaskState => MesosTaskState, Volume} import org.apache.mesos.Protos.ContainerInfo.{DockerInfo, MesosInfo} import org.apache.mesos.Protos.Environment.Variable import org.apache.mesos.protobuf.ByteString -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, TaskState} import org.apache.spark.SparkException import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.mesos.config.MesosSecretConfig import org.apache.spark.internal.Logging /** - * A collection of utility functions which can be used by both the - * MesosSchedulerBackend and the [[MesosFineGrainedSchedulerBackend]]. + * A collection of utility functions which can be used by the + * MesosSchedulerBackend, [[MesosFineGrainedSchedulerBackend]] and the MesosExecutorBackend. */ -private[mesos] object MesosSchedulerBackendUtil extends Logging { +private[spark] object MesosSchedulerBackendUtil extends Logging { /** * Parse a list of volume specs, each of which * takes the form [host-dir:]container-dir[:rw|:ro]. @@ -294,4 +295,13 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { .setImage(imageProto) .build } + + def taskStateToMesos(state: TaskState.TaskState): MesosTaskState = state match { + case TaskState.LAUNCHING => MesosTaskState.TASK_STARTING + case TaskState.RUNNING => MesosTaskState.TASK_RUNNING + case TaskState.FINISHED => MesosTaskState.TASK_FINISHED + case TaskState.FAILED => MesosTaskState.TASK_FAILED + case TaskState.KILLED => MesosTaskState.TASK_KILLED + case TaskState.LOST => MesosTaskState.TASK_LOST + } } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index ed3bd358d4082..5784ee314aa17 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -29,7 +29,7 @@ import scala.util.control.NonFatal import com.google.common.base.Splitter import com.google.common.io.Files import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler, SchedulerDriver} -import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} +import org.apache.mesos.Protos.{SlaveID => AgentID, TaskState => MesosTaskState, _} import org.apache.mesos.Protos.FrameworkInfo.Capability import org.apache.mesos.Protos.Resource.ReservationInfo import org.apache.mesos.protobuf.{ByteString, GeneratedMessageV3} @@ -304,12 +304,12 @@ trait MesosSchedulerUtils extends Logging { * Match the requirements (if any) to the offer attributes. * if attribute requirements are not specified - return true * else if attribute is defined and no values are given, simple attribute presence is performed - * else if attribute name and value is specified, subset match is performed on slave attributes + * else if attribute name and value is specified, subset match is performed on agent attributes */ def matchesAttributeRequirements( - slaveOfferConstraints: Map[String, Set[String]], + agentOfferConstraints: Map[String, Set[String]], offerAttributes: Map[String, GeneratedMessageV3]): Boolean = { - slaveOfferConstraints.forall { + agentOfferConstraints.forall { // offer has the required attribute and subsumes the required values for that attribute case (name, requiredValues) => offerAttributes.get(name) match { @@ -574,15 +574,6 @@ trait MesosSchedulerUtils extends Logging { MesosTaskState.TASK_UNREACHABLE => TaskState.LOST } - def taskStateToMesos(state: TaskState.TaskState): MesosTaskState = state match { - case TaskState.LAUNCHING => MesosTaskState.TASK_STARTING - case TaskState.RUNNING => MesosTaskState.TASK_RUNNING - case TaskState.FINISHED => MesosTaskState.TASK_FINISHED - case TaskState.FAILED => MesosTaskState.TASK_FAILED - case TaskState.KILLED => MesosTaskState.TASK_KILLED - case TaskState.LOST => MesosTaskState.TASK_LOST - } - protected def declineOffer( driver: org.apache.mesos.SchedulerDriver, offer: Offer, @@ -612,4 +603,3 @@ trait MesosSchedulerUtils extends Logging { } } } - diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index 9a50142b51d97..bb37bbd2d8046 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -413,7 +413,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi new MesosDriverDescription("d1", "jar", 100, 1, true, command, Map((config.EXECUTOR_HOME.key, "test"), ("spark.app.name", "test")), "s1", new Date())) assert(response.success) - val slaveId = SlaveID.newBuilder().setValue("s1").build() + val agentId = SlaveID.newBuilder().setValue("s1").build() val offer = Offer.newBuilder() .addResources( Resource.newBuilder().setRole("*") @@ -425,7 +425,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi .setType(Type.SCALAR)) .setId(OfferID.newBuilder().setValue("o1").build()) .setFrameworkId(FrameworkID.newBuilder().setValue("f1").build()) - .setSlaveId(slaveId) + .setSlaveId(agentId) .setHostname("host1") .build() // Offer the resource to launch the submitted driver @@ -438,7 +438,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val taskStatus = TaskStatus.newBuilder() .setTaskId(TaskID.newBuilder().setValue(response.submissionId).build()) - .setSlaveId(slaveId) + .setSlaveId(agentId) .setState(MesosTaskState.TASK_KILLED) .build() // Update the status of the killed task diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 5ab277ed87a72..4d7f6441020b7 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -105,7 +105,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite backend.statusUpdate(driver, status) verify(driver, times(1)).reviveOffers() - // Launches a new task on a valid offer from the same slave + // Launches a new task on a valid offer from the same agent offerResources(List(offer2)) verifyTaskLaunched(driver, "o2") } @@ -250,7 +250,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite verifyTaskLaunched(driver, "o2") } - test("mesos creates multiple executors on a single slave") { + test("mesos creates multiple executors on a single agent") { val executorCores = 4 setBackend(Map(EXECUTOR_CORES.key -> executorCores.toString)) @@ -727,10 +727,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite private case class Resources(mem: Int, cpus: Int, gpus: Int = 0) - private def registerMockExecutor(executorId: String, slaveId: String, cores: Integer) = { + private def registerMockExecutor(executorId: String, agentId: String, cores: Integer) = { val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] - val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty, + val message = RegisterExecutor(executorId, mockEndpointRef, agentId, cores, Map.empty, Map.empty, Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) backend.driverEndpoint.askSync[Boolean](message) @@ -766,10 +766,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } } - private def createTaskStatus(taskId: String, slaveId: String, state: TaskState): TaskStatus = { + private def createTaskStatus(taskId: String, agentId: String, state: TaskState): TaskStatus = { TaskStatus.newBuilder() .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) + .setSlaveId(SlaveID.newBuilder().setValue(agentId).build()) .setState(state) .build } diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala index 5a4bf1dd2d409..92676cc4e7395 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala @@ -46,7 +46,7 @@ object Utils { def createOffer( offerId: String, - slaveId: String, + agentId: String, mem: Int, cpus: Int, ports: Option[(Long, Long)] = None, @@ -77,8 +77,8 @@ object Utils { builder.setId(createOfferId(offerId)) .setFrameworkId(FrameworkID.newBuilder() .setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId)) - .setHostname(s"host${slaveId}") + .setSlaveId(SlaveID.newBuilder().setValue(agentId)) + .setHostname(s"host${agentId}") .addAllAttributes(attributes.asJava) .build() } @@ -101,8 +101,8 @@ object Utils { OfferID.newBuilder().setValue(offerId).build() } - def createSlaveId(slaveId: String): SlaveID = { - SlaveID.newBuilder().setValue(slaveId).build() + def createAgentId(agentId: String): SlaveID = { + SlaveID.newBuilder().setValue(agentId).build() } def createExecutorId(executorId: String): ExecutorID = { @@ -227,4 +227,3 @@ object Utils { .build() } } - diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 0475b0aed0ec4..3f2e8846e85b3 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -264,13 +264,14 @@ private[spark] abstract class YarnSchedulerBackend( case NonFatal(e) => logWarning(s"Attempted to get executor loss reason" + s" for executor id ${executorId} at RPC address ${executorRpcAddress}," + - s" but got no response. Marking as slave lost.", e) - RemoveExecutor(executorId, SlaveLost()) + s" but got no response. Marking as agent lost.", e) + RemoveExecutor(executorId, ExecutorProcessLost()) }(ThreadUtils.sameThread) case None => logWarning("Attempted to check for an executor loss reason" + " before the AM has registered!") - Future.successful(RemoveExecutor(executorId, SlaveLost("AM is not yet registered."))) + Future.successful(RemoveExecutor(executorId, + ExecutorProcessLost("AM is not yet registered."))) } removeExecutorMessage.foreach { message => driverEndpoint.send(message) } diff --git a/sbin/decommission-slave.sh b/sbin/decommission-slave.sh old mode 100644 new mode 100755 index 4bbf257ff1d3a..858bede1d2878 --- a/sbin/decommission-slave.sh +++ b/sbin/decommission-slave.sh @@ -17,41 +17,7 @@ # limitations under the License. # -# A shell script to decommission all workers on a single slave -# -# Environment variables -# -# SPARK_WORKER_INSTANCES The number of worker instances that should be -# running on this slave. Default is 1. - -# Usage: decommission-slave.sh [--block-until-exit] -# Decommissions all slaves on this worker machine - -set -ex - -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -. "${SPARK_HOME}/sbin/spark-config.sh" - -. "${SPARK_HOME}/bin/load-spark-env.sh" - -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - "${SPARK_HOME}/sbin"/spark-daemon.sh decommission org.apache.spark.deploy.worker.Worker 1 -else - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "${SPARK_HOME}/sbin"/spark-daemon.sh decommission org.apache.spark.deploy.worker.Worker $(( $i + 1 )) - done -fi +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -# Check if --block-until-exit is set. -# This is done for systems which block on the decomissioning script and on exit -# shut down the entire system (e.g. K8s). -if [ "$1" == "--block-until-exit" ]; then - shift - # For now we only block on the 0th instance if there multiple instances. - instance=$1 - pid="$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid" - wait $pid -fi +>&2 echo "This script is deprecated, use decommission-worker.sh" +"${DIR}/decommission-worker.sh" "$@" diff --git a/sbin/decommission-worker.sh b/sbin/decommission-worker.sh new file mode 100755 index 0000000000000..cf81a53f395c2 --- /dev/null +++ b/sbin/decommission-worker.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# A shell script to decommission all workers on a single worker +# +# Environment variables +# +# SPARK_WORKER_INSTANCES The number of worker instances that should be +# running on this worker machine. Default is 1. + +# Usage: decommission-worker.sh [--block-until-exit] +# Decommissions all workers on this worker machine. + +set -ex + +if [ -z "${SPARK_HOME}" ]; then + export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${SPARK_HOME}/sbin/spark-config.sh" + +. "${SPARK_HOME}/bin/load-spark-env.sh" + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + "${SPARK_HOME}/sbin"/spark-daemon.sh decommission org.apache.spark.deploy.worker.Worker 1 +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "${SPARK_HOME}/sbin"/spark-daemon.sh decommission org.apache.spark.deploy.worker.Worker $(( $i + 1 )) + done +fi + +# Check if --block-until-exit is set. +# This is done for systems which block on the decomissioning script and on exit +# shut down the entire system (e.g. K8s). +if [ "$1" == "--block-until-exit" ]; then + shift + # For now we only block on the 0th instance if there multiple instances. + instance=$1 + pid="$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid" + wait $pid +fi diff --git a/sbin/slaves.sh b/sbin/slaves.sh index c971aa3296b09..b92007ecdfad5 100755 --- a/sbin/slaves.sh +++ b/sbin/slaves.sh @@ -17,87 +17,7 @@ # limitations under the License. # -# Run a shell command on all slave hosts. -# -# Environment Variables -# -# SPARK_SLAVES File naming remote hosts. -# Default is ${SPARK_CONF_DIR}/slaves. -# SPARK_CONF_DIR Alternate conf dir. Default is ${SPARK_HOME}/conf. -# SPARK_SLAVE_SLEEP Seconds to sleep between spawning remote commands. -# SPARK_SSH_OPTS Options passed to ssh when running remote commands. -## - -usage="Usage: slaves.sh [--config ] command..." - -# if no args specified, show usage -if [ $# -le 0 ]; then - echo $usage - exit 1 -fi - -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -. "${SPARK_HOME}/sbin/spark-config.sh" - -# If the slaves file is specified in the command line, -# then it takes precedence over the definition in -# spark-env.sh. Save it here. -if [ -f "$SPARK_SLAVES" ]; then - HOSTLIST=`cat "$SPARK_SLAVES"` -fi - -# Check if --config is passed as an argument. It is an optional parameter. -# Exit if the argument is not a directory. -if [ "$1" == "--config" ] -then - shift - conf_dir="$1" - if [ ! -d "$conf_dir" ] - then - echo "ERROR : $conf_dir is not a directory" - echo $usage - exit 1 - else - export SPARK_CONF_DIR="$conf_dir" - fi - shift -fi - -. "${SPARK_HOME}/bin/load-spark-env.sh" - -if [ "$HOSTLIST" = "" ]; then - if [ "$SPARK_SLAVES" = "" ]; then - if [ -f "${SPARK_CONF_DIR}/slaves" ]; then - HOSTLIST=`cat "${SPARK_CONF_DIR}/slaves"` - else - HOSTLIST=localhost - fi - else - HOSTLIST=`cat "${SPARK_SLAVES}"` - fi -fi - - - -# By default disable strict host key checking -if [ "$SPARK_SSH_OPTS" = "" ]; then - SPARK_SSH_OPTS="-o StrictHostKeyChecking=no" -fi - -for slave in `echo "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do - if [ -n "${SPARK_SSH_FOREGROUND}" ]; then - ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ - 2>&1 | sed "s/^/$slave: /" - else - ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ - 2>&1 | sed "s/^/$slave: /" & - fi - if [ "$SPARK_SLAVE_SLEEP" != "" ]; then - sleep $SPARK_SLAVE_SLEEP - fi -done +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -wait +>&2 echo "This script is deprecated, use workers.sh" +"${DIR}/workers.sh" "$@" diff --git a/sbin/spark-daemons.sh b/sbin/spark-daemons.sh index dec2f4432df39..9a5e5f3a09c1d 100755 --- a/sbin/spark-daemons.sh +++ b/sbin/spark-daemons.sh @@ -17,7 +17,7 @@ # limitations under the License. # -# Run a Spark command on all slave hosts. +# Run a Spark command on all worker hosts. usage="Usage: spark-daemons.sh [--config ] [start|stop] command instance-number args..." @@ -33,4 +33,4 @@ fi . "${SPARK_HOME}/sbin/spark-config.sh" -exec "${SPARK_HOME}/sbin/slaves.sh" cd "${SPARK_HOME}" \; "${SPARK_HOME}/sbin/spark-daemon.sh" "$@" +exec "${SPARK_HOME}/sbin/workers.sh" cd "${SPARK_HOME}" \; "${SPARK_HOME}/sbin/spark-daemon.sh" "$@" diff --git a/sbin/start-all.sh b/sbin/start-all.sh index a5d30d274ea6e..064074e07922b 100755 --- a/sbin/start-all.sh +++ b/sbin/start-all.sh @@ -19,7 +19,7 @@ # Start all spark daemons. # Starts the master on this node. -# Starts a worker on each node specified in conf/slaves +# Starts a worker on each node specified in conf/workers if [ -z "${SPARK_HOME}" ]; then export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" @@ -32,4 +32,4 @@ fi "${SPARK_HOME}/sbin"/start-master.sh # Start Workers -"${SPARK_HOME}/sbin"/start-slaves.sh +"${SPARK_HOME}/sbin"/start-workers.sh diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 9b3b26b07842b..68682532f02ee 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -17,76 +17,7 @@ # limitations under the License. # -# Starts a slave on the machine this script is executed on. -# -# Environment Variables -# -# SPARK_WORKER_INSTANCES The number of worker instances to run on this -# slave. Default is 1. Note it has been deprecate since Spark 3.0. -# SPARK_WORKER_PORT The base port number for the first worker. If set, -# subsequent workers will increment this number. If -# unset, Spark will find a valid port number, but -# with no guarantee of a predictable pattern. -# SPARK_WORKER_WEBUI_PORT The base port for the web interface of the first -# worker. Subsequent workers will increment this -# number. Default is 8081. - -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -# NOTE: This exact class name is matched downstream by SparkSubmit. -# Any changes need to be reflected there. -CLASS="org.apache.spark.deploy.worker.Worker" - -if [[ $# -lt 1 ]] || [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/start-slave.sh [options]" - pattern="Usage:" - pattern+="\|Using Spark's default log4j profile:" - pattern+="\|Started daemon with process name" - pattern+="\|Registered signal handler for" - - "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 - exit 1 -fi - -. "${SPARK_HOME}/sbin/spark-config.sh" - -. "${SPARK_HOME}/bin/load-spark-env.sh" - -# First argument should be the master; we need to store it aside because we may -# need to insert arguments between it and the other arguments -MASTER=$1 -shift - -# Determine desired worker port -if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then - SPARK_WORKER_WEBUI_PORT=8081 -fi - -# Start up the appropriate number of workers on this machine. -# quick local function to start a worker -function start_instance { - WORKER_NUM=$1 - shift - - if [ "$SPARK_WORKER_PORT" = "" ]; then - PORT_FLAG= - PORT_NUM= - else - PORT_FLAG="--port" - PORT_NUM=$(( $SPARK_WORKER_PORT + $WORKER_NUM - 1 )) - fi - WEBUI_PORT=$(( $SPARK_WORKER_WEBUI_PORT + $WORKER_NUM - 1 )) - - "${SPARK_HOME}/sbin"/spark-daemon.sh start $CLASS $WORKER_NUM \ - --webui-port "$WEBUI_PORT" $PORT_FLAG $PORT_NUM $MASTER "$@" -} +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - start_instance 1 "$@" -else - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - start_instance $(( 1 + $i )) "$@" - done -fi +>&2 echo "This script is deprecated, use start-worker.sh" +"${DIR}/start-worker.sh" "$@" diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index f5269df523dac..9b113d9f2e0f4 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -17,30 +17,7 @@ # limitations under the License. # -# Starts a slave instance on each machine specified in the conf/slaves file. +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -. "${SPARK_HOME}/sbin/spark-config.sh" -. "${SPARK_HOME}/bin/load-spark-env.sh" - -# Find the port number for the master -if [ "$SPARK_MASTER_PORT" = "" ]; then - SPARK_MASTER_PORT=7077 -fi - -if [ "$SPARK_MASTER_HOST" = "" ]; then - case `uname` in - (SunOS) - SPARK_MASTER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" - ;; - (*) - SPARK_MASTER_HOST="`hostname -f`" - ;; - esac -fi - -# Launch the slaves -"${SPARK_HOME}/sbin/slaves.sh" cd "${SPARK_HOME}" \; "${SPARK_HOME}/sbin/start-slave.sh" "spark://$SPARK_MASTER_HOST:$SPARK_MASTER_PORT" +>&2 echo "This script is deprecated, use start-workers.sh" +"${DIR}/start-workers.sh" "$@" diff --git a/sbin/start-worker.sh b/sbin/start-worker.sh new file mode 100755 index 0000000000000..fd58f01bac2eb --- /dev/null +++ b/sbin/start-worker.sh @@ -0,0 +1,92 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Starts a worker on the machine this script is executed on. +# +# Environment Variables +# +# SPARK_WORKER_INSTANCES The number of worker instances to run on this +# worker. Default is 1. Note it has been deprecate since Spark 3.0. +# SPARK_WORKER_PORT The base port number for the first worker. If set, +# subsequent workers will increment this number. If +# unset, Spark will find a valid port number, but +# with no guarantee of a predictable pattern. +# SPARK_WORKER_WEBUI_PORT The base port for the web interface of the first +# worker. Subsequent workers will increment this +# number. Default is 8081. + +if [ -z "${SPARK_HOME}" ]; then + export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +# NOTE: This exact class name is matched downstream by SparkSubmit. +# Any changes need to be reflected there. +CLASS="org.apache.spark.deploy.worker.Worker" + +if [[ $# -lt 1 ]] || [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/start-worker.sh [options]" + pattern="Usage:" + pattern+="\|Using Spark's default log4j profile:" + pattern+="\|Started daemon with process name" + pattern+="\|Registered signal handler for" + + "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + exit 1 +fi + +. "${SPARK_HOME}/sbin/spark-config.sh" + +. "${SPARK_HOME}/bin/load-spark-env.sh" + +# First argument should be the master; we need to store it aside because we may +# need to insert arguments between it and the other arguments +MASTER=$1 +shift + +# Determine desired worker port +if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then + SPARK_WORKER_WEBUI_PORT=8081 +fi + +# Start up the appropriate number of workers on this machine. +# quick local function to start a worker +function start_instance { + WORKER_NUM=$1 + shift + + if [ "$SPARK_WORKER_PORT" = "" ]; then + PORT_FLAG= + PORT_NUM= + else + PORT_FLAG="--port" + PORT_NUM=$(( $SPARK_WORKER_PORT + $WORKER_NUM - 1 )) + fi + WEBUI_PORT=$(( $SPARK_WORKER_WEBUI_PORT + $WORKER_NUM - 1 )) + + "${SPARK_HOME}/sbin"/spark-daemon.sh start $CLASS $WORKER_NUM \ + --webui-port "$WEBUI_PORT" $PORT_FLAG $PORT_NUM $MASTER "$@" +} + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + start_instance 1 "$@" +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + start_instance $(( 1 + $i )) "$@" + done +fi diff --git a/sbin/start-workers.sh b/sbin/start-workers.sh new file mode 100755 index 0000000000000..3867ef3ccf255 --- /dev/null +++ b/sbin/start-workers.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Starts a worker instance on each machine specified in the conf/workers file. + +if [ -z "${SPARK_HOME}" ]; then + export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${SPARK_HOME}/sbin/spark-config.sh" +. "${SPARK_HOME}/bin/load-spark-env.sh" + +# Find the port number for the master +if [ "$SPARK_MASTER_PORT" = "" ]; then + SPARK_MASTER_PORT=7077 +fi + +if [ "$SPARK_MASTER_HOST" = "" ]; then + case `uname` in + (SunOS) + SPARK_MASTER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + SPARK_MASTER_HOST="`hostname -f`" + ;; + esac +fi + +# Launch the workers +"${SPARK_HOME}/sbin/workers.sh" cd "${SPARK_HOME}" \; "${SPARK_HOME}/sbin/start-worker.sh" "spark://$SPARK_MASTER_HOST:$SPARK_MASTER_PORT" diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh index 4e476ca05cb05..2c40905cd499b 100755 --- a/sbin/stop-all.sh +++ b/sbin/stop-all.sh @@ -27,8 +27,8 @@ fi # Load the Spark configuration . "${SPARK_HOME}/sbin/spark-config.sh" -# Stop the slaves, then the master -"${SPARK_HOME}/sbin"/stop-slaves.sh +# Stop the workers, then the master +"${SPARK_HOME}/sbin"/stop-workers.sh "${SPARK_HOME}/sbin"/stop-master.sh if [ "$1" == "--wait" ] @@ -36,7 +36,7 @@ then printf "Waiting for workers to shut down..." while true do - running=`${SPARK_HOME}/sbin/slaves.sh ps -ef | grep -v grep | grep deploy.worker.Worker` + running=`${SPARK_HOME}/sbin/workers.sh ps -ef | grep -v grep | grep deploy.worker.Worker` if [ -z "$running" ] then printf "\nAll workers successfully shut down.\n" diff --git a/sbin/stop-slave.sh b/sbin/stop-slave.sh index 685bcf59b33aa..71ed29987d4a1 100755 --- a/sbin/stop-slave.sh +++ b/sbin/stop-slave.sh @@ -17,28 +17,7 @@ # limitations under the License. # -# A shell script to stop all workers on a single slave -# -# Environment variables -# -# SPARK_WORKER_INSTANCES The number of worker instances that should be -# running on this slave. Default is 1. - -# Usage: stop-slave.sh -# Stops all slaves on this worker machine - -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -. "${SPARK_HOME}/sbin/spark-config.sh" - -. "${SPARK_HOME}/bin/load-spark-env.sh" +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - "${SPARK_HOME}/sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker 1 -else - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "${SPARK_HOME}/sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) - done -fi +>&2 echo "This script is deprecated, use stop-worker.sh" +"${DIR}/stop-worker.sh" "$@" diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh index a57441b52a04a..c0aca6868efe3 100755 --- a/sbin/stop-slaves.sh +++ b/sbin/stop-slaves.sh @@ -17,12 +17,7 @@ # limitations under the License. # -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -. "${SPARK_HOME}/sbin/spark-config.sh" - -. "${SPARK_HOME}/bin/load-spark-env.sh" - -"${SPARK_HOME}/sbin/slaves.sh" cd "${SPARK_HOME}" \; "${SPARK_HOME}/sbin"/stop-slave.sh +>&2 echo "This script is deprecated, use stop-workers.sh" +"${DIR}/stop-workers.sh" "$@" diff --git a/sbin/stop-worker.sh b/sbin/stop-worker.sh new file mode 100755 index 0000000000000..112b62ecffa27 --- /dev/null +++ b/sbin/stop-worker.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# A shell script to stop all workers on a single worker +# +# Environment variables +# +# SPARK_WORKER_INSTANCES The number of worker instances that should be +# running on this worker machine. Default is 1. + +# Usage: stop-worker.sh +# Stops all workers on this worker machine + +if [ -z "${SPARK_HOME}" ]; then + export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${SPARK_HOME}/sbin/spark-config.sh" + +. "${SPARK_HOME}/bin/load-spark-env.sh" + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + "${SPARK_HOME}/sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker 1 +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "${SPARK_HOME}/sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) + done +fi diff --git a/sbin/stop-workers.sh b/sbin/stop-workers.sh new file mode 100755 index 0000000000000..552800f522222 --- /dev/null +++ b/sbin/stop-workers.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +if [ -z "${SPARK_HOME}" ]; then + export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${SPARK_HOME}/sbin/spark-config.sh" + +. "${SPARK_HOME}/bin/load-spark-env.sh" + +"${SPARK_HOME}/sbin/workers.sh" cd "${SPARK_HOME}" \; "${SPARK_HOME}/sbin"/stop-worker.sh diff --git a/sbin/workers.sh b/sbin/workers.sh new file mode 100755 index 0000000000000..cab0330723a6c --- /dev/null +++ b/sbin/workers.sh @@ -0,0 +1,120 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Run a shell command on all worker hosts. +# +# Environment Variables +# +# SPARK_WORKERS File naming remote hosts. +# Default is ${SPARK_CONF_DIR}/workers. +# SPARK_CONF_DIR Alternate conf dir. Default is ${SPARK_HOME}/conf. +# SPARK_WORKER_SLEEP Seconds to sleep between spawning remote commands. +# SPARK_SSH_OPTS Options passed to ssh when running remote commands. +## + +usage="Usage: workers.sh [--config ] command..." + +# if no args specified, show usage +if [ $# -le 0 ]; then + echo $usage + exit 1 +fi + +if [ -z "${SPARK_HOME}" ]; then + export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${SPARK_HOME}/sbin/spark-config.sh" + +# If the workers file is specified in the command line, +# then it takes precedence over the definition in +# spark-env.sh. Save it here. +if [ -f "$SPARK_WORKERS" ]; then + HOSTLIST=`cat "$SPARK_WORKERS"` +fi +if [ -f "$SPARK_SLAVES" ]; then + >&2 echo "SPARK_SLAVES is deprecated, use SPARK_WORKERS" + HOSTLIST=`cat "$SPARK_SLAVES"` +fi + + +# Check if --config is passed as an argument. It is an optional parameter. +# Exit if the argument is not a directory. +if [ "$1" == "--config" ] +then + shift + conf_dir="$1" + if [ ! -d "$conf_dir" ] + then + echo "ERROR : $conf_dir is not a directory" + echo $usage + exit 1 + else + export SPARK_CONF_DIR="$conf_dir" + fi + shift +fi + +. "${SPARK_HOME}/bin/load-spark-env.sh" + +if [ "$HOSTLIST" = "" ]; then + if [ "$SPARK_SLAVES" = "" ] && [ "$SPARK_WORKERS" = "" ]; then + if [ -f "${SPARK_CONF_DIR}/workers" ]; then + HOSTLIST=`cat "${SPARK_CONF_DIR}/workers"` + elif [ -f "${SPARK_CONF_DIR}/slaves" ]; then + HOSTLIST=`cat "${SPARK_CONF_DIR}/slaves"` + else + HOSTLIST=localhost + fi + else + if [ -f "$SPARK_WORKERS" ]; then + HOSTLIST=`cat "$SPARK_WORKERS"` + fi + if [ -f "$SPARK_SLAVES" ]; then + >&2 echo "SPARK_SLAVES is deprecated, use SPARK_WORKERS" + HOSTLIST=`cat "$SPARK_SLAVES"` + fi + fi +fi + + + +# By default disable strict host key checking +if [ "$SPARK_SSH_OPTS" = "" ]; then + SPARK_SSH_OPTS="-o StrictHostKeyChecking=no" +fi + +for host in `echo "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do + if [ -n "${SPARK_SSH_FOREGROUND}" ]; then + ssh $SPARK_SSH_OPTS "$host" $"${@// /\\ }" \ + 2>&1 | sed "s/^/$host: /" + else + ssh $SPARK_SSH_OPTS "$host" $"${@// /\\ }" \ + 2>&1 | sed "s/^/$host: /" & + fi + if [ "$SPARK_WORKER_SLEEP" != "" ]; then + sleep $SPARK_WORKER_SLEEP + fi + if [ "$SPARK_SLAVE_SLEEP" != "" ]; then + >&2 echo "SPARK_SLAVE_SLEEP is deprecated, use SPARK_WORKER_SLEEP" + sleep $SPARK_SLAVE_SLEEP + fi +done + +wait diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 233e6224a10d9..109b7f4bb31bb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils -/** A singleton object for the master program. The slaves should not access this. */ +/** A singleton object for the master program. The executors should not access this. */ private[hive] object SparkSQLEnv extends Logging { logDebug("Initializing SparkSQLEnv") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 8ddb979d7713a..3e0d44160c8a1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -308,7 +308,7 @@ class HadoopTableReader( /** * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be - * applied locally on each slave. + * applied locally on each executor. */ private def createOldHadoopRDD(tableDesc: TableDesc, path: String): RDD[Writable] = { val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ @@ -330,7 +330,7 @@ class HadoopTableReader( /** * Creates a NewHadoopRDD based on the broadcasted HiveConf and other job properties that will be - * applied locally on each slave. + * applied locally on each executor. */ private def createNewHadoopRDD(tableDesc: TableDesc, path: String): RDD[Writable] = { val newJobConf = new JobConf(hadoopConf) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 2d53a1b4c78b6..7850285be83af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -67,7 +67,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * @param master Name of the Spark Master * @param appName Name to be used when registering with the scheduler * @param batchDuration The time interval at which streaming data will be divided into batches - * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jarFile JAR file containing job code, to ship to cluster. This can be a path on the * local file system or an HDFS, HTTP, HTTPS, or FTP URL. */ @@ -84,7 +84,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * @param master Name of the Spark Master * @param appName Name to be used when registering with the scheduler * @param batchDuration The time interval at which streaming data will be divided into batches - * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. */ @@ -101,7 +101,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * @param master Name of the Spark Master * @param appName Name to be used when registering with the scheduler * @param batchDuration The time interval at which streaming data will be divided into batches - * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. * @param environment Environment variables to set on worker nodes diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 13cf5cc0e71ea..51c4b0fd4a2d8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -415,7 +415,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } /** - * Run the dummy Spark job to ensure that all slaves have registered. This avoids all the + * Run the dummy Spark job to ensure that all executors have registered. This avoids all the * receivers to be scheduled on the same node. * * TODO Should poll the executor number and wait for executors according to diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index 9cdfdb8374322..e207dab7de068 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -93,7 +93,7 @@ object RawTextHelper { } /** - * Warms up the SparkContext in master and slave by running tasks to force JIT kick in + * Warms up the SparkContext in master and executor by running tasks to force JIT kick in * before real workload starts. */ def warmUp(sc: SparkContext): Unit = { From 4ad9bfd53b84a6d2497668c73af6899bae14c187 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 14 Jul 2020 11:22:44 +0900 Subject: [PATCH 184/384] [SPARK-32138] Drop Python 2.7, 3.4 and 3.5 ### What changes were proposed in this pull request? This PR aims to drop Python 2.7, 3.4 and 3.5. Roughly speaking, it removes all the widely known Python 2 compatibility workarounds such as `sys.version` comparison, `__future__`. Also, it removes the Python 2 dedicated codes such as `ArrayConstructor` in Spark. ### Why are the changes needed? 1. Unsupport EOL Python versions 2. Reduce maintenance overhead and remove a bit of legacy codes and hacks for Python 2. 3. PyPy2 has a critical bug that causes a flaky test, SPARK-28358 given my testing and investigation. 4. Users can use Python type hints with Pandas UDFs without thinking about Python version 5. Users can leverage one latest cloudpickle, https://github.com/apache/spark/pull/28950. With Python 3.8+ it can also leverage C pickle. ### Does this PR introduce _any_ user-facing change? Yes, users cannot use Python 2.7, 3.4 and 3.5 in the upcoming Spark version. ### How was this patch tested? Manually tested and also tested in Jenkins. Closes #28957 from HyukjinKwon/SPARK-32138. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .github/workflows/master.yml | 3 +- .../apache/spark/api/python/SerDeUtil.scala | 66 ----- dev/create-release/releaseutils.py | 7 +- dev/github_jira_sync.py | 11 +- dev/lint-python | 10 +- dev/merge_spark_pr.py | 12 +- dev/run-tests-jenkins.py | 12 +- dev/sparktestsupport/toposort.py | 3 +- docs/configuration.md | 2 +- docs/index.md | 3 +- docs/rdd-programming-guide.md | 12 +- examples/src/main/python/als.py | 2 - examples/src/main/python/avro_inputformat.py | 2 - examples/src/main/python/kmeans.py | 2 - .../src/main/python/logistic_regression.py | 2 - .../main/python/ml/aft_survival_regression.py | 2 - examples/src/main/python/ml/als_example.py | 8 +- .../main/python/ml/anova_selector_example.py | 2 - .../src/main/python/ml/anova_test_example.py | 2 - .../src/main/python/ml/binarizer_example.py | 2 - .../python/ml/bisecting_k_means_example.py | 2 - .../bucketed_random_projection_lsh_example.py | 2 - .../src/main/python/ml/bucketizer_example.py | 2 - .../main/python/ml/chi_square_test_example.py | 2 - .../main/python/ml/chisq_selector_example.py | 2 - .../src/main/python/ml/correlation_example.py | 2 - .../python/ml/count_vectorizer_example.py | 2 - .../src/main/python/ml/cross_validator.py | 2 - .../src/main/python/ml/dataframe_example.py | 2 - examples/src/main/python/ml/dct_example.py | 2 - .../decision_tree_classification_example.py | 2 - .../ml/decision_tree_regression_example.py | 2 - .../python/ml/elementwise_product_example.py | 2 - .../ml/estimator_transformer_param_example.py | 2 - .../main/python/ml/feature_hasher_example.py | 2 - .../main/python/ml/fm_classifier_example.py | 2 - .../main/python/ml/fm_regressor_example.py | 2 - .../main/python/ml/fvalue_selector_example.py | 2 - .../src/main/python/ml/fvalue_test_example.py | 2 - .../python/ml/gaussian_mixture_example.py | 2 - .../generalized_linear_regression_example.py | 2 - ...radient_boosted_tree_classifier_example.py | 2 - ...gradient_boosted_tree_regressor_example.py | 2 - .../main/python/ml/index_to_string_example.py | 2 - .../src/main/python/ml/interaction_example.py | 2 - .../python/ml/isotonic_regression_example.py | 2 - examples/src/main/python/ml/kmeans_example.py | 2 - examples/src/main/python/ml/lda_example.py | 2 - .../ml/linear_regression_with_elastic_net.py | 2 - examples/src/main/python/ml/linearsvc.py | 2 - .../ml/logistic_regression_summary_example.py | 2 - .../logistic_regression_with_elastic_net.py | 2 - .../main/python/ml/max_abs_scaler_example.py | 2 - .../main/python/ml/min_hash_lsh_example.py | 2 - .../main/python/ml/min_max_scaler_example.py | 2 - ...ss_logistic_regression_with_elastic_net.py | 2 - .../multilayer_perceptron_classification.py | 2 - examples/src/main/python/ml/n_gram_example.py | 2 - .../src/main/python/ml/naive_bayes_example.py | 2 - .../src/main/python/ml/normalizer_example.py | 2 - .../src/main/python/ml/one_vs_rest_example.py | 2 - .../main/python/ml/onehot_encoder_example.py | 2 - examples/src/main/python/ml/pca_example.py | 2 - .../python/ml/polynomial_expansion_example.py | 2 - .../python/ml/quantile_discretizer_example.py | 2 - .../ml/random_forest_classifier_example.py | 2 - .../ml/random_forest_regressor_example.py | 2 - .../src/main/python/ml/rformula_example.py | 2 - .../main/python/ml/robust_scaler_example.py | 2 - .../src/main/python/ml/sql_transformer.py | 2 - .../main/python/ml/standard_scaler_example.py | 2 - .../python/ml/stopwords_remover_example.py | 2 - .../main/python/ml/string_indexer_example.py | 2 - .../src/main/python/ml/summarizer_example.py | 2 - examples/src/main/python/ml/tf_idf_example.py | 2 - .../src/main/python/ml/tokenizer_example.py | 2 - .../ml/variance_threshold_selector_example.py | 2 - .../python/ml/vector_assembler_example.py | 2 - .../main/python/ml/vector_indexer_example.py | 2 - .../python/ml/vector_size_hint_example.py | 2 - .../main/python/ml/vector_slicer_example.py | 2 - .../src/main/python/ml/word2vec_example.py | 2 - .../binary_classification_metrics_example.py | 1 - .../python/mllib/bisecting_k_means_example.py | 2 - .../src/main/python/mllib/correlations.py | 2 - .../main/python/mllib/correlations_example.py | 2 - .../decision_tree_classification_example.py | 2 - .../mllib/decision_tree_regression_example.py | 2 - .../mllib/elementwise_product_example.py | 2 - .../python/mllib/gaussian_mixture_example.py | 2 - .../python/mllib/gaussian_mixture_model.py | 7 +- ...radient_boosting_classification_example.py | 2 - .../gradient_boosting_regression_example.py | 2 - .../mllib/hypothesis_testing_example.py | 2 - ...testing_kolmogorov_smirnov_test_example.py | 2 - .../mllib/isotonic_regression_example.py | 2 - .../src/main/python/mllib/k_means_example.py | 2 - .../kernel_density_estimation_example.py | 2 - examples/src/main/python/mllib/kmeans.py | 2 - .../latent_dirichlet_allocation_example.py | 2 - .../linear_regression_with_sgd_example.py | 2 - .../main/python/mllib/logistic_regression.py | 2 - .../logistic_regression_with_lbfgs_example.py | 2 - .../main/python/mllib/naive_bayes_example.py | 2 - .../main/python/mllib/normalizer_example.py | 2 - .../power_iteration_clustering_example.py | 2 - .../random_forest_classification_example.py | 2 - .../mllib/random_forest_regression_example.py | 2 - .../python/mllib/random_rdd_generation.py | 2 - .../python/mllib/recommendation_example.py | 2 - .../src/main/python/mllib/sampled_rdds.py | 2 - .../python/mllib/standard_scaler_example.py | 2 - .../mllib/stratified_sampling_example.py | 2 - .../python/mllib/streaming_k_means_example.py | 2 - .../streaming_linear_regression_example.py | 2 - .../mllib/summary_statistics_example.py | 2 - .../src/main/python/mllib/tf_idf_example.py | 2 - examples/src/main/python/mllib/word2vec.py | 2 - .../src/main/python/mllib/word2vec_example.py | 2 - examples/src/main/python/pagerank.py | 2 - .../src/main/python/parquet_inputformat.py | 2 - examples/src/main/python/pi.py | 2 - examples/src/main/python/sort.py | 2 - examples/src/main/python/sql/arrow.py | 9 - examples/src/main/python/sql/basic.py | 2 - examples/src/main/python/sql/datasource.py | 2 - examples/src/main/python/sql/hive.py | 2 - .../streaming/structured_kafka_wordcount.py | 2 - .../streaming/structured_network_wordcount.py | 2 - .../structured_network_wordcount_windowed.py | 2 - examples/src/main/python/status_api_demo.py | 7 +- .../main/python/streaming/hdfs_wordcount.py | 2 - .../python/streaming/network_wordcount.py | 2 - .../streaming/network_wordjoinsentiments.py | 2 - .../recoverable_network_wordcount.py | 2 - .../python/streaming/sql_network_wordcount.py | 2 - .../streaming/stateful_network_wordcount.py | 2 - .../src/main/python/transitive_closure.py | 2 - examples/src/main/python/wordcount.py | 2 - .../streaming/kinesis_wordcount_asl.py | 2 - python/pyspark/accumulators.py | 5 +- python/pyspark/broadcast.py | 10 +- python/pyspark/conf.py | 25 +- python/pyspark/context.py | 44 +-- python/pyspark/find_spark_home.py | 31 +- python/pyspark/java_gateway.py | 7 +- python/pyspark/ml/classification.py | 8 +- python/pyspark/ml/common.py | 7 +- python/pyspark/ml/feature.py | 39 +-- python/pyspark/ml/fpm.py | 6 +- python/pyspark/ml/image.py | 3 +- python/pyspark/ml/linalg/__init__.py | 33 +-- python/pyspark/ml/param/__init__.py | 25 +- .../ml/param/_shared_params_code_gen.py | 2 - python/pyspark/ml/pipeline.py | 6 +- python/pyspark/ml/tests/test_feature.py | 5 +- python/pyspark/ml/tests/test_param.py | 15 +- .../pyspark/ml/tests/test_training_summary.py | 5 +- python/pyspark/ml/tree.py | 6 +- python/pyspark/ml/tuning.py | 3 +- python/pyspark/ml/util.py | 20 +- python/pyspark/ml/wrapper.py | 12 +- python/pyspark/mllib/__init__.py | 2 - python/pyspark/mllib/clustering.py | 22 +- python/pyspark/mllib/common.py | 7 +- python/pyspark/mllib/feature.py | 19 +- python/pyspark/mllib/fpm.py | 7 +- python/pyspark/mllib/linalg/__init__.py | 32 +-- python/pyspark/mllib/linalg/distributed.py | 35 ++- python/pyspark/mllib/stat/KernelDensity.py | 5 - python/pyspark/mllib/stat/_statistics.py | 14 +- python/pyspark/mllib/tests/test_linalg.py | 5 +- python/pyspark/mllib/tree.py | 2 - python/pyspark/mllib/util.py | 14 +- python/pyspark/rdd.py | 39 +-- python/pyspark/resultiterable.py | 5 +- python/pyspark/serializers.py | 50 +--- python/pyspark/shell.py | 5 +- python/pyspark/sql/__init__.py | 3 - python/pyspark/sql/avro/functions.py | 7 +- python/pyspark/sql/catalog.py | 10 +- python/pyspark/sql/column.py | 80 +++--- python/pyspark/sql/conf.py | 10 +- python/pyspark/sql/context.py | 46 ++- python/pyspark/sql/dataframe.py | 272 ++++++++---------- python/pyspark/sql/functions.py | 212 +++++--------- python/pyspark/sql/group.py | 8 +- python/pyspark/sql/pandas/conversion.py | 24 +- python/pyspark/sql/pandas/functions.py | 45 ++- python/pyspark/sql/pandas/serializers.py | 13 +- python/pyspark/sql/readwriter.py | 39 ++- python/pyspark/sql/session.py | 45 +-- python/pyspark/sql/streaming.py | 28 +- python/pyspark/sql/tests/test_arrow.py | 14 +- python/pyspark/sql/tests/test_column.py | 10 +- python/pyspark/sql/tests/test_context.py | 6 +- python/pyspark/sql/tests/test_functions.py | 4 - .../sql/tests/test_pandas_cogrouped_map.py | 15 +- .../sql/tests/test_pandas_grouped_map.py | 39 ++- python/pyspark/sql/tests/test_pandas_map.py | 3 - .../sql/tests/test_pandas_udf_scalar.py | 27 +- .../sql/tests/test_pandas_udf_typehints.py | 252 ++++++---------- python/pyspark/sql/tests/test_types.py | 27 +- python/pyspark/sql/types.py | 101 +------ python/pyspark/sql/udf.py | 14 +- python/pyspark/sql/utils.py | 21 +- python/pyspark/streaming/context.py | 2 - python/pyspark/streaming/dstream.py | 7 +- python/pyspark/taskcontext.py | 4 - python/pyspark/testing/sqlutils.py | 7 +- python/pyspark/tests/test_profiler.py | 6 +- python/pyspark/tests/test_rdd.py | 21 +- python/pyspark/tests/test_readwrite.py | 183 ------------ python/pyspark/tests/test_shuffle.py | 13 +- python/pyspark/tests/test_taskcontext.py | 7 +- python/pyspark/tests/test_util.py | 6 +- python/pyspark/tests/test_worker.py | 27 +- python/pyspark/util.py | 42 --- python/pyspark/worker.py | 32 +-- python/run-tests.py | 9 +- python/setup.py | 11 +- .../integration-tests/tests/pyfiles.py | 3 - .../tests/worker_memory_check.py | 2 - .../src/test/resources/data/scripts/cat.py | 1 - .../resources/data/scripts/dumpdata_script.py | 9 +- 225 files changed, 735 insertions(+), 2033 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d6458bf44fd36..5cf00c6ed9e67 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -133,7 +133,8 @@ jobs: architecture: x64 - name: Install Python 3.6 uses: actions/setup-python@v2 - if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) + # Yarn has a Python specific test too, for example, YarnClusterSuite. + if: contains(matrix.modules, 'yarn') || contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) with: python-version: 3.6 architecture: x64 diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 01e64b6972ae2..5a6fa507963f0 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -45,71 +45,6 @@ private[spark] object SerDeUtil extends Logging { } } } - // Unpickle array.array generated by Python 2.6 - class ArrayConstructor extends net.razorvine.pickle.objects.ArrayConstructor { - // /* Description of types */ - // static struct arraydescr descriptors[] = { - // {'c', sizeof(char), c_getitem, c_setitem}, - // {'b', sizeof(char), b_getitem, b_setitem}, - // {'B', sizeof(char), BB_getitem, BB_setitem}, - // #ifdef Py_USING_UNICODE - // {'u', sizeof(Py_UNICODE), u_getitem, u_setitem}, - // #endif - // {'h', sizeof(short), h_getitem, h_setitem}, - // {'H', sizeof(short), HH_getitem, HH_setitem}, - // {'i', sizeof(int), i_getitem, i_setitem}, - // {'I', sizeof(int), II_getitem, II_setitem}, - // {'l', sizeof(long), l_getitem, l_setitem}, - // {'L', sizeof(long), LL_getitem, LL_setitem}, - // {'f', sizeof(float), f_getitem, f_setitem}, - // {'d', sizeof(double), d_getitem, d_setitem}, - // {'\0', 0, 0, 0} /* Sentinel */ - // }; - val machineCodes: Map[Char, Int] = if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) { - Map('B' -> 0, 'b' -> 1, 'H' -> 3, 'h' -> 5, 'I' -> 7, 'i' -> 9, - 'L' -> 11, 'l' -> 13, 'f' -> 15, 'd' -> 17, 'u' -> 21 - ) - } else { - Map('B' -> 0, 'b' -> 1, 'H' -> 2, 'h' -> 4, 'I' -> 6, 'i' -> 8, - 'L' -> 10, 'l' -> 12, 'f' -> 14, 'd' -> 16, 'u' -> 20 - ) - } - override def construct(args: Array[Object]): Object = { - if (args.length == 1) { - construct(args ++ Array("")) - } else if (args.length == 2 && args(1).isInstanceOf[String]) { - val typecode = args(0).asInstanceOf[String].charAt(0) - // This must be ISO 8859-1 / Latin 1, not UTF-8, to interoperate correctly - val data = args(1).asInstanceOf[String].getBytes(StandardCharsets.ISO_8859_1) - if (typecode == 'c') { - // It seems like the pickle of pypy uses the similar protocol to Python 2.6, which uses - // a string for array data instead of list as Python 2.7, and handles an array of - // typecode 'c' as 1-byte character. - val result = new Array[Char](data.length) - var i = 0 - while (i < data.length) { - result(i) = data(i).toChar - i += 1 - } - result - } else { - construct(typecode, machineCodes(typecode), data) - } - } else if (args.length == 2 && args(0) == "l") { - // On Python 2, an array of typecode 'l' should be handled as long rather than int. - val values = args(1).asInstanceOf[JArrayList[_]] - val result = new Array[Long](values.size) - var i = 0 - while (i < values.size) { - result(i) = values.get(i).asInstanceOf[Number].longValue() - i += 1 - } - result - } else { - super.construct(args) - } - } - } private var initialized = false // This should be called before trying to unpickle array.array from Python @@ -117,7 +52,6 @@ private[spark] object SerDeUtil extends Logging { def initialize(): Unit = { synchronized{ if (!initialized) { - Unpickler.registerConstructor("array", "array", new ArrayConstructor()) Unpickler.registerConstructor("__builtin__", "bytearray", new ByteArrayConstructor()) Unpickler.registerConstructor("builtins", "bytearray", new ByteArrayConstructor()) Unpickler.registerConstructor("__builtin__", "bytes", new ByteArrayConstructor()) diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index a5a26ae8f5354..241b7ed539ae9 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -49,8 +49,6 @@ print("Install using 'sudo pip install unidecode'") sys.exit(-1) -if sys.version < '3': - input = raw_input # noqa # Contributors list file name contributors_file_name = "contributors.txt" @@ -152,10 +150,7 @@ def get_commits(tag): if not is_valid_author(author): author = github_username # Guard against special characters - try: # Python 2 - author = unicode(author, "UTF-8") - except NameError: # Python 3 - author = str(author) + author = str(author) author = unidecode.unidecode(author).strip() commit = Commit(_hash, author, title, pr_number) commits.append(commit) diff --git a/dev/github_jira_sync.py b/dev/github_jira_sync.py index b444b74d4027c..b90afeebc5238 100755 --- a/dev/github_jira_sync.py +++ b/dev/github_jira_sync.py @@ -22,14 +22,9 @@ import os import re import sys -if sys.version < '3': - from urllib2 import urlopen - from urllib2 import Request - from urllib2 import HTTPError -else: - from urllib.request import urlopen - from urllib.request import Request - from urllib.error import HTTPError +from urllib.request import urlopen +from urllib.request import Request +from urllib.error import HTTPError try: import jira.client diff --git a/dev/lint-python b/dev/lint-python index d5491f2447176..1fddbfa64b32c 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -168,7 +168,15 @@ function sphinx_test { # Check that the documentation builds acceptably, skip check if sphinx is not installed. if ! hash "$SPHINX_BUILD" 2> /dev/null; then - echo "The $SPHINX_BUILD command was not found. Skipping pydoc checks for now." + echo "The $SPHINX_BUILD command was not found. Skipping Sphinx build for now." + echo + return + fi + + # TODO(SPARK-32279): Install Sphinx in Python 3 of Jenkins machines + PYTHON_HAS_SPHINX=$("$PYTHON_EXECUTABLE" -c 'import importlib.util; print(importlib.util.find_spec("sphinx") is not None)') + if [[ "$PYTHON_HAS_SPHINX" == "False" ]]; then + echo "$PYTHON_EXECUTABLE does not have Sphinx installed. Skipping Sphinx build for now." echo return fi diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 967cdace60dc9..b42429d7175b1 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -31,15 +31,9 @@ import subprocess import sys import traceback -if sys.version < '3': - input = raw_input # noqa - from urllib2 import urlopen - from urllib2 import Request - from urllib2 import HTTPError -else: - from urllib.request import urlopen - from urllib.request import Request - from urllib.error import HTTPError +from urllib.request import urlopen +from urllib.request import Request +from urllib.error import HTTPError try: import jira.client diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 13be9592d771f..4ff5b327e3325 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -22,15 +22,9 @@ import json import functools import subprocess -if sys.version < '3': - from urllib2 import urlopen - from urllib2 import Request - from urllib2 import HTTPError, URLError -else: - from urllib.request import urlopen - from urllib.request import Request - from urllib.error import HTTPError, URLError - +from urllib.request import urlopen +from urllib.request import Request +from urllib.error import HTTPError, URLError from sparktestsupport import SPARK_HOME, ERROR_CODES from sparktestsupport.shellutils import run_cmd diff --git a/dev/sparktestsupport/toposort.py b/dev/sparktestsupport/toposort.py index 8b2688d20039f..6785e481b56b5 100644 --- a/dev/sparktestsupport/toposort.py +++ b/dev/sparktestsupport/toposort.py @@ -24,8 +24,7 @@ # Moved functools import to the top of the file. # Changed assert to a ValueError. # Changed iter[items|keys] to [items|keys], for python 3 -# compatibility. I don't think it matters for python 2 these are -# now lists instead of iterables. +# compatibility. # Copy the input so as to leave it unmodified. # Renamed function from toposort2 to toposort. # Handle empty input. diff --git a/docs/configuration.md b/docs/configuration.md index 42f706b296d30..abf76105ae77d 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2917,7 +2917,7 @@ The following variables can be set in `spark-env.sh`: - diff --git a/docs/index.md b/docs/index.md index c0771ca170af5..8fd169e63f608 100644 --- a/docs/index.md +++ b/docs/index.md @@ -44,9 +44,8 @@ source, visit [Building Spark](building-spark.html). Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS), and it should run on any platform that runs a supported version of Java. This should include JVMs on x86_64 and ARM64. It's easy to run locally on one machine --- all you need is to have `java` installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. -Spark runs on Java 8/11, Scala 2.12, Python 2.7+/3.4+ and R 3.5+. +Spark runs on Java 8/11, Scala 2.12, Python 3.6+ and R 3.5+. Java 8 prior to version 8u92 support is deprecated as of Spark 3.0.0. -Python 2 and Python 3 prior to version 3.6 support is deprecated as of Spark 3.0.0. For the Scala API, Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version ({{site.SCALA_BINARY_VERSION}}.x). diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index 70bfefce475a1..07207f62bb9bd 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -101,10 +101,10 @@ import org.apache.spark.SparkConf;
-Spark {{site.SPARK_VERSION}} works with Python 2.7+ or Python 3.4+. It can use the standard CPython interpreter, +Spark {{site.SPARK_VERSION}} works with Python 3.6+. It can use the standard CPython interpreter, so C libraries like NumPy can be used. It also works with PyPy 2.3+. -Note that Python 2 support is deprecated as of Spark 3.0.0. +Python 2, 3.4 and 3.5 supports were removed in Spark 3.1.0. Spark applications in Python can either be run with the `bin/spark-submit` script which includes Spark at runtime, or by including it in your setup.py as: @@ -134,8 +134,8 @@ PySpark requires the same minor version of Python in both driver and workers. It you can specify which version of Python you want to use by `PYSPARK_PYTHON`, for example: {% highlight bash %} -$ PYSPARK_PYTHON=python3.4 bin/pyspark -$ PYSPARK_PYTHON=/opt/pypy-2.5/bin/pypy bin/spark-submit examples/src/main/python/pi.py +$ PYSPARK_PYTHON=python3.8 bin/pyspark +$ PYSPARK_PYTHON=/path-to-your-pypy/pypy bin/spark-submit examples/src/main/python/pi.py {% endhighlight %}
@@ -276,7 +276,7 @@ $ PYSPARK_DRIVER_PYTHON=jupyter PYSPARK_DRIVER_PYTHON_OPTS=notebook ./bin/pyspar You can customize the `ipython` or `jupyter` commands by setting `PYSPARK_DRIVER_PYTHON_OPTS`. -After the Jupyter Notebook server is launched, you can create a new "Python 2" notebook from +After the Jupyter Notebook server is launched, you can create a new notebook from the "Files" tab. Inside the notebook, you can input the command `%pylab inline` as part of your notebook before you start to try Spark from the Jupyter notebook. @@ -447,7 +447,7 @@ Writables are automatically converted:
Default timeout for all network interactions. This config will be used in place of spark.core.connection.ack.wait.timeout, - spark.storage.blockManagerSlaveTimeoutMs, + spark.storage.blockManagerHeartbeatTimeoutMs, spark.shuffle.io.connectionTimeout, spark.rpc.askTimeout or spark.rpc.lookupTimeout if they are not configured.
Mesos Slave ID{state.slaveId.getValue}Mesos Agent ID{state.agentId.getValue}
Mesos Task ID{state.driverDescription.command.mainClass} cpus: {state.driverDescription.cores}, mem: {state.driverDescription.mem} {UIUtils.formatDate(state.startDate)}{state.slaveId.getValue}{state.agentId.getValue} {stateString(state.mesosTaskStatus)} {sandboxCol}
PYSPARK_PYTHONPython binary executable to use for PySpark in both driver and workers (default is python2.7 if available, otherwise python). + Python binary executable to use for PySpark in both driver and workers (default is python3 if available, otherwise python). Property spark.pyspark.python take precedence if it is set
- + diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 6d3241876ad51..511634fd8f6c2 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -21,8 +21,6 @@ This example requires numpy (http://www.numpy.org/) """ -from __future__ import print_function - import sys import numpy as np diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index a18722c687f8b..49ab37e7b3286 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -43,8 +43,6 @@ {u'favorite_color': None, u'name': u'Alyssa'} {u'favorite_color': u'red', u'name': u'Ben'} """ -from __future__ import print_function - import sys from functools import reduce diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index a42d711fc505f..022378619c97f 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -22,8 +22,6 @@ This example requires NumPy (http://www.numpy.org/). """ -from __future__ import print_function - import sys import numpy as np diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index bcc4e0f4e8eae..4b83740152ca4 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -22,8 +22,6 @@ In practice, one may prefer to use the LogisticRegression algorithm in ML, as shown in examples/src/main/python/ml/logistic_regression_with_elastic_net.py. """ -from __future__ import print_function - import sys import numpy as np diff --git a/examples/src/main/python/ml/aft_survival_regression.py b/examples/src/main/python/ml/aft_survival_regression.py index 0a71f76418ea6..2040a7876c7fa 100644 --- a/examples/src/main/python/ml/aft_survival_regression.py +++ b/examples/src/main/python/ml/aft_survival_regression.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/aft_survival_regression.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.regression import AFTSurvivalRegression from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index 8b7ec9c439f9f..b39263978402b 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -15,12 +15,6 @@ # limitations under the License. # -from __future__ import print_function - -import sys -if sys.version >= '3': - long = int - from pyspark.sql import SparkSession # $example on$ @@ -39,7 +33,7 @@ lines = spark.read.text("data/mllib/als/sample_movielens_ratings.txt").rdd parts = lines.map(lambda row: row.value.split("::")) ratingsRDD = parts.map(lambda p: Row(userId=int(p[0]), movieId=int(p[1]), - rating=float(p[2]), timestamp=long(p[3]))) + rating=float(p[2]), timestamp=int(p[3]))) ratings = spark.createDataFrame(ratingsRDD) (training, test) = ratings.randomSplit([0.8, 0.2]) diff --git a/examples/src/main/python/ml/anova_selector_example.py b/examples/src/main/python/ml/anova_selector_example.py index f8458f5d6e487..da80fa62316d7 100644 --- a/examples/src/main/python/ml/anova_selector_example.py +++ b/examples/src/main/python/ml/anova_selector_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/anova_selector_example.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import ANOVASelector diff --git a/examples/src/main/python/ml/anova_test_example.py b/examples/src/main/python/ml/anova_test_example.py index 4119441cdeab6..451e078f60e56 100644 --- a/examples/src/main/python/ml/anova_test_example.py +++ b/examples/src/main/python/ml/anova_test_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/anova_test_example.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/binarizer_example.py b/examples/src/main/python/ml/binarizer_example.py index 669bb2aeabecd..5d5ae4122e1d4 100644 --- a/examples/src/main/python/ml/binarizer_example.py +++ b/examples/src/main/python/ml/binarizer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import Binarizer diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index 82adb338b5d91..513f80a09ef05 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/bisecting_k_means_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.clustering import BisectingKMeans from pyspark.ml.evaluation import ClusteringEvaluator diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py index 610176ea596ca..f5836091f35ba 100644 --- a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/bucketed_random_projection_lsh_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.feature import BucketedRandomProjectionLSH from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/bucketizer_example.py b/examples/src/main/python/ml/bucketizer_example.py index 742f35093b9d2..5de67f7126b5e 100644 --- a/examples/src/main/python/ml/bucketizer_example.py +++ b/examples/src/main/python/ml/bucketizer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import Bucketizer diff --git a/examples/src/main/python/ml/chi_square_test_example.py b/examples/src/main/python/ml/chi_square_test_example.py index 2af7e683cdb72..bf15a03d9cb4c 100644 --- a/examples/src/main/python/ml/chi_square_test_example.py +++ b/examples/src/main/python/ml/chi_square_test_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/chi_square_test_example.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/chisq_selector_example.py b/examples/src/main/python/ml/chisq_selector_example.py index 028a9ea9d67b1..c83a8c1bc7b27 100644 --- a/examples/src/main/python/ml/chisq_selector_example.py +++ b/examples/src/main/python/ml/chisq_selector_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import ChiSqSelector diff --git a/examples/src/main/python/ml/correlation_example.py b/examples/src/main/python/ml/correlation_example.py index 1f4e402ac1a51..9006d541491fb 100644 --- a/examples/src/main/python/ml/correlation_example.py +++ b/examples/src/main/python/ml/correlation_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/correlation_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.linalg import Vectors from pyspark.ml.stat import Correlation diff --git a/examples/src/main/python/ml/count_vectorizer_example.py b/examples/src/main/python/ml/count_vectorizer_example.py index f2e41db77d898..b3ddfb128c3d0 100644 --- a/examples/src/main/python/ml/count_vectorizer_example.py +++ b/examples/src/main/python/ml/count_vectorizer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import CountVectorizer diff --git a/examples/src/main/python/ml/cross_validator.py b/examples/src/main/python/ml/cross_validator.py index 6256d11504afb..0ad0865486959 100644 --- a/examples/src/main/python/ml/cross_validator.py +++ b/examples/src/main/python/ml/cross_validator.py @@ -22,8 +22,6 @@ bin/spark-submit examples/src/main/python/ml/cross_validator.py """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression diff --git a/examples/src/main/python/ml/dataframe_example.py b/examples/src/main/python/ml/dataframe_example.py index cabc3de68f2f4..d2bf93744113b 100644 --- a/examples/src/main/python/ml/dataframe_example.py +++ b/examples/src/main/python/ml/dataframe_example.py @@ -19,8 +19,6 @@ An example of how to use DataFrame for ML. Run with:: bin/spark-submit examples/src/main/python/ml/dataframe_example.py """ -from __future__ import print_function - import os import sys import tempfile diff --git a/examples/src/main/python/ml/dct_example.py b/examples/src/main/python/ml/dct_example.py index c0457f8d0f43b..37da4f5e8f1cb 100644 --- a/examples/src/main/python/ml/dct_example.py +++ b/examples/src/main/python/ml/dct_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import DCT from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py index d6e2977de0082..eb7177b845357 100644 --- a/examples/src/main/python/ml/decision_tree_classification_example.py +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -18,8 +18,6 @@ """ Decision Tree Classification Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import DecisionTreeClassifier diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py index 58d7ad921d8e0..1ed1636a3d962 100644 --- a/examples/src/main/python/ml/decision_tree_regression_example.py +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -18,8 +18,6 @@ """ Decision Tree Regression Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import DecisionTreeRegressor diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py index 590053998bccc..71eec8d432998 100644 --- a/examples/src/main/python/ml/elementwise_product_example.py +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import ElementwiseProduct from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/estimator_transformer_param_example.py b/examples/src/main/python/ml/estimator_transformer_param_example.py index eb21051435393..1dcca6c201119 100644 --- a/examples/src/main/python/ml/estimator_transformer_param_example.py +++ b/examples/src/main/python/ml/estimator_transformer_param_example.py @@ -18,8 +18,6 @@ """ Estimator Transformer Param Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml.linalg import Vectors from pyspark.ml.classification import LogisticRegression diff --git a/examples/src/main/python/ml/feature_hasher_example.py b/examples/src/main/python/ml/feature_hasher_example.py index 6cf9ecc396400..4fe573d19dfbc 100644 --- a/examples/src/main/python/ml/feature_hasher_example.py +++ b/examples/src/main/python/ml/feature_hasher_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import FeatureHasher diff --git a/examples/src/main/python/ml/fm_classifier_example.py b/examples/src/main/python/ml/fm_classifier_example.py index 6e7c2ccf021ed..b47bdc5275beb 100644 --- a/examples/src/main/python/ml/fm_classifier_example.py +++ b/examples/src/main/python/ml/fm_classifier_example.py @@ -18,8 +18,6 @@ """ FMClassifier Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import FMClassifier diff --git a/examples/src/main/python/ml/fm_regressor_example.py b/examples/src/main/python/ml/fm_regressor_example.py index afd76396800b7..5c8133996ae83 100644 --- a/examples/src/main/python/ml/fm_regressor_example.py +++ b/examples/src/main/python/ml/fm_regressor_example.py @@ -18,8 +18,6 @@ """ FMRegressor Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import FMRegressor diff --git a/examples/src/main/python/ml/fvalue_selector_example.py b/examples/src/main/python/ml/fvalue_selector_example.py index 3158953a5dfc4..f164af47eb309 100644 --- a/examples/src/main/python/ml/fvalue_selector_example.py +++ b/examples/src/main/python/ml/fvalue_selector_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/fvalue_selector_example.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import FValueSelector diff --git a/examples/src/main/python/ml/fvalue_test_example.py b/examples/src/main/python/ml/fvalue_test_example.py index 410b39e4493f8..dfa8073e5afc9 100644 --- a/examples/src/main/python/ml/fvalue_test_example.py +++ b/examples/src/main/python/ml/fvalue_test_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/fvalue_test_example.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/gaussian_mixture_example.py b/examples/src/main/python/ml/gaussian_mixture_example.py index 4938a904189f9..1441faa792983 100644 --- a/examples/src/main/python/ml/gaussian_mixture_example.py +++ b/examples/src/main/python/ml/gaussian_mixture_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/gaussian_mixture_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.clustering import GaussianMixture # $example off$ diff --git a/examples/src/main/python/ml/generalized_linear_regression_example.py b/examples/src/main/python/ml/generalized_linear_regression_example.py index a52f4650c1c6f..06a8a5a2e9428 100644 --- a/examples/src/main/python/ml/generalized_linear_regression_example.py +++ b/examples/src/main/python/ml/generalized_linear_regression_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/generalized_linear_regression_example.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.regression import GeneralizedLinearRegression diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py index c2042fd7b7b07..a7efa2170a069 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -18,8 +18,6 @@ """ Gradient Boosted Tree Classifier Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import GBTClassifier diff --git a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py index cc96c973e4b23..5e09b96c1ea3a 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py @@ -18,8 +18,6 @@ """ Gradient Boosted Tree Regressor Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import GBTRegressor diff --git a/examples/src/main/python/ml/index_to_string_example.py b/examples/src/main/python/ml/index_to_string_example.py index 33d104e8e3f41..98bdb89ce3039 100644 --- a/examples/src/main/python/ml/index_to_string_example.py +++ b/examples/src/main/python/ml/index_to_string_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import IndexToString, StringIndexer # $example off$ diff --git a/examples/src/main/python/ml/interaction_example.py b/examples/src/main/python/ml/interaction_example.py index 4b632271916f5..ac365179b0c20 100644 --- a/examples/src/main/python/ml/interaction_example.py +++ b/examples/src/main/python/ml/interaction_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import Interaction, VectorAssembler # $example off$ diff --git a/examples/src/main/python/ml/isotonic_regression_example.py b/examples/src/main/python/ml/isotonic_regression_example.py index 89cba9dfc7e8f..d7b893894fc71 100644 --- a/examples/src/main/python/ml/isotonic_regression_example.py +++ b/examples/src/main/python/ml/isotonic_regression_example.py @@ -21,8 +21,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/isotonic_regression_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.regression import IsotonicRegression # $example off$ diff --git a/examples/src/main/python/ml/kmeans_example.py b/examples/src/main/python/ml/kmeans_example.py index 80a878af679f4..47223fd953d17 100644 --- a/examples/src/main/python/ml/kmeans_example.py +++ b/examples/src/main/python/ml/kmeans_example.py @@ -22,8 +22,6 @@ This example requires NumPy (http://www.numpy.org/). """ -from __future__ import print_function - # $example on$ from pyspark.ml.clustering import KMeans from pyspark.ml.evaluation import ClusteringEvaluator diff --git a/examples/src/main/python/ml/lda_example.py b/examples/src/main/python/ml/lda_example.py index 97d1a042d1479..a47dfa383c895 100644 --- a/examples/src/main/python/ml/lda_example.py +++ b/examples/src/main/python/ml/lda_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/lda_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.clustering import LDA # $example off$ diff --git a/examples/src/main/python/ml/linear_regression_with_elastic_net.py b/examples/src/main/python/ml/linear_regression_with_elastic_net.py index 6639e9160ab71..864fc76cff132 100644 --- a/examples/src/main/python/ml/linear_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/linear_regression_with_elastic_net.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.regression import LinearRegression # $example off$ diff --git a/examples/src/main/python/ml/linearsvc.py b/examples/src/main/python/ml/linearsvc.py index 9b79abbf96f88..61d726cf3f1ae 100644 --- a/examples/src/main/python/ml/linearsvc.py +++ b/examples/src/main/python/ml/linearsvc.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.classification import LinearSVC # $example off$ diff --git a/examples/src/main/python/ml/logistic_regression_summary_example.py b/examples/src/main/python/ml/logistic_regression_summary_example.py index 2274ff707b2a3..6d045108da0aa 100644 --- a/examples/src/main/python/ml/logistic_regression_summary_example.py +++ b/examples/src/main/python/ml/logistic_regression_summary_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/logistic_regression_summary_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.classification import LogisticRegression # $example off$ diff --git a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py index d095fbd373408..916fdade27623 100644 --- a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.classification import LogisticRegression # $example off$ diff --git a/examples/src/main/python/ml/max_abs_scaler_example.py b/examples/src/main/python/ml/max_abs_scaler_example.py index 45eda3cdadde3..d7ff3561ce429 100644 --- a/examples/src/main/python/ml/max_abs_scaler_example.py +++ b/examples/src/main/python/ml/max_abs_scaler_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import MaxAbsScaler from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/min_hash_lsh_example.py b/examples/src/main/python/ml/min_hash_lsh_example.py index 93136e6ae3cae..683f97a055ede 100644 --- a/examples/src/main/python/ml/min_hash_lsh_example.py +++ b/examples/src/main/python/ml/min_hash_lsh_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/min_hash_lsh_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.feature import MinHashLSH from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/min_max_scaler_example.py b/examples/src/main/python/ml/min_max_scaler_example.py index b5f272e59bc30..cd74243699894 100644 --- a/examples/src/main/python/ml/min_max_scaler_example.py +++ b/examples/src/main/python/ml/min_max_scaler_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import MinMaxScaler from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/multiclass_logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/multiclass_logistic_regression_with_elastic_net.py index bec9860c79a2d..3bb4a72864101 100644 --- a/examples/src/main/python/ml/multiclass_logistic_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/multiclass_logistic_regression_with_elastic_net.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.classification import LogisticRegression # $example off$ diff --git a/examples/src/main/python/ml/multilayer_perceptron_classification.py b/examples/src/main/python/ml/multilayer_perceptron_classification.py index 88fc69f753953..74f532193573d 100644 --- a/examples/src/main/python/ml/multilayer_perceptron_classification.py +++ b/examples/src/main/python/ml/multilayer_perceptron_classification.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.classification import MultilayerPerceptronClassifier from pyspark.ml.evaluation import MulticlassClassificationEvaluator diff --git a/examples/src/main/python/ml/n_gram_example.py b/examples/src/main/python/ml/n_gram_example.py index 31676e076a11b..8c8031b939458 100644 --- a/examples/src/main/python/ml/n_gram_example.py +++ b/examples/src/main/python/ml/n_gram_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import NGram # $example off$ diff --git a/examples/src/main/python/ml/naive_bayes_example.py b/examples/src/main/python/ml/naive_bayes_example.py index 7290ab81cd0ec..8d1777c6f9e39 100644 --- a/examples/src/main/python/ml/naive_bayes_example.py +++ b/examples/src/main/python/ml/naive_bayes_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.classification import NaiveBayes from pyspark.ml.evaluation import MulticlassClassificationEvaluator diff --git a/examples/src/main/python/ml/normalizer_example.py b/examples/src/main/python/ml/normalizer_example.py index 510bd825fd286..2aa012961a2ee 100644 --- a/examples/src/main/python/ml/normalizer_example.py +++ b/examples/src/main/python/ml/normalizer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import Normalizer from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/one_vs_rest_example.py b/examples/src/main/python/ml/one_vs_rest_example.py index 956e94ae4ab62..4cae1a99808e8 100644 --- a/examples/src/main/python/ml/one_vs_rest_example.py +++ b/examples/src/main/python/ml/one_vs_rest_example.py @@ -21,8 +21,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/one_vs_rest_example.py """ -from __future__ import print_function - # $example on$ from pyspark.ml.classification import LogisticRegression, OneVsRest from pyspark.ml.evaluation import MulticlassClassificationEvaluator diff --git a/examples/src/main/python/ml/onehot_encoder_example.py b/examples/src/main/python/ml/onehot_encoder_example.py index 73775b79e36cb..6deb84ed785ca 100644 --- a/examples/src/main/python/ml/onehot_encoder_example.py +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import OneHotEncoder # $example off$ diff --git a/examples/src/main/python/ml/pca_example.py b/examples/src/main/python/ml/pca_example.py index 38746aced096a..03fb709c8e91d 100644 --- a/examples/src/main/python/ml/pca_example.py +++ b/examples/src/main/python/ml/pca_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import PCA from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py index 40bcb7b13a3de..75f436e768dc5 100644 --- a/examples/src/main/python/ml/polynomial_expansion_example.py +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import PolynomialExpansion from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/quantile_discretizer_example.py b/examples/src/main/python/ml/quantile_discretizer_example.py index 0fc1d1949a77d..82be3936d2598 100644 --- a/examples/src/main/python/ml/quantile_discretizer_example.py +++ b/examples/src/main/python/ml/quantile_discretizer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import QuantileDiscretizer # $example off$ diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index 4eaa94dd7f489..8983d1f2e979b 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -18,8 +18,6 @@ """ Random Forest Classifier Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import RandomForestClassifier diff --git a/examples/src/main/python/ml/random_forest_regressor_example.py b/examples/src/main/python/ml/random_forest_regressor_example.py index a34edff2ecaa2..b9306ddf2f82c 100644 --- a/examples/src/main/python/ml/random_forest_regressor_example.py +++ b/examples/src/main/python/ml/random_forest_regressor_example.py @@ -18,8 +18,6 @@ """ Random Forest Regressor Example. """ -from __future__ import print_function - # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import RandomForestRegressor diff --git a/examples/src/main/python/ml/rformula_example.py b/examples/src/main/python/ml/rformula_example.py index 6629239db29ec..25bb6dac56e81 100644 --- a/examples/src/main/python/ml/rformula_example.py +++ b/examples/src/main/python/ml/rformula_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import RFormula # $example off$ diff --git a/examples/src/main/python/ml/robust_scaler_example.py b/examples/src/main/python/ml/robust_scaler_example.py index 435e9ccb806c6..9f7c6d6507c78 100644 --- a/examples/src/main/python/ml/robust_scaler_example.py +++ b/examples/src/main/python/ml/robust_scaler_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import RobustScaler # $example off$ diff --git a/examples/src/main/python/ml/sql_transformer.py b/examples/src/main/python/ml/sql_transformer.py index 0bf8f35720c95..c8ac5c46aa5e9 100644 --- a/examples/src/main/python/ml/sql_transformer.py +++ b/examples/src/main/python/ml/sql_transformer.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import SQLTransformer # $example off$ diff --git a/examples/src/main/python/ml/standard_scaler_example.py b/examples/src/main/python/ml/standard_scaler_example.py index c0027480e69b3..9021c10075d81 100644 --- a/examples/src/main/python/ml/standard_scaler_example.py +++ b/examples/src/main/python/ml/standard_scaler_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import StandardScaler # $example off$ diff --git a/examples/src/main/python/ml/stopwords_remover_example.py b/examples/src/main/python/ml/stopwords_remover_example.py index 3b8e7855e3e79..832a7c7d0ad88 100644 --- a/examples/src/main/python/ml/stopwords_remover_example.py +++ b/examples/src/main/python/ml/stopwords_remover_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import StopWordsRemover # $example off$ diff --git a/examples/src/main/python/ml/string_indexer_example.py b/examples/src/main/python/ml/string_indexer_example.py index 2255bfb9c1a60..f2ac63eabd71c 100644 --- a/examples/src/main/python/ml/string_indexer_example.py +++ b/examples/src/main/python/ml/string_indexer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import StringIndexer # $example off$ diff --git a/examples/src/main/python/ml/summarizer_example.py b/examples/src/main/python/ml/summarizer_example.py index 8835f189a1ad4..4982746450132 100644 --- a/examples/src/main/python/ml/summarizer_example.py +++ b/examples/src/main/python/ml/summarizer_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/summarizer_example.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.stat import Summarizer diff --git a/examples/src/main/python/ml/tf_idf_example.py b/examples/src/main/python/ml/tf_idf_example.py index d43244fa68e97..b4bb0dfa3183c 100644 --- a/examples/src/main/python/ml/tf_idf_example.py +++ b/examples/src/main/python/ml/tf_idf_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import HashingTF, IDF, Tokenizer # $example off$ diff --git a/examples/src/main/python/ml/tokenizer_example.py b/examples/src/main/python/ml/tokenizer_example.py index 5c65c5c9f8260..c6b5fac227315 100644 --- a/examples/src/main/python/ml/tokenizer_example.py +++ b/examples/src/main/python/ml/tokenizer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import Tokenizer, RegexTokenizer from pyspark.sql.functions import col, udf diff --git a/examples/src/main/python/ml/variance_threshold_selector_example.py b/examples/src/main/python/ml/variance_threshold_selector_example.py index b7edb86653530..0a996e0e28264 100644 --- a/examples/src/main/python/ml/variance_threshold_selector_example.py +++ b/examples/src/main/python/ml/variance_threshold_selector_example.py @@ -20,8 +20,6 @@ Run with: bin/spark-submit examples/src/main/python/ml/variance_threshold_selector_example.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import VarianceThresholdSelector diff --git a/examples/src/main/python/ml/vector_assembler_example.py b/examples/src/main/python/ml/vector_assembler_example.py index 98de1d5ea7dac..0ce31cf0eabc9 100644 --- a/examples/src/main/python/ml/vector_assembler_example.py +++ b/examples/src/main/python/ml/vector_assembler_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.linalg import Vectors from pyspark.ml.feature import VectorAssembler diff --git a/examples/src/main/python/ml/vector_indexer_example.py b/examples/src/main/python/ml/vector_indexer_example.py index 5c2956077d6ce..51a4191606fb8 100644 --- a/examples/src/main/python/ml/vector_indexer_example.py +++ b/examples/src/main/python/ml/vector_indexer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import VectorIndexer # $example off$ diff --git a/examples/src/main/python/ml/vector_size_hint_example.py b/examples/src/main/python/ml/vector_size_hint_example.py index fb77dacec629d..355d85aee8729 100644 --- a/examples/src/main/python/ml/vector_size_hint_example.py +++ b/examples/src/main/python/ml/vector_size_hint_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.linalg import Vectors from pyspark.ml.feature import (VectorSizeHint, VectorAssembler) diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py index 68c8cfe27e375..86e089d152c5a 100644 --- a/examples/src/main/python/ml/vector_slicer_example.py +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import VectorSlicer from pyspark.ml.linalg import Vectors diff --git a/examples/src/main/python/ml/word2vec_example.py b/examples/src/main/python/ml/word2vec_example.py index 77f8951df0883..0eabeda3dce4b 100644 --- a/examples/src/main/python/ml/word2vec_example.py +++ b/examples/src/main/python/ml/word2vec_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from pyspark.ml.feature import Word2Vec # $example off$ diff --git a/examples/src/main/python/mllib/binary_classification_metrics_example.py b/examples/src/main/python/mllib/binary_classification_metrics_example.py index d14ce7982e24f..741746e6e35ae 100644 --- a/examples/src/main/python/mllib/binary_classification_metrics_example.py +++ b/examples/src/main/python/mllib/binary_classification_metrics_example.py @@ -17,7 +17,6 @@ """ Binary Classification Metrics Example. """ -from __future__ import print_function from pyspark import SparkContext # $example on$ from pyspark.mllib.classification import LogisticRegressionWithLBFGS diff --git a/examples/src/main/python/mllib/bisecting_k_means_example.py b/examples/src/main/python/mllib/bisecting_k_means_example.py index 36e36fc6897f3..d7b6ad9d424a6 100644 --- a/examples/src/main/python/mllib/bisecting_k_means_example.py +++ b/examples/src/main/python/mllib/bisecting_k_means_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from numpy import array # $example off$ diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py index 089504fa7064b..27d07b22a5645 100755 --- a/examples/src/main/python/mllib/correlations.py +++ b/examples/src/main/python/mllib/correlations.py @@ -18,8 +18,6 @@ """ Correlations using MLlib. """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/correlations_example.py b/examples/src/main/python/mllib/correlations_example.py index 66d18f6e5df17..bb71b968687cb 100644 --- a/examples/src/main/python/mllib/correlations_example.py +++ b/examples/src/main/python/mllib/correlations_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - import numpy as np from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/decision_tree_classification_example.py b/examples/src/main/python/mllib/decision_tree_classification_example.py index 7eecf500584ad..009e393226c01 100644 --- a/examples/src/main/python/mllib/decision_tree_classification_example.py +++ b/examples/src/main/python/mllib/decision_tree_classification_example.py @@ -18,8 +18,6 @@ """ Decision Tree Classification Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import DecisionTree, DecisionTreeModel diff --git a/examples/src/main/python/mllib/decision_tree_regression_example.py b/examples/src/main/python/mllib/decision_tree_regression_example.py index acf9e25fdf31c..71dfbf0790175 100644 --- a/examples/src/main/python/mllib/decision_tree_regression_example.py +++ b/examples/src/main/python/mllib/decision_tree_regression_example.py @@ -18,8 +18,6 @@ """ Decision Tree Regression Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import DecisionTree, DecisionTreeModel diff --git a/examples/src/main/python/mllib/elementwise_product_example.py b/examples/src/main/python/mllib/elementwise_product_example.py index 8ae9afb1dc477..15e6a43f736cf 100644 --- a/examples/src/main/python/mllib/elementwise_product_example.py +++ b/examples/src/main/python/mllib/elementwise_product_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.feature import ElementwiseProduct diff --git a/examples/src/main/python/mllib/gaussian_mixture_example.py b/examples/src/main/python/mllib/gaussian_mixture_example.py index a60e799d62eb1..3b19478f457ec 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_example.py +++ b/examples/src/main/python/mllib/gaussian_mixture_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from numpy import array # $example off$ diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index 6b46e27ddaaa8..96ce6b6f6ab25 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -18,11 +18,6 @@ """ A Gaussian Mixture Model clustering program using MLlib. """ -from __future__ import print_function - -import sys -if sys.version >= '3': - long = int import random import argparse @@ -53,7 +48,7 @@ def parseVector(line): parser.add_argument('--convergenceTol', default=1e-3, type=float, help='convergence threshold') parser.add_argument('--maxIterations', default=100, type=int, help='Number of iterations') parser.add_argument('--seed', default=random.getrandbits(19), - type=long, help='Random seed') + type=int, help='Random seed') args = parser.parse_args() conf = SparkConf().setAppName("GMM") diff --git a/examples/src/main/python/mllib/gradient_boosting_classification_example.py b/examples/src/main/python/mllib/gradient_boosting_classification_example.py index 65a03572be9b5..eb12f206196fe 100644 --- a/examples/src/main/python/mllib/gradient_boosting_classification_example.py +++ b/examples/src/main/python/mllib/gradient_boosting_classification_example.py @@ -18,8 +18,6 @@ """ Gradient Boosted Trees Classification Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel diff --git a/examples/src/main/python/mllib/gradient_boosting_regression_example.py b/examples/src/main/python/mllib/gradient_boosting_regression_example.py index 877f8ab461ccd..eb59a992df539 100644 --- a/examples/src/main/python/mllib/gradient_boosting_regression_example.py +++ b/examples/src/main/python/mllib/gradient_boosting_regression_example.py @@ -18,8 +18,6 @@ """ Gradient Boosted Trees Regression Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel diff --git a/examples/src/main/python/mllib/hypothesis_testing_example.py b/examples/src/main/python/mllib/hypothesis_testing_example.py index 21a5584fd6e06..321be8b76f1b9 100644 --- a/examples/src/main/python/mllib/hypothesis_testing_example.py +++ b/examples/src/main/python/mllib/hypothesis_testing_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.linalg import Matrices, Vectors diff --git a/examples/src/main/python/mllib/hypothesis_testing_kolmogorov_smirnov_test_example.py b/examples/src/main/python/mllib/hypothesis_testing_kolmogorov_smirnov_test_example.py index ef380dee79d3d..12a186900e358 100644 --- a/examples/src/main/python/mllib/hypothesis_testing_kolmogorov_smirnov_test_example.py +++ b/examples/src/main/python/mllib/hypothesis_testing_kolmogorov_smirnov_test_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.stat import Statistics diff --git a/examples/src/main/python/mllib/isotonic_regression_example.py b/examples/src/main/python/mllib/isotonic_regression_example.py index f5322d79c45ba..a5a0cfeae9d75 100644 --- a/examples/src/main/python/mllib/isotonic_regression_example.py +++ b/examples/src/main/python/mllib/isotonic_regression_example.py @@ -18,8 +18,6 @@ """ Isotonic Regression Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ import math diff --git a/examples/src/main/python/mllib/k_means_example.py b/examples/src/main/python/mllib/k_means_example.py index d6058f45020c4..ead1e56de55c6 100644 --- a/examples/src/main/python/mllib/k_means_example.py +++ b/examples/src/main/python/mllib/k_means_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - # $example on$ from numpy import array from math import sqrt diff --git a/examples/src/main/python/mllib/kernel_density_estimation_example.py b/examples/src/main/python/mllib/kernel_density_estimation_example.py index 3e8f7241a4a1e..22d191716057c 100644 --- a/examples/src/main/python/mllib/kernel_density_estimation_example.py +++ b/examples/src/main/python/mllib/kernel_density_estimation_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.stat import KernelDensity diff --git a/examples/src/main/python/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py index 1bdb3e9b4a2af..2560384b6a0e2 100755 --- a/examples/src/main/python/mllib/kmeans.py +++ b/examples/src/main/python/mllib/kmeans.py @@ -20,8 +20,6 @@ This example requires NumPy (http://www.numpy.org/). """ -from __future__ import print_function - import sys import numpy as np diff --git a/examples/src/main/python/mllib/latent_dirichlet_allocation_example.py b/examples/src/main/python/mllib/latent_dirichlet_allocation_example.py index 2a1bef5f207b7..f82a28aadc5a3 100644 --- a/examples/src/main/python/mllib/latent_dirichlet_allocation_example.py +++ b/examples/src/main/python/mllib/latent_dirichlet_allocation_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.clustering import LDA, LDAModel diff --git a/examples/src/main/python/mllib/linear_regression_with_sgd_example.py b/examples/src/main/python/mllib/linear_regression_with_sgd_example.py index 6744463d40ef1..cb67396332312 100644 --- a/examples/src/main/python/mllib/linear_regression_with_sgd_example.py +++ b/examples/src/main/python/mllib/linear_regression_with_sgd_example.py @@ -18,8 +18,6 @@ """ Linear Regression With SGD Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD, LinearRegressionModel diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 87efe17375226..7b90615a53424 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -20,8 +20,6 @@ This example requires NumPy (http://www.numpy.org/). """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py b/examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py index c9b768b3147d2..ac5ab1d1b5d91 100644 --- a/examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py +++ b/examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py @@ -18,8 +18,6 @@ """ Logistic Regression With LBFGS Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.classification import LogisticRegressionWithLBFGS, LogisticRegressionModel diff --git a/examples/src/main/python/mllib/naive_bayes_example.py b/examples/src/main/python/mllib/naive_bayes_example.py index a29fcccac5bfc..74d18233d533a 100644 --- a/examples/src/main/python/mllib/naive_bayes_example.py +++ b/examples/src/main/python/mllib/naive_bayes_example.py @@ -22,8 +22,6 @@ `spark-submit --master local[4] examples/src/main/python/mllib/naive_bayes_example.py` """ -from __future__ import print_function - import shutil from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/normalizer_example.py b/examples/src/main/python/mllib/normalizer_example.py index a4e028ca9af8b..d46110d9a0300 100644 --- a/examples/src/main/python/mllib/normalizer_example.py +++ b/examples/src/main/python/mllib/normalizer_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.feature import Normalizer diff --git a/examples/src/main/python/mllib/power_iteration_clustering_example.py b/examples/src/main/python/mllib/power_iteration_clustering_example.py index ca19c0ccb60c8..60eedef5fab30 100644 --- a/examples/src/main/python/mllib/power_iteration_clustering_example.py +++ b/examples/src/main/python/mllib/power_iteration_clustering_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.clustering import PowerIterationClustering, PowerIterationClusteringModel diff --git a/examples/src/main/python/mllib/random_forest_classification_example.py b/examples/src/main/python/mllib/random_forest_classification_example.py index 5ac67520daee0..a929c10d5a573 100644 --- a/examples/src/main/python/mllib/random_forest_classification_example.py +++ b/examples/src/main/python/mllib/random_forest_classification_example.py @@ -18,8 +18,6 @@ """ Random Forest Classification Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import RandomForest, RandomForestModel diff --git a/examples/src/main/python/mllib/random_forest_regression_example.py b/examples/src/main/python/mllib/random_forest_regression_example.py index 7e986a0d307f0..4e05937768211 100644 --- a/examples/src/main/python/mllib/random_forest_regression_example.py +++ b/examples/src/main/python/mllib/random_forest_regression_example.py @@ -18,8 +18,6 @@ """ Random Forest Regression Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import RandomForest, RandomForestModel diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py index 9a429b5f8abdf..49afcfe9391ab 100755 --- a/examples/src/main/python/mllib/random_rdd_generation.py +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -18,8 +18,6 @@ """ Randomly generated RDDs. """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/recommendation_example.py b/examples/src/main/python/mllib/recommendation_example.py index 00e683c3ae938..719f3f904b246 100644 --- a/examples/src/main/python/mllib/recommendation_example.py +++ b/examples/src/main/python/mllib/recommendation_example.py @@ -18,8 +18,6 @@ """ Collaborative Filtering Classification Example. """ -from __future__ import print_function - from pyspark import SparkContext # $example on$ diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py index 00e7cf4bbcdbf..9095c2b2d70d6 100755 --- a/examples/src/main/python/mllib/sampled_rdds.py +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -18,8 +18,6 @@ """ Randomly sampled RDDs. """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/standard_scaler_example.py b/examples/src/main/python/mllib/standard_scaler_example.py index 11ed34427dfe2..c8fd64dfbbf4a 100644 --- a/examples/src/main/python/mllib/standard_scaler_example.py +++ b/examples/src/main/python/mllib/standard_scaler_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.feature import StandardScaler diff --git a/examples/src/main/python/mllib/stratified_sampling_example.py b/examples/src/main/python/mllib/stratified_sampling_example.py index a13f8f08dd68b..2d29f74a19c1a 100644 --- a/examples/src/main/python/mllib/stratified_sampling_example.py +++ b/examples/src/main/python/mllib/stratified_sampling_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext if __name__ == "__main__": diff --git a/examples/src/main/python/mllib/streaming_k_means_example.py b/examples/src/main/python/mllib/streaming_k_means_example.py index e82509ad3ffb6..4904a9ebcf544 100644 --- a/examples/src/main/python/mllib/streaming_k_means_example.py +++ b/examples/src/main/python/mllib/streaming_k_means_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext from pyspark.streaming import StreamingContext # $example on$ diff --git a/examples/src/main/python/mllib/streaming_linear_regression_example.py b/examples/src/main/python/mllib/streaming_linear_regression_example.py index 714c9a0de7217..1d52e00fbfb5e 100644 --- a/examples/src/main/python/mllib/streaming_linear_regression_example.py +++ b/examples/src/main/python/mllib/streaming_linear_regression_example.py @@ -18,8 +18,6 @@ """ Streaming Linear Regression Example. """ -from __future__ import print_function - # $example on$ import sys # $example off$ diff --git a/examples/src/main/python/mllib/summary_statistics_example.py b/examples/src/main/python/mllib/summary_statistics_example.py index d55d1a2c2d0e1..d86e841145501 100644 --- a/examples/src/main/python/mllib/summary_statistics_example.py +++ b/examples/src/main/python/mllib/summary_statistics_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ import numpy as np diff --git a/examples/src/main/python/mllib/tf_idf_example.py b/examples/src/main/python/mllib/tf_idf_example.py index b66412b2334e7..4449066f5b0a6 100644 --- a/examples/src/main/python/mllib/tf_idf_example.py +++ b/examples/src/main/python/mllib/tf_idf_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.feature import HashingTF, IDF diff --git a/examples/src/main/python/mllib/word2vec.py b/examples/src/main/python/mllib/word2vec.py index 4e7d4f7610c24..3e5720b4df4d6 100644 --- a/examples/src/main/python/mllib/word2vec.py +++ b/examples/src/main/python/mllib/word2vec.py @@ -23,8 +23,6 @@ # grep -o -E '\w+(\W+\w+){0,15}' text8 > text8_lines # This was done so that the example can be run in local mode -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/word2vec_example.py b/examples/src/main/python/mllib/word2vec_example.py index ad1090c77ee11..d37a6e7137b8f 100644 --- a/examples/src/main/python/mllib/word2vec_example.py +++ b/examples/src/main/python/mllib/word2vec_example.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark import SparkContext # $example on$ from pyspark.mllib.feature import Word2Vec diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index 2c19e8700ab16..0ab7249a82185 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -22,8 +22,6 @@ Example Usage: bin/spark-submit examples/src/main/python/pagerank.py data/mllib/pagerank_data.txt 10 """ -from __future__ import print_function - import re import sys from operator import add diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index 83041f0040a0c..ca8dd25e6dabf 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -29,8 +29,6 @@ {u'favorite_color': u'red', u'name': u'Ben', u'favorite_numbers': []} <...more log output...> """ -from __future__ import print_function - import sys from pyspark.sql import SparkSession diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 5839cc2874956..e646722533f68 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - import sys from random import random from operator import add diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index d3cd985d197e3..9efb00a6f1532 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - import sys from pyspark.sql import SparkSession diff --git a/examples/src/main/python/sql/arrow.py b/examples/src/main/python/sql/arrow.py index b7d8467172fab..e46449dbefbcd 100644 --- a/examples/src/main/python/sql/arrow.py +++ b/examples/src/main/python/sql/arrow.py @@ -21,21 +21,12 @@ ./bin/spark-submit examples/src/main/python/sql/arrow.py """ -from __future__ import print_function - -import sys - from pyspark.sql import SparkSession from pyspark.sql.pandas.utils import require_minimum_pandas_version, require_minimum_pyarrow_version require_minimum_pandas_version() require_minimum_pyarrow_version() -if sys.version_info < (3, 6): - raise Exception( - "Running this example file requires Python 3.6+; however, " - "your Python version was:\n %s" % sys.version) - def dataframe_with_arrow_example(spark): # $example on:dataframe_with_arrow$ diff --git a/examples/src/main/python/sql/basic.py b/examples/src/main/python/sql/basic.py index c8fb25d0533b5..eba8e6ad99d17 100644 --- a/examples/src/main/python/sql/basic.py +++ b/examples/src/main/python/sql/basic.py @@ -20,8 +20,6 @@ Run with: ./bin/spark-submit examples/src/main/python/sql/basic.py """ -from __future__ import print_function - # $example on:init_session$ from pyspark.sql import SparkSession # $example off:init_session$ diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index 265f135e1e5f2..94a41a7e5e7b4 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -20,8 +20,6 @@ Run with: ./bin/spark-submit examples/src/main/python/sql/datasource.py """ -from __future__ import print_function - from pyspark.sql import SparkSession # $example on:schema_merging$ from pyspark.sql import Row diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py index e96a8af71adc3..bc23dcd9bd2b2 100644 --- a/examples/src/main/python/sql/hive.py +++ b/examples/src/main/python/sql/hive.py @@ -20,8 +20,6 @@ Run with: ./bin/spark-submit examples/src/main/python/sql/hive.py """ -from __future__ import print_function - # $example on:spark_hive$ from os.path import join, abspath diff --git a/examples/src/main/python/sql/streaming/structured_kafka_wordcount.py b/examples/src/main/python/sql/streaming/structured_kafka_wordcount.py index 921067891352a..40a955a46c9b9 100644 --- a/examples/src/main/python/sql/streaming/structured_kafka_wordcount.py +++ b/examples/src/main/python/sql/streaming/structured_kafka_wordcount.py @@ -36,8 +36,6 @@ `$ bin/spark-submit examples/src/main/python/sql/streaming/structured_kafka_wordcount.py \ host1:port1,host2:port2 subscribe topic1,topic2` """ -from __future__ import print_function - import sys from pyspark.sql import SparkSession diff --git a/examples/src/main/python/sql/streaming/structured_network_wordcount.py b/examples/src/main/python/sql/streaming/structured_network_wordcount.py index 9ac392164735b..c8f43c9dcf2eb 100644 --- a/examples/src/main/python/sql/streaming/structured_network_wordcount.py +++ b/examples/src/main/python/sql/streaming/structured_network_wordcount.py @@ -27,8 +27,6 @@ `$ bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999` """ -from __future__ import print_function - import sys from pyspark.sql import SparkSession diff --git a/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py b/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py index c4e3bbf44cd5a..cc39d8afa6be9 100644 --- a/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py +++ b/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py @@ -39,8 +39,6 @@ One recommended , pair is 10, 5 """ -from __future__ import print_function - import sys from pyspark.sql import SparkSession diff --git a/examples/src/main/python/status_api_demo.py b/examples/src/main/python/status_api_demo.py index 8cc8cc820cfce..7b408c87260c0 100644 --- a/examples/src/main/python/status_api_demo.py +++ b/examples/src/main/python/status_api_demo.py @@ -15,15 +15,10 @@ # limitations under the License. # -from __future__ import print_function - import time import threading import sys -if sys.version >= '3': - import queue as Queue -else: - import Queue +import queue as Queue from pyspark import SparkConf, SparkContext diff --git a/examples/src/main/python/streaming/hdfs_wordcount.py b/examples/src/main/python/streaming/hdfs_wordcount.py index f9a5c43a8eaa9..fac07727b7b12 100644 --- a/examples/src/main/python/streaming/hdfs_wordcount.py +++ b/examples/src/main/python/streaming/hdfs_wordcount.py @@ -25,8 +25,6 @@ Then create a text file in `localdir` and the words in the file will get counted. """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/streaming/network_wordcount.py b/examples/src/main/python/streaming/network_wordcount.py index f3099d2517cd5..b57f4e9e38b82 100644 --- a/examples/src/main/python/streaming/network_wordcount.py +++ b/examples/src/main/python/streaming/network_wordcount.py @@ -25,8 +25,6 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999` """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/streaming/network_wordjoinsentiments.py b/examples/src/main/python/streaming/network_wordjoinsentiments.py index 2b5434c0c845a..5b03546fb4d83 100644 --- a/examples/src/main/python/streaming/network_wordjoinsentiments.py +++ b/examples/src/main/python/streaming/network_wordjoinsentiments.py @@ -30,8 +30,6 @@ localhost 9999` """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py index a39c4d0b5b8cd..8424556e88818 100644 --- a/examples/src/main/python/streaming/recoverable_network_wordcount.py +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -35,8 +35,6 @@ checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from the checkpoint data. """ -from __future__ import print_function - import os import sys diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index ab3cfc067994d..59a8a11a45b19 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -27,8 +27,6 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/sql_network_wordcount.py localhost 9999` """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/streaming/stateful_network_wordcount.py b/examples/src/main/python/streaming/stateful_network_wordcount.py index d5d1eba6c5969..7a45be663a765 100644 --- a/examples/src/main/python/streaming/stateful_network_wordcount.py +++ b/examples/src/main/python/streaming/stateful_network_wordcount.py @@ -29,8 +29,6 @@ `$ bin/spark-submit examples/src/main/python/streaming/stateful_network_wordcount.py \ localhost 9999` """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 49551d40851cc..9f543daecd3dd 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - import sys from random import Random diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index a05e24ff3ff95..037c1e8aa379d 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - import sys from operator import add diff --git a/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py b/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py index 5370b793897bb..df8c64e531cfa 100644 --- a/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py +++ b/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py @@ -55,8 +55,6 @@ See http://spark.apache.org/docs/latest/streaming-kinesis-integration.html for more details on the Kinesis Spark Streaming integration. """ -from __future__ import print_function - import sys from pyspark import SparkContext diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index a5d513262b266..2a19d233bc652 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -89,10 +89,7 @@ import sys import select import struct -if sys.version < '3': - import SocketServer -else: - import socketserver as SocketServer +import socketserver as SocketServer import threading from pyspark.serializers import read_int, PickleSerializer diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 803d857055dc0..c2daf7600ff26 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -20,16 +20,12 @@ import sys from tempfile import NamedTemporaryFile import threading +import pickle from pyspark.java_gateway import local_connect_and_auth from pyspark.serializers import ChunkedStream, pickle_protocol -from pyspark.util import _exception_message, print_exec +from pyspark.util import print_exec -if sys.version < '3': - import cPickle as pickle -else: - import pickle - unicode = str __all__ = ['Broadcast'] @@ -113,7 +109,7 @@ def dump(self, value, f): raise except Exception as e: msg = "Could not serialize broadcast: %s: %s" \ - % (e.__class__.__name__, _exception_message(e)) + % (e.__class__.__name__, str(e)) print_exec(sys.stderr) raise pickle.PicklingError(msg) f.close() diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 2024260868197..efd8b6d633e0c 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -22,14 +22,14 @@ >>> conf.setMaster("local").setAppName("My app") >>> conf.get("spark.master") -u'local' +'local' >>> conf.get("spark.app.name") -u'My app' +'My app' >>> sc = SparkContext(conf=conf) >>> sc.master -u'local' +'local' >>> sc.appName -u'My app' +'My app' >>> sc.sparkHome is None True @@ -37,21 +37,21 @@ >>> conf.setSparkHome("/path") >>> conf.get("spark.home") -u'/path' +'/path' >>> conf.setExecutorEnv("VAR1", "value1") >>> conf.setExecutorEnv(pairs = [("VAR3", "value3"), ("VAR4", "value4")]) >>> conf.get("spark.executorEnv.VAR1") -u'value1' +'value1' >>> print(conf.toDebugString()) spark.executorEnv.VAR1=value1 spark.executorEnv.VAR3=value3 spark.executorEnv.VAR4=value4 spark.home=/path >>> sorted(conf.getAll(), key=lambda p: p[0]) -[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), \ -(u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] +[('spark.executorEnv.VAR1', 'value1'), ('spark.executorEnv.VAR3', 'value3'), \ +('spark.executorEnv.VAR4', 'value4'), ('spark.home', '/path')] >>> conf._jconf.setExecutorEnv("VAR5", "value5") JavaObject id... >>> print(conf.toDebugString()) @@ -65,11 +65,6 @@ __all__ = ['SparkConf'] import sys -import re - -if sys.version > '3': - unicode = str - __doc__ = re.sub(r"(\W|^)[uU](['])", r'\1\2', __doc__) class SparkConf(object): @@ -124,9 +119,9 @@ def set(self, key, value): """Set a configuration property.""" # Try to set self._jconf first if JVM is created, set self._conf if JVM is not created yet. if self._jconf is not None: - self._jconf.set(key, unicode(value)) + self._jconf.set(key, str(value)) else: - self._conf[key] = unicode(value) + self._conf[key] = str(value) return self def setIfMissing(self, key, value): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6d58e1d14484c..2e105cc38260d 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -21,6 +21,7 @@ import sys import threading import warnings +import importlib from threading import RLock from tempfile import NamedTemporaryFile @@ -37,15 +38,12 @@ PairDeserializer, AutoBatchedSerializer, NoOpSerializer, ChunkedStream from pyspark.storagelevel import StorageLevel from pyspark.resource.information import ResourceInformation -from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix +from pyspark.rdd import RDD, _load_from_socket from pyspark.taskcontext import TaskContext from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler -if sys.version > '3': - xrange = range - __all__ = ['SparkContext'] @@ -213,15 +211,6 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') self.pythonVer = "%d.%d" % sys.version_info[:2] - if sys.version_info < (3, 6): - with warnings.catch_warnings(): - warnings.simplefilter("once") - warnings.warn( - "Support for Python 2 and Python 3 prior to version 3.6 is deprecated as " - "of Spark 3.0. See also the plan for dropping Python 2 support at " - "https://spark.apache.org/news/plan-for-dropping-python-2-support.html.", - DeprecationWarning) - # Broadcast's __reduce__ method stores Broadcast instances here. # This allows other code to determine which Broadcast instances have # been pickled, so it can determine which Java broadcast objects to @@ -398,7 +387,6 @@ def version(self): return self._jsc.version() @property - @ignore_unicode_prefix def applicationId(self): """ A unique identifier for the Spark application. @@ -408,7 +396,7 @@ def applicationId(self): * in case of YARN something like 'application_1433865536131_34483' >>> sc.applicationId # doctest: +ELLIPSIS - u'local-...' + 'local-...' """ return self._jsc.sc().applicationId() @@ -490,20 +478,20 @@ def range(self, start, end=None, step=1, numSlices=None): end = start start = 0 - return self.parallelize(xrange(start, end, step), numSlices) + return self.parallelize(range(start, end, step), numSlices) def parallelize(self, c, numSlices=None): """ - Distribute a local Python collection to form an RDD. Using xrange + Distribute a local Python collection to form an RDD. Using range is recommended if the input represents a range for performance. >>> sc.parallelize([0, 2, 3, 4, 6], 5).glom().collect() [[0], [2], [3], [4], [6]] - >>> sc.parallelize(xrange(0, 6, 2), 5).glom().collect() + >>> sc.parallelize(range(0, 6, 2), 5).glom().collect() [[], [0], [], [2], [4]] """ numSlices = int(numSlices) if numSlices is not None else self.defaultParallelism - if isinstance(c, xrange): + if isinstance(c, range): size = len(c) if size == 0: return self.parallelize([], numSlices) @@ -522,7 +510,7 @@ def f(split, iterator): # the empty iterator to a list, thus make sure worker reuse takes effect. # See more details in SPARK-26549. assert len(list(iterator)) == 0 - return xrange(getStart(split), getStart(split + 1), step) + return range(getStart(split), getStart(split + 1), step) return self.parallelize([], numSlices).mapPartitionsWithIndex(f) @@ -591,7 +579,6 @@ def pickleFile(self, name, minPartitions=None): minPartitions = minPartitions or self.defaultMinPartitions return RDD(self._jsc.objectFile(name, minPartitions), self) - @ignore_unicode_prefix def textFile(self, name, minPartitions=None, use_unicode=True): """ Read a text file from HDFS, a local file system (available on all @@ -608,13 +595,12 @@ def textFile(self, name, minPartitions=None, use_unicode=True): ... _ = testFile.write("Hello world!") >>> textFile = sc.textFile(path) >>> textFile.collect() - [u'Hello world!'] + ['Hello world!'] """ minPartitions = minPartitions or min(self.defaultParallelism, 2) return RDD(self._jsc.textFile(name, minPartitions), self, UTF8Deserializer(use_unicode)) - @ignore_unicode_prefix def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): """ Read a directory of text files from HDFS, a local file system @@ -658,7 +644,7 @@ def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): ... _ = file2.write("2") >>> textFiles = sc.wholeTextFiles(dirPath) >>> sorted(textFiles.collect()) - [(u'.../1.txt', u'1'), (u'.../2.txt', u'2')] + [('.../1.txt', '1'), ('.../2.txt', '2')] """ minPartitions = minPartitions or self.defaultMinPartitions return RDD(self._jsc.wholeTextFiles(path, minPartitions), self, @@ -846,7 +832,6 @@ def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) return RDD(jrdd, self, input_deserializer) - @ignore_unicode_prefix def union(self, rdds): """ Build the union of a list of RDDs. @@ -860,10 +845,10 @@ def union(self, rdds): ... _ = testFile.write("Hello") >>> textFile = sc.textFile(path) >>> textFile.collect() - [u'Hello'] + ['Hello'] >>> parallelized = sc.parallelize(["World!"]) >>> sorted(sc.union([textFile, parallelized]).collect()) - [u'Hello', 'World!'] + ['Hello', 'World!'] """ first_jrdd_deserializer = rdds[0]._jrdd_deserializer if any(x._jrdd_deserializer != first_jrdd_deserializer for x in rdds): @@ -959,9 +944,8 @@ def addPyFile(self, path): self._python_includes.append(filename) # for tests in local mode sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) - if sys.version > '3': - import importlib - importlib.invalidate_caches() + + importlib.invalidate_caches() def setCheckpointDir(self, dirName): """ diff --git a/python/pyspark/find_spark_home.py b/python/pyspark/find_spark_home.py index 52f6ea9a37100..920c04009dd11 100755 --- a/python/pyspark/find_spark_home.py +++ b/python/pyspark/find_spark_home.py @@ -20,7 +20,6 @@ # This script attempt to determine the correct setting for SPARK_HOME given # that Spark may have been installed on the system with pip. -from __future__ import print_function import os import sys @@ -41,26 +40,15 @@ def is_spark_home(path): # Add the path of the PySpark module if it exists import_error_raised = False - if sys.version < "3": - import imp - try: - module_home = imp.find_module("pyspark")[1] - paths.append(module_home) - # If we are installed in edit mode also look two dirs up - paths.append(os.path.join(module_home, "../../")) - except ImportError: - # Not pip installed no worries - import_error_raised = True - else: - from importlib.util import find_spec - try: - module_home = os.path.dirname(find_spec("pyspark").origin) - paths.append(module_home) - # If we are installed in edit mode also look two dirs up - paths.append(os.path.join(module_home, "../../")) - except ImportError: - # Not pip installed no worries - import_error_raised = True + from importlib.util import find_spec + try: + module_home = os.path.dirname(find_spec("pyspark").origin) + paths.append(module_home) + # If we are installed in edit mode also look two dirs up + paths.append(os.path.join(module_home, "../../")) + except ImportError: + # Not pip installed no worries + import_error_raised = True # Normalize the paths paths = [os.path.abspath(p) for p in paths] @@ -84,5 +72,6 @@ def is_spark_home(path): "'PYSPARK_PYTHON=python3 pyspark'.\n", file=sys.stderr) sys.exit(-1) + if __name__ == "__main__": print(_find_spark_home()) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 0daf09b17a82a..fba92a96ae1a1 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -17,7 +17,6 @@ import atexit import os -import sys import signal import shlex import shutil @@ -27,14 +26,10 @@ import time from subprocess import Popen, PIPE -if sys.version >= '3': - xrange = range - from py4j.java_gateway import java_import, JavaGateway, JavaObject, GatewayParameters from py4j.clientserver import ClientServer, JavaParameters, PythonParameters from pyspark.find_spark_home import _find_spark_home from pyspark.serializers import read_int, write_with_length, UTF8Deserializer -from pyspark.util import _exception_message def launch_gateway(conf=None, popen_kwargs=None): @@ -197,7 +192,7 @@ def local_connect_and_auth(port, auth_secret): _do_server_auth(sockfile, auth_secret) return (sockfile, sock) except socket.error as e: - emsg = _exception_message(e) + emsg = str(e) errors.append("tried to connect to %s, but an error occured: %s" % (sa, emsg)) sock.close() sock = None diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index cc8ce0567bd7f..7c8cbe3a9fe3e 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -16,20 +16,20 @@ # import operator -import sys +import warnings from abc import ABCMeta, abstractmethod, abstractproperty from multiprocessing.pool import ThreadPool -from pyspark import since, keyword_only +from pyspark import keyword_only from pyspark.ml import Estimator, Predictor, PredictionModel, Model from pyspark.ml.param.shared import * from pyspark.ml.tree import _DecisionTreeModel, _DecisionTreeParams, \ _TreeEnsembleModel, _RandomForestParams, _GBTParams, \ - _HasVarianceImpurity, _TreeClassifierParams, _TreeEnsembleParams + _HasVarianceImpurity, _TreeClassifierParams from pyspark.ml.regression import _FactorizationMachinesParams, DecisionTreeRegressionModel from pyspark.ml.util import * from pyspark.ml.base import _PredictorParams -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, \ +from pyspark.ml.wrapper import JavaParams, \ JavaPredictor, JavaPredictionModel, JavaWrapper from pyspark.ml.common import inherit_doc, _java2py, _py2java from pyspark.ml.linalg import Vectors diff --git a/python/pyspark/ml/common.py b/python/pyspark/ml/common.py index 387c5d7309dea..4e1d7f93aef9b 100644 --- a/python/pyspark/ml/common.py +++ b/python/pyspark/ml/common.py @@ -15,11 +15,6 @@ # limitations under the License. # -import sys -if sys.version >= '3': - long = int - unicode = str - import py4j.protocol from py4j.protocol import Py4JJavaError from py4j.java_gateway import JavaObject @@ -79,7 +74,7 @@ def _py2java(sc, obj): obj = [_py2java(sc, x) for x in obj] elif isinstance(obj, JavaObject): pass - elif isinstance(obj, (int, long, float, bool, bytes, unicode)): + elif isinstance(obj, (int, float, bool, bytes, str)): pass else: data = bytearray(PickleSerializer().dumps(obj)) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 498629cea846c..c52ea62686ab9 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,12 +15,7 @@ # limitations under the License. # -import sys -if sys.version > '3': - basestring = str - from pyspark import since, keyword_only, SparkContext -from pyspark.rdd import ignore_unicode_prefix from pyspark.ml.linalg import _convert_to_vector from pyspark.ml.param.shared import * from pyspark.ml.util import JavaMLReadable, JavaMLWritable @@ -2178,7 +2173,6 @@ def originalMax(self): @inherit_doc -@ignore_unicode_prefix class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ A feature transformer that converts the input array of strings into an array of n-grams. Null @@ -2196,15 +2190,15 @@ class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWr >>> ngram.setOutputCol("nGrams") NGram... >>> ngram.transform(df).head() - Row(inputTokens=[u'a', u'b', u'c', u'd', u'e'], nGrams=[u'a b', u'b c', u'c d', u'd e']) + Row(inputTokens=['a', 'b', 'c', 'd', 'e'], nGrams=['a b', 'b c', 'c d', 'd e']) >>> # Change n-gram length >>> ngram.setParams(n=4).transform(df).head() - Row(inputTokens=[u'a', u'b', u'c', u'd', u'e'], nGrams=[u'a b c d', u'b c d e']) + Row(inputTokens=['a', 'b', 'c', 'd', 'e'], nGrams=['a b c d', 'b c d e']) >>> # Temporarily modify output column. >>> ngram.transform(df, {ngram.outputCol: "output"}).head() - Row(inputTokens=[u'a', u'b', u'c', u'd', u'e'], output=[u'a b c d', u'b c d e']) + Row(inputTokens=['a', 'b', 'c', 'd', 'e'], output=['a b c d', 'b c d e']) >>> ngram.transform(df).head() - Row(inputTokens=[u'a', u'b', u'c', u'd', u'e'], nGrams=[u'a b c d', u'b c d e']) + Row(inputTokens=['a', 'b', 'c', 'd', 'e'], nGrams=['a b c d', 'b c d e']) >>> # Must use keyword arguments to specify params. >>> ngram.setParams("text") Traceback (most recent call last): @@ -3082,7 +3076,6 @@ def range(self): @inherit_doc -@ignore_unicode_prefix class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ A regex based tokenizer that extracts tokens either by using the @@ -3099,15 +3092,15 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, >>> reTokenizer.setOutputCol("words") RegexTokenizer... >>> reTokenizer.transform(df).head() - Row(text=u'A B c', words=[u'a', u'b', u'c']) + Row(text='A B c', words=['a', 'b', 'c']) >>> # Change a parameter. >>> reTokenizer.setParams(outputCol="tokens").transform(df).head() - Row(text=u'A B c', tokens=[u'a', u'b', u'c']) + Row(text='A B c', tokens=['a', 'b', 'c']) >>> # Temporarily modify a parameter. >>> reTokenizer.transform(df, {reTokenizer.outputCol: "words"}).head() - Row(text=u'A B c', words=[u'a', u'b', u'c']) + Row(text='A B c', words=['a', 'b', 'c']) >>> reTokenizer.transform(df).head() - Row(text=u'A B c', tokens=[u'a', u'b', u'c']) + Row(text='A B c', tokens=['a', 'b', 'c']) >>> # Must use keyword arguments to specify params. >>> reTokenizer.setParams("text") Traceback (most recent call last): @@ -3935,7 +3928,6 @@ def loadDefaultStopWords(language): @inherit_doc -@ignore_unicode_prefix class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ A tokenizer that converts the input string to lowercase and then @@ -3946,15 +3938,15 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java >>> tokenizer.setInputCol("text") Tokenizer... >>> tokenizer.transform(df).head() - Row(text=u'a b c', words=[u'a', u'b', u'c']) + Row(text='a b c', words=['a', 'b', 'c']) >>> # Change a parameter. >>> tokenizer.setParams(outputCol="tokens").transform(df).head() - Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + Row(text='a b c', tokens=['a', 'b', 'c']) >>> # Temporarily modify a parameter. >>> tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() - Row(text=u'a b c', words=[u'a', u'b', u'c']) + Row(text='a b c', words=['a', 'b', 'c']) >>> tokenizer.transform(df).head() - Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + Row(text='a b c', tokens=['a', 'b', 'c']) >>> # Must use keyword arguments to specify params. >>> tokenizer.setParams("text") Traceback (most recent call last): @@ -4476,7 +4468,6 @@ def getMaxSentenceLength(self): @inherit_doc -@ignore_unicode_prefix class Word2Vec(JavaEstimator, _Word2VecParams, JavaMLReadable, JavaMLWritable): """ Word2Vec trains a model of `Map(String, Vector)`, i.e. transforms a word into a code for further @@ -4505,7 +4496,7 @@ class Word2Vec(JavaEstimator, _Word2VecParams, JavaMLReadable, JavaMLWritable): +----+--------------------+ ... >>> model.findSynonymsArray("a", 2) - [(u'b', 0.015859870240092278), (u'c', -0.5680795907974243)] + [('b', 0.015859870240092278), ('c', -0.5680795907974243)] >>> from pyspark.sql.functions import format_number as fmt >>> model.findSynonyms("a", 2).select("word", fmt("similarity", 5).alias("similarity")).show() +----+----------+ @@ -4668,7 +4659,7 @@ def findSynonyms(self, word, num): Returns a dataframe with two fields word and similarity (which gives the cosine similarity). """ - if not isinstance(word, basestring): + if not isinstance(word, str): word = _convert_to_vector(word) return self._call_java("findSynonyms", word, num) @@ -4680,7 +4671,7 @@ def findSynonymsArray(self, word, num): Returns an array with two fields word and similarity (which gives the cosine similarity). """ - if not isinstance(word, basestring): + if not isinstance(word, str): word = _convert_to_vector(word) tuples = self._java_obj.findSynonymsArray(word, num) return list(map(lambda st: (st._1(), st._2()), list(tuples))) diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index 7a5591f3fbf76..b91788a82c19a 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -15,8 +15,7 @@ # limitations under the License. # -from pyspark import keyword_only, since -from pyspark.rdd import ignore_unicode_prefix +from pyspark import keyword_only from pyspark.sql import DataFrame from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams @@ -132,7 +131,6 @@ def associationRules(self): return self._call_java("associationRules") -@ignore_unicode_prefix class FPGrowth(JavaEstimator, _FPGrowthParams, JavaMLWritable, JavaMLReadable): r""" A parallel FP-growth algorithm to mine frequent itemsets. The algorithm is described in @@ -193,7 +191,7 @@ class FPGrowth(JavaEstimator, _FPGrowthParams, JavaMLWritable, JavaMLReadable): ... >>> new_data = spark.createDataFrame([(["t", "s"], )], ["items"]) >>> sorted(fpm.transform(new_data).first().newPrediction) - [u'x', u'y', u'z'] + ['x', 'y', 'z'] .. versionadded:: 2.2.0 """ diff --git a/python/pyspark/ml/image.py b/python/pyspark/ml/image.py index 4fb1036fbab89..20b24559b182d 100644 --- a/python/pyspark/ml/image.py +++ b/python/pyspark/ml/image.py @@ -25,14 +25,13 @@ """ import sys -import warnings import numpy as np from distutils.version import LooseVersion from pyspark import SparkContext from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string -from pyspark.sql import DataFrame, SparkSession +from pyspark.sql import SparkSession __all__ = ["ImageSchema"] diff --git a/python/pyspark/ml/linalg/__init__.py b/python/pyspark/ml/linalg/__init__.py index a79d5e5dcbb16..8be440da4fef8 100644 --- a/python/pyspark/ml/linalg/__init__.py +++ b/python/pyspark/ml/linalg/__init__.py @@ -27,18 +27,8 @@ import array import struct -if sys.version >= '3': - basestring = str - xrange = range - import copyreg as copy_reg - long = int -else: - from itertools import izip as zip - import copy_reg - import numpy as np -from pyspark import since from pyspark.sql.types import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ IntegerType, ByteType, BooleanType @@ -47,13 +37,6 @@ 'Matrix', 'DenseMatrix', 'SparseMatrix', 'Matrices'] -if sys.version_info[:2] == (2, 7): - # speed up pickling array in Python 2.7 - def fast_pickle_array(ar): - return array.array, (ar.typecode, ar.tostring()) - copy_reg.pickle(array.array, fast_pickle_array) - - # Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, # such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. @@ -68,7 +51,7 @@ def fast_pickle_array(ar): def _convert_to_vector(l): if isinstance(l, Vector): return l - elif type(l) in (array.array, np.array, np.ndarray, list, tuple, xrange): + elif type(l) in (array.array, np.array, np.ndarray, list, tuple, range): return DenseVector(l) elif _have_scipy and scipy.sparse.issparse(l): assert l.shape[1] == 1, "Expected column vector" @@ -102,7 +85,7 @@ def _vector_size(v): """ if isinstance(v, Vector): return len(v) - elif type(v) in (array.array, list, tuple, xrange): + elif type(v) in (array.array, list, tuple, range): return len(v) elif type(v) == np.ndarray: if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): @@ -415,7 +398,7 @@ def __eq__(self, other): elif isinstance(other, SparseVector): if len(self) != other.size: return False - return Vectors._equals(list(xrange(len(self))), self.array, other.indices, other.values) + return Vectors._equals(list(range(len(self))), self.array, other.indices, other.values) return False def __ne__(self, other): @@ -520,7 +503,7 @@ def __init__(self, size, *args): self.indices = np.array(args[0], dtype=np.int32) self.values = np.array(args[1], dtype=np.float64) assert len(self.indices) == len(self.values), "index and value arrays not same length" - for i in xrange(len(self.indices) - 1): + for i in range(len(self.indices) - 1): if self.indices[i] >= self.indices[i + 1]: raise TypeError( "Indices %s and %s are not strictly increasing" @@ -699,7 +682,7 @@ def __repr__(self): inds = self.indices vals = self.values entries = ", ".join(["{0}: {1}".format(inds[i], _format_float(vals[i])) - for i in xrange(len(inds))]) + for i in range(len(inds))]) return "SparseVector({0}, {{{1}}})".format(self.size, entries) def __eq__(self, other): @@ -709,7 +692,7 @@ def __eq__(self, other): elif isinstance(other, DenseVector): if self.size != len(other): return False - return Vectors._equals(self.indices, self.values, list(xrange(len(other))), other.array) + return Vectors._equals(self.indices, self.values, list(range(len(other))), other.array) return False def __getitem__(self, index): @@ -791,7 +774,7 @@ def dense(*elements): >>> Vectors.dense(1.0, 2.0) DenseVector([1.0, 2.0]) """ - if len(elements) == 1 and not isinstance(elements[0], (float, int, long)): + if len(elements) == 1 and not isinstance(elements[0], (float, int)): # it's list, numpy.array or other iterable object. elements = elements[0] return DenseVector(elements) @@ -1124,7 +1107,7 @@ def toArray(self): Return a numpy.ndarray """ A = np.zeros((self.numRows, self.numCols), dtype=np.float64, order='F') - for k in xrange(self.colPtrs.size - 1): + for k in range(self.colPtrs.size - 1): startptr = self.colPtrs[k] endptr = self.colPtrs[k + 1] if self.isTransposed: diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 1be8755c7b982..96b07bfa5f14f 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -16,15 +16,10 @@ # import array import sys -if sys.version > '3': - basestring = str - xrange = range - unicode = str - from abc import ABCMeta import copy -import numpy as np +import numpy as np from py4j.java_gateway import JavaObject from pyspark.ml.linalg import DenseVector, Vector, Matrix @@ -93,12 +88,12 @@ def _is_integer(value): @staticmethod def _can_convert_to_list(value): vtype = type(value) - return vtype in [list, np.ndarray, tuple, xrange, array.array] or isinstance(value, Vector) + return vtype in [list, np.ndarray, tuple, range, array.array] or isinstance(value, Vector) @staticmethod def _can_convert_to_string(value): vtype = type(value) - return isinstance(value, basestring) or vtype in [np.unicode_, np.string_, np.str_] + return isinstance(value, str) or vtype in [np.unicode_, np.string_, np.str_] @staticmethod def identity(value): @@ -114,7 +109,7 @@ def toList(value): """ if type(value) == list: return value - elif type(value) in [np.ndarray, tuple, xrange, array.array]: + elif type(value) in [np.ndarray, tuple, range, array.array]: return list(value) elif isinstance(value, Vector): return list(value.toArray()) @@ -211,12 +206,10 @@ def toString(value): """ Convert a value to a string, if possible. """ - if isinstance(value, basestring): + if isinstance(value, str): return value - elif type(value) in [np.string_, np.str_]: + elif type(value) in [np.string_, np.str_, np.unicode_]: return str(value) - elif type(value) == np.unicode_: - return unicode(value) else: raise TypeError("Could not convert %s to string type" % type(value)) @@ -338,7 +331,7 @@ def hasParam(self, paramName): Tests whether this instance contains a param with a given (string) name. """ - if isinstance(paramName, basestring): + if isinstance(paramName, str): p = getattr(self, paramName, None) return isinstance(p, Param) else: @@ -421,7 +414,7 @@ def _resolveParam(self, param): if isinstance(param, Param): self._shouldOwn(param) return param - elif isinstance(param, basestring): + elif isinstance(param, str): return self.getParam(param) else: raise ValueError("Cannot resolve %r as a param." % param) @@ -510,7 +503,7 @@ def _resetUid(self, newUid): :return: same instance, but with the uid and Param.parent values updated, including within param maps """ - newUid = unicode(newUid) + newUid = str(newUid) self.uid = newUid newDefaultParamMap = dict() newParamMap = dict() diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 2086e831f4282..bc1ea87ad629c 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - header = """# # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 53d07ec9660d9..eacb8b82b5244 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -16,12 +16,8 @@ # import sys -import os -if sys.version > '3': - basestring = str - -from pyspark import since, keyword_only, SparkContext +from pyspark import keyword_only from pyspark.ml.base import Estimator, Model, Transformer from pyspark.ml.param import Param, Params from pyspark.ml.util import * diff --git a/python/pyspark/ml/tests/test_feature.py b/python/pyspark/ml/tests/test_feature.py index 4c6bfa696b110..7856a317c261d 100644 --- a/python/pyspark/ml/tests/test_feature.py +++ b/python/pyspark/ml/tests/test_feature.py @@ -19,9 +19,6 @@ import sys import unittest -if sys.version > '3': - basestring = str - from pyspark.ml.feature import Binarizer, CountVectorizer, CountVectorizerModel, HashingTF, IDF, \ NGram, RFormula, StopWordsRemover, StringIndexer, StringIndexerModel, VectorSizeHint from pyspark.ml.linalg import DenseVector, SparseVector, Vectors @@ -91,7 +88,7 @@ def test_stopwordsremover(self): transformedDF = stopWordRemover.transform(dataset) self.assertEqual(transformedDF.head().output, ["panda"]) self.assertEqual(type(stopWordRemover.getStopWords()), list) - self.assertTrue(isinstance(stopWordRemover.getStopWords()[0], basestring)) + self.assertTrue(isinstance(stopWordRemover.getStopWords()[0], str)) # Custom stopwords = ["panda"] stopWordRemover.setStopWords(stopwords) diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index 1b2b1914cc036..e1abd59a2d7b2 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -35,10 +35,6 @@ from pyspark.testing.mlutils import check_params, PySparkTestCase, SparkSessionTestCase -if sys.version > '3': - xrange = range - - class ParamTypeConversionTests(PySparkTestCase): """ Test that param type conversion happens. @@ -67,14 +63,14 @@ def test_vector(self): def test_list(self): l = [0, 1] for lst_like in [l, np.array(l), DenseVector(l), SparseVector(len(l), range(len(l)), l), - pyarray.array('l', l), xrange(2), tuple(l)]: + pyarray.array('l', l), range(2), tuple(l)]: converted = TypeConverters.toList(lst_like) self.assertEqual(type(converted), list) self.assertListEqual(converted, l) def test_list_int(self): for indices in [[1.0, 2.0], np.array([1.0, 2.0]), DenseVector([1.0, 2.0]), - SparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), + SparseVector(2, {0: 1.0, 1: 2.0}), range(1, 3), (1.0, 2.0), pyarray.array('d', [1.0, 2.0])]: vs = VectorSlicer(indices=indices) self.assertListEqual(vs.getIndices(), [1, 2]) @@ -200,12 +196,7 @@ def test_resolveparam(self): self.assertEqual(testParams._resolveParam("maxIter"), testParams.maxIter) self.assertEqual(testParams._resolveParam(u"maxIter"), testParams.maxIter) - if sys.version_info[0] >= 3: - # In Python 3, it is allowed to get/set attributes with non-ascii characters. - e_cls = AttributeError - else: - e_cls = UnicodeEncodeError - self.assertRaises(e_cls, lambda: testParams._resolveParam(u"아")) + self.assertRaises(AttributeError, lambda: testParams._resolveParam(u"아")) def test_params(self): testParams = TestParams() diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index 7d905793188bf..15e9ebb0f5ce8 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -18,9 +18,6 @@ import sys import unittest -if sys.version > '3': - basestring = str - from pyspark.ml.classification import BinaryLogisticRegressionSummary, LinearSVC, \ LinearSVCSummary, BinaryRandomForestClassificationSummary, LogisticRegression, \ LogisticRegressionSummary, RandomForestClassificationSummary, \ @@ -101,7 +98,7 @@ def test_glr_summary(self): self.assertEqual(s.residualDegreeOfFreedom, 1) self.assertEqual(s.residualDegreeOfFreedomNull, 2) self.assertEqual(s.rank, 1) - self.assertTrue(isinstance(s.solver, basestring)) + self.assertTrue(isinstance(s.solver, str)) self.assertTrue(isinstance(s.aic, float)) self.assertTrue(isinstance(s.deviance, float)) self.assertTrue(isinstance(s.nullDeviance, float)) diff --git a/python/pyspark/ml/tree.py b/python/pyspark/ml/tree.py index a13b27ec8a79c..460c76fabc375 100644 --- a/python/pyspark/ml/tree.py +++ b/python/pyspark/ml/tree.py @@ -15,12 +15,10 @@ # limitations under the License. # -from pyspark import since, keyword_only from pyspark.ml.param.shared import * from pyspark.ml.util import * -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, \ - JavaPredictor, JavaPredictionModel -from pyspark.ml.common import inherit_doc, _java2py, _py2java +from pyspark.ml.wrapper import JavaPredictionModel +from pyspark.ml.common import inherit_doc @inherit_doc diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index e00753b2ffc20..7f3d942e2e456 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -15,12 +15,11 @@ # limitations under the License. # import itertools -import sys from multiprocessing.pool import ThreadPool import numpy as np -from pyspark import since, keyword_only +from pyspark import keyword_only from pyspark.ml import Estimator, Model from pyspark.ml.common import _py2java, _java2py from pyspark.ml.param import Params, Param, TypeConverters diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index aac2b38d3f57d..9ab6bfa9ba968 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -20,12 +20,6 @@ import os import time import uuid -import warnings - -if sys.version > '3': - basestring = str - unicode = str - long = int from pyspark import SparkContext, since from pyspark.ml.common import inherit_doc @@ -60,10 +54,10 @@ def __repr__(self): @classmethod def _randomUID(cls): """ - Generate a unique unicode id for the object. The default implementation + Generate a unique string id for the object. The default implementation concatenates the class name, "_", and 12 random hex chars. """ - return unicode(cls.__name__ + "_" + uuid.uuid4().hex[-12:]) + return str(cls.__name__ + "_" + uuid.uuid4().hex[-12:]) @inherit_doc @@ -170,8 +164,8 @@ def __init__(self, instance): def save(self, path): """Save the ML instance to the input path.""" - if not isinstance(path, basestring): - raise TypeError("path should be a basestring, got type %s" % type(path)) + if not isinstance(path, str): + raise TypeError("path should be a string, got type %s" % type(path)) self._jwrite.save(path) def overwrite(self): @@ -275,8 +269,8 @@ def __init__(self, clazz): def load(self, path): """Load the ML instance from the input path.""" - if not isinstance(path, basestring): - raise TypeError("path should be a basestring, got type %s" % type(path)) + if not isinstance(path, str): + raise TypeError("path should be a string, got type %s" % type(path)) java_obj = self._jread.load(path) if not hasattr(self._clazz, "_from_java"): raise NotImplementedError("This Java ML type cannot be loaded into Python currently: %r" @@ -430,7 +424,7 @@ def _get_metadata_to_save(instance, sc, extraMetadata=None, paramMap=None): for p in instance._defaultParamMap: jsonDefaultParams[p.name] = instance._defaultParamMap[p] - basicMetadata = {"class": cls, "timestamp": long(round(time.time() * 1000)), + basicMetadata = {"class": cls, "timestamp": int(round(time.time() * 1000)), "sparkVersion": sc.version, "uid": uid, "paramMap": jsonParams, "defaultParamMap": jsonDefaultParams} if extraMetadata is not None: diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index e59c6c7b250a8..c1d060a51cf9d 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -16,9 +16,6 @@ # from abc import ABCMeta, abstractmethod -import sys -if sys.version >= '3': - xrange = range from pyspark import since from pyspark import SparkContext @@ -26,7 +23,6 @@ from pyspark.ml import Estimator, Predictor, PredictionModel, Transformer, Model from pyspark.ml.base import _PredictorParams from pyspark.ml.param import Params -from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol from pyspark.ml.util import _jvm from pyspark.ml.common import inherit_doc, _java2py, _py2java @@ -99,15 +95,15 @@ def _new_java_array(pylist, java_class): # If pylist is a 2D array, then a 2D java array will be created. # The 2D array is a square, non-jagged 2D array that is big enough for all elements. inner_array_length = 0 - for i in xrange(len(pylist)): + for i in range(len(pylist)): inner_array_length = max(inner_array_length, len(pylist[i])) java_array = sc._gateway.new_array(java_class, len(pylist), inner_array_length) - for i in xrange(len(pylist)): - for j in xrange(len(pylist[i])): + for i in range(len(pylist)): + for j in range(len(pylist[i])): java_array[i][j] = pylist[i][j] else: java_array = sc._gateway.new_array(java_class, len(pylist)) - for i in xrange(len(pylist)): + for i in range(len(pylist)): java_array[i] = pylist[i] return java_array diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index ae26521ea96bf..6067693111547 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -21,8 +21,6 @@ The `pyspark.mllib` package is in maintenance mode as of the Spark 2.0.0 release to encourage migration to the DataFrame-based APIs under the `pyspark.ml` package. """ -from __future__ import absolute_import - # MLlib currently needs NumPy 1.4+, so complain if lower import numpy diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index e41e5c9cc8e89..85cfe583fd5c5 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -17,20 +17,13 @@ import sys import array as pyarray -import warnings - -if sys.version > '3': - xrange = range - basestring = str - from math import exp, log +from collections import namedtuple from numpy import array, random, tile -from collections import namedtuple - from pyspark import SparkContext, since -from pyspark.rdd import RDD, ignore_unicode_prefix +from pyspark.rdd import RDD from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, callJavaFunc, _py2java, _java2py from pyspark.mllib.linalg import SparseVector, _convert_to_vector, DenseVector from pyspark.mllib.stat.distribution import MultivariateGaussian @@ -257,7 +250,7 @@ def predict(self, x): return x.map(self.predict) x = _convert_to_vector(x) - for i in xrange(len(self.centers)): + for i in range(len(self.centers)): distance = x.squared_distance(self.centers[i]) if distance < best_distance: best = i @@ -708,7 +701,7 @@ class StreamingKMeansModel(KMeansModel): >>> stkm = StreamingKMeansModel(initCenters, initWeights) >>> data = sc.parallelize([[-0.1, -0.1], [0.1, 0.1], ... [0.9, 0.9], [1.1, 1.1]]) - >>> stkm = stkm.update(data, 1.0, u"batches") + >>> stkm = stkm.update(data, 1.0, "batches") >>> stkm.centers array([[ 0., 0.], [ 1., 1.]]) @@ -720,7 +713,7 @@ class StreamingKMeansModel(KMeansModel): [3.0, 3.0] >>> decayFactor = 0.0 >>> data = sc.parallelize([DenseVector([1.5, 1.5]), DenseVector([0.2, 0.2])]) - >>> stkm = stkm.update(data, 0.0, u"batches") + >>> stkm = stkm.update(data, 0.0, "batches") >>> stkm.centers array([[ 0.2, 0.2], [ 1.5, 1.5]]) @@ -743,7 +736,6 @@ def clusterWeights(self): """Return the cluster weights.""" return self._clusterWeights - @ignore_unicode_prefix @since('1.5.0') def update(self, data, decayFactor, timeUnit): """Update the centroids, according to data @@ -979,8 +971,8 @@ def load(cls, sc, path): """ if not isinstance(sc, SparkContext): raise TypeError("sc should be a SparkContext, got type %s" % type(sc)) - if not isinstance(path, basestring): - raise TypeError("path should be a basestring, got type %s" % type(path)) + if not isinstance(path, str): + raise TypeError("path should be a string, got type %s" % type(path)) model = callMLlibFunc("loadLDAModel", sc, path) return LDAModel(model) diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index bac8f350563ec..24e2f198251ad 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -15,11 +15,6 @@ # limitations under the License. # -import sys -if sys.version >= '3': - long = int - unicode = str - import py4j.protocol from py4j.protocol import Py4JJavaError from py4j.java_gateway import JavaObject @@ -81,7 +76,7 @@ def _py2java(sc, obj): obj = [_py2java(sc, x) for x in obj] elif isinstance(obj, JavaObject): pass - elif isinstance(obj, (int, long, float, bool, bytes, unicode)): + elif isinstance(obj, (int, float, bool, bytes, str)): pass else: data = bytearray(PickleSerializer().dumps(obj)) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 3efae6ff0ecc3..80a197eaa7494 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -18,21 +18,15 @@ """ Python package for feature in MLlib. """ -from __future__ import absolute_import - import sys import warnings -if sys.version >= '3': - basestring = str - unicode = str - from py4j.protocol import Py4JJavaError from pyspark import since -from pyspark.rdd import RDD, ignore_unicode_prefix +from pyspark.rdd import RDD from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.linalg import ( - Vector, Vectors, DenseVector, SparseVector, _convert_to_vector) + Vectors, DenseVector, SparseVector, _convert_to_vector) from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.util import JavaLoader, JavaSaveable @@ -616,7 +610,7 @@ def findSynonyms(self, word, num): .. note:: Local use only """ - if not isinstance(word, basestring): + if not isinstance(word, str): word = _convert_to_vector(word) words, similarity = self.call("findSynonyms", word, num) return zip(words, similarity) @@ -640,7 +634,6 @@ def load(cls, sc, path): return Word2VecModel(model) -@ignore_unicode_prefix class Word2Vec(object): """Word2Vec creates vector representation of words in a text corpus. The algorithm first constructs a vocabulary from the corpus @@ -668,7 +661,7 @@ class Word2Vec(object): >>> syms = model.findSynonyms("a", 2) >>> [s[0] for s in syms] - [u'b', u'c'] + ['b', 'c'] But querying for synonyms of a vector may return the word whose representation is that vector: @@ -676,7 +669,7 @@ class Word2Vec(object): >>> vec = model.transform("a") >>> syms = model.findSynonyms(vec, 2) >>> [s[0] for s in syms] - [u'a', u'b'] + ['a', 'b'] >>> import os, tempfile >>> path = tempfile.mkdtemp() @@ -686,7 +679,7 @@ class Word2Vec(object): True >>> syms = sameModel.findSynonyms("a", 2) >>> [s[0] for s in syms] - [u'b', u'c'] + ['b', 'c'] >>> from shutil import rmtree >>> try: ... rmtree(path) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index 373a141456b2f..cbbd7b351b20d 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -20,7 +20,6 @@ from collections import namedtuple from pyspark import since -from pyspark.rdd import ignore_unicode_prefix from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc from pyspark.mllib.util import JavaSaveable, JavaLoader, inherit_doc @@ -28,7 +27,6 @@ @inherit_doc -@ignore_unicode_prefix class FPGrowthModel(JavaModelWrapper, JavaSaveable, JavaLoader): """ A FP-Growth model for mining frequent itemsets @@ -38,7 +36,7 @@ class FPGrowthModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> rdd = sc.parallelize(data, 2) >>> model = FPGrowth.train(rdd, 0.6, 2) >>> sorted(model.freqItemsets().collect()) - [FreqItemset(items=[u'a'], freq=4), FreqItemset(items=[u'c'], freq=3), ... + [FreqItemset(items=['a'], freq=4), FreqItemset(items=['c'], freq=3), ... >>> model_path = temp_path + "/fpm" >>> model.save(sc, model_path) >>> sameModel = FPGrowthModel.load(sc, model_path) @@ -101,7 +99,6 @@ class FreqItemset(namedtuple("FreqItemset", ["items", "freq"])): @inherit_doc -@ignore_unicode_prefix class PrefixSpanModel(JavaModelWrapper): """ Model fitted by PrefixSpan @@ -114,7 +111,7 @@ class PrefixSpanModel(JavaModelWrapper): >>> rdd = sc.parallelize(data, 2) >>> model = PrefixSpan.train(rdd) >>> sorted(model.freqSequences().collect()) - [FreqSequence(sequence=[[u'a']], freq=3), FreqSequence(sequence=[[u'a'], [u'a']], freq=1), ... + [FreqSequence(sequence=[['a']], freq=3), FreqSequence(sequence=[['a'], ['a']], freq=1), ... .. versionadded:: 1.6.0 """ diff --git a/python/pyspark/mllib/linalg/__init__.py b/python/pyspark/mllib/linalg/__init__.py index cd09621b13b56..c1402fb98a50d 100644 --- a/python/pyspark/mllib/linalg/__init__.py +++ b/python/pyspark/mllib/linalg/__init__.py @@ -27,15 +27,6 @@ import array import struct -if sys.version >= '3': - basestring = str - xrange = range - import copyreg as copy_reg - long = int -else: - from itertools import izip as zip - import copy_reg - import numpy as np from pyspark import since @@ -49,13 +40,6 @@ 'QRDecomposition'] -if sys.version_info[:2] == (2, 7): - # speed up pickling array in Python 2.7 - def fast_pickle_array(ar): - return array.array, (ar.typecode, ar.tostring()) - copy_reg.pickle(array.array, fast_pickle_array) - - # Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, # such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. @@ -70,7 +54,7 @@ def fast_pickle_array(ar): def _convert_to_vector(l): if isinstance(l, Vector): return l - elif type(l) in (array.array, np.array, np.ndarray, list, tuple, xrange): + elif type(l) in (array.array, np.array, np.ndarray, list, tuple, range): return DenseVector(l) elif _have_scipy and scipy.sparse.issparse(l): assert l.shape[1] == 1, "Expected column vector" @@ -104,7 +88,7 @@ def _vector_size(v): """ if isinstance(v, Vector): return len(v) - elif type(v) in (array.array, list, tuple, xrange): + elif type(v) in (array.array, list, tuple, range): return len(v) elif type(v) == np.ndarray: if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): @@ -459,7 +443,7 @@ def __eq__(self, other): elif isinstance(other, SparseVector): if len(self) != other.size: return False - return Vectors._equals(list(xrange(len(self))), self.array, other.indices, other.values) + return Vectors._equals(list(range(len(self))), self.array, other.indices, other.values) return False def __ne__(self, other): @@ -556,7 +540,7 @@ def __init__(self, size, *args): self.indices = np.array(args[0], dtype=np.int32) self.values = np.array(args[1], dtype=np.float64) assert len(self.indices) == len(self.values), "index and value arrays not same length" - for i in xrange(len(self.indices) - 1): + for i in range(len(self.indices) - 1): if self.indices[i] >= self.indices[i + 1]: raise TypeError( "Indices %s and %s are not strictly increasing" @@ -788,7 +772,7 @@ def __repr__(self): inds = self.indices vals = self.values entries = ", ".join(["{0}: {1}".format(inds[i], _format_float(vals[i])) - for i in xrange(len(inds))]) + for i in range(len(inds))]) return "SparseVector({0}, {{{1}}})".format(self.size, entries) def __eq__(self, other): @@ -798,7 +782,7 @@ def __eq__(self, other): elif isinstance(other, DenseVector): if self.size != len(other): return False - return Vectors._equals(self.indices, self.values, list(xrange(len(other))), other.array) + return Vectors._equals(self.indices, self.values, list(range(len(other))), other.array) return False def __getitem__(self, index): @@ -880,7 +864,7 @@ def dense(*elements): >>> Vectors.dense(1.0, 2.0) DenseVector([1.0, 2.0]) """ - if len(elements) == 1 and not isinstance(elements[0], (float, int, long)): + if len(elements) == 1 and not isinstance(elements[0], (float, int)): # it's list, numpy.array or other iterable object. elements = elements[0] return DenseVector(elements) @@ -1279,7 +1263,7 @@ def toArray(self): Return an numpy.ndarray """ A = np.zeros((self.numRows, self.numCols), dtype=np.float64, order='F') - for k in xrange(self.colPtrs.size - 1): + for k in range(self.colPtrs.size - 1): startptr = self.colPtrs[k] endptr = self.colPtrs[k + 1] if self.isTransposed: diff --git a/python/pyspark/mllib/linalg/distributed.py b/python/pyspark/mllib/linalg/distributed.py index 56701758c89c9..603d31d3d7b26 100644 --- a/python/pyspark/mllib/linalg/distributed.py +++ b/python/pyspark/mllib/linalg/distributed.py @@ -21,9 +21,6 @@ import sys -if sys.version >= '3': - long = int - from py4j.java_gateway import JavaObject from pyspark import RDD, since @@ -95,9 +92,9 @@ def __init__(self, rows, numRows=0, numCols=0): """ if isinstance(rows, RDD): rows = rows.map(_convert_to_vector) - java_matrix = callMLlibFunc("createRowMatrix", rows, long(numRows), int(numCols)) + java_matrix = callMLlibFunc("createRowMatrix", rows, int(numRows), int(numCols)) elif isinstance(rows, DataFrame): - java_matrix = callMLlibFunc("createRowMatrix", rows, long(numRows), int(numCols)) + java_matrix = callMLlibFunc("createRowMatrix", rows, int(numRows), int(numCols)) elif (isinstance(rows, JavaObject) and rows.getClass().getSimpleName() == "RowMatrix"): java_matrix = rows @@ -439,13 +436,13 @@ class IndexedRow(object): """ Represents a row of an IndexedRowMatrix. - Just a wrapper over a (long, vector) tuple. + Just a wrapper over a (int, vector) tuple. :param index: The index for the given row. :param vector: The row in the matrix at the given index. """ def __init__(self, index, vector): - self.index = long(index) + self.index = int(index) self.vector = _convert_to_vector(vector) def __repr__(self): @@ -465,8 +462,8 @@ class IndexedRowMatrix(DistributedMatrix): """ Represents a row-oriented distributed Matrix with indexed rows. - :param rows: An RDD of IndexedRows or (long, vector) tuples or a DataFrame consisting of a - long typed column of indices and a vector typed column. + :param rows: An RDD of IndexedRows or (int, vector) tuples or a DataFrame consisting of a + int typed column of indices and a vector typed column. :param numRows: Number of rows in the matrix. A non-positive value means unknown, at which point the number of rows will be determined by the max row @@ -510,14 +507,14 @@ def __init__(self, rows, numRows=0, numCols=0): # both be easily serialized. We will convert back to # IndexedRows on the Scala side. java_matrix = callMLlibFunc("createIndexedRowMatrix", rows.toDF(), - long(numRows), int(numCols)) + int(numRows), int(numCols)) elif isinstance(rows, DataFrame): - java_matrix = callMLlibFunc("createIndexedRowMatrix", rows, long(numRows), int(numCols)) + java_matrix = callMLlibFunc("createIndexedRowMatrix", rows, int(numRows), int(numCols)) elif (isinstance(rows, JavaObject) and rows.getClass().getSimpleName() == "IndexedRowMatrix"): java_matrix = rows else: - raise TypeError("rows should be an RDD of IndexedRows or (long, vector) tuples, " + raise TypeError("rows should be an RDD of IndexedRows or (int, vector) tuples, " "got %s" % type(rows)) self._java_matrix_wrapper = JavaModelWrapper(java_matrix) @@ -731,15 +728,15 @@ class MatrixEntry(object): """ Represents an entry of a CoordinateMatrix. - Just a wrapper over a (long, long, float) tuple. + Just a wrapper over a (int, int, float) tuple. :param i: The row index of the matrix. :param j: The column index of the matrix. :param value: The (i, j)th entry of the matrix, as a float. """ def __init__(self, i, j, value): - self.i = long(i) - self.j = long(j) + self.i = int(i) + self.j = int(j) self.value = float(value) def __repr__(self): @@ -760,7 +757,7 @@ class CoordinateMatrix(DistributedMatrix): Represents a matrix in coordinate format. :param entries: An RDD of MatrixEntry inputs or - (long, long, float) tuples. + (int, int, float) tuples. :param numRows: Number of rows in the matrix. A non-positive value means unknown, at which point the number of rows will be determined by the max row @@ -804,13 +801,13 @@ def __init__(self, entries, numRows=0, numCols=0): # each be easily serialized. We will convert back to # MatrixEntry inputs on the Scala side. java_matrix = callMLlibFunc("createCoordinateMatrix", entries.toDF(), - long(numRows), long(numCols)) + int(numRows), int(numCols)) elif (isinstance(entries, JavaObject) and entries.getClass().getSimpleName() == "CoordinateMatrix"): java_matrix = entries else: raise TypeError("entries should be an RDD of MatrixEntry entries or " - "(long, long, float) tuples, got %s" % type(entries)) + "(int, int, float) tuples, got %s" % type(entries)) self._java_matrix_wrapper = JavaModelWrapper(java_matrix) @@ -1044,7 +1041,7 @@ def __init__(self, blocks, rowsPerBlock, colsPerBlock, numRows=0, numCols=0): # the Scala side. java_matrix = callMLlibFunc("createBlockMatrix", blocks.toDF(), int(rowsPerBlock), int(colsPerBlock), - long(numRows), long(numCols)) + int(numRows), int(numCols)) elif (isinstance(blocks, JavaObject) and blocks.getClass().getSimpleName() == "BlockMatrix"): java_matrix = blocks diff --git a/python/pyspark/mllib/stat/KernelDensity.py b/python/pyspark/mllib/stat/KernelDensity.py index 7250eab6705a7..56444c152f0ba 100644 --- a/python/pyspark/mllib/stat/KernelDensity.py +++ b/python/pyspark/mllib/stat/KernelDensity.py @@ -15,11 +15,6 @@ # limitations under the License. # -import sys - -if sys.version > '3': - xrange = range - import numpy as np from pyspark.mllib.common import callMLlibFunc diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index d49f741a2f44a..43454ba5187dd 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -16,10 +16,8 @@ # import sys -if sys.version >= '3': - basestring = str -from pyspark.rdd import RDD, ignore_unicode_prefix +from pyspark.rdd import RDD from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.linalg import Matrix, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -157,7 +155,6 @@ def corr(x, y=None, method=None): return callMLlibFunc("corr", x.map(float), y.map(float), method) @staticmethod - @ignore_unicode_prefix def chiSqTest(observed, expected=None): """ If `observed` is Vector, conduct Pearson's chi-squared goodness @@ -199,9 +196,9 @@ def chiSqTest(observed, expected=None): >>> print(round(pearson.pValue, 4)) 0.8187 >>> pearson.method - u'pearson' + 'pearson' >>> pearson.nullHypothesis - u'observed follows the same distribution as expected.' + 'observed follows the same distribution as expected.' >>> observed = Vectors.dense([21, 38, 43, 80]) >>> expected = Vectors.dense([3, 5, 7, 20]) @@ -242,7 +239,6 @@ def chiSqTest(observed, expected=None): return ChiSqTestResult(jmodel) @staticmethod - @ignore_unicode_prefix def kolmogorovSmirnovTest(data, distName="norm", *params): """ Performs the Kolmogorov-Smirnov (KS) test for data sampled from @@ -282,7 +278,7 @@ def kolmogorovSmirnovTest(data, distName="norm", *params): >>> print(round(ksmodel.statistic, 3)) 0.175 >>> ksmodel.nullHypothesis - u'Sample follows theoretical distribution' + 'Sample follows theoretical distribution' >>> data = sc.parallelize([2.0, 3.0, 4.0]) >>> ksmodel = kstest(data, "norm", 3.0, 1.0) @@ -293,7 +289,7 @@ def kolmogorovSmirnovTest(data, distName="norm", *params): """ if not isinstance(data, RDD): raise TypeError("data should be an RDD, got %s." % type(data)) - if not isinstance(distName, basestring): + if not isinstance(distName, str): raise TypeError("distName should be a string, got %s." % type(distName)) params = [float(param) for param in params] diff --git a/python/pyspark/mllib/tests/test_linalg.py b/python/pyspark/mllib/tests/test_linalg.py index 312730e8aff8b..21c2bb422a3c3 100644 --- a/python/pyspark/mllib/tests/test_linalg.py +++ b/python/pyspark/mllib/tests/test_linalg.py @@ -31,9 +31,6 @@ from pyspark.testing.mllibutils import MLlibTestCase from pyspark.testing.utils import have_scipy -if sys.version >= '3': - long = int - class VectorTests(MLlibTestCase): @@ -447,7 +444,7 @@ def test_row_matrix_from_dataframe(self): def test_indexed_row_matrix_from_dataframe(self): from pyspark.sql.utils import IllegalArgumentException - df = self.spark.createDataFrame([Row(long(0), Vectors.dense(1))]) + df = self.spark.createDataFrame([Row(int(0), Vectors.dense(1))]) matrix = IndexedRowMatrix(df) self.assertEqual(matrix.numRows(), 1) self.assertEqual(matrix.numCols(), 1) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 2d8df461acf9f..e05dfdb953ceb 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import absolute_import - import sys import random diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index f0f9cda4672b1..a0be29a82e3dc 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -18,10 +18,6 @@ import sys import numpy as np -if sys.version > '3': - xrange = range - basestring = str - from pyspark import SparkContext, since from pyspark.mllib.common import callMLlibFunc, inherit_doc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector @@ -46,7 +42,7 @@ def _parse_libsvm_line(line): nnz = len(items) - 1 indices = np.zeros(nnz, dtype=np.int32) values = np.zeros(nnz) - for i in xrange(nnz): + for i in range(nnz): index, value = items[1 + i].split(":") indices[i] = int(index) - 1 values[i] = float(value) @@ -61,10 +57,10 @@ def _convert_labeled_point_to_libsvm(p): v = _convert_to_vector(p.features) if isinstance(v, SparseVector): nnz = len(v.indices) - for i in xrange(nnz): + for i in range(nnz): items.append(str(v.indices[i] + 1) + ":" + str(v.values[i])) else: - for i in xrange(len(v)): + for i in range(len(v)): items.append(str(i + 1) + ":" + str(v[i])) return " ".join(items) @@ -396,8 +392,8 @@ def save(self, sc, path): """Save this model to the given path.""" if not isinstance(sc, SparkContext): raise TypeError("sc should be a SparkContext, got type %s" % type(sc)) - if not isinstance(path, basestring): - raise TypeError("path should be a basestring, got type %s" % type(path)) + if not isinstance(path, str): + raise TypeError("path should be a string, got type %s" % type(path)) self._java_model.save(sc._jsc.sc(), path) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index db0c1971cd2fe..437b2c446529a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -33,15 +33,10 @@ from functools import reduce from math import sqrt, log, isinf, isnan, pow, ceil -if sys.version > '3': - basestring = unicode = str -else: - from itertools import imap as map, ifilter as filter - from pyspark.java_gateway import local_connect_and_auth from pyspark.serializers import AutoBatchedSerializer, BatchedSerializer, NoOpSerializer, \ CartesianDeserializer, CloudPickleSerializer, PairDeserializer, PickleSerializer, \ - UTF8Deserializer, pack_long, read_int, write_int + pack_long, read_int, write_int from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_full_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -93,7 +88,7 @@ def portable_hash(x): 219750521 """ - if sys.version_info >= (3, 2, 3) and 'PYTHONHASHSEED' not in os.environ: + if 'PYTHONHASHSEED' not in os.environ: raise Exception("Randomness of hash of string should be disabled via PYTHONHASHSEED") if x is None: @@ -204,19 +199,6 @@ def __del__(self): return iter(PyLocalIterable(sock_info, serializer)) -def ignore_unicode_prefix(f): - """ - Ignore the 'u' prefix of string in doc tests, to make it works - in both python 2 and 3 - """ - if sys.version >= '3': - # the representation of unicode string in Python 3 does not have prefix 'u', - # so remove the prefix 'u' for doc tests - literal_re = re.compile(r"(\W|^)[uU](['])", re.UNICODE) - f.__doc__ = literal_re.sub(r'\1\2', f.__doc__) - return f - - class Partitioner(object): def __init__(self, numPartitions, partitionFunc): self.numPartitions = numPartitions @@ -797,13 +779,12 @@ def groupBy(self, f, numPartitions=None, partitionFunc=portable_hash): """ return self.map(lambda x: (f(x), x)).groupByKey(numPartitions, partitionFunc) - @ignore_unicode_prefix def pipe(self, command, env=None, checkCode=False): """ Return an RDD created by piping elements to a forked external process. >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect() - [u'1', u'2', u'', u'3'] + ['1', '2', '', '3'] :param checkCode: whether or not to check the return value of the shell command. """ @@ -816,7 +797,7 @@ def func(iterator): def pipe_objs(out): for obj in iterator: - s = unicode(obj).rstrip('\n') + '\n' + s = str(obj).rstrip('\n') + '\n' out.write(s.encode('utf-8')) out.close() Thread(target=pipe_objs, args=[pipe.stdin]).start() @@ -1591,7 +1572,6 @@ def saveAsPickleFile(self, path, batchSize=10): ser = BatchedSerializer(PickleSerializer(), batchSize) self._reserialize(ser)._jrdd.saveAsObjectFile(path) - @ignore_unicode_prefix def saveAsTextFile(self, path, compressionCodecClass=None): """ Save this RDD as a text file, using string representations of elements. @@ -1625,13 +1605,13 @@ def saveAsTextFile(self, path, compressionCodecClass=None): >>> from fileinput import input, hook_compressed >>> result = sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed)) >>> b''.join(result).decode('utf-8') - u'bar\\nfoo\\n' + 'bar\\nfoo\\n' """ def func(split, iterator): for x in iterator: - if not isinstance(x, (unicode, bytes)): - x = unicode(x) - if isinstance(x, unicode): + if not isinstance(x, (str, bytes)): + x = str(x) + if isinstance(x, str): x = x.encode("utf-8") yield x keyed = self.mapPartitionsWithIndex(func) @@ -2281,14 +2261,13 @@ def name(self): if n: return n - @ignore_unicode_prefix def setName(self, name): """ Assign a name to this RDD. >>> rdd1 = sc.parallelize([1, 2]) >>> rdd1.setName('RDD1').name() - u'RDD1' + 'RDD1' """ self._jrdd.setName(name) return self diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index c867b51877ffe..cd2a59513bb17 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -15,10 +15,7 @@ # limitations under the License. # -try: - from collections.abc import Iterable -except ImportError: - from collections import Iterable +from collections.abc import Iterable __all__ = ["ResultIterable"] diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 49b7cb4546676..80ce9b8408d4e 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -58,18 +58,11 @@ import collections import zlib import itertools - -if sys.version < '3': - import cPickle as pickle - from itertools import izip as zip, imap as map -else: - import pickle - basestring = unicode = str - xrange = range +import pickle pickle_protocol = pickle.HIGHEST_PROTOCOL from pyspark import cloudpickle -from pyspark.util import _exception_message, print_exec +from pyspark.util import print_exec __all__ = ["PickleSerializer", "MarshalSerializer", "UTF8Deserializer"] @@ -132,11 +125,6 @@ class FramedSerializer(Serializer): where `length` is a 32-bit integer and data is `length` bytes. """ - def __init__(self): - # On Python 2.6, we can't write bytearrays to streams, so we need to convert them - # to strings first. Check if the version number is that old. - self._only_write_strings = sys.version_info[0:2] <= (2, 6) - def dump_stream(self, iterator, stream): for obj in iterator: self._write_with_length(obj, stream) @@ -155,10 +143,7 @@ def _write_with_length(self, obj, stream): if len(serialized) > (1 << 31): raise ValueError("can not serialize object larger than 2G") write_int(len(serialized), stream) - if self._only_write_strings: - stream.write(str(serialized)) - else: - stream.write(serialized) + stream.write(serialized) def _read_with_length(self, stream): length = read_int(stream) @@ -204,7 +189,7 @@ def _batched(self, iterator): yield list(iterator) elif hasattr(iterator, "__len__") and hasattr(iterator, "__getslice__"): n = len(iterator) - for i in xrange(0, n, self.batchSize): + for i in range(0, n, self.batchSize): yield iterator[i: i + self.batchSize] else: items = [] @@ -395,23 +380,8 @@ def _copy_func(f): return types.FunctionType(f.__code__, f.__globals__, f.__name__, f.__defaults__, f.__closure__) - def _kwdefaults(f): - # __kwdefaults__ contains the default values of keyword-only arguments which are - # introduced from Python 3. The possible cases for __kwdefaults__ in namedtuple - # are as below: - # - # - Does not exist in Python 2. - # - Returns None in <= Python 3.5.x. - # - Returns a dictionary containing the default values to the keys from Python 3.6.x - # (See https://bugs.python.org/issue25628). - kargs = getattr(f, "__kwdefaults__", None) - if kargs is None: - return {} - else: - return kargs - _old_namedtuple = _copy_func(collections.namedtuple) - _old_namedtuple_kwdefaults = _kwdefaults(collections.namedtuple) + _old_namedtuple_kwdefaults = collections.namedtuple.__kwdefaults__ def namedtuple(*args, **kwargs): for k, v in _old_namedtuple_kwdefaults.items(): @@ -453,12 +423,8 @@ class PickleSerializer(FramedSerializer): def dumps(self, obj): return pickle.dumps(obj, pickle_protocol) - if sys.version >= '3': - def loads(self, obj, encoding="bytes"): - return pickle.loads(obj, encoding=encoding) - else: - def loads(self, obj, encoding=None): - return pickle.loads(obj) + def loads(self, obj, encoding="bytes"): + return pickle.loads(obj, encoding=encoding) class CloudPickleSerializer(PickleSerializer): @@ -469,7 +435,7 @@ def dumps(self, obj): except pickle.PickleError: raise except Exception as e: - emsg = _exception_message(e) + emsg = str(e) if "'i' format requires" in emsg: msg = "Object too large to serialize: %s" % emsg else: diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 65e3bdbc05ce8..cde163bd2d73d 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -26,11 +26,8 @@ import platform import warnings -import py4j - -from pyspark import SparkConf from pyspark.context import SparkContext -from pyspark.sql import SparkSession, SQLContext +from pyspark.sql import SparkSession if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index c28cb8c3b9cbe..af32469e82b43 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -39,9 +39,6 @@ - :class:`pyspark.sql.Window` For working with window functions. """ -from __future__ import absolute_import - - from pyspark.sql.types import Row from pyspark.sql.context import SQLContext, HiveContext, UDFRegistration from pyspark.sql.session import SparkSession diff --git a/python/pyspark/sql/avro/functions.py b/python/pyspark/sql/avro/functions.py index ed62a72d6c8fb..974412ee4efea 100644 --- a/python/pyspark/sql/avro/functions.py +++ b/python/pyspark/sql/avro/functions.py @@ -21,12 +21,10 @@ from pyspark import since, SparkContext -from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.column import Column, _to_java_column from pyspark.util import _print_missing_jar -@ignore_unicode_prefix @since(3.0) def from_avro(data, jsonFormatSchema, options={}): """ @@ -45,7 +43,7 @@ def from_avro(data, jsonFormatSchema, options={}): >>> from pyspark.sql import Row >>> from pyspark.sql.avro.functions import from_avro, to_avro - >>> data = [(1, Row(name='Alice', age=2))] + >>> data = [(1, Row(age=2, name='Alice'))] >>> df = spark.createDataFrame(data, ("key", "value")) >>> avroDf = df.select(to_avro(df.value).alias("avro")) >>> avroDf.collect() @@ -55,7 +53,7 @@ def from_avro(data, jsonFormatSchema, options={}): ... "fields":[{"name":"age","type":["long","null"]}, ... {"name":"name","type":["string","null"]}]},"null"]}]}''' >>> avroDf.select(from_avro(avroDf.avro, jsonFormatSchema).alias("value")).collect() - [Row(value=Row(avro=Row(age=2, name=u'Alice')))] + [Row(value=Row(avro=Row(age=2, name='Alice')))] """ sc = SparkContext._active_spark_context @@ -69,7 +67,6 @@ def from_avro(data, jsonFormatSchema, options={}): return Column(jc) -@ignore_unicode_prefix @since(3.0) def to_avro(data, jsonFormatSchema=""): """ diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 974251f63b37a..25fc696dac051 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -20,10 +20,8 @@ from collections import namedtuple from pyspark import since -from pyspark.rdd import ignore_unicode_prefix, PythonEvalType from pyspark.sql.dataframe import DataFrame -from pyspark.sql.udf import UserDefinedFunction -from pyspark.sql.types import IntegerType, StringType, StructType +from pyspark.sql.types import StructType Database = namedtuple("Database", "name description locationUri") @@ -44,19 +42,16 @@ def __init__(self, sparkSession): self._jsparkSession = sparkSession._jsparkSession self._jcatalog = sparkSession._jsparkSession.catalog() - @ignore_unicode_prefix @since(2.0) def currentDatabase(self): """Returns the current default database in this session.""" return self._jcatalog.currentDatabase() - @ignore_unicode_prefix @since(2.0) def setCurrentDatabase(self, dbName): """Sets the current default database in this session.""" return self._jcatalog.setCurrentDatabase(dbName) - @ignore_unicode_prefix @since(2.0) def listDatabases(self): """Returns a list of databases available across all sessions.""" @@ -70,7 +65,6 @@ def listDatabases(self): locationUri=jdb.locationUri())) return databases - @ignore_unicode_prefix @since(2.0) def listTables(self, dbName=None): """Returns a list of tables/views in the specified database. @@ -92,7 +86,6 @@ def listTables(self, dbName=None): isTemporary=jtable.isTemporary())) return tables - @ignore_unicode_prefix @since(2.0) def listFunctions(self, dbName=None): """Returns a list of functions registered in the specified database. @@ -113,7 +106,6 @@ def listFunctions(self, dbName=None): isTemporary=jfunction.isTemporary())) return functions - @ignore_unicode_prefix @since(2.0) def listColumns(self, tableName, dbName=None): """Returns a list of columns for the given table/view in the specified database. diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index ef4944c9121a4..bd4c35576214e 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -19,15 +19,8 @@ import json import warnings -if sys.version >= '3': - basestring = str - long = int - -from py4j.java_gateway import is_instance_of - from pyspark import copy_func, since from pyspark.context import SparkContext -from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.types import * __all__ = ["Column"] @@ -46,7 +39,7 @@ def _create_column_from_name(name): def _to_java_column(col): if isinstance(col, Column): jcol = col._jc - elif isinstance(col, basestring): + elif isinstance(col, str): jcol = _create_column_from_name(col) else: raise TypeError( @@ -359,7 +352,7 @@ def __iter__(self): :param other: string in line >>> df.filter(df.name.contains('o')).collect() - [Row(age=5, name=u'Bob')] + [Row(age=5, name='Bob')] """ _rlike_doc = """ SQL RLIKE expression (LIKE with Regex). Returns a boolean :class:`Column` based on a regex @@ -368,7 +361,7 @@ def __iter__(self): :param other: an extended regex expression >>> df.filter(df.name.rlike('ice$')).collect() - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] """ _like_doc = """ SQL like expression. Returns a boolean :class:`Column` based on a SQL LIKE match. @@ -378,7 +371,7 @@ def __iter__(self): See :func:`rlike` for a regex version >>> df.filter(df.name.like('Al%')).collect() - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] """ _startswith_doc = """ String starts with. Returns a boolean :class:`Column` based on a string match. @@ -386,7 +379,7 @@ def __iter__(self): :param other: string at start of line (do not use a regex `^`) >>> df.filter(df.name.startswith('Al')).collect() - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] >>> df.filter(df.name.startswith('^Al')).collect() [] """ @@ -396,18 +389,17 @@ def __iter__(self): :param other: string at end of line (do not use a regex `$`) >>> df.filter(df.name.endswith('ice')).collect() - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] >>> df.filter(df.name.endswith('ice$')).collect() [] """ - contains = ignore_unicode_prefix(_bin_op("contains", _contains_doc)) - rlike = ignore_unicode_prefix(_bin_op("rlike", _rlike_doc)) - like = ignore_unicode_prefix(_bin_op("like", _like_doc)) - startswith = ignore_unicode_prefix(_bin_op("startsWith", _startswith_doc)) - endswith = ignore_unicode_prefix(_bin_op("endsWith", _endswith_doc)) + contains = _bin_op("contains", _contains_doc) + rlike = _bin_op("rlike", _rlike_doc) + like = _bin_op("like", _like_doc) + startswith = _bin_op("startsWith", _startswith_doc) + endswith = _bin_op("endsWith", _endswith_doc) - @ignore_unicode_prefix @since(1.3) def substr(self, startPos, length): """ @@ -417,7 +409,7 @@ def substr(self, startPos, length): :param length: length of the substring (int or Column) >>> df.select(df.name.substr(1, 3).alias("col")).collect() - [Row(col=u'Ali'), Row(col=u'Bob')] + [Row(col='Ali'), Row(col='Bob')] """ if type(startPos) != type(length): raise TypeError( @@ -435,7 +427,6 @@ def substr(self, startPos, length): raise TypeError("Unexpected type: %s" % type(startPos)) return Column(jc) - @ignore_unicode_prefix @since(1.5) def isin(self, *cols): """ @@ -443,9 +434,9 @@ def isin(self, *cols): expression is contained by the evaluated values of the arguments. >>> df[df.name.isin("Bob", "Mike")].collect() - [Row(age=5, name=u'Bob')] + [Row(age=5, name='Bob')] >>> df[df.age.isin([1, 2, 3])].collect() - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] """ if len(cols) == 1 and isinstance(cols[0], (list, set)): cols = cols[0] @@ -461,7 +452,7 @@ def isin(self, *cols): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.asc()).collect() - [Row(name=u'Alice'), Row(name=u'Tom')] + [Row(name='Alice'), Row(name='Tom')] """ _asc_nulls_first_doc = """ Returns a sort expression based on ascending order of the column, and null values @@ -470,7 +461,7 @@ def isin(self, *cols): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.asc_nulls_first()).collect() - [Row(name=None), Row(name=u'Alice'), Row(name=u'Tom')] + [Row(name=None), Row(name='Alice'), Row(name='Tom')] .. versionadded:: 2.4 """ @@ -481,7 +472,7 @@ def isin(self, *cols): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.asc_nulls_last()).collect() - [Row(name=u'Alice'), Row(name=u'Tom'), Row(name=None)] + [Row(name='Alice'), Row(name='Tom'), Row(name=None)] .. versionadded:: 2.4 """ @@ -491,7 +482,7 @@ def isin(self, *cols): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.desc()).collect() - [Row(name=u'Tom'), Row(name=u'Alice')] + [Row(name='Tom'), Row(name='Alice')] """ _desc_nulls_first_doc = """ Returns a sort expression based on the descending order of the column, and null values @@ -500,7 +491,7 @@ def isin(self, *cols): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.desc_nulls_first()).collect() - [Row(name=None), Row(name=u'Tom'), Row(name=u'Alice')] + [Row(name=None), Row(name='Tom'), Row(name='Alice')] .. versionadded:: 2.4 """ @@ -511,37 +502,37 @@ def isin(self, *cols): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.desc_nulls_last()).collect() - [Row(name=u'Tom'), Row(name=u'Alice'), Row(name=None)] + [Row(name='Tom'), Row(name='Alice'), Row(name=None)] .. versionadded:: 2.4 """ - asc = ignore_unicode_prefix(_unary_op("asc", _asc_doc)) - asc_nulls_first = ignore_unicode_prefix(_unary_op("asc_nulls_first", _asc_nulls_first_doc)) - asc_nulls_last = ignore_unicode_prefix(_unary_op("asc_nulls_last", _asc_nulls_last_doc)) - desc = ignore_unicode_prefix(_unary_op("desc", _desc_doc)) - desc_nulls_first = ignore_unicode_prefix(_unary_op("desc_nulls_first", _desc_nulls_first_doc)) - desc_nulls_last = ignore_unicode_prefix(_unary_op("desc_nulls_last", _desc_nulls_last_doc)) + asc = _unary_op("asc", _asc_doc) + asc_nulls_first = _unary_op("asc_nulls_first", _asc_nulls_first_doc) + asc_nulls_last = _unary_op("asc_nulls_last", _asc_nulls_last_doc) + desc = _unary_op("desc", _desc_doc) + desc_nulls_first = _unary_op("desc_nulls_first", _desc_nulls_first_doc) + desc_nulls_last = _unary_op("desc_nulls_last", _desc_nulls_last_doc) _isNull_doc = """ True if the current expression is null. >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)]) + >>> df = spark.createDataFrame([Row(name='Tom', height=80), Row(name='Alice', height=None)]) >>> df.filter(df.height.isNull()).collect() - [Row(height=None, name=u'Alice')] + [Row(name='Alice', height=None)] """ _isNotNull_doc = """ True if the current expression is NOT null. >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)]) + >>> df = spark.createDataFrame([Row(name='Tom', height=80), Row(name='Alice', height=None)]) >>> df.filter(df.height.isNotNull()).collect() - [Row(height=80, name=u'Tom')] + [Row(name='Tom', height=80)] """ - isNull = ignore_unicode_prefix(_unary_op("isNull", _isNull_doc)) - isNotNull = ignore_unicode_prefix(_unary_op("isNotNull", _isNotNull_doc)) + isNull = _unary_op("isNull", _isNull_doc) + isNotNull = _unary_op("isNotNull", _isNotNull_doc) @since(1.3) def alias(self, *alias, **kwargs): @@ -581,17 +572,16 @@ def alias(self, *alias, **kwargs): name = copy_func(alias, sinceversion=2.0, doc=":func:`name` is an alias for :func:`alias`.") - @ignore_unicode_prefix @since(1.3) def cast(self, dataType): """ Convert the column into type ``dataType``. >>> df.select(df.age.cast("string").alias('ages')).collect() - [Row(ages=u'2'), Row(ages=u'5')] + [Row(ages='2'), Row(ages='5')] >>> df.select(df.age.cast(StringType()).alias('ages')).collect() - [Row(ages=u'2'), Row(ages=u'5')] + [Row(ages='2'), Row(ages='5')] """ - if isinstance(dataType, basestring): + if isinstance(dataType, str): jc = self._jc.cast(dataType) elif isinstance(dataType, DataType): from pyspark.sql import SparkSession diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index 71ea1631718f1..eab084a1faddf 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -18,10 +18,6 @@ import sys from pyspark import since, _NoValue -from pyspark.rdd import ignore_unicode_prefix - -if sys.version_info[0] >= 3: - basestring = str class RuntimeConfig(object): @@ -34,13 +30,11 @@ def __init__(self, jconf): """Create a new RuntimeConfig that wraps the underlying JVM object.""" self._jconf = jconf - @ignore_unicode_prefix @since(2.0) def set(self, key, value): """Sets the given Spark runtime configuration property.""" self._jconf.set(key, value) - @ignore_unicode_prefix @since(2.0) def get(self, key, default=_NoValue): """Returns the value of Spark runtime configuration property for the given key, @@ -54,7 +48,6 @@ def get(self, key, default=_NoValue): self._checkType(default, "default") return self._jconf.get(key, default) - @ignore_unicode_prefix @since(2.0) def unset(self, key): """Resets the configuration property for the given key.""" @@ -62,11 +55,10 @@ def unset(self, key): def _checkType(self, obj, identifier): """Assert that an object is of type str.""" - if not isinstance(obj, basestring): + if not isinstance(obj, str): raise TypeError("expected %s '%s' to be a string (was '%s')" % (identifier, obj, type(obj).__name__)) - @ignore_unicode_prefix @since(2.4) def isModifiable(self, key): """Indicates whether the configuration property with the given key diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 956343a2310b8..7fbcf85cb1d50 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -15,15 +15,10 @@ # limitations under the License. # -from __future__ import print_function import sys import warnings -if sys.version >= '3': - basestring = unicode = str - from pyspark import since, _NoValue -from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.session import _monkey_patch_RDD, SparkSession from pyspark.sql.dataframe import DataFrame from pyspark.sql.readwriter import DataFrameReader @@ -52,7 +47,6 @@ class SQLContext(object): _instantiatedContext = None - @ignore_unicode_prefix def __init__(self, sparkContext, sparkSession=None, jsqlContext=None): """Creates a new SQLContext. @@ -70,7 +64,7 @@ def __init__(self, sparkContext, sparkSession=None, jsqlContext=None): [Row((i + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=2.0, (NOT b)=False, list[1]=2, \ dict[s]=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] >>> df.rdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() - [(1, u'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] + [(1, 'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ warnings.warn( "Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.", @@ -142,7 +136,6 @@ def setConf(self, key, value): """ self.sparkSession.conf.set(key, value) - @ignore_unicode_prefix @since(1.3) def getConf(self, key, defaultValue=_NoValue): """Returns the value of Spark SQL configuration property for the given key. @@ -152,12 +145,12 @@ def getConf(self, key, defaultValue=_NoValue): the system default value. >>> sqlContext.getConf("spark.sql.shuffle.partitions") - u'200' - >>> sqlContext.getConf("spark.sql.shuffle.partitions", u"10") - u'10' - >>> sqlContext.setConf("spark.sql.shuffle.partitions", u"50") - >>> sqlContext.getConf("spark.sql.shuffle.partitions", u"10") - u'50' + '200' + >>> sqlContext.getConf("spark.sql.shuffle.partitions", "10") + '10' + >>> sqlContext.setConf("spark.sql.shuffle.partitions", "50") + >>> sqlContext.getConf("spark.sql.shuffle.partitions", "10") + '50' """ return self.sparkSession.conf.get(key, defaultValue) @@ -229,7 +222,6 @@ def _inferSchema(self, rdd, samplingRatio=None): return self.sparkSession._inferSchema(rdd, samplingRatio) @since(1.3) - @ignore_unicode_prefix def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=True): """ Creates a :class:`DataFrame` from an :class:`RDD`, a list or a :class:`pandas.DataFrame`. @@ -274,27 +266,27 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr >>> l = [('Alice', 1)] >>> sqlContext.createDataFrame(l).collect() - [Row(_1=u'Alice', _2=1)] + [Row(_1='Alice', _2=1)] >>> sqlContext.createDataFrame(l, ['name', 'age']).collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> d = [{'name': 'Alice', 'age': 1}] >>> sqlContext.createDataFrame(d).collect() - [Row(age=1, name=u'Alice')] + [Row(age=1, name='Alice')] >>> rdd = sc.parallelize(l) >>> sqlContext.createDataFrame(rdd).collect() - [Row(_1=u'Alice', _2=1)] + [Row(_1='Alice', _2=1)] >>> df = sqlContext.createDataFrame(rdd, ['name', 'age']) >>> df.collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> from pyspark.sql import Row >>> Person = Row('name', 'age') >>> person = rdd.map(lambda r: Person(*r)) >>> df2 = sqlContext.createDataFrame(person) >>> df2.collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> from pyspark.sql.types import * >>> schema = StructType([ @@ -302,15 +294,15 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr ... StructField("age", IntegerType(), True)]) >>> df3 = sqlContext.createDataFrame(rdd, schema) >>> df3.collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> sqlContext.createDataFrame(df.toPandas()).collect() # doctest: +SKIP - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> sqlContext.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP [Row(0=1, 1=2)] >>> sqlContext.createDataFrame(rdd, "a: string, b: int").collect() - [Row(a=u'Alice', b=1)] + [Row(a='Alice', b=1)] >>> rdd = rdd.map(lambda row: row[1]) >>> sqlContext.createDataFrame(rdd, "int").collect() [Row(value=1)] @@ -358,7 +350,6 @@ def createExternalTable(self, tableName, path=None, source=None, schema=None, ** return self.sparkSession.catalog.createExternalTable( tableName, path, source, schema, **options) - @ignore_unicode_prefix @since(1.0) def sql(self, sqlQuery): """Returns a :class:`DataFrame` representing the result of the given query. @@ -368,7 +359,7 @@ def sql(self, sqlQuery): >>> sqlContext.registerDataFrameAsTable(df, "table1") >>> df2 = sqlContext.sql("SELECT field1 AS f1, field2 as f2 from table1") >>> df2.collect() - [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] + [Row(f1=1, f2='row1'), Row(f1=2, f2='row2'), Row(f1=3, f2='row3')] """ return self.sparkSession.sql(sqlQuery) @@ -385,7 +376,6 @@ def table(self, tableName): """ return self.sparkSession.table(tableName) - @ignore_unicode_prefix @since(1.3) def tables(self, dbName=None): """Returns a :class:`DataFrame` containing names of tables in the given database. @@ -401,7 +391,7 @@ def tables(self, dbName=None): >>> sqlContext.registerDataFrameAsTable(df, "table1") >>> df2 = sqlContext.tables() >>> df2.filter("tableName = 'table1'").first() - Row(database=u'', tableName=u'table1', isTemporary=True) + Row(database='', tableName='table1', isTemporary=True) """ if dbName is None: return DataFrame(self._ssql_ctx.tables(), self) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 3ad899bcc3670..023fbeabcbabc 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -17,21 +17,12 @@ import sys import random - -if sys.version >= '3': - basestring = unicode = str - long = int - from functools import reduce - from html import escape as html_escape -else: - from itertools import imap as map - from cgi import escape as html_escape - import warnings +from functools import reduce +from html import escape as html_escape from pyspark import copy_func, since, _NoValue -from pyspark.rdd import RDD, _load_from_socket, _local_iterator_from_socket, \ - ignore_unicode_prefix +from pyspark.rdd import RDD, _load_from_socket, _local_iterator_from_socket from pyspark.serializers import BatchedSerializer, PickleSerializer, \ UTF8Deserializer from pyspark.storagelevel import StorageLevel @@ -109,7 +100,6 @@ def stat(self): """ return DataFrameStatFunctions(self) - @ignore_unicode_prefix @since(1.3) def toJSON(self, use_unicode=True): """Converts a :class:`DataFrame` into a :class:`RDD` of string. @@ -117,7 +107,7 @@ def toJSON(self, use_unicode=True): Each row is turned into a JSON document as one element in the returned RDD. >>> df.toJSON().first() - u'{"age":2,"name":"Alice"}' + '{"age":2,"name":"Alice"}' """ rdd = self._jdf.toJSON() return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) @@ -330,11 +320,11 @@ def explain(self, extended=None, mode=None): # For the case when extended is mode: # df.explain("formatted") - is_extended_as_mode = isinstance(extended, basestring) and mode is None + is_extended_as_mode = isinstance(extended, str) and mode is None # For the mode specified: # df.explain(mode="formatted") - is_mode_case = extended is None and isinstance(mode, basestring) + is_mode_case = extended is None and isinstance(mode, str) if not (is_no_argument or is_extended_case or is_extended_as_mode or is_mode_case): argtypes = [ @@ -568,7 +558,7 @@ def hint(self, name, *parameters): if not isinstance(name, str): raise TypeError("name should be provided as str, got {0}".format(type(name))) - allowed_types = (basestring, list, float, int) + allowed_types = (str, list, float, int) for p in parameters: if not isinstance(p, allowed_types): raise TypeError( @@ -587,19 +577,17 @@ def count(self): """ return int(self._jdf.count()) - @ignore_unicode_prefix @since(1.3) def collect(self): """Returns all the records as a list of :class:`Row`. >>> df.collect() - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + [Row(age=2, name='Alice'), Row(age=5, name='Bob')] """ with SCCallSiteSync(self._sc) as css: sock_info = self._jdf.collectToPython() return list(_load_from_socket(sock_info, BatchedSerializer(PickleSerializer()))) - @ignore_unicode_prefix @since(2.0) def toLocalIterator(self, prefetchPartitions=False): """ @@ -612,36 +600,33 @@ def toLocalIterator(self, prefetchPartitions=False): before it is needed. >>> list(df.toLocalIterator()) - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + [Row(age=2, name='Alice'), Row(age=5, name='Bob')] """ with SCCallSiteSync(self._sc) as css: sock_info = self._jdf.toPythonIterator(prefetchPartitions) return _local_iterator_from_socket(sock_info, BatchedSerializer(PickleSerializer())) - @ignore_unicode_prefix @since(1.3) def limit(self, num): """Limits the result count to the number specified. >>> df.limit(1).collect() - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] >>> df.limit(0).collect() [] """ jdf = self._jdf.limit(num) return DataFrame(jdf, self.sql_ctx) - @ignore_unicode_prefix @since(1.3) def take(self, num): """Returns the first ``num`` rows as a :class:`list` of :class:`Row`. >>> df.take(2) - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + [Row(age=2, name='Alice'), Row(age=5, name='Bob')] """ return self.limit(num).collect() - @ignore_unicode_prefix @since(3.0) def tail(self, num): """ @@ -651,7 +636,7 @@ def tail(self, num): a very large ``num`` can crash the driver process with OutOfMemoryError. >>> df.tail(1) - [Row(age=5, name=u'Bob')] + [Row(age=5, name='Bob')] """ with SCCallSiteSync(self._sc): sock_info = self._jdf.tailToPython(num) @@ -818,7 +803,7 @@ def repartition(self, numPartitions, *cols): else: return DataFrame( self._jdf.repartition(numPartitions, self._jcols(*cols)), self.sql_ctx) - elif isinstance(numPartitions, (basestring, Column)): + elif isinstance(numPartitions, (str, Column)): cols = (numPartitions, ) + cols return DataFrame(self._jdf.repartition(self._jcols(*cols)), self.sql_ctx) else: @@ -869,7 +854,7 @@ def repartitionByRange(self, numPartitions, *cols): else: return DataFrame( self._jdf.repartitionByRange(numPartitions, self._jcols(*cols)), self.sql_ctx) - elif isinstance(numPartitions, (basestring, Column)): + elif isinstance(numPartitions, (str, Column)): cols = (numPartitions,) + cols return DataFrame(self._jdf.repartitionByRange(self._jcols(*cols)), self.sql_ctx) else: @@ -944,7 +929,7 @@ def sample(self, withReplacement=None, fraction=None, seed=None): fraction = withReplacement withReplacement = None - seed = long(seed) if seed is not None else None + seed = int(seed) if seed is not None else None args = [arg for arg in [withReplacement, fraction, seed] if arg is not None] jdf = self._jdf.sample(*args) return DataFrame(jdf, self.sql_ctx) @@ -978,15 +963,15 @@ def sampleBy(self, col, fractions, seed=None): .. versionchanged:: 3.0 Added sampling by a column of :class:`Column` """ - if isinstance(col, basestring): + if isinstance(col, str): col = Column(col) elif not isinstance(col, Column): raise ValueError("col must be a string or a column, but got %r" % type(col)) if not isinstance(fractions, dict): raise ValueError("fractions must be a dict but got %r" % type(fractions)) for k, v in fractions.items(): - if not isinstance(k, (float, int, long, basestring)): - raise ValueError("key must be float, int, long, or string, but got %r" % type(k)) + if not isinstance(k, (float, int, str)): + raise ValueError("key must be float, int, or string, but got %r" % type(k)) fractions[k] = float(v) col = col._jc seed = seed if seed is not None else random.randint(0, sys.maxsize) @@ -1011,7 +996,7 @@ def randomSplit(self, weights, seed=None): if w < 0.0: raise ValueError("Weights must be positive. Found weight value: %s" % w) seed = seed if seed is not None else random.randint(0, sys.maxsize) - rdd_array = self._jdf.randomSplit(_to_list(self.sql_ctx._sc, weights), long(seed)) + rdd_array = self._jdf.randomSplit(_to_list(self.sql_ctx._sc, weights), int(seed)) return [DataFrame(rdd, self.sql_ctx) for rdd in rdd_array] @property @@ -1052,12 +1037,11 @@ def colRegex(self, colName): | 3| +----+ """ - if not isinstance(colName, basestring): + if not isinstance(colName, str): raise ValueError("colName should be provided as string") jc = self._jdf.colRegex(colName) return Column(jc) - @ignore_unicode_prefix @since(1.3) def alias(self, alias): """Returns a new :class:`DataFrame` with an alias set. @@ -1070,12 +1054,11 @@ def alias(self, alias): >>> joined_df = df_as1.join(df_as2, col("df_as1.name") == col("df_as2.name"), 'inner') >>> joined_df.select("df_as1.name", "df_as2.name", "df_as2.age") \ .sort(desc("df_as1.name")).collect() - [Row(name=u'Bob', name=u'Bob', age=5), Row(name=u'Alice', name=u'Alice', age=2)] + [Row(name='Bob', name='Bob', age=5), Row(name='Alice', name='Alice', age=2)] """ - assert isinstance(alias, basestring), "alias should be a string" + assert isinstance(alias, str), "alias should be a string" return DataFrame(getattr(self._jdf, "as")(alias), self.sql_ctx) - @ignore_unicode_prefix @since(2.1) def crossJoin(self, other): """Returns the cartesian product with another :class:`DataFrame`. @@ -1083,18 +1066,17 @@ def crossJoin(self, other): :param other: Right side of the cartesian product. >>> df.select("age", "name").collect() - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + [Row(age=2, name='Alice'), Row(age=5, name='Bob')] >>> df2.select("name", "height").collect() - [Row(name=u'Tom', height=80), Row(name=u'Bob', height=85)] + [Row(name='Tom', height=80), Row(name='Bob', height=85)] >>> df.crossJoin(df2.select("height")).select("age", "name", "height").collect() - [Row(age=2, name=u'Alice', height=80), Row(age=2, name=u'Alice', height=85), - Row(age=5, name=u'Bob', height=80), Row(age=5, name=u'Bob', height=85)] + [Row(age=2, name='Alice', height=80), Row(age=2, name='Alice', height=85), + Row(age=5, name='Bob', height=80), Row(age=5, name='Bob', height=85)] """ jdf = self._jdf.crossJoin(other._jdf) return DataFrame(jdf, self.sql_ctx) - @ignore_unicode_prefix @since(1.3) def join(self, other, on=None, how=None): """Joins with another :class:`DataFrame`, using the given join expression. @@ -1113,27 +1095,27 @@ def join(self, other, on=None, how=None): >>> from pyspark.sql.functions import desc >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height) \ .sort(desc("name")).collect() - [Row(name=u'Bob', height=85), Row(name=u'Alice', height=None), Row(name=None, height=80)] + [Row(name='Bob', height=85), Row(name='Alice', height=None), Row(name=None, height=80)] >>> df.join(df2, 'name', 'outer').select('name', 'height').sort(desc("name")).collect() - [Row(name=u'Tom', height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] + [Row(name='Tom', height=80), Row(name='Bob', height=85), Row(name='Alice', height=None)] >>> cond = [df.name == df3.name, df.age == df3.age] >>> df.join(df3, cond, 'outer').select(df.name, df3.age).collect() - [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] + [Row(name='Alice', age=2), Row(name='Bob', age=5)] >>> df.join(df2, 'name').select(df.name, df2.height).collect() - [Row(name=u'Bob', height=85)] + [Row(name='Bob', height=85)] >>> df.join(df4, ['name', 'age']).select(df.name, df.age).collect() - [Row(name=u'Bob', age=5)] + [Row(name='Bob', age=5)] """ if on is not None and not isinstance(on, list): on = [on] if on is not None: - if isinstance(on[0], basestring): + if isinstance(on[0], str): on = self._jseq(on) else: assert isinstance(on[0], Column), "on should be Column or list of Column" @@ -1147,7 +1129,7 @@ def join(self, other, on=None, how=None): how = "inner" if on is None: on = self._jseq([]) - assert isinstance(how, basestring), "how should be basestring" + assert isinstance(how, str), "how should be a string" jdf = self._jdf.join(other._jdf, on, how) return DataFrame(jdf, self.sql_ctx) @@ -1171,7 +1153,6 @@ def sortWithinPartitions(self, *cols, **kwargs): jdf = self._jdf.sortWithinPartitions(self._sort_cols(cols, kwargs)) return DataFrame(jdf, self.sql_ctx) - @ignore_unicode_prefix @since(1.3) def sort(self, *cols, **kwargs): """Returns a new :class:`DataFrame` sorted by the specified column(s). @@ -1182,18 +1163,18 @@ def sort(self, *cols, **kwargs): If a list is specified, length of the list must equal length of the `cols`. >>> df.sort(df.age.desc()).collect() - [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + [Row(age=5, name='Bob'), Row(age=2, name='Alice')] >>> df.sort("age", ascending=False).collect() - [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + [Row(age=5, name='Bob'), Row(age=2, name='Alice')] >>> df.orderBy(df.age.desc()).collect() - [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + [Row(age=5, name='Bob'), Row(age=2, name='Alice')] >>> from pyspark.sql.functions import * >>> df.sort(asc("age")).collect() - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + [Row(age=2, name='Alice'), Row(age=5, name='Bob')] >>> df.orderBy(desc("age"), "name").collect() - [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + [Row(age=5, name='Bob'), Row(age=2, name='Alice')] >>> df.orderBy(["age", "name"], ascending=[0, 1]).collect() - [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + [Row(age=5, name='Bob'), Row(age=2, name='Alice')] """ jdf = self._jdf.sort(self._sort_cols(cols, kwargs)) return DataFrame(jdf, self.sql_ctx) @@ -1333,7 +1314,6 @@ def summary(self, *statistics): jdf = self._jdf.summary(self._jseq(statistics)) return DataFrame(jdf, self.sql_ctx) - @ignore_unicode_prefix @since(1.3) def head(self, n=None): """Returns the first ``n`` rows. @@ -1346,26 +1326,24 @@ def head(self, n=None): If n is 1, return a single Row. >>> df.head() - Row(age=2, name=u'Alice') + Row(age=2, name='Alice') >>> df.head(1) - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] """ if n is None: rs = self.head(1) return rs[0] if rs else None return self.take(n) - @ignore_unicode_prefix @since(1.3) def first(self): """Returns the first row as a :class:`Row`. >>> df.first() - Row(age=2, name=u'Alice') + Row(age=2, name='Alice') """ return self.head() - @ignore_unicode_prefix @since(1.3) def __getitem__(self, item): """Returns the column as a :class:`Column`. @@ -1373,13 +1351,13 @@ def __getitem__(self, item): >>> df.select(df['age']).collect() [Row(age=2), Row(age=5)] >>> df[ ["name", "age"]].collect() - [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] + [Row(name='Alice', age=2), Row(name='Bob', age=5)] >>> df[ df.age > 3 ].collect() - [Row(age=5, name=u'Bob')] + [Row(age=5, name='Bob')] >>> df[df[0] > 3].collect() - [Row(age=5, name=u'Bob')] + [Row(age=5, name='Bob')] """ - if isinstance(item, basestring): + if isinstance(item, str): jc = self._jdf.apply(item) return Column(jc) elif isinstance(item, Column): @@ -1405,7 +1383,6 @@ def __getattr__(self, name): jc = self._jdf.apply(name) return Column(jc) - @ignore_unicode_prefix @since(1.3) def select(self, *cols): """Projects a set of expressions and returns a new :class:`DataFrame`. @@ -1415,11 +1392,11 @@ def select(self, *cols): in the current :class:`DataFrame`. >>> df.select('*').collect() - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + [Row(age=2, name='Alice'), Row(age=5, name='Bob')] >>> df.select('name', 'age').collect() - [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] + [Row(name='Alice', age=2), Row(name='Bob', age=5)] >>> df.select(df.name, (df.age + 10).alias('age')).collect() - [Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)] + [Row(name='Alice', age=12), Row(name='Bob', age=15)] """ jdf = self._jdf.select(self._jcols(*cols)) return DataFrame(jdf, self.sql_ctx) @@ -1438,7 +1415,6 @@ def selectExpr(self, *expr): jdf = self._jdf.selectExpr(self._jseq(expr)) return DataFrame(jdf, self.sql_ctx) - @ignore_unicode_prefix @since(1.3) def filter(self, condition): """Filters rows using the given condition. @@ -1449,16 +1425,16 @@ def filter(self, condition): or a string of SQL expression. >>> df.filter(df.age > 3).collect() - [Row(age=5, name=u'Bob')] + [Row(age=5, name='Bob')] >>> df.where(df.age == 2).collect() - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] >>> df.filter("age > 3").collect() - [Row(age=5, name=u'Bob')] + [Row(age=5, name='Bob')] >>> df.where("age = 2").collect() - [Row(age=2, name=u'Alice')] + [Row(age=2, name='Alice')] """ - if isinstance(condition, basestring): + if isinstance(condition, str): jdf = self._jdf.filter(condition) elif isinstance(condition, Column): jdf = self._jdf.filter(condition._jc) @@ -1466,7 +1442,6 @@ def filter(self, condition): raise TypeError("condition should be string or Column") return DataFrame(jdf, self.sql_ctx) - @ignore_unicode_prefix @since(1.3) def groupBy(self, *cols): """Groups the :class:`DataFrame` using the specified columns, @@ -1481,11 +1456,11 @@ def groupBy(self, *cols): >>> df.groupBy().avg().collect() [Row(avg(age)=3.5)] >>> sorted(df.groupBy('name').agg({'age': 'mean'}).collect()) - [Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)] + [Row(name='Alice', avg(age)=2.0), Row(name='Bob', avg(age)=5.0)] >>> sorted(df.groupBy(df.name).avg().collect()) - [Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)] + [Row(name='Alice', avg(age)=2.0), Row(name='Bob', avg(age)=5.0)] >>> sorted(df.groupBy(['name', df.age]).count().collect()) - [Row(name=u'Alice', age=2, count=1), Row(name=u'Bob', age=5, count=1)] + [Row(name='Alice', age=2, count=1), Row(name='Bob', age=5, count=1)] """ jgd = self._jdf.groupBy(self._jcols(*cols)) from pyspark.sql.group import GroupedData @@ -1655,19 +1630,19 @@ def dropDuplicates(self, subset=None): ... Row(name='Alice', age=5, height=80), \\ ... Row(name='Alice', age=10, height=80)]).toDF() >>> df.dropDuplicates().show() - +---+------+-----+ - |age|height| name| - +---+------+-----+ - | 5| 80|Alice| - | 10| 80|Alice| - +---+------+-----+ + +-----+---+------+ + | name|age|height| + +-----+---+------+ + |Alice| 5| 80| + |Alice| 10| 80| + +-----+---+------+ >>> df.dropDuplicates(['name', 'height']).show() - +---+------+-----+ - |age|height| name| - +---+------+-----+ - | 5| 80|Alice| - +---+------+-----+ + +-----+---+------+ + | name|age|height| + +-----+---+------+ + |Alice| 5| 80| + +-----+---+------+ """ if subset is None: jdf = self._jdf.dropDuplicates() @@ -1700,7 +1675,7 @@ def dropna(self, how='any', thresh=None, subset=None): if subset is None: subset = self.columns - elif isinstance(subset, basestring): + elif isinstance(subset, str): subset = [subset] elif not isinstance(subset, (list, tuple)): raise ValueError("subset should be a list or tuple of column names") @@ -1715,11 +1690,11 @@ def fillna(self, value, subset=None): """Replace null values, alias for ``na.fill()``. :func:`DataFrame.fillna` and :func:`DataFrameNaFunctions.fill` are aliases of each other. - :param value: int, long, float, string, bool or dict. + :param value: int, float, string, bool or dict. Value to replace null values with. If the value is a dict, then `subset` is ignored and `value` must be a mapping from column name (string) to replacement value. The replacement value must be - an int, long, float, boolean, or string. + an int, float, boolean, or string. :param subset: optional list of column names to consider. Columns specified in subset that do not have matching data type are ignored. For example, if `value` is a string, and subset contains a non-string column, @@ -1754,13 +1729,13 @@ def fillna(self, value, subset=None): | 50| null|unknown| +---+------+-------+ """ - if not isinstance(value, (float, int, long, basestring, bool, dict)): - raise ValueError("value should be a float, int, long, string, bool or dict") + if not isinstance(value, (float, int, str, bool, dict)): + raise ValueError("value should be a float, int, string, bool or dict") # Note that bool validates isinstance(int), but we don't want to # convert bools to floats - if not isinstance(value, bool) and isinstance(value, (int, long)): + if not isinstance(value, bool) and isinstance(value, int): value = float(value) if isinstance(value, dict): @@ -1768,7 +1743,7 @@ def fillna(self, value, subset=None): elif subset is None: return DataFrame(self._jdf.na().fill(value), self.sql_ctx) else: - if isinstance(subset, basestring): + if isinstance(subset, str): subset = [subset] elif not isinstance(subset, (list, tuple)): raise ValueError("subset should be a list or tuple of column names") @@ -1787,12 +1762,12 @@ def replace(self, to_replace, value=_NoValue, subset=None): floating point representation. In case of conflicts (for example with `{42: -1, 42.0: 1}`) and arbitrary replacement will be used. - :param to_replace: bool, int, long, float, string, list or dict. + :param to_replace: bool, int, float, string, list or dict. Value to be replaced. If the value is a dict, then `value` is ignored or can be omitted, and `to_replace` must be a mapping between a value and a replacement. - :param value: bool, int, long, float, string, list or None. - The replacement value must be a bool, int, long, float, string or None. If `value` is a + :param value: bool, int, float, string, list or None. + The replacement value must be a bool, int, float, string or None. If `value` is a list, `value` should be of the same length and type as `to_replace`. If `value` is a scalar and `to_replace` is a sequence, then `value` is used as a replacement for each item in `to_replace`. @@ -1854,7 +1829,7 @@ def all_of(types): >>> all_of(bool)([True, False]) True - >>> all_of(basestring)(["a", 1]) + >>> all_of(str)(["a", 1]) False """ def all_of_(xs): @@ -1862,20 +1837,20 @@ def all_of_(xs): return all_of_ all_of_bool = all_of(bool) - all_of_str = all_of(basestring) - all_of_numeric = all_of((float, int, long)) + all_of_str = all_of(str) + all_of_numeric = all_of((float, int)) # Validate input types - valid_types = (bool, float, int, long, basestring, list, tuple) + valid_types = (bool, float, int, str, list, tuple) if not isinstance(to_replace, valid_types + (dict, )): raise ValueError( - "to_replace should be a bool, float, int, long, string, list, tuple, or dict. " + "to_replace should be a bool, float, int, string, list, tuple, or dict. " "Got {0}".format(type(to_replace))) if not isinstance(value, valid_types) and value is not None \ and not isinstance(to_replace, dict): raise ValueError("If to_replace is not a dict, value should be " - "a bool, float, int, long, string, list, tuple or None. " + "a bool, float, int, string, list, tuple or None. " "Got {0}".format(type(value))) if isinstance(to_replace, (list, tuple)) and isinstance(value, (list, tuple)): @@ -1883,12 +1858,12 @@ def all_of_(xs): raise ValueError("to_replace and value lists should be of the same length. " "Got {0} and {1}".format(len(to_replace), len(value))) - if not (subset is None or isinstance(subset, (list, tuple, basestring))): + if not (subset is None or isinstance(subset, (list, tuple, str))): raise ValueError("subset should be a list or tuple of column names, " "column name or None. Got {0}".format(type(subset))) # Reshape input arguments if necessary - if isinstance(to_replace, (float, int, long, basestring)): + if isinstance(to_replace, (float, int, str)): to_replace = [to_replace] if isinstance(to_replace, dict): @@ -1896,11 +1871,11 @@ def all_of_(xs): if value is not None: warnings.warn("to_replace is a dict and value is not None. value will be ignored.") else: - if isinstance(value, (float, int, long, basestring)) or value is None: + if isinstance(value, (float, int, str)) or value is None: value = [value for _ in range(len(to_replace))] rep_dict = dict(zip(to_replace, value)) - if isinstance(subset, basestring): + if isinstance(subset, str): subset = [subset] # Verify we were not passed in mixed type generics. @@ -1957,10 +1932,10 @@ def approxQuantile(self, col, probabilities, relativeError): Added support for multiple columns. """ - if not isinstance(col, (basestring, list, tuple)): + if not isinstance(col, (str, list, tuple)): raise ValueError("col should be a string, list or tuple, but got %r" % type(col)) - isStr = isinstance(col, basestring) + isStr = isinstance(col, str) if isinstance(col, tuple): col = list(col) @@ -1968,7 +1943,7 @@ def approxQuantile(self, col, probabilities, relativeError): col = [col] for c in col: - if not isinstance(c, basestring): + if not isinstance(c, str): raise ValueError("columns should be strings, but got %r" % type(c)) col = _to_list(self._sc, col) @@ -1977,12 +1952,12 @@ def approxQuantile(self, col, probabilities, relativeError): if isinstance(probabilities, tuple): probabilities = list(probabilities) for p in probabilities: - if not isinstance(p, (float, int, long)) or p < 0 or p > 1: - raise ValueError("probabilities should be numerical (float, int, long) in [0,1].") + if not isinstance(p, (float, int)) or p < 0 or p > 1: + raise ValueError("probabilities should be numerical (float, int) in [0,1].") probabilities = _to_list(self._sc, probabilities) - if not isinstance(relativeError, (float, int, long)) or relativeError < 0: - raise ValueError("relativeError should be numerical (float, int, long) >= 0.") + if not isinstance(relativeError, (float, int)) or relativeError < 0: + raise ValueError("relativeError should be numerical (float, int) >= 0.") relativeError = float(relativeError) jaq = self._jdf.stat().approxQuantile(col, probabilities, relativeError) @@ -2000,9 +1975,9 @@ def corr(self, col1, col2, method=None): :param col2: The name of the second column :param method: The correlation method. Currently only supports "pearson" """ - if not isinstance(col1, basestring): + if not isinstance(col1, str): raise ValueError("col1 should be a string.") - if not isinstance(col2, basestring): + if not isinstance(col2, str): raise ValueError("col2 should be a string.") if not method: method = "pearson" @@ -2020,9 +1995,9 @@ def cov(self, col1, col2): :param col1: The name of the first column :param col2: The name of the second column """ - if not isinstance(col1, basestring): + if not isinstance(col1, str): raise ValueError("col1 should be a string.") - if not isinstance(col2, basestring): + if not isinstance(col2, str): raise ValueError("col2 should be a string.") return self._jdf.stat().cov(col1, col2) @@ -2042,9 +2017,9 @@ def crosstab(self, col1, col2): :param col2: The name of the second column. Distinct items will make the column names of the :class:`DataFrame`. """ - if not isinstance(col1, basestring): + if not isinstance(col1, str): raise ValueError("col1 should be a string.") - if not isinstance(col2, basestring): + if not isinstance(col2, str): raise ValueError("col2 should be a string.") return DataFrame(self._jdf.stat().crosstab(col1, col2), self.sql_ctx) @@ -2073,7 +2048,6 @@ def freqItems(self, cols, support=None): support = 0.01 return DataFrame(self._jdf.stat().freqItems(_to_seq(self._sc, cols), support), self.sql_ctx) - @ignore_unicode_prefix @since(1.3) def withColumn(self, colName, col): """ @@ -2092,13 +2066,12 @@ def withColumn(self, colName, col): To avoid this, use :func:`select` with the multiple columns at once. >>> df.withColumn('age2', df.age + 2).collect() - [Row(age=2, name=u'Alice', age2=4), Row(age=5, name=u'Bob', age2=7)] + [Row(age=2, name='Alice', age2=4), Row(age=5, name='Bob', age2=7)] """ assert isinstance(col, Column), "col should be Column" return DataFrame(self._jdf.withColumn(colName, col._jc), self.sql_ctx) - @ignore_unicode_prefix @since(1.3) def withColumnRenamed(self, existing, new): """Returns a new :class:`DataFrame` by renaming an existing column. @@ -2108,12 +2081,11 @@ def withColumnRenamed(self, existing, new): :param new: string, new name of the column. >>> df.withColumnRenamed('age', 'age2').collect() - [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')] + [Row(age2=2, name='Alice'), Row(age2=5, name='Bob')] """ return DataFrame(self._jdf.withColumnRenamed(existing, new), self.sql_ctx) @since(1.4) - @ignore_unicode_prefix def drop(self, *cols): """Returns a new :class:`DataFrame` that drops the specified column. This is a no-op if schema doesn't contain the given column name(s). @@ -2122,23 +2094,23 @@ def drop(self, *cols): :class:`Column` to drop, or a list of string name of the columns to drop. >>> df.drop('age').collect() - [Row(name=u'Alice'), Row(name=u'Bob')] + [Row(name='Alice'), Row(name='Bob')] >>> df.drop(df.age).collect() - [Row(name=u'Alice'), Row(name=u'Bob')] + [Row(name='Alice'), Row(name='Bob')] >>> df.join(df2, df.name == df2.name, 'inner').drop(df.name).collect() - [Row(age=5, height=85, name=u'Bob')] + [Row(age=5, height=85, name='Bob')] >>> df.join(df2, df.name == df2.name, 'inner').drop(df2.name).collect() - [Row(age=5, name=u'Bob', height=85)] + [Row(age=5, name='Bob', height=85)] >>> df.join(df2, 'name', 'inner').drop('age', 'height').collect() - [Row(name=u'Bob')] + [Row(name='Bob')] """ if len(cols) == 1: col = cols[0] - if isinstance(col, basestring): + if isinstance(col, str): jdf = self._jdf.drop(col) elif isinstance(col, Column): jdf = self._jdf.drop(col._jc) @@ -2146,20 +2118,19 @@ def drop(self, *cols): raise TypeError("col should be a string or a Column") else: for col in cols: - if not isinstance(col, basestring): + if not isinstance(col, str): raise TypeError("each col in the param list should be a string") jdf = self._jdf.drop(self._jseq(cols)) return DataFrame(jdf, self.sql_ctx) - @ignore_unicode_prefix def toDF(self, *cols): """Returns a new :class:`DataFrame` that with new specified column names :param cols: list of new column names (string) >>> df.toDF('f1', 'f2').collect() - [Row(f1=2, f2=u'Alice'), Row(f1=5, f2=u'Bob')] + [Row(f1=2, f2='Alice'), Row(f1=5, f2='Bob')] """ jdf = self._jdf.toDF(self._jseq(cols)) return DataFrame(jdf, self.sql_ctx) @@ -2347,7 +2318,6 @@ def _test(): from pyspark.context import SparkContext from pyspark.sql import Row, SQLContext, SparkSession import pyspark.sql.dataframe - from pyspark.sql.functions import from_unixtime globs = pyspark.sql.dataframe.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc @@ -2356,16 +2326,16 @@ def _test(): globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\ .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())])) - globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() - globs['df3'] = sc.parallelize([Row(name='Alice', age=2), - Row(name='Bob', age=5)]).toDF() - globs['df4'] = sc.parallelize([Row(name='Alice', age=10, height=80), - Row(name='Bob', age=5, height=None), - Row(name='Tom', age=None, height=None), - Row(name=None, age=None, height=None)]).toDF() - globs['df5'] = sc.parallelize([Row(name='Alice', spy=False, age=10), - Row(name='Bob', spy=None, age=5), - Row(name='Mallory', spy=True, age=None)]).toDF() + globs['df2'] = sc.parallelize([Row(height=80, name='Tom'), Row(height=85, name='Bob')]).toDF() + globs['df3'] = sc.parallelize([Row(age=2, name='Alice'), + Row(age=5, name='Bob')]).toDF() + globs['df4'] = sc.parallelize([Row(age=10, height=80, name='Alice'), + Row(age=5, height=None, name='Bob'), + Row(age=None, height=None, name='Tom'), + Row(age=None, height=None, name=None)]).toDF() + globs['df5'] = sc.parallelize([Row(age=10, name='Alice', spy=False), + Row(age=5, name='Bob', spy=None), + Row(age=None, name='Mallory', spy=True)]).toDF() globs['sdf'] = sc.parallelize([Row(name='Tom', time=1479441846), Row(name='Bob', time=1479442946)]).toDF() diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index b5a7c18904b14..63b049999f255 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -22,14 +22,8 @@ import functools import warnings -if sys.version < "3": - from itertools import imap as map - -if sys.version >= '3': - basestring = str - from pyspark import since, SparkContext -from pyspark.rdd import ignore_unicode_prefix, PythonEvalType +from pyspark.rdd import PythonEvalType from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_from_literal, \ _create_column_from_name from pyspark.sql.dataframe import DataFrame @@ -88,14 +82,14 @@ def _(col1, col2): # if they are not columns or strings. if isinstance(col1, Column): arg1 = col1._jc - elif isinstance(col1, basestring): + elif isinstance(col1, str): arg1 = _create_column_from_name(col1) else: arg1 = float(col1) if isinstance(col2, Column): arg2 = col2._jc - elif isinstance(col2, basestring): + elif isinstance(col2, str): arg2 = _create_column_from_name(col2) else: arg2 = float(col2) @@ -648,7 +642,6 @@ def percentile_approx(col, percentage, accuracy=10000): return Column(sc._jvm.functions.percentile_approx(_to_java_column(col), percentage, accuracy)) -@ignore_unicode_prefix @since(1.4) def rand(seed=None): """Generates a random column with independent and identically distributed (i.i.d.) samples @@ -657,8 +650,8 @@ def rand(seed=None): .. note:: The function is non-deterministic in general case. >>> df.withColumn('rand', rand(seed=42) * 3).collect() - [Row(age=2, name=u'Alice', rand=2.4052597283576684), - Row(age=5, name=u'Bob', rand=2.3913904055683974)] + [Row(age=2, name='Alice', rand=2.4052597283576684), + Row(age=5, name='Bob', rand=2.3913904055683974)] """ sc = SparkContext._active_spark_context if seed is not None: @@ -668,7 +661,6 @@ def rand(seed=None): return Column(jc) -@ignore_unicode_prefix @since(1.4) def randn(seed=None): """Generates a column with independent and identically distributed (i.i.d.) samples from @@ -677,8 +669,8 @@ def randn(seed=None): .. note:: The function is non-deterministic in general case. >>> df.withColumn('randn', randn(seed=42)).collect() - [Row(age=2, name=u'Alice', randn=1.1027054481455365), - Row(age=5, name=u'Bob', randn=0.7400395449950132)] + [Row(age=2, name='Alice', randn=1.1027054481455365), + Row(age=5, name='Bob', randn=0.7400395449950132)] """ sc = SparkContext._active_spark_context if seed is not None: @@ -774,7 +766,6 @@ def expr(str): return Column(sc._jvm.functions.expr(str)) -@ignore_unicode_prefix @since(1.4) def struct(*cols): """Creates a new struct column. @@ -782,9 +773,9 @@ def struct(*cols): :param cols: list of column names (string) or list of :class:`Column` expressions >>> df.select(struct('age', 'name').alias("struct")).collect() - [Row(struct=Row(age=2, name=u'Alice')), Row(struct=Row(age=5, name=u'Bob'))] + [Row(struct=Row(age=2, name='Alice')), Row(struct=Row(age=5, name='Bob'))] >>> df.select(struct([df.age, df.name]).alias("struct")).collect() - [Row(struct=Row(age=2, name=u'Alice')), Row(struct=Row(age=5, name=u'Bob'))] + [Row(struct=Row(age=2, name='Alice')), Row(struct=Row(age=5, name='Bob'))] """ sc = SparkContext._active_spark_context if len(cols) == 1 and isinstance(cols[0], (list, set)): @@ -879,14 +870,13 @@ def log2(col): @since(1.5) -@ignore_unicode_prefix def conv(col, fromBase, toBase): """ Convert a number in a string column from one base to another. >>> df = spark.createDataFrame([("010101",)], ['n']) >>> df.select(conv(df.n, 2, 16).alias('hex')).collect() - [Row(hex=u'15')] + [Row(hex='15')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.conv(_to_java_column(col), fromBase, toBase)) @@ -976,7 +966,6 @@ def current_timestamp(): return Column(sc._jvm.functions.current_timestamp()) -@ignore_unicode_prefix @since(1.5) def date_format(date, format): """ @@ -992,7 +981,7 @@ def date_format(date, format): >>> df = spark.createDataFrame([('2015-04-08',)], ['dt']) >>> df.select(date_format('dt', 'MM/dd/yyy').alias('date')).collect() - [Row(date=u'04/08/2015')] + [Row(date='04/08/2015')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.date_format(_to_java_column(date), format)) @@ -1310,7 +1299,6 @@ def last_day(date): return Column(sc._jvm.functions.last_day(_to_java_column(date))) -@ignore_unicode_prefix @since(1.5) def from_unixtime(timestamp, format="yyyy-MM-dd HH:mm:ss"): """ @@ -1321,7 +1309,7 @@ def from_unixtime(timestamp, format="yyyy-MM-dd HH:mm:ss"): >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> time_df = spark.createDataFrame([(1428476400,)], ['unix_time']) >>> time_df.select(from_unixtime('unix_time').alias('ts')).collect() - [Row(ts=u'2015-04-08 00:00:00')] + [Row(ts='2015-04-08 00:00:00')] >>> spark.conf.unset("spark.sql.session.timeZone") """ sc = SparkContext._active_spark_context @@ -1447,7 +1435,6 @@ def timestamp_seconds(col): @since(2.0) -@ignore_unicode_prefix def window(timeColumn, windowDuration, slideDuration=None, startTime=None): """Bucketize rows into one or more time windows given a timestamp specifying column. Window starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window @@ -1471,7 +1458,7 @@ def window(timeColumn, windowDuration, slideDuration=None, startTime=None): >>> w = df.groupBy(window("date", "5 seconds")).agg(sum("val").alias("sum")) >>> w.select(w.window.start.cast("string").alias("start"), ... w.window.end.cast("string").alias("end"), "sum").collect() - [Row(start=u'2016-03-11 09:00:05', end=u'2016-03-11 09:00:10', sum=1)] + [Row(start='2016-03-11 09:00:05', end='2016-03-11 09:00:10', sum=1)] """ def check_string_field(field, fieldName): if not field or type(field) is not str: @@ -1498,7 +1485,6 @@ def check_string_field(field, fieldName): # ---------------------------- misc functions ---------------------------------- @since(1.5) -@ignore_unicode_prefix def crc32(col): """ Calculates the cyclic redundancy check value (CRC32) of a binary column and @@ -1511,33 +1497,30 @@ def crc32(col): return Column(sc._jvm.functions.crc32(_to_java_column(col))) -@ignore_unicode_prefix @since(1.5) def md5(col): """Calculates the MD5 digest and returns the value as a 32 character hex string. >>> spark.createDataFrame([('ABC',)], ['a']).select(md5('a').alias('hash')).collect() - [Row(hash=u'902fbdd2b1df0c4f70b4a5d23525e932')] + [Row(hash='902fbdd2b1df0c4f70b4a5d23525e932')] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.md5(_to_java_column(col)) return Column(jc) -@ignore_unicode_prefix @since(1.5) def sha1(col): """Returns the hex string result of SHA-1. >>> spark.createDataFrame([('ABC',)], ['a']).select(sha1('a').alias('hash')).collect() - [Row(hash=u'3c01bdbb26f358bab27f267924aa2c9a03fcfdb8')] + [Row(hash='3c01bdbb26f358bab27f267924aa2c9a03fcfdb8')] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.sha1(_to_java_column(col)) return Column(jc) -@ignore_unicode_prefix @since(1.5) def sha2(col, numBits): """Returns the hex string result of SHA-2 family of hash functions (SHA-224, SHA-256, SHA-384, @@ -1546,9 +1529,9 @@ def sha2(col, numBits): >>> digests = df.select(sha2(df.name, 256).alias('s')).collect() >>> digests[0] - Row(s=u'3bc51062973c458d5a6f2d8d64a023246354ad7e064b1e4e009ec8a0699a3043') + Row(s='3bc51062973c458d5a6f2d8d64a023246354ad7e064b1e4e009ec8a0699a3043') >>> digests[1] - Row(s=u'cd9fb1e148ccd8442e5aa74904cc73bf6fb54d1d54d333bd596aa9bb4bb4e961') + Row(s='cd9fb1e148ccd8442e5aa74904cc73bf6fb54d1d54d333bd596aa9bb4bb4e961') """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.sha2(_to_java_column(col), numBits) @@ -1600,7 +1583,6 @@ def xxhash64(*cols): @since(1.5) -@ignore_unicode_prefix def concat_ws(sep, *cols): """ Concatenates multiple input string columns together into a single string column, @@ -1608,7 +1590,7 @@ def concat_ws(sep, *cols): >>> df = spark.createDataFrame([('abcd','123')], ['s', 'd']) >>> df.select(concat_ws('-', df.s, df.d).alias('s')).collect() - [Row(s=u'abcd-123')] + [Row(s='abcd-123')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.concat_ws(sep, _to_seq(sc, cols, _to_java_column))) @@ -1634,7 +1616,6 @@ def encode(col, charset): return Column(sc._jvm.functions.encode(_to_java_column(col), charset)) -@ignore_unicode_prefix @since(1.5) def format_number(col, d): """ @@ -1645,13 +1626,12 @@ def format_number(col, d): :param d: the N decimal places >>> spark.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect() - [Row(v=u'5.0000')] + [Row(v='5.0000')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.format_number(_to_java_column(col), d)) -@ignore_unicode_prefix @since(1.5) def format_string(format, *cols): """ @@ -1663,7 +1643,7 @@ def format_string(format, *cols): >>> df = spark.createDataFrame([(5, "hello")], ['a', 'b']) >>> df.select(format_string('%d %s', df.a, df.b).alias('v')).collect() - [Row(v=u'5 hello')] + [Row(v='5 hello')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.format_string(format, _to_seq(sc, cols, _to_java_column))) @@ -1721,7 +1701,6 @@ def overlay(src, replace, pos, len=-1): @since(1.5) -@ignore_unicode_prefix def substring(str, pos, len): """ Substring starts at `pos` and is of length `len` when str is String type or @@ -1732,14 +1711,13 @@ def substring(str, pos, len): >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(substring(df.s, 1, 2).alias('s')).collect() - [Row(s=u'ab')] + [Row(s='ab')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.substring(_to_java_column(str), pos, len)) @since(1.5) -@ignore_unicode_prefix def substring_index(str, delim, count): """ Returns the substring from string str before count occurrences of the delimiter delim. @@ -1749,15 +1727,14 @@ def substring_index(str, delim, count): >>> df = spark.createDataFrame([('a.b.c.d',)], ['s']) >>> df.select(substring_index(df.s, '.', 2).alias('s')).collect() - [Row(s=u'a.b')] + [Row(s='a.b')] >>> df.select(substring_index(df.s, '.', -3).alias('s')).collect() - [Row(s=u'b.c.d')] + [Row(s='b.c.d')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.substring_index(_to_java_column(str), delim, count)) -@ignore_unicode_prefix @since(1.5) def levenshtein(left, right): """Computes the Levenshtein distance of the two given strings. @@ -1792,49 +1769,45 @@ def locate(substr, str, pos=1): @since(1.5) -@ignore_unicode_prefix def lpad(col, len, pad): """ Left-pad the string column to width `len` with `pad`. >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(lpad(df.s, 6, '#').alias('s')).collect() - [Row(s=u'##abcd')] + [Row(s='##abcd')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.lpad(_to_java_column(col), len, pad)) @since(1.5) -@ignore_unicode_prefix def rpad(col, len, pad): """ Right-pad the string column to width `len` with `pad`. >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(rpad(df.s, 6, '#').alias('s')).collect() - [Row(s=u'abcd##')] + [Row(s='abcd##')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.rpad(_to_java_column(col), len, pad)) @since(1.5) -@ignore_unicode_prefix def repeat(col, n): """ Repeats a string column n times, and returns it as a new string column. >>> df = spark.createDataFrame([('ab',)], ['s',]) >>> df.select(repeat(df.s, 3).alias('s')).collect() - [Row(s=u'ababab')] + [Row(s='ababab')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.repeat(_to_java_column(col), n)) @since(1.5) -@ignore_unicode_prefix def split(str, pattern, limit=-1): """ Splits str around matches of the given pattern. @@ -1855,15 +1828,14 @@ def split(str, pattern, limit=-1): >>> df = spark.createDataFrame([('oneAtwoBthreeC',)], ['s',]) >>> df.select(split(df.s, '[ABC]', 2).alias('s')).collect() - [Row(s=[u'one', u'twoBthreeC'])] + [Row(s=['one', 'twoBthreeC'])] >>> df.select(split(df.s, '[ABC]', -1).alias('s')).collect() - [Row(s=[u'one', u'two', u'three', u''])] + [Row(s=['one', 'two', 'three', ''])] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.split(_to_java_column(str), pattern, limit)) -@ignore_unicode_prefix @since(1.5) def regexp_extract(str, pattern, idx): r"""Extract a specific group matched by a Java regex, from the specified string column. @@ -1871,73 +1843,68 @@ def regexp_extract(str, pattern, idx): >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_extract('str', r'(\d+)-(\d+)', 1).alias('d')).collect() - [Row(d=u'100')] + [Row(d='100')] >>> df = spark.createDataFrame([('foo',)], ['str']) >>> df.select(regexp_extract('str', r'(\d+)', 1).alias('d')).collect() - [Row(d=u'')] + [Row(d='')] >>> df = spark.createDataFrame([('aaaac',)], ['str']) >>> df.select(regexp_extract('str', '(a+)(b)?(c)', 2).alias('d')).collect() - [Row(d=u'')] + [Row(d='')] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.regexp_extract(_to_java_column(str), pattern, idx) return Column(jc) -@ignore_unicode_prefix @since(1.5) def regexp_replace(str, pattern, replacement): r"""Replace all substrings of the specified string value that match regexp with rep. >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_replace('str', r'(\d+)', '--').alias('d')).collect() - [Row(d=u'-----')] + [Row(d='-----')] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.regexp_replace(_to_java_column(str), pattern, replacement) return Column(jc) -@ignore_unicode_prefix @since(1.5) def initcap(col): """Translate the first letter of each word to upper case in the sentence. >>> spark.createDataFrame([('ab cd',)], ['a']).select(initcap("a").alias('v')).collect() - [Row(v=u'Ab Cd')] + [Row(v='Ab Cd')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.initcap(_to_java_column(col))) @since(1.5) -@ignore_unicode_prefix def soundex(col): """ Returns the SoundEx encoding for a string >>> df = spark.createDataFrame([("Peters",),("Uhrbach",)], ['name']) >>> df.select(soundex(df.name).alias("soundex")).collect() - [Row(soundex=u'P362'), Row(soundex=u'U612')] + [Row(soundex='P362'), Row(soundex='U612')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.soundex(_to_java_column(col))) -@ignore_unicode_prefix @since(1.5) def bin(col): """Returns the string representation of the binary value of the given column. >>> df.select(bin(df.age).alias('c')).collect() - [Row(c=u'10'), Row(c=u'101')] + [Row(c='10'), Row(c='101')] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.bin(_to_java_column(col)) return Column(jc) -@ignore_unicode_prefix @since(1.5) def hex(col): """Computes hex value of the given column, which could be :class:`pyspark.sql.types.StringType`, @@ -1945,14 +1912,13 @@ def hex(col): :class:`pyspark.sql.types.LongType`. >>> spark.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect() - [Row(hex(a)=u'414243', hex(b)=u'3')] + [Row(hex(a)='414243', hex(b)='3')] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.hex(_to_java_column(col)) return Column(jc) -@ignore_unicode_prefix @since(1.5) def unhex(col): """Inverse of hex. Interprets each pair of characters as a hexadecimal number @@ -1965,7 +1931,6 @@ def unhex(col): return Column(sc._jvm.functions.unhex(_to_java_column(col))) -@ignore_unicode_prefix @since(1.5) def length(col): """Computes the character length of string data or number of bytes of binary data. @@ -1979,7 +1944,6 @@ def length(col): return Column(sc._jvm.functions.length(_to_java_column(col))) -@ignore_unicode_prefix @since(1.5) def translate(srcCol, matching, replace): """A function translate any character in the `srcCol` by a character in `matching`. @@ -1989,7 +1953,7 @@ def translate(srcCol, matching, replace): >>> spark.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123") \\ ... .alias('r')).collect() - [Row(r=u'1a2s3ae')] + [Row(r='1a2s3ae')] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.translate(_to_java_column(srcCol), matching, replace)) @@ -1997,7 +1961,6 @@ def translate(srcCol, matching, replace): # ---------------------- Collection functions ------------------------------ -@ignore_unicode_prefix @since(2.0) def create_map(*cols): """Creates a new map column. @@ -2006,9 +1969,9 @@ def create_map(*cols): grouped as key-value pairs, e.g. (key1, value1, key2, value2, ...). >>> df.select(create_map('name', 'age').alias("map")).collect() - [Row(map={u'Alice': 2}), Row(map={u'Bob': 5})] + [Row(map={'Alice': 2}), Row(map={'Bob': 5})] >>> df.select(create_map([df.name, df.age]).alias("map")).collect() - [Row(map={u'Alice': 2}), Row(map={u'Bob': 5})] + [Row(map={'Alice': 2}), Row(map={'Bob': 5})] """ sc = SparkContext._active_spark_context if len(cols) == 1 and isinstance(cols[0], (list, set)): @@ -2108,7 +2071,6 @@ def slice(x, start, length): return Column(sc._jvm.functions.slice(_to_java_column(x), start, length)) -@ignore_unicode_prefix @since(2.4) def array_join(col, delimiter, null_replacement=None): """ @@ -2117,9 +2079,9 @@ def array_join(col, delimiter, null_replacement=None): >>> df = spark.createDataFrame([(["a", "b", "c"],), (["a", None],)], ['data']) >>> df.select(array_join(df.data, ",").alias("joined")).collect() - [Row(joined=u'a,b,c'), Row(joined=u'a')] + [Row(joined='a,b,c'), Row(joined='a')] >>> df.select(array_join(df.data, ",", "NULL").alias("joined")).collect() - [Row(joined=u'a,b,c'), Row(joined=u'a,NULL')] + [Row(joined='a,b,c'), Row(joined='a,NULL')] """ sc = SparkContext._active_spark_context if null_replacement is None: @@ -2130,7 +2092,6 @@ def array_join(col, delimiter, null_replacement=None): @since(1.5) -@ignore_unicode_prefix def concat(*cols): """ Concatenates multiple input columns together into a single column. @@ -2138,7 +2099,7 @@ def concat(*cols): >>> df = spark.createDataFrame([('abcd','123')], ['s', 'd']) >>> df.select(concat(df.s, df.d).alias('s')).collect() - [Row(s=u'abcd123')] + [Row(s='abcd123')] >>> df = spark.createDataFrame([([1, 2], [3, 4], [5]), ([1, 2], None, [3])], ['a', 'b', 'c']) >>> df.select(concat(df.a, df.b, df.c).alias("arr")).collect() @@ -2165,7 +2126,6 @@ def array_position(col, value): return Column(sc._jvm.functions.array_position(_to_java_column(col), value)) -@ignore_unicode_prefix @since(2.4) def element_at(col, extraction): """ @@ -2179,7 +2139,7 @@ def element_at(col, extraction): >>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) >>> df.select(element_at(df.data, 1)).collect() - [Row(element_at(data, 1)=u'a'), Row(element_at(data, 1)=None)] + [Row(element_at(data, 1)='a'), Row(element_at(data, 1)=None)] >>> df = spark.createDataFrame([({"a": 1.0, "b": 2.0},), ({},)], ['data']) >>> df.select(element_at(df.data, lit("a"))).collect() @@ -2221,7 +2181,6 @@ def array_distinct(col): return Column(sc._jvm.functions.array_distinct(_to_java_column(col))) -@ignore_unicode_prefix @since(2.4) def array_intersect(col1, col2): """ @@ -2234,13 +2193,12 @@ def array_intersect(col1, col2): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])]) >>> df.select(array_intersect(df.c1, df.c2)).collect() - [Row(array_intersect(c1, c2)=[u'a', u'c'])] + [Row(array_intersect(c1, c2)=['a', 'c'])] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.array_intersect(_to_java_column(col1), _to_java_column(col2))) -@ignore_unicode_prefix @since(2.4) def array_union(col1, col2): """ @@ -2253,13 +2211,12 @@ def array_union(col1, col2): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])]) >>> df.select(array_union(df.c1, df.c2)).collect() - [Row(array_union(c1, c2)=[u'b', u'a', u'c', u'd', u'f'])] + [Row(array_union(c1, c2)=['b', 'a', 'c', 'd', 'f'])] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.array_union(_to_java_column(col1), _to_java_column(col2))) -@ignore_unicode_prefix @since(2.4) def array_except(col1, col2): """ @@ -2272,7 +2229,7 @@ def array_except(col1, col2): >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])]) >>> df.select(array_except(df.c1, df.c2)).collect() - [Row(array_except(c1, c2)=[u'b'])] + [Row(array_except(c1, c2)=['b'])] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.array_except(_to_java_column(col1), _to_java_column(col2))) @@ -2397,7 +2354,6 @@ def posexplode_outer(col): return Column(jc) -@ignore_unicode_prefix @since(1.6) def get_json_object(col, path): """ @@ -2411,14 +2367,13 @@ def get_json_object(col, path): >>> df = spark.createDataFrame(data, ("key", "jstring")) >>> df.select(df.key, get_json_object(df.jstring, '$.f1').alias("c0"), \\ ... get_json_object(df.jstring, '$.f2').alias("c1") ).collect() - [Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)] + [Row(key='1', c0='value1', c1='value2'), Row(key='2', c0='value12', c1=None)] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.get_json_object(_to_java_column(col), path) return Column(jc) -@ignore_unicode_prefix @since(1.6) def json_tuple(col, *fields): """Creates a new row for a json column according to the given field names. @@ -2429,14 +2384,13 @@ def json_tuple(col, *fields): >>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')] >>> df = spark.createDataFrame(data, ("key", "jstring")) >>> df.select(df.key, json_tuple(df.jstring, 'f1', 'f2')).collect() - [Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)] + [Row(key='1', c0='value1', c1='value2'), Row(key='2', c0='value12', c1=None)] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.json_tuple(_to_java_column(col), _to_seq(sc, fields)) return Column(jc) -@ignore_unicode_prefix @since(2.1) def from_json(col, schema, options={}): """ @@ -2460,7 +2414,7 @@ def from_json(col, schema, options={}): >>> df.select(from_json(df.value, "a INT").alias("json")).collect() [Row(json=Row(a=1))] >>> df.select(from_json(df.value, "MAP").alias("json")).collect() - [Row(json={u'a': 1})] + [Row(json={'a': 1})] >>> data = [(1, '''[{"a": 1}]''')] >>> schema = ArrayType(StructType([StructField("a", IntegerType())])) >>> df = spark.createDataFrame(data, ("key", "value")) @@ -2485,7 +2439,6 @@ def from_json(col, schema, options={}): return Column(jc) -@ignore_unicode_prefix @since(2.1) def to_json(col, options={}): """ @@ -2499,26 +2452,26 @@ def to_json(col, options={}): >>> from pyspark.sql import Row >>> from pyspark.sql.types import * - >>> data = [(1, Row(name='Alice', age=2))] + >>> data = [(1, Row(age=2, name='Alice'))] >>> df = spark.createDataFrame(data, ("key", "value")) >>> df.select(to_json(df.value).alias("json")).collect() - [Row(json=u'{"age":2,"name":"Alice"}')] - >>> data = [(1, [Row(name='Alice', age=2), Row(name='Bob', age=3)])] + [Row(json='{"age":2,"name":"Alice"}')] + >>> data = [(1, [Row(age=2, name='Alice'), Row(age=3, name='Bob')])] >>> df = spark.createDataFrame(data, ("key", "value")) >>> df.select(to_json(df.value).alias("json")).collect() - [Row(json=u'[{"age":2,"name":"Alice"},{"age":3,"name":"Bob"}]')] + [Row(json='[{"age":2,"name":"Alice"},{"age":3,"name":"Bob"}]')] >>> data = [(1, {"name": "Alice"})] >>> df = spark.createDataFrame(data, ("key", "value")) >>> df.select(to_json(df.value).alias("json")).collect() - [Row(json=u'{"name":"Alice"}')] + [Row(json='{"name":"Alice"}')] >>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])] >>> df = spark.createDataFrame(data, ("key", "value")) >>> df.select(to_json(df.value).alias("json")).collect() - [Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')] + [Row(json='[{"name":"Alice"},{"name":"Bob"}]')] >>> data = [(1, ["Alice", "Bob"])] >>> df = spark.createDataFrame(data, ("key", "value")) >>> df.select(to_json(df.value).alias("json")).collect() - [Row(json=u'["Alice","Bob"]')] + [Row(json='["Alice","Bob"]')] """ sc = SparkContext._active_spark_context @@ -2526,7 +2479,6 @@ def to_json(col, options={}): return Column(jc) -@ignore_unicode_prefix @since(2.4) def schema_of_json(json, options={}): """ @@ -2540,12 +2492,12 @@ def schema_of_json(json, options={}): >>> df = spark.range(1) >>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect() - [Row(json=u'struct')] + [Row(json='struct')] >>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'}) >>> df.select(schema.alias("json")).collect() - [Row(json=u'struct')] + [Row(json='struct')] """ - if isinstance(json, basestring): + if isinstance(json, str): col = _create_column_from_literal(json) elif isinstance(json, Column): col = _to_java_column(json) @@ -2557,7 +2509,6 @@ def schema_of_json(json, options={}): return Column(jc) -@ignore_unicode_prefix @since(3.0) def schema_of_csv(csv, options={}): """ @@ -2568,11 +2519,11 @@ def schema_of_csv(csv, options={}): >>> df = spark.range(1) >>> df.select(schema_of_csv(lit('1|a'), {'sep':'|'}).alias("csv")).collect() - [Row(csv=u'struct<_c0:int,_c1:string>')] + [Row(csv='struct<_c0:int,_c1:string>')] >>> df.select(schema_of_csv('1|a', {'sep':'|'}).alias("csv")).collect() - [Row(csv=u'struct<_c0:int,_c1:string>')] + [Row(csv='struct<_c0:int,_c1:string>')] """ - if isinstance(csv, basestring): + if isinstance(csv, str): col = _create_column_from_literal(csv) elif isinstance(csv, Column): col = _to_java_column(csv) @@ -2584,7 +2535,6 @@ def schema_of_csv(csv, options={}): return Column(jc) -@ignore_unicode_prefix @since(3.0) def to_csv(col, options={}): """ @@ -2595,10 +2545,10 @@ def to_csv(col, options={}): :param options: options to control converting. accepts the same options as the CSV datasource. >>> from pyspark.sql import Row - >>> data = [(1, Row(name='Alice', age=2))] + >>> data = [(1, Row(age=2, name='Alice'))] >>> df = spark.createDataFrame(data, ("key", "value")) >>> df.select(to_csv(df.value).alias("csv")).collect() - [Row(csv=u'2,Alice')] + [Row(csv='2,Alice')] """ sc = SparkContext._active_spark_context @@ -2705,7 +2655,6 @@ def shuffle(col): @since(1.5) -@ignore_unicode_prefix def reverse(col): """ Collection function: returns a reversed string or an array with reverse order of elements. @@ -2714,7 +2663,7 @@ def reverse(col): >>> df = spark.createDataFrame([('Spark SQL',)], ['data']) >>> df.select(reverse(df.data).alias('s')).collect() - [Row(s=u'LQS krapS')] + [Row(s='LQS krapS')] >>> df = spark.createDataFrame([([2, 1, 3],) ,([1],) ,([],)], ['data']) >>> df.select(reverse(df.data).alias('r')).collect() [Row(r=[3, 1, 2]), Row(r=[1]), Row(r=[])] @@ -2820,7 +2769,6 @@ def map_from_entries(col): return Column(sc._jvm.functions.map_from_entries(_to_java_column(col))) -@ignore_unicode_prefix @since(2.4) def array_repeat(col, count): """ @@ -2828,7 +2776,7 @@ def array_repeat(col, count): >>> df = spark.createDataFrame([('ab',)], ['data']) >>> df.select(array_repeat(df.data, 3).alias('r')).collect() - [Row(r=[u'ab', u'ab', u'ab'])] + [Row(r=['ab', 'ab', 'ab'])] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.array_repeat( @@ -2898,7 +2846,6 @@ def sequence(start, stop, step=None): _to_java_column(start), _to_java_column(stop), _to_java_column(step))) -@ignore_unicode_prefix @since(3.0) def from_csv(col, schema, options={}): """ @@ -2920,11 +2867,11 @@ def from_csv(col, schema, options={}): >>> df = spark.createDataFrame(data, ("value",)) >>> options = {'ignoreLeadingWhiteSpace': True} >>> df.select(from_csv(df.value, "s string", options).alias("csv")).collect() - [Row(csv=Row(s=u'abc'))] + [Row(csv=Row(s='abc'))] """ sc = SparkContext._active_spark_context - if isinstance(schema, basestring): + if isinstance(schema, str): schema = _create_column_from_literal(schema) elif isinstance(schema, Column): schema = _to_java_column(schema) @@ -2984,20 +2931,6 @@ def _get_lambda_parameters(f): return parameters -def _get_lambda_parameters_legacy(f): - # TODO (SPARK-29909) Remove once 2.7 support is dropped - import inspect - - spec = inspect.getargspec(f) - if not 1 <= len(spec.args) <= 3 or spec.varargs or spec.keywords: - raise ValueError( - "f should take between 1 and 3 arguments, but provided function takes {}".format( - spec - ) - ) - return spec.args - - def _create_lambda(f): """ Create `o.a.s.sql.expressions.LambdaFunction` corresponding @@ -3008,10 +2941,7 @@ def _create_lambda(f): - (Column, Column) -> Column: ... - (Column, Column, Column) -> Column: ... """ - if sys.version_info >= (3, 3): - parameters = _get_lambda_parameters(f) - else: - parameters = _get_lambda_parameters_legacy(f) + parameters = _get_lambda_parameters(f) sc = SparkContext._active_spark_context expressions = sc._jvm.org.apache.spark.sql.catalyst.expressions @@ -3481,7 +3411,7 @@ def udf(f=None, returnType=StringType()): evalType=PythonEvalType.SQL_BATCHED_UDF) -blacklist = ['map', 'since', 'ignore_unicode_prefix'] +blacklist = ['map', 'since'] __all__ = [k for k, v in globals().items() if not k.startswith('_') and k[0].islower() and callable(v) and k not in blacklist] __all__ += ["PandasUDFType"] @@ -3500,7 +3430,7 @@ def _test(): sc = spark.sparkContext globs['sc'] = sc globs['spark'] = spark - globs['df'] = spark.createDataFrame([Row(name='Alice', age=2), Row(name='Bob', age=5)]) + globs['df'] = spark.createDataFrame([Row(age=2, name='Alice'), Row(age=5, name='Bob')]) (failure_count, test_count) = doctest.testmod( pyspark.sql.functions, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index ac826bc64ad7e..83e2baa8f0002 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -18,7 +18,6 @@ import sys from pyspark import since -from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.column import Column, _to_seq from pyspark.sql.dataframe import DataFrame from pyspark.sql.pandas.group_ops import PandasGroupedOpsMixin @@ -60,7 +59,6 @@ def __init__(self, jgd, df): self._df = df self.sql_ctx = df.sql_ctx - @ignore_unicode_prefix @since(1.3) def agg(self, *exprs): """Compute aggregates and returns the result as a :class:`DataFrame`. @@ -91,18 +89,18 @@ def agg(self, *exprs): >>> gdf = df.groupBy(df.name) >>> sorted(gdf.agg({"*": "count"}).collect()) - [Row(name=u'Alice', count(1)=1), Row(name=u'Bob', count(1)=1)] + [Row(name='Alice', count(1)=1), Row(name='Bob', count(1)=1)] >>> from pyspark.sql import functions as F >>> sorted(gdf.agg(F.min(df.age)).collect()) - [Row(name=u'Alice', min(age)=2), Row(name=u'Bob', min(age)=5)] + [Row(name='Alice', min(age)=2), Row(name='Bob', min(age)=5)] >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> @pandas_udf('int', PandasUDFType.GROUPED_AGG) # doctest: +SKIP ... def min_udf(v): ... return v.min() >>> sorted(gdf.agg(min_udf(df.age)).collect()) # doctest: +SKIP - [Row(name=u'Alice', min_udf(age)=2), Row(name=u'Bob', min_udf(age)=5)] + [Row(name='Alice', min_udf(age)=2), Row(name='Bob', min_udf(age)=5)] """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): diff --git a/python/pyspark/sql/pandas/conversion.py b/python/pyspark/sql/pandas/conversion.py index e6d8e9f24a557..3842bc2357c6c 100644 --- a/python/pyspark/sql/pandas/conversion.py +++ b/python/pyspark/sql/pandas/conversion.py @@ -16,11 +16,6 @@ # import sys import warnings -if sys.version >= '3': - basestring = unicode = str - xrange = range -else: - from itertools import izip as zip from collections import Counter from pyspark import since @@ -29,7 +24,6 @@ from pyspark.sql.types import IntegralType from pyspark.sql.types import * from pyspark.traceback_utils import SCCallSiteSync -from pyspark.util import _exception_message class PandasConversionMixin(object): @@ -84,7 +78,7 @@ def toPandas(self): "failed by the reason below:\n %s\n" "Attempting non-optimization as " "'spark.sql.execution.arrow.pyspark.fallback.enabled' is set to " - "true." % _exception_message(e)) + "true." % str(e)) warnings.warn(msg) use_arrow = False else: @@ -93,7 +87,7 @@ def toPandas(self): "'spark.sql.execution.arrow.pyspark.enabled' is set to true, but has " "reached the error below and will not continue because automatic fallback " "with 'spark.sql.execution.arrow.pyspark.fallback.enabled' has been set to " - "false.\n %s" % _exception_message(e)) + "false.\n %s" % str(e)) warnings.warn(msg) raise @@ -130,7 +124,7 @@ def toPandas(self): "reached the error below and can not continue. Note that " "'spark.sql.execution.arrow.pyspark.fallback.enabled' does not have an " "effect on failures in the middle of " - "computation.\n %s" % _exception_message(e)) + "computation.\n %s" % str(e)) warnings.warn(msg) raise @@ -268,7 +262,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr # If no schema supplied by user then get the names of columns only if schema is None: - schema = [str(x) if not isinstance(x, basestring) else + schema = [str(x) if not isinstance(x, str) else (x.encode('utf-8') if not isinstance(x, str) else x) for x in data.columns] @@ -276,8 +270,6 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr try: return self._create_from_pandas_with_arrow(data, schema, timezone) except Exception as e: - from pyspark.util import _exception_message - if self._wrapped._conf.arrowPySparkFallbackEnabled(): msg = ( "createDataFrame attempted Arrow optimization because " @@ -285,7 +277,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr "failed by the reason below:\n %s\n" "Attempting non-optimization as " "'spark.sql.execution.arrow.pyspark.fallback.enabled' is set to " - "true." % _exception_message(e)) + "true." % str(e)) warnings.warn(msg) else: msg = ( @@ -293,7 +285,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr "'spark.sql.execution.arrow.pyspark.enabled' is set to true, but has " "reached the error below and will not continue because automatic " "fallback with 'spark.sql.execution.arrow.pyspark.fallback.enabled' " - "has been set to false.\n %s" % _exception_message(e)) + "has been set to false.\n %s" % str(e)) warnings.warn(msg) raise data = self._convert_from_pandas(data, schema, timezone) @@ -358,7 +350,7 @@ def _get_numpy_record_dtype(self, rec): col_names = cur_dtypes.names record_type_list = [] has_rec_fix = False - for i in xrange(len(cur_dtypes)): + for i in range(len(cur_dtypes)): curr_type = cur_dtypes[i] # If type is a datetime64 timestamp, convert to microseconds # NOTE: if dtype is datetime[ns] then np.record.tolist() will output values as longs, @@ -413,7 +405,7 @@ def _create_from_pandas_with_arrow(self, pdf, schema, timezone): # Slice the DataFrame to be batched step = -(-len(pdf) // self.sparkContext.defaultParallelism) # round int up - pdf_slices = (pdf.iloc[start:start + step] for start in xrange(0, len(pdf), step)) + pdf_slices = (pdf.iloc[start:start + step] for start in range(0, len(pdf), step)) # Create list of Arrow (columns, type) for serializer dump_stream arrow_data = [[(c, t) for (_, c), t in zip(pdf_slice.iteritems(), arrow_types)] diff --git a/python/pyspark/sql/pandas/functions.py b/python/pyspark/sql/pandas/functions.py index 094dc357b6822..ba4dec82d4eb4 100644 --- a/python/pyspark/sql/pandas/functions.py +++ b/python/pyspark/sql/pandas/functions.py @@ -18,6 +18,7 @@ import functools import sys import warnings +from inspect import getfullargspec from pyspark import since from pyspark.rdd import PythonEvalType @@ -25,7 +26,6 @@ from pyspark.sql.pandas.utils import require_minimum_pandas_version, require_minimum_pyarrow_version from pyspark.sql.types import DataType from pyspark.sql.udf import _create_udf -from pyspark.util import _get_argspec class PandasUDFType(object): @@ -371,30 +371,29 @@ def calculate(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]: def _create_pandas_udf(f, returnType, evalType): - argspec = _get_argspec(f) + argspec = getfullargspec(f) # pandas UDF by type hints. - if sys.version_info >= (3, 6): - from inspect import signature - - if evalType in [PythonEvalType.SQL_SCALAR_PANDAS_UDF, - PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, - PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF]: - warnings.warn( - "In Python 3.6+ and Spark 3.0+, it is preferred to specify type hints for " - "pandas UDF instead of specifying pandas UDF type which will be deprecated " - "in the future releases. See SPARK-28264 for more details.", UserWarning) - elif evalType in [PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, - PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, - PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]: - # In case of 'SQL_GROUPED_MAP_PANDAS_UDF', deprecation warning is being triggered - # at `apply` instead. - # In case of 'SQL_MAP_PANDAS_ITER_UDF' and 'SQL_COGROUPED_MAP_PANDAS_UDF', the - # evaluation type will always be set. - pass - elif len(argspec.annotations) > 0: - evalType = infer_eval_type(signature(f)) - assert evalType is not None + from inspect import signature + + if evalType in [PythonEvalType.SQL_SCALAR_PANDAS_UDF, + PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, + PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF]: + warnings.warn( + "In Python 3.6+ and Spark 3.0+, it is preferred to specify type hints for " + "pandas UDF instead of specifying pandas UDF type which will be deprecated " + "in the future releases. See SPARK-28264 for more details.", UserWarning) + elif evalType in [PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, + PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]: + # In case of 'SQL_GROUPED_MAP_PANDAS_UDF', deprecation warning is being triggered + # at `apply` instead. + # In case of 'SQL_MAP_PANDAS_ITER_UDF' and 'SQL_COGROUPED_MAP_PANDAS_UDF', the + # evaluation type will always be set. + pass + elif len(argspec.annotations) > 0: + evalType = infer_eval_type(signature(f)) + assert evalType is not None if evalType is None: # Set default is scalar UDF. diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 42562e1fb9c46..4b91c6a0f8730 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -19,13 +19,6 @@ Serializers for PyArrow and pandas conversions. See `pyspark.serializers` for more details. """ -import sys -if sys.version < '3': - from itertools import izip as zip -else: - basestring = unicode = str - xrange = range - from pyspark.serializers import Serializer, read_int, write_int, UTF8Deserializer @@ -67,7 +60,7 @@ def load_stream(self, stream): raise RuntimeError("An error occurred while calling " "ArrowCollectSerializer.load_stream: {}".format(error_msg)) batch_order = [] - for i in xrange(num): + for i in range(num): index = read_int(stream) batch_order.append(index) yield batch_order @@ -180,7 +173,7 @@ def create_array(s, t): if len(s) == 0 and len(s.columns) == 0: arrs_names = [(pa.array([], type=field.type), field.name) for field in t] # Assign result columns by schema name if user labeled with strings - elif self._assign_cols_by_name and any(isinstance(name, basestring) + elif self._assign_cols_by_name and any(isinstance(name, str) for name in s.columns): arrs_names = [(create_array(s[field.name], field.type), field.name) for field in t] @@ -194,7 +187,7 @@ def create_array(s, t): else: arrs.append(create_array(s, t)) - return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in xrange(len(arrs))]) + return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in range(len(arrs))]) def dump_stream(self, iterator, stream): """ diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 336345e383729..a83aece2e485d 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -15,15 +15,9 @@ # limitations under the License. # -import sys - -if sys.version >= '3': - basestring = unicode = str - from py4j.java_gateway import JavaClass from pyspark import RDD, since -from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.column import _to_seq from pyspark.sql.types import * from pyspark.sql import utils @@ -94,7 +88,7 @@ def schema(self, schema): if isinstance(schema, StructType): jschema = spark._jsparkSession.parseDataType(schema.json()) self._jreader = self._jreader.schema(jschema) - elif isinstance(schema, basestring): + elif isinstance(schema, str): self._jreader = self._jreader.schema(schema) else: raise TypeError("schema should be StructType or string") @@ -174,7 +168,7 @@ def load(self, path=None, format=None, schema=None, **options): if schema is not None: self.schema(schema) self.options(**options) - if isinstance(path, basestring): + if isinstance(path, str): return self._df(self._jreader.load(path)) elif path is not None: if type(path) != list: @@ -294,16 +288,16 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, samplingRatio=samplingRatio, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, locale=locale, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(path, basestring): + if isinstance(path, str): path = [path] if type(path) == list: return self._df(self._jreader.json(self._spark._sc._jvm.PythonUtils.toSeq(path))) elif isinstance(path, RDD): def func(iterator): for x in iterator: - if not isinstance(x, basestring): - x = unicode(x) - if isinstance(x, unicode): + if not isinstance(x, str): + x = str(x) + if isinstance(x, str): x = x.encode("utf-8") yield x keyed = path.mapPartitions(func) @@ -352,7 +346,6 @@ def parquet(self, *paths, **options): recursiveFileLookup=recursiveFileLookup) return self._df(self._jreader.parquet(_to_seq(self._spark._sc, paths))) - @ignore_unicode_prefix @since(1.6) def text(self, paths, wholetext=False, lineSep=None, pathGlobFilter=None, recursiveFileLookup=None): @@ -376,15 +369,15 @@ def text(self, paths, wholetext=False, lineSep=None, pathGlobFilter=None, >>> df = spark.read.text('python/test_support/sql/text-test.txt') >>> df.collect() - [Row(value=u'hello'), Row(value=u'this')] + [Row(value='hello'), Row(value='this')] >>> df = spark.read.text('python/test_support/sql/text-test.txt', wholetext=True) >>> df.collect() - [Row(value=u'hello\\nthis')] + [Row(value='hello\\nthis')] """ self._set_opts( wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(paths, basestring): + if isinstance(paths, str): paths = [paths] return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(paths))) @@ -529,16 +522,16 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, samplingRatio=samplingRatio, enforceSchema=enforceSchema, emptyValue=emptyValue, locale=locale, lineSep=lineSep, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(path, basestring): + if isinstance(path, str): path = [path] if type(path) == list: return self._df(self._jreader.csv(self._spark._sc._jvm.PythonUtils.toSeq(path))) elif isinstance(path, RDD): def func(iterator): for x in iterator: - if not isinstance(x, basestring): - x = unicode(x) - if isinstance(x, unicode): + if not isinstance(x, str): + x = str(x) + if isinstance(x, str): x = x.encode("utf-8") yield x keyed = path.mapPartitions(func) @@ -574,7 +567,7 @@ def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=N """ self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(path, basestring): + if isinstance(path, str): path = [path] return self._df(self._jreader.orc(_to_seq(self._spark._sc, path))) @@ -763,7 +756,7 @@ def bucketBy(self, numBuckets, col, *cols): col, cols = col[0], col[1:] - if not all(isinstance(c, basestring) for c in cols) or not(isinstance(col, basestring)): + if not all(isinstance(c, str) for c in cols) or not(isinstance(col, str)): raise TypeError("all names should be `str`") self._jwrite = self._jwrite.bucketBy(numBuckets, col, _to_seq(self._spark._sc, cols)) @@ -788,7 +781,7 @@ def sortBy(self, col, *cols): col, cols = col[0], col[1:] - if not all(isinstance(c, basestring) for c in cols) or not(isinstance(col, basestring)): + if not all(isinstance(c, str) for c in cols) or not(isinstance(col, str)): raise TypeError("all names should be `str`") self._jwrite = self._jwrite.sortBy(col, _to_seq(self._spark._sc, cols)) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 61891c478dbe4..a5d102712d5e4 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -15,22 +15,13 @@ # limitations under the License. # -# To disallow implicit relative import. Remove this once we drop Python 2. -from __future__ import absolute_import -from __future__ import print_function import sys import warnings from functools import reduce from threading import RLock -if sys.version >= '3': - basestring = unicode = str - xrange = range -else: - from itertools import imap as map - from pyspark import since -from pyspark.rdd import RDD, ignore_unicode_prefix +from pyspark.rdd import RDD from pyspark.sql.conf import RuntimeConfig from pyspark.sql.dataframe import DataFrame from pyspark.sql.pandas.conversion import SparkConversionMixin @@ -56,7 +47,7 @@ def toDF(self, schema=None, sampleRatio=None): :return: a DataFrame >>> rdd.toDF().collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] """ return sparkSession.createDataFrame(self, schema, sampleRatio) @@ -197,7 +188,6 @@ def getOrCreate(self): _instantiatedSession = None _activeSession = None - @ignore_unicode_prefix def __init__(self, sparkContext, jsparkSession=None): """Creates a new SparkSession. @@ -213,7 +203,7 @@ def __init__(self, sparkContext, jsparkSession=None): [Row((i + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=2.0, (NOT b)=False, list[1]=2, \ dict[s]=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] >>> df.rdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() - [(1, u'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] + [(1, 'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ from pyspark.sql.context import SQLContext self._sc = sparkContext @@ -492,7 +482,6 @@ def _create_shell_session(): return SparkSession.builder.getOrCreate() @since(2.0) - @ignore_unicode_prefix def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=True): """ Creates a :class:`DataFrame` from an :class:`RDD`, a list or a :class:`pandas.DataFrame`. @@ -530,34 +519,29 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr .. note:: Usage with spark.sql.execution.arrow.pyspark.enabled=True is experimental. - .. note:: When Arrow optimization is enabled, strings inside Pandas DataFrame in Python - 2 are converted into bytes as they are bytes in Python 2 whereas regular strings are - left as strings. When using strings in Python 2, use unicode `u""` as Python standard - practice. - >>> l = [('Alice', 1)] >>> spark.createDataFrame(l).collect() - [Row(_1=u'Alice', _2=1)] + [Row(_1='Alice', _2=1)] >>> spark.createDataFrame(l, ['name', 'age']).collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> d = [{'name': 'Alice', 'age': 1}] >>> spark.createDataFrame(d).collect() - [Row(age=1, name=u'Alice')] + [Row(age=1, name='Alice')] >>> rdd = sc.parallelize(l) >>> spark.createDataFrame(rdd).collect() - [Row(_1=u'Alice', _2=1)] + [Row(_1='Alice', _2=1)] >>> df = spark.createDataFrame(rdd, ['name', 'age']) >>> df.collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> from pyspark.sql import Row >>> Person = Row('name', 'age') >>> person = rdd.map(lambda r: Person(*r)) >>> df2 = spark.createDataFrame(person) >>> df2.collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> from pyspark.sql.types import * >>> schema = StructType([ @@ -565,15 +549,15 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr ... StructField("age", IntegerType(), True)]) >>> df3 = spark.createDataFrame(rdd, schema) >>> df3.collect() - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> spark.createDataFrame(df.toPandas()).collect() # doctest: +SKIP - [Row(name=u'Alice', age=1)] + [Row(name='Alice', age=1)] >>> spark.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP [Row(0=1, 1=2)] >>> spark.createDataFrame(rdd, "a: string, b: int").collect() - [Row(a=u'Alice', b=1)] + [Row(a='Alice', b=1)] >>> rdd = rdd.map(lambda row: row[1]) >>> spark.createDataFrame(rdd, "int").collect() [Row(value=1)] @@ -587,7 +571,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr if isinstance(data, DataFrame): raise TypeError("data is already a DataFrame") - if isinstance(schema, basestring): + if isinstance(schema, str): schema = _parse_datatype_string(schema) elif isinstance(schema, (list, tuple)): # Must re-encode any unicode strings to be consistent with StructField names @@ -634,7 +618,6 @@ def prepare(obj): df._schema = schema return df - @ignore_unicode_prefix @since(2.0) def sql(self, sqlQuery): """Returns a :class:`DataFrame` representing the result of the given query. @@ -644,7 +627,7 @@ def sql(self, sqlQuery): >>> df.createOrReplaceTempView("table1") >>> df2 = spark.sql("SELECT field1 AS f1, field2 as f2 from table1") >>> df2.collect() - [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] + [Row(f1=1, f2='row1'), Row(f1=2, f2='row2'), Row(f1=3, f2='row3')] """ return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 2450a4c93c460..5c528c1d54df7 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -18,13 +18,9 @@ import sys import json -if sys.version >= '3': - basestring = str - from py4j.java_gateway import java_import from pyspark import since, keyword_only -from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.column import _to_seq from pyspark.sql.readwriter import OptionUtils, to_str from pyspark.sql.types import * @@ -204,7 +200,6 @@ def __init__(self, jsqm): self._jsqm = jsqm @property - @ignore_unicode_prefix @since(2.0) def active(self): """Returns a list of active queries associated with this SQLContext @@ -213,12 +208,11 @@ def active(self): >>> sqm = spark.streams >>> # get the list of active streaming queries >>> [q.name for q in sqm.active] - [u'this_query'] + ['this_query'] >>> sq.stop() """ return [StreamingQuery(jsq) for jsq in self._jsqm.active()] - @ignore_unicode_prefix @since(2.0) def get(self, id): """Returns an active query from this SQLContext or throws exception if an active query @@ -226,7 +220,7 @@ def get(self, id): >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() >>> sq.name - u'this_query' + 'this_query' >>> sq = spark.streams.get(sq.id) >>> sq.isActive True @@ -328,7 +322,7 @@ def schema(self, schema): if isinstance(schema, StructType): jschema = spark._jsparkSession.parseDataType(schema.json()) self._jreader = self._jreader.schema(jschema) - elif isinstance(schema, basestring): + elif isinstance(schema, str): self._jreader = self._jreader.schema(schema) else: raise TypeError("schema should be StructType or string") @@ -527,7 +521,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(path, basestring): + if isinstance(path, str): return self._df(self._jreader.json(path)) else: raise TypeError("path can be only a single string") @@ -555,7 +549,7 @@ def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=N """ self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(path, basestring): + if isinstance(path, str): return self._df(self._jreader.orc(path)) else: raise TypeError("path can be only a single string") @@ -585,12 +579,11 @@ def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLook """ self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(path, basestring): + if isinstance(path, str): return self._df(self._jreader.parquet(path)) else: raise TypeError("path can be only a single string") - @ignore_unicode_prefix @since(2.0) def text(self, path, wholetext=False, lineSep=None, pathGlobFilter=None, recursiveFileLookup=None): @@ -623,7 +616,7 @@ def text(self, path, wholetext=False, lineSep=None, pathGlobFilter=None, self._set_opts( wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(path, basestring): + if isinstance(path, str): return self._df(self._jreader.text(path)) else: raise TypeError("path can be only a single string") @@ -762,7 +755,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema, emptyValue=emptyValue, locale=locale, lineSep=lineSep, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) - if isinstance(path, basestring): + if isinstance(path, str): return self._df(self._jreader.csv(path)) else: raise TypeError("path can be only a single string") @@ -1153,7 +1146,6 @@ def foreachBatch(self, func): ensure_callback_server_started(gw) return self - @ignore_unicode_prefix @since(2.0) def start(self, path=None, format=None, outputMode=None, partitionBy=None, queryName=None, **options): @@ -1186,14 +1178,14 @@ def start(self, path=None, format=None, outputMode=None, partitionBy=None, query >>> sq.isActive True >>> sq.name - u'this_query' + 'this_query' >>> sq.stop() >>> sq.isActive False >>> sq = sdf.writeStream.trigger(processingTime='5 seconds').start( ... queryName='that_query', outputMode="append", format='memory') >>> sq.name - u'that_query' + 'that_query' >>> sq.isActive True >>> sq.stop() diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index a96354e3ecb58..90fc983aec021 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -21,9 +21,6 @@ import time import unittest import warnings -import sys -if sys.version >= '3': - basestring = unicode = str from pyspark import SparkContext, SparkConf from pyspark.sql import Row, SparkSession @@ -32,7 +29,6 @@ from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ pandas_requirement_message, pyarrow_requirement_message from pyspark.testing.utils import QuietTest -from pyspark.util import _exception_message if have_pandas: import pandas as pd @@ -130,7 +126,7 @@ def test_toPandas_fallback_enabled(self): warn.message for warn in warns if isinstance(warn.message, UserWarning)] self.assertTrue(len(user_warns) > 0) self.assertTrue( - "Attempting non-optimization" in _exception_message(user_warns[-1])) + "Attempting non-optimization" in str(user_warns[-1])) assert_frame_equal(pdf, pd.DataFrame({u'map': [{u'a': 1}]})) def test_toPandas_fallback_disabled(self): @@ -358,7 +354,7 @@ def test_createDataFrame_fallback_enabled(self): warn.message for warn in warns if isinstance(warn.message, UserWarning)] self.assertTrue(len(user_warns) > 0) self.assertTrue( - "Attempting non-optimization" in _exception_message(user_warns[-1])) + "Attempting non-optimization" in str(user_warns[-1])) self.assertEqual(df.collect(), [Row(a={u'a': 1})]) def test_createDataFrame_fallback_disabled(self): @@ -438,12 +434,12 @@ def test_createDateFrame_with_category_type(self): assert_frame_equal(result_spark, result_arrow) # ensure original category elements are string - self.assertIsInstance(category_first_element, basestring) + self.assertIsInstance(category_first_element, str) # spark data frame and arrow execution mode enabled data frame type must match pandas self.assertEqual(spark_type, 'string') self.assertEqual(arrow_type, 'string') - self.assertIsInstance(arrow_first_category_element, basestring) - self.assertIsInstance(spark_first_category_element, basestring) + self.assertIsInstance(arrow_first_category_element, str) + self.assertIsInstance(spark_first_category_element, str) def test_createDataFrame_with_float_index(self): # SPARK-32098: float index should not produce duplicated or truncated Spark DataFrame diff --git a/python/pyspark/sql/tests/test_column.py b/python/pyspark/sql/tests/test_column.py index 58bf896a10c2a..e0b8bf45a2c70 100644 --- a/python/pyspark/sql/tests/test_column.py +++ b/python/pyspark/sql/tests/test_column.py @@ -16,8 +16,6 @@ # limitations under the License. # -import sys - from pyspark.sql import Column, Row from pyspark.sql.types import * from pyspark.sql.utils import AnalysisException @@ -109,12 +107,8 @@ def test_access_column(self): self.assertRaises(TypeError, lambda: df[{}]) def test_column_name_with_non_ascii(self): - if sys.version >= '3': - columnName = "数量" - self.assertTrue(isinstance(columnName, str)) - else: - columnName = unicode("数量", "utf-8") - self.assertTrue(isinstance(columnName, unicode)) + columnName = "数量" + self.assertTrue(isinstance(columnName, str)) schema = StructType([StructField(columnName, LongType(), True)]) df = self.spark.createDataFrame([(1,)], schema) self.assertEqual(schema, df.schema) diff --git a/python/pyspark/sql/tests/test_context.py b/python/pyspark/sql/tests/test_context.py index 3b1b638ed4aa6..ff953ba4b4b76 100644 --- a/python/pyspark/sql/tests/test_context.py +++ b/python/pyspark/sql/tests/test_context.py @@ -19,11 +19,7 @@ import sys import tempfile import unittest -try: - from importlib import reload # Python 3.4+ only. -except ImportError: - # Otherwise, we will stick to Python 2's built-in reload. - pass +from importlib import reload import py4j diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 52ae74df5d4f2..7dcc19f3ba45d 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -167,10 +167,6 @@ def test_string_functions(self): TypeError, "must be the same type", lambda: df.select(col('name').substr(0, lit(1)))) - if sys.version_info.major == 2: - self.assertRaises( - TypeError, - lambda: df.select(col('name').substr(long(0), long(1)))) for name in _string_functions.keys(): self.assertEqual( diff --git a/python/pyspark/sql/tests/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py index c1cb30c3caa91..24a73918d8be4 100644 --- a/python/pyspark/sql/tests/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py @@ -32,11 +32,6 @@ import pyarrow as pa -# Tests below use pd.DataFrame.assign that will infer mixed types (unicode/str) for column names -# From kwargs w/ Python 2, so need to set check_column_type=False and avoid this check -_check_column_type = sys.version >= '3' - - @unittest.skipIf( not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message) @@ -109,7 +104,7 @@ def merge_pandas(l, r): 'v2': [90, 100, 110] }) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_empty_group_by(self): left = self.data1 @@ -130,7 +125,7 @@ def merge_pandas(l, r): .merge(left, right, on=['id', 'k']) \ .sort_values(by=['id', 'k']) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_mixed_scalar_udfs_followed_by_cogrouby_apply(self): df = self.spark.range(0, 10).toDF('v1') @@ -173,7 +168,7 @@ def left_assign_key(key, l, _): expected = self.data1.toPandas() expected = expected.assign(key=expected.id % 2 == 0) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_wrong_return_type(self): # Test that we get a sensible exception invalid values passed to apply @@ -224,7 +219,7 @@ def right_assign_key(key, l, r): expected = left.toPandas() if isLeft else right.toPandas() expected = expected.assign(key=expected.id) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) @staticmethod def _test_merge(left, right, output_schema='id long, k int, v int, v2 int'): @@ -246,7 +241,7 @@ def merge_pandas(l, r): .merge(left, right, on=['id', 'k']) \ .sort_values(by=['id', 'k']) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_pandas_grouped_map.py b/python/pyspark/sql/tests/test_pandas_grouped_map.py index cc6167e619285..00cc9b3a64c73 100644 --- a/python/pyspark/sql/tests/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_grouped_map.py @@ -38,11 +38,6 @@ import pyarrow as pa -# Tests below use pd.DataFrame.assign that will infer mixed types (unicode/str) for column names -# from kwargs w/ Python 2, so need to set check_column_type=False and avoid this check -_check_column_type = sys.version >= '3' - - @unittest.skipIf( not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message) @@ -139,9 +134,9 @@ def test_supported_types(self): result3 = df.groupby('id').apply(udf3).sort('id').toPandas() expected3 = expected1 - assert_frame_equal(expected1, result1, check_column_type=_check_column_type) - assert_frame_equal(expected2, result2, check_column_type=_check_column_type) - assert_frame_equal(expected3, result3, check_column_type=_check_column_type) + assert_frame_equal(expected1, result1) + assert_frame_equal(expected2, result2) + assert_frame_equal(expected3, result3) def test_array_type_correct(self): df = self.data.withColumn("arr", array(col("id"))).repartition(1, "id") @@ -159,7 +154,7 @@ def test_array_type_correct(self): result = df.groupby('id').apply(udf).sort('id').toPandas() expected = df.toPandas().groupby('id').apply(udf.func).reset_index(drop=True) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_register_grouped_map_udf(self): foo_udf = pandas_udf(lambda x: x, "id long", PandasUDFType.GROUPED_MAP) @@ -181,7 +176,7 @@ def foo(pdf): result = df.groupby('id').apply(foo).sort('id').toPandas() expected = df.toPandas().groupby('id').apply(foo.func).reset_index(drop=True) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_coerce(self): df = self.data @@ -195,7 +190,7 @@ def test_coerce(self): result = df.groupby('id').apply(foo).sort('id').toPandas() expected = df.toPandas().groupby('id').apply(foo.func).reset_index(drop=True) expected = expected.assign(v=expected.v.astype('float64')) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_complex_groupby(self): df = self.data @@ -213,7 +208,7 @@ def normalize(pdf): expected = pdf.groupby(pdf['id'] % 2 == 0, as_index=False).apply(normalize.func) expected = expected.sort_values(['id', 'v']).reset_index(drop=True) expected = expected.assign(norm=expected.norm.astype('float64')) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_empty_groupby(self): df = self.data @@ -231,7 +226,7 @@ def normalize(pdf): expected = normalize.func(pdf) expected = expected.sort_values(['id', 'v']).reset_index(drop=True) expected = expected.assign(norm=expected.norm.astype('float64')) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_datatype_string(self): df = self.data @@ -244,7 +239,7 @@ def test_datatype_string(self): result = df.groupby('id').apply(foo_udf).sort('id').toPandas() expected = df.toPandas().groupby('id').apply(foo_udf.func).reset_index(drop=True) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) def test_wrong_return_type(self): with QuietTest(self.sc): @@ -301,7 +296,7 @@ def test_timestamp_dst(self): df = self.spark.createDataFrame(dt, 'timestamp').toDF('time') foo_udf = pandas_udf(lambda pdf: pdf, 'time timestamp', PandasUDFType.GROUPED_MAP) result = df.groupby('time').apply(foo_udf).sort('time') - assert_frame_equal(df.toPandas(), result.toPandas(), check_column_type=_check_column_type) + assert_frame_equal(df.toPandas(), result.toPandas()) def test_udf_with_key(self): import numpy as np @@ -355,26 +350,26 @@ def foo3(key, pdf): expected1 = pdf.groupby('id', as_index=False)\ .apply(lambda x: udf1.func((x.id.iloc[0],), x))\ .sort_values(['id', 'v']).reset_index(drop=True) - assert_frame_equal(expected1, result1, check_column_type=_check_column_type) + assert_frame_equal(expected1, result1) # Test groupby expression result2 = df.groupby(df.id % 2).apply(udf1).sort('id', 'v').toPandas() expected2 = pdf.groupby(pdf.id % 2, as_index=False)\ .apply(lambda x: udf1.func((x.id.iloc[0] % 2,), x))\ .sort_values(['id', 'v']).reset_index(drop=True) - assert_frame_equal(expected2, result2, check_column_type=_check_column_type) + assert_frame_equal(expected2, result2) # Test complex groupby result3 = df.groupby(df.id, df.v % 2).apply(udf2).sort('id', 'v').toPandas() expected3 = pdf.groupby([pdf.id, pdf.v % 2], as_index=False)\ .apply(lambda x: udf2.func((x.id.iloc[0], (x.v % 2).iloc[0],), x))\ .sort_values(['id', 'v']).reset_index(drop=True) - assert_frame_equal(expected3, result3, check_column_type=_check_column_type) + assert_frame_equal(expected3, result3) # Test empty groupby result4 = df.groupby().apply(udf3).sort('id', 'v').toPandas() expected4 = udf3.func((), pdf) - assert_frame_equal(expected4, result4, check_column_type=_check_column_type) + assert_frame_equal(expected4, result4) def test_column_order(self): @@ -407,7 +402,7 @@ def change_col_order(pdf): .select('id', 'u', 'v').toPandas() pd_result = grouped_pdf.apply(change_col_order) expected = pd_result.sort_values(['id', 'v']).reset_index(drop=True) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) # Function returns a pdf with positional columns, indexed by range def range_col_order(pdf): @@ -426,7 +421,7 @@ def range_col_order(pdf): pd_result = grouped_pdf.apply(range_col_order) rename_pdf(pd_result, ['id', 'u', 'v']) expected = pd_result.sort_values(['id', 'v']).reset_index(drop=True) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) # Function returns a pdf with columns indexed with integers def int_index(pdf): @@ -444,7 +439,7 @@ def int_index(pdf): pd_result = grouped_pdf.apply(int_index) rename_pdf(pd_result, ['id', 'u', 'v']) expected = pd_result.sort_values(['id', 'v']).reset_index(drop=True) - assert_frame_equal(expected, result, check_column_type=_check_column_type) + assert_frame_equal(expected, result) @pandas_udf('id long, v int', PandasUDFType.GROUPED_MAP) def column_name_typo(pdf): diff --git a/python/pyspark/sql/tests/test_pandas_map.py b/python/pyspark/sql/tests/test_pandas_map.py index f1956a2523e48..02ae6a86f9ab3 100644 --- a/python/pyspark/sql/tests/test_pandas_map.py +++ b/python/pyspark/sql/tests/test_pandas_map.py @@ -19,9 +19,6 @@ import time import unittest -if sys.version >= '3': - unicode = str - from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ pandas_requirement_message, pyarrow_requirement_message diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index 2d38efd39f902..75e2a0929ed29 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -22,10 +22,6 @@ import tempfile import time import unittest - -if sys.version >= '3': - unicode = str - from datetime import date, datetime from decimal import Decimal @@ -319,7 +315,7 @@ def test_vectorized_udf_struct_type(self): StructField('str', StringType())]) def scalar_func(id): - return pd.DataFrame({'id': id, 'str': id.apply(unicode)}) + return pd.DataFrame({'id': id, 'str': id.apply(str)}) def iter_func(it): for id in it: @@ -486,14 +482,14 @@ def test_vectorized_udf_chained_struct_type(self): @pandas_udf(return_type) def scalar_f(id): - return pd.DataFrame({'id': id, 'str': id.apply(unicode)}) + return pd.DataFrame({'id': id, 'str': id.apply(str)}) scalar_g = pandas_udf(lambda x: x, return_type) @pandas_udf(return_type, PandasUDFType.SCALAR_ITER) def iter_f(it): for id in it: - yield pd.DataFrame({'id': id, 'str': id.apply(unicode)}) + yield pd.DataFrame({'id': id, 'str': id.apply(str)}) iter_g = pandas_udf(lambda x: x, return_type, PandasUDFType.SCALAR_ITER) @@ -915,21 +911,12 @@ def to_category_func(x): # Check result of column 'B' must be equal to column 'A' in type and values pd.testing.assert_series_equal(result_spark["A"], result_spark["B"], check_names=False) - @unittest.skipIf(sys.version_info[:2] < (3, 5), "Type hints are supported from Python 3.5.") def test_type_annotation(self): # Regression test to check if type hints can be used. See SPARK-23569. - # Note that it throws an error during compilation in lower Python versions if 'exec' - # is not used. Also, note that we explicitly use another dictionary to avoid modifications - # in the current 'locals()'. - # - # Hyukjin: I think it's an ugly way to test issues about syntax specific in - # higher versions of Python, which we shouldn't encourage. This was the last resort - # I could come up with at that time. - _locals = {} - exec( - "import pandas as pd\ndef noop(col: pd.Series) -> pd.Series: return col", - _locals) - df = self.spark.range(1).select(pandas_udf(f=_locals['noop'], returnType='bigint')('id')) + def noop(col: pd.Series) -> pd.Series: + return col + + df = self.spark.range(1).select(pandas_udf(f=noop, returnType='bigint')('id')) self.assertEqual(df.first()[0], 0) def test_mixed_udf(self): diff --git a/python/pyspark/sql/tests/test_pandas_udf_typehints.py b/python/pyspark/sql/tests/test_pandas_udf_typehints.py index 2582080056864..618164fa8496f 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_typehints.py +++ b/python/pyspark/sql/tests/test_pandas_udf_typehints.py @@ -14,9 +14,9 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import sys import unittest import inspect +from typing import Union, Iterator, Tuple from pyspark.sql.functions import mean, lit from pyspark.testing.sqlutils import ReusedSQLTestCase, \ @@ -24,209 +24,162 @@ pyarrow_requirement_message from pyspark.sql.pandas.typehints import infer_eval_type from pyspark.sql.pandas.functions import pandas_udf, PandasUDFType +from pyspark.sql import Row if have_pandas: import pandas as pd + import numpy as np from pandas.util.testing import assert_frame_equal -python_requirement_message = "pandas UDF with type hints are supported with Python 3.6+." - @unittest.skipIf( - not have_pandas or not have_pyarrow or sys.version_info[:2] < (3, 6), - pandas_requirement_message or pyarrow_requirement_message or python_requirement_message) + not have_pandas or not have_pyarrow, + pandas_requirement_message or pyarrow_requirement_message) class PandasUDFTypeHintsTests(ReusedSQLTestCase): - # Note that, we should remove `exec` once we drop Python 2 in this class. - - def setUp(self): - self.local = {'pd': pd} - def test_type_annotation_scalar(self): - exec( - "def func(col: pd.Series) -> pd.Series: pass", - self.local) + def func(col: pd.Series) -> pd.Series: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR) - exec( - "def func(col: pd.DataFrame, col1: pd.Series) -> pd.DataFrame: pass", - self.local) + def func(col: pd.DataFrame, col1: pd.Series) -> pd.DataFrame: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR) - exec( - "def func(col: pd.DataFrame, *args: pd.Series) -> pd.Series: pass", - self.local) + def func(col: pd.DataFrame, *args: pd.Series) -> pd.Series: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR) - exec( - "def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame) -> pd.Series:\n" - " pass", - self.local) + def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame) -> pd.Series: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR) - exec( - "def func(col: pd.Series, *, col2: pd.DataFrame) -> pd.DataFrame:\n" - " pass", - self.local) + def func(col: pd.Series, *, col2: pd.DataFrame) -> pd.DataFrame: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR) - exec( - "from typing import Union\n" - "def func(col: Union[pd.Series, pd.DataFrame], *, col2: pd.DataFrame) -> pd.Series:\n" - " pass", - self.local) + def func(col: Union[pd.Series, pd.DataFrame], *, col2: pd.DataFrame) -> pd.Series: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR) def test_type_annotation_scalar_iter(self): - exec( - "from typing import Iterator\n" - "def func(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: pass", - self.local) + def func(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR_ITER) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR_ITER) - exec( - "from typing import Iterator, Tuple\n" - "def func(iter: Iterator[Tuple[pd.DataFrame, pd.Series]]) -> Iterator[pd.DataFrame]:\n" - " pass", - self.local) + def func(iter: Iterator[Tuple[pd.DataFrame, pd.Series]]) -> Iterator[pd.DataFrame]: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR_ITER) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR_ITER) - exec( - "from typing import Iterator, Tuple\n" - "def func(iter: Iterator[Tuple[pd.DataFrame, ...]]) -> Iterator[pd.Series]: pass", - self.local) + def func(iter: Iterator[Tuple[pd.DataFrame, ...]]) -> Iterator[pd.Series]: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR_ITER) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR_ITER) - exec( - "from typing import Iterator, Tuple, Union\n" - "def func(iter: Iterator[Tuple[Union[pd.DataFrame, pd.Series], ...]])" - " -> Iterator[pd.Series]: pass", - self.local) + def func( + iter: Iterator[Tuple[Union[pd.DataFrame, pd.Series], ...]] + ) -> Iterator[pd.Series]: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR_ITER) + infer_eval_type(inspect.signature(func)), PandasUDFType.SCALAR_ITER) def test_type_annotation_group_agg(self): - exec( - "def func(col: pd.Series) -> str: pass", - self.local) + + def func(col: pd.Series) -> str: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + infer_eval_type(inspect.signature(func)), PandasUDFType.GROUPED_AGG) - exec( - "def func(col: pd.DataFrame, col1: pd.Series) -> int: pass", - self.local) + def func(col: pd.DataFrame, col1: pd.Series) -> int: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + infer_eval_type(inspect.signature(func)), PandasUDFType.GROUPED_AGG) - exec( - "from pyspark.sql import Row\n" - "def func(col: pd.DataFrame, *args: pd.Series) -> Row: pass", - self.local) + def func(col: pd.DataFrame, *args: pd.Series) -> Row: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + infer_eval_type(inspect.signature(func)), PandasUDFType.GROUPED_AGG) - exec( - "def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame) -> str:\n" - " pass", - self.local) + def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame) -> str: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + infer_eval_type(inspect.signature(func)), PandasUDFType.GROUPED_AGG) - exec( - "def func(col: pd.Series, *, col2: pd.DataFrame) -> float:\n" - " pass", - self.local) + def func(col: pd.Series, *, col2: pd.DataFrame) -> float: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + infer_eval_type(inspect.signature(func)), PandasUDFType.GROUPED_AGG) - exec( - "from typing import Union\n" - "def func(col: Union[pd.Series, pd.DataFrame], *, col2: pd.DataFrame) -> float:\n" - " pass", - self.local) + def func(col: Union[pd.Series, pd.DataFrame], *, col2: pd.DataFrame) -> float: + pass self.assertEqual( - infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + infer_eval_type(inspect.signature(func)), PandasUDFType.GROUPED_AGG) def test_type_annotation_negative(self): - exec( - "def func(col: str) -> pd.Series: pass", - self.local) + + def func(col: str) -> pd.Series: + pass self.assertRaisesRegex( NotImplementedError, "Unsupported signature.*str", - infer_eval_type, inspect.signature(self.local['func'])) + infer_eval_type, inspect.signature(func)) - exec( - "def func(col: pd.DataFrame, col1: int) -> pd.DataFrame: pass", - self.local) + def func(col: pd.DataFrame, col1: int) -> pd.DataFrame: + pass self.assertRaisesRegex( NotImplementedError, "Unsupported signature.*int", - infer_eval_type, inspect.signature(self.local['func'])) + infer_eval_type, inspect.signature(func)) - exec( - "from typing import Union\n" - "def func(col: Union[pd.DataFrame, str], col1: int) -> pd.DataFrame: pass", - self.local) + def func(col: Union[pd.DataFrame, str], col1: int) -> pd.DataFrame: + pass self.assertRaisesRegex( NotImplementedError, "Unsupported signature.*str", - infer_eval_type, inspect.signature(self.local['func'])) + infer_eval_type, inspect.signature(func)) - exec( - "from typing import Tuple\n" - "def func(col: pd.Series) -> Tuple[pd.DataFrame]: pass", - self.local) + def func(col: pd.Series) -> Tuple[pd.DataFrame]: + pass self.assertRaisesRegex( NotImplementedError, "Unsupported signature.*Tuple", - infer_eval_type, inspect.signature(self.local['func'])) + infer_eval_type, inspect.signature(func)) - exec( - "def func(col, *args: pd.Series) -> pd.Series: pass", - self.local) + def func(col, *args: pd.Series) -> pd.Series: + pass self.assertRaisesRegex( ValueError, "should be specified.*Series", - infer_eval_type, inspect.signature(self.local['func'])) + infer_eval_type, inspect.signature(func)) - exec( - "def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame):\n" - " pass", - self.local) + def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame): + pass self.assertRaisesRegex( ValueError, "should be specified.*Series", - infer_eval_type, inspect.signature(self.local['func'])) + infer_eval_type, inspect.signature(func)) - exec( - "def func(col: pd.Series, *, col2) -> pd.DataFrame:\n" - " pass", - self.local) + def func(col: pd.Series, *, col2) -> pd.DataFrame: + pass self.assertRaisesRegex( ValueError, "should be specified.*Series", - infer_eval_type, inspect.signature(self.local['func'])) + infer_eval_type, inspect.signature(func)) def test_scalar_udf_type_hint(self): df = self.spark.range(10).selectExpr("id", "id as v") - exec( - "import typing\n" - "def plus_one(v: typing.Union[pd.Series, pd.DataFrame]) -> pd.Series:\n" - " return v + 1", - self.local) - - plus_one = pandas_udf("long")(self.local["plus_one"]) + def plus_one(v: Union[pd.Series, pd.DataFrame]) -> pd.Series: + return v + 1 + plus_one = pandas_udf("long")(plus_one) actual = df.select(plus_one(df.v).alias("plus_one")) expected = df.selectExpr("(v + 1) as plus_one") assert_frame_equal(expected.toPandas(), actual.toPandas()) @@ -234,14 +187,11 @@ def test_scalar_udf_type_hint(self): def test_scalar_iter_udf_type_hint(self): df = self.spark.range(10).selectExpr("id", "id as v") - exec( - "import typing\n" - "def plus_one(itr: typing.Iterator[pd.Series]) -> typing.Iterator[pd.Series]:\n" - " for s in itr:\n" - " yield s + 1", - self.local) + def plus_one(itr: Iterator[pd.Series]) -> Iterator[pd.Series]: + for s in itr: + yield s + 1 - plus_one = pandas_udf("long")(self.local["plus_one"]) + plus_one = pandas_udf("long")(plus_one) actual = df.select(plus_one(df.v).alias("plus_one")) expected = df.selectExpr("(v + 1) as plus_one") @@ -249,13 +199,11 @@ def test_scalar_iter_udf_type_hint(self): def test_group_agg_udf_type_hint(self): df = self.spark.range(10).selectExpr("id", "id as v") - exec( - "import numpy as np\n" - "def weighted_mean(v: pd.Series, w: pd.Series) -> float:\n" - " return np.average(v, weights=w)", - self.local) - weighted_mean = pandas_udf("double")(self.local["weighted_mean"]) + def weighted_mean(v: pd.Series, w: pd.Series) -> float: + return np.average(v, weights=w) + + weighted_mean = pandas_udf("double")(weighted_mean) actual = df.groupby('id').agg(weighted_mean(df.v, lit(1.0))).sort('id') expected = df.groupby('id').agg(mean(df.v).alias('weighted_mean(v, 1.0)')).sort('id') @@ -263,12 +211,9 @@ def test_group_agg_udf_type_hint(self): def test_ignore_type_hint_in_group_apply_in_pandas(self): df = self.spark.range(10) - exec( - "def pandas_plus_one(v: pd.DataFrame) -> pd.DataFrame:\n" - " return v + 1", - self.local) - pandas_plus_one = self.local["pandas_plus_one"] + def pandas_plus_one(v: pd.DataFrame) -> pd.DataFrame: + return v + 1 actual = df.groupby('id').applyInPandas(pandas_plus_one, schema=df.schema).sort('id') expected = df.selectExpr("id + 1 as id") @@ -276,12 +221,9 @@ def test_ignore_type_hint_in_group_apply_in_pandas(self): def test_ignore_type_hint_in_cogroup_apply_in_pandas(self): df = self.spark.range(10) - exec( - "def pandas_plus_one(left: pd.DataFrame, right: pd.DataFrame) -> pd.DataFrame:\n" - " return left + 1", - self.local) - pandas_plus_one = self.local["pandas_plus_one"] + def pandas_plus_one(left: pd.DataFrame, right: pd.DataFrame) -> pd.DataFrame: + return left + 1 actual = df.groupby('id').cogroup( self.spark.range(10).groupby("id") @@ -291,13 +233,9 @@ def test_ignore_type_hint_in_cogroup_apply_in_pandas(self): def test_ignore_type_hint_in_map_in_pandas(self): df = self.spark.range(10) - exec( - "from typing import Iterator\n" - "def pandas_plus_one(iter: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]:\n" - " return map(lambda v: v + 1, iter)", - self.local) - pandas_plus_one = self.local["pandas_plus_one"] + def pandas_plus_one(iter: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: + return map(lambda v: v + 1, iter) actual = df.mapInPandas(pandas_plus_one, schema=df.schema) expected = df.selectExpr("id + 1 as id") diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 016cafd669019..051c8bde50ad9 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -56,7 +56,7 @@ def test_infer_schema_to_local(self): self.assertEqual(10, df3.count()) def test_apply_schema_to_dict_and_rows(self): - schema = StructType().add("b", StringType()).add("a", IntegerType()) + schema = StructType().add("a", IntegerType()).add("b", StringType()) input = [{"a": 1}, {"b": "coffee"}] rdd = self.sc.parallelize(input) for verify in [False, True]: @@ -72,7 +72,6 @@ def test_apply_schema_to_dict_and_rows(self): self.assertEqual(10, df4.count()) def test_create_dataframe_schema_mismatch(self): - input = [Row(a=1)] rdd = self.sc.parallelize(range(3)).map(lambda i: Row(a=i)) schema = StructType([StructField("a", IntegerType()), StructField("b", StringType())]) df = self.spark.createDataFrame(rdd, schema) @@ -540,7 +539,6 @@ def test_infer_long_type(self): self.assertEqual(_infer_type(2**61), LongType()) self.assertEqual(_infer_type(2**71), LongType()) - @unittest.skipIf(sys.version < "3", "only Python 3 infers bytes as binary type") def test_infer_binary_type(self): binaryrow = [Row(f1='a', f2=b"abcd")] df = self.sc.parallelize(binaryrow).toDF() @@ -665,10 +663,6 @@ def assertCollectSuccess(typecode, value): supported_string_types += ['u'] # test unicode assertCollectSuccess('u', u'a') - if sys.version_info[0] < 3: - supported_string_types += ['c'] - # test string - assertCollectSuccess('c', 'a') # supported float and double # @@ -721,12 +715,8 @@ def assertCollectSuccess(typecode, value): # # Keys in _array_type_mappings is a complete list of all supported types, # and types not in _array_type_mappings are considered unsupported. - # `array.typecodes` are not supported in python 2. - if sys.version_info[0] < 3: - all_types = set(['c', 'b', 'B', 'u', 'h', 'H', 'i', 'I', 'l', 'L', 'f', 'd']) - else: - # PyPy seems not having array.typecodes. - all_types = set(['b', 'B', 'u', 'h', 'H', 'i', 'I', 'l', 'L', 'q', 'Q', 'f', 'd']) + # PyPy seems not having array.typecodes. + all_types = set(['b', 'B', 'u', 'h', 'H', 'i', 'I', 'l', 'L', 'q', 'Q', 'f', 'd']) unsupported_types = all_types - set(supported_types) # test unsupported types for t in unsupported_types: @@ -767,10 +757,7 @@ def test_row_without_column_name(self): self.assertEqual(repr(row), "") # test __repr__ with unicode values - if sys.version_info.major >= 3: - self.assertEqual(repr(Row("数", "量")), "") - else: - self.assertEqual(repr(Row(u"数", u"量")), r"") + self.assertEqual(repr(Row("数", "量")), "") def test_empty_row(self): row = Row() @@ -888,7 +875,6 @@ def __init__(self, **kwargs): ({"s": "a", "f": 1.0}, schema), (Row(s="a", i=1), schema), (Row(s="a", i=None), schema), - (Row(s="a", i=1, f=1.0), schema), (["a", 1], schema), (["a", None], schema), (("a", 1), schema), @@ -973,18 +959,13 @@ def __init__(self, **kwargs): with self.assertRaises(exp, msg=msg): _make_type_verifier(data_type, nullable=False)(obj) - @unittest.skipIf(sys.version_info[:2] < (3, 6), "Create Row without sorting fields") def test_row_without_field_sorting(self): - sorting_enabled_tmp = Row._row_field_sorting_enabled - Row._row_field_sorting_enabled = False - r = Row(b=1, a=2) TestRow = Row("b", "a") expected = TestRow(1, 2) self.assertEqual(r, expected) self.assertEqual(repr(r), "Row(b=1, a=2)") - Row._row_field_sorting_enabled = sorting_enabled_tmp if __name__ == "__main__": diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 320a68dffe7a3..cc08482c735b1 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -15,7 +15,6 @@ # limitations under the License. # -import os import sys import decimal import time @@ -26,11 +25,6 @@ import base64 from array import array import ctypes -import warnings - -if sys.version >= "3": - long = int - basestring = unicode = str from py4j.protocol import register_input_converter from py4j.java_gateway import JavaClass @@ -409,9 +403,7 @@ def __init__(self, name, dataType, nullable=True, metadata=None): """ assert isinstance(dataType, DataType),\ "dataType %s should be an instance of %s" % (dataType, DataType) - assert isinstance(name, basestring), "field name %s should be string" % (name) - if not isinstance(name, str): - name = name.encode('utf-8') + assert isinstance(name, str), "field name %s should be a string" % (name) self.name = name self.dataType = dataType self.nullable = nullable @@ -613,8 +605,6 @@ def toInternal(self, obj): else: if isinstance(obj, dict): return tuple(obj.get(n) for n in self.names) - elif isinstance(obj, Row) and getattr(obj, "__from_dict__", False): - return tuple(obj[n] for n in self.names) elif isinstance(obj, (list, tuple)): return tuple(obj) elif hasattr(obj, "__dict__"): @@ -904,19 +894,9 @@ def _parse_datatype_json_value(json_value): datetime.date: DateType, datetime.datetime: TimestampType, datetime.time: TimestampType, + bytes: BinaryType, } -if sys.version < "3": - _type_mappings.update({ - unicode: StringType, - long: LongType, - }) - -if sys.version >= "3": - _type_mappings.update({ - bytes: BinaryType, - }) - # Mapping Python array types to Spark SQL DataType # We should be careful here. The size of these types in python depends on C # implementation. We need to make sure that this conversion does not lose any @@ -990,20 +970,6 @@ def _int_size_to_type(size): if sys.version_info[0] < 4: _array_type_mappings['u'] = StringType -# Type code 'c' are only available at python 2 -if sys.version_info[0] < 3: - _array_type_mappings['c'] = StringType - -# SPARK-21465: -# In python2, array of 'L' happened to be mistakenly, just partially supported. To -# avoid breaking user's code, we should keep this partial support. Below is a -# dirty hacking to keep this partial support and pass the unit test. -import platform -if sys.version_info[0] < 3 and platform.python_implementation() != 'PyPy': - if 'L' not in _array_type_mappings.keys(): - _array_type_mappings['L'] = LongType - _array_unsigned_int_typecode_ctype_mappings['L'] = ctypes.c_uint - def _infer_type(obj): """Infer the DataType from obj @@ -1187,14 +1153,14 @@ def convert_struct(obj): _acceptable_types = { BooleanType: (bool,), - ByteType: (int, long), - ShortType: (int, long), - IntegerType: (int, long), - LongType: (int, long), + ByteType: (int,), + ShortType: (int,), + IntegerType: (int,), + LongType: (int,), FloatType: (float,), DoubleType: (float,), DecimalType: (decimal.Decimal,), - StringType: (str, unicode), + StringType: (str,), BinaryType: (bytearray, bytes), DateType: (datetime.date, datetime.datetime), TimestampType: (datetime.datetime,), @@ -1376,10 +1342,6 @@ def verify_struct(obj): if isinstance(obj, dict): for f, verifier in verifiers: verifier(obj.get(f)) - elif isinstance(obj, Row) and getattr(obj, "__from_dict__", False): - # the order in obj could be different than dataType.fields - for f, verifier in verifiers: - verifier(obj[f]) elif isinstance(obj, (tuple, list)): if len(obj) != len(verifiers): raise ValueError( @@ -1438,21 +1400,11 @@ class Row(tuple): NOTE: As of Spark 3.0.0, Rows created from named arguments no longer have field names sorted alphabetically and will be ordered in the position as - entered. To enable sorting for Rows compatible with Spark 2.x, set the - environment variable "PYSPARK_ROW_FIELD_SORTING_ENABLED" to "true". This - option is deprecated and will be removed in future versions of Spark. For - Python versions < 3.6, the order of named arguments is not guaranteed to - be the same as entered, see https://www.python.org/dev/peps/pep-0468. In - this case, a warning will be issued and the Row will fallback to sort the - field names automatically. - - NOTE: Examples with Row in pydocs are run with the environment variable - "PYSPARK_ROW_FIELD_SORTING_ENABLED" set to "true" which results in output - where fields are sorted. + entered. >>> row = Row(name="Alice", age=11) >>> row - Row(age=11, name='Alice') + Row(name='Alice', age=11) >>> row['name'], row['age'] ('Alice', 11) >>> row.name, row.age @@ -1476,47 +1428,22 @@ class Row(tuple): Row(name='Alice', age=11) This form can also be used to create rows as tuple values, i.e. with unnamed - fields. Beware that such Row objects have different equality semantics: + fields. >>> row1 = Row("Alice", 11) >>> row2 = Row(name="Alice", age=11) >>> row1 == row2 - False - >>> row3 = Row(a="Alice", b=11) - >>> row1 == row3 True """ - # Remove after Python < 3.6 dropped, see SPARK-29748 - _row_field_sorting_enabled = \ - os.environ.get('PYSPARK_ROW_FIELD_SORTING_ENABLED', 'false').lower() == 'true' - - if _row_field_sorting_enabled: - warnings.warn("The environment variable 'PYSPARK_ROW_FIELD_SORTING_ENABLED' " - "is deprecated and will be removed in future versions of Spark") - def __new__(cls, *args, **kwargs): if args and kwargs: raise ValueError("Can not use both args " "and kwargs to create Row") if kwargs: - if not Row._row_field_sorting_enabled and sys.version_info[:2] < (3, 6): - warnings.warn("To use named arguments for Python version < 3.6, Row fields will be " - "automatically sorted. This warning can be skipped by setting the " - "environment variable 'PYSPARK_ROW_FIELD_SORTING_ENABLED' to 'true'.") - Row._row_field_sorting_enabled = True - # create row objects - if Row._row_field_sorting_enabled: - # Remove after Python < 3.6 dropped, see SPARK-29748 - names = sorted(kwargs.keys()) - row = tuple.__new__(cls, [kwargs[n] for n in names]) - row.__fields__ = names - row.__from_dict__ = True - else: - row = tuple.__new__(cls, list(kwargs.values())) - row.__fields__ = list(kwargs.keys()) - + row = tuple.__new__(cls, list(kwargs.values())) + row.__fields__ = list(kwargs.keys()) return row else: # create row class or objects @@ -1537,7 +1464,7 @@ def asDict(self, recursive=False): >>> Row(name="Alice", age=11).asDict() == {'name': 'Alice', 'age': 11} True >>> row = Row(key=1, value=Row(name='a', age=2)) - >>> row.asDict() == {'key': 1, 'value': Row(age=2, name='a')} + >>> row.asDict() == {'key': 1, 'value': Row(name='a', age=2)} True >>> row.asDict(True) == {'key': 1, 'value': {'name': 'a', 'age': 2}} True @@ -1600,7 +1527,7 @@ def __getattr__(self, item): raise AttributeError(item) def __setattr__(self, key, value): - if key != '__fields__' and key != "__from_dict__": + if key != '__fields__': raise Exception("Row is read-only") self.__dict__[key] = value diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index da68583b04e1c..100481cf12899 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -21,7 +21,7 @@ import sys from pyspark import SparkContext, since -from pyspark.rdd import _prepare_for_python_RDD, PythonEvalType, ignore_unicode_prefix +from pyspark.rdd import _prepare_for_python_RDD, PythonEvalType from pyspark.sql.column import Column, _to_java_column, _to_seq from pyspark.sql.types import StringType, DataType, StructType, _parse_datatype_string from pyspark.sql.pandas.types import to_arrow_type @@ -232,7 +232,6 @@ class UDFRegistration(object): def __init__(self, sparkSession): self.sparkSession = sparkSession - @ignore_unicode_prefix @since("1.3.1") def register(self, name, f, returnType=None): """Register a Python function (including lambda function) or a user-defined function @@ -261,10 +260,10 @@ def register(self, name, f, returnType=None): >>> strlen = spark.udf.register("stringLengthString", lambda x: len(x)) >>> spark.sql("SELECT stringLengthString('test')").collect() - [Row(stringLengthString(test)=u'4')] + [Row(stringLengthString(test)='4')] >>> spark.sql("SELECT 'foo' AS text").select(strlen("text")).collect() - [Row(stringLengthString(text)=u'3')] + [Row(stringLengthString(text)='3')] >>> from pyspark.sql.types import IntegerType >>> _ = spark.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) @@ -349,7 +348,6 @@ def register(self, name, f, returnType=None): self.sparkSession._jsparkSession.udf().registerPython(name, register_udf._judf) return return_udf - @ignore_unicode_prefix @since(2.3) def registerJavaFunction(self, name, javaClassName, returnType=None): """Register a Java user-defined function as a SQL function. @@ -389,7 +387,6 @@ def registerJavaFunction(self, name, javaClassName, returnType=None): jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) self.sparkSession._jsparkSession.udf().registerJava(name, javaClassName, jdt) - @ignore_unicode_prefix @since(2.3) def registerJavaUDAF(self, name, javaClassName): """Register a Java user-defined aggregate function as a SQL function. @@ -403,7 +400,7 @@ def registerJavaUDAF(self, name, javaClassName): >>> df.createOrReplaceTempView("df") >>> q = "SELECT name, javaUDAF(id) as avg from df group by name order by name desc" >>> spark.sql(q).collect() # doctest: +SKIP - [Row(name=u'b', avg=102.0), Row(name=u'a', avg=102.0)] + [Row(name='b', avg=102.0), Row(name='a', avg=102.0)] """ self.sparkSession._jsparkSession.udf().registerJavaUDAF(name, javaClassName) @@ -419,9 +416,6 @@ def _test(): .appName("sql.udf tests")\ .getOrCreate() globs['spark'] = spark - # Hack to skip the unit tests in register. These are currently being tested in proper tests. - # We should reenable this test once we completely drop Python 2. - del pyspark.sql.udf.UDFRegistration.register (failure_count, test_count) = doctest.testmod( pyspark.sql.udf, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 1d5bc49d252e2..bd76d880055cd 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -16,22 +16,9 @@ # import py4j -import sys from pyspark import SparkContext -if sys.version_info.major >= 3: - unicode = str - # Disable exception chaining (PEP 3134) in captured exceptions - # in order to hide JVM stacktace. - exec(""" -def raise_from(e): - raise e from None -""") -else: - def raise_from(e): - raise e - class CapturedException(Exception): def __init__(self, desc, stackTrace, cause=None): @@ -45,11 +32,7 @@ def __str__(self): desc = self.desc if debug_enabled: desc = desc + "\n\nJVM stacktrace:\n%s" % self.stackTrace - # encode unicode instance for python2 for human readable description - if sys.version_info.major < 3 and isinstance(desc, unicode): - return str(desc.encode('utf-8')) - else: - return str(desc) + return str(desc) class AnalysisException(CapturedException): @@ -131,7 +114,7 @@ def deco(*a, **kw): if not isinstance(converted, UnknownException): # Hide where the exception came from that shows a non-Pythonic # JVM exception message. - raise_from(converted) + raise converted from None else: raise return deco diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 6199611940dc9..170f0c0ef7593 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from py4j.java_gateway import java_import, is_instance_of from pyspark import RDD, SparkConf diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 60562a6c92aff..000318588ef88 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -21,11 +21,6 @@ from itertools import chain from datetime import datetime -if sys.version < "3": - from itertools import imap as map, ifilter as filter -else: - long = int - from py4j.protocol import Py4JJavaError from pyspark import RDD @@ -404,7 +399,7 @@ def _jtime(self, timestamp): """ if isinstance(timestamp, datetime): timestamp = time.mktime(timestamp.timetuple()) - return self._sc._jvm.Time(long(timestamp * 1000)) + return self._sc._jvm.Time(int(timestamp * 1000)) def slice(self, begin, end): """ diff --git a/python/pyspark/taskcontext.py b/python/pyspark/taskcontext.py index 8f419a5e8446a..d8aa5f93182e2 100644 --- a/python/pyspark/taskcontext.py +++ b/python/pyspark/taskcontext.py @@ -14,10 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # - -from __future__ import print_function -import json - from pyspark.java_gateway import local_connect_and_auth from pyspark.serializers import read_int, write_int, write_with_length, UTF8Deserializer diff --git a/python/pyspark/testing/sqlutils.py b/python/pyspark/testing/sqlutils.py index 085fce6daa4ec..e85cae7dda2c6 100644 --- a/python/pyspark/testing/sqlutils.py +++ b/python/pyspark/testing/sqlutils.py @@ -24,7 +24,6 @@ from pyspark.sql import SparkSession from pyspark.sql.types import ArrayType, DoubleType, UserDefinedType, Row from pyspark.testing.utils import ReusedPySparkTestCase -from pyspark.util import _exception_message pandas_requirement_message = None @@ -33,7 +32,7 @@ require_minimum_pandas_version() except ImportError as e: # If Pandas version requirement is not satisfied, skip related tests. - pandas_requirement_message = _exception_message(e) + pandas_requirement_message = str(e) pyarrow_requirement_message = None try: @@ -41,14 +40,14 @@ require_minimum_pyarrow_version() except ImportError as e: # If Arrow version requirement is not satisfied, skip related tests. - pyarrow_requirement_message = _exception_message(e) + pyarrow_requirement_message = str(e) test_not_compiled_message = None try: from pyspark.sql.utils import require_test_compiled require_test_compiled() except Exception as e: - test_not_compiled_message = _exception_message(e) + test_not_compiled_message = str(e) have_pandas = pandas_requirement_message is None have_pyarrow = pyarrow_requirement_message is None diff --git a/python/pyspark/tests/test_profiler.py b/python/pyspark/tests/test_profiler.py index 04ca5a3896bf4..dbce72a0d3489 100644 --- a/python/pyspark/tests/test_profiler.py +++ b/python/pyspark/tests/test_profiler.py @@ -19,15 +19,11 @@ import sys import tempfile import unittest +from io import StringIO from pyspark import SparkConf, SparkContext, BasicProfiler from pyspark.testing.utils import PySparkTestCase -if sys.version >= "3": - from io import StringIO -else: - from StringIO import StringIO - class ProfilerTests(PySparkTestCase): diff --git a/python/pyspark/tests/test_rdd.py b/python/pyspark/tests/test_rdd.py index 6c5b818056f2d..1a580e27ea527 100644 --- a/python/pyspark/tests/test_rdd.py +++ b/python/pyspark/tests/test_rdd.py @@ -32,9 +32,6 @@ MarshalSerializer, UTF8Deserializer, NoOpSerializer from pyspark.testing.utils import ReusedPySparkTestCase, SPARK_HOME, QuietTest -if sys.version_info[0] >= 3: - xrange = range - global_func = lambda: "Hi" @@ -193,15 +190,13 @@ def test_deleting_input_files(self): def test_sampling_default_seed(self): # Test for SPARK-3995 (default seed setting) - data = self.sc.parallelize(xrange(1000), 1) + data = self.sc.parallelize(range(1000), 1) subset = data.takeSample(False, 10) self.assertEqual(len(subset), 10) def test_aggregate_mutable_zero_value(self): # Test for SPARK-9021; uses aggregate and treeAggregate to build dict # representing a counter of ints - # NOTE: dict is used instead of collections.Counter for Python 2.6 - # compatibility from collections import defaultdict # Show that single or multiple partitions work @@ -262,8 +257,6 @@ def comboOp(x, y): def test_fold_mutable_zero_value(self): # Test for SPARK-9021; uses fold to merge an RDD of dict counters into # a single dict - # NOTE: dict is used instead of collections.Counter for Python 2.6 - # compatibility from collections import defaultdict counts1 = defaultdict(int, dict((i, 1) for i in range(10))) @@ -439,7 +432,7 @@ def run(f, sc): def test_large_closure(self): N = 200000 - data = [float(i) for i in xrange(N)] + data = [float(i) for i in range(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) self.assertEqual(N, rdd.first()) # regression test for SPARK-6886 @@ -464,8 +457,8 @@ def test_zip_with_different_serializers(self): def test_zip_with_different_object_sizes(self): # regress test for SPARK-5973 - a = self.sc.parallelize(xrange(10000)).map(lambda i: '*' * i) - b = self.sc.parallelize(xrange(10000, 20000)).map(lambda i: '*' * i) + a = self.sc.parallelize(range(10000)).map(lambda i: '*' * i) + b = self.sc.parallelize(range(10000, 20000)).map(lambda i: '*' * i) self.assertEqual(10000, a.zip(b).count()) def test_zip_with_different_number_of_items(self): @@ -487,7 +480,7 @@ def test_zip_with_different_number_of_items(self): self.assertRaises(Exception, lambda: a.zip(b).count()) def test_count_approx_distinct(self): - rdd = self.sc.parallelize(xrange(1000)) + rdd = self.sc.parallelize(range(1000)) self.assertTrue(950 < rdd.countApproxDistinct(0.03) < 1050) self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.03) < 1050) self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.03) < 1050) @@ -641,7 +634,7 @@ def test_distinct(self): def test_external_group_by_key(self): self.sc._conf.set("spark.python.worker.memory", "1m") N = 2000001 - kv = self.sc.parallelize(xrange(N)).map(lambda x: (x % 3, x)) + kv = self.sc.parallelize(range(N)).map(lambda x: (x % 3, x)) gkv = kv.groupByKey().cache() self.assertEqual(3, gkv.count()) filtered = gkv.filter(lambda kv: kv[0] == 1) @@ -698,7 +691,7 @@ def test_multiple_python_java_RDD_conversions(self): # Regression test for SPARK-6294 def test_take_on_jrdd(self): - rdd = self.sc.parallelize(xrange(1 << 20)).map(lambda x: str(x)) + rdd = self.sc.parallelize(range(1 << 20)).map(lambda x: str(x)) rdd._jrdd.first() def test_sortByKey_uses_all_partitions_not_only_first_and_last(self): diff --git a/python/pyspark/tests/test_readwrite.py b/python/pyspark/tests/test_readwrite.py index 734b7e4789f61..faa006c7d82e5 100644 --- a/python/pyspark/tests/test_readwrite.py +++ b/python/pyspark/tests/test_readwrite.py @@ -38,104 +38,6 @@ def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() shutil.rmtree(cls.tempdir.name) - @unittest.skipIf(sys.version >= "3", "serialize array of byte") - def test_sequencefiles(self): - basepath = self.tempdir.name - ints = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfint/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.Text").collect()) - ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] - self.assertEqual(ints, ei) - - doubles = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfdouble/", - "org.apache.hadoop.io.DoubleWritable", - "org.apache.hadoop.io.Text").collect()) - ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')] - self.assertEqual(doubles, ed) - - bytes = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbytes/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.BytesWritable").collect()) - ebs = [(1, bytearray('aa', 'utf-8')), - (1, bytearray('aa', 'utf-8')), - (2, bytearray('aa', 'utf-8')), - (2, bytearray('bb', 'utf-8')), - (2, bytearray('bb', 'utf-8')), - (3, bytearray('cc', 'utf-8'))] - self.assertEqual(bytes, ebs) - - text = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sftext/", - "org.apache.hadoop.io.Text", - "org.apache.hadoop.io.Text").collect()) - et = [(u'1', u'aa'), - (u'1', u'aa'), - (u'2', u'aa'), - (u'2', u'bb'), - (u'2', u'bb'), - (u'3', u'cc')] - self.assertEqual(text, et) - - bools = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbool/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.BooleanWritable").collect()) - eb = [(1, False), (1, True), (2, False), (2, False), (2, True), (3, True)] - self.assertEqual(bools, eb) - - nulls = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfnull/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.BooleanWritable").collect()) - en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)] - self.assertEqual(nulls, en) - - maps = self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect() - em = [(1, {}), - (1, {3.0: u'bb'}), - (2, {1.0: u'aa'}), - (2, {1.0: u'cc'}), - (3, {2.0: u'dd'})] - for v in maps: - self.assertTrue(v in em) - - # arrays get pickled to tuples by default - tuples = sorted(self.sc.sequenceFile( - basepath + "/sftestdata/sfarray/", - "org.apache.hadoop.io.IntWritable", - "org.apache.spark.api.python.DoubleArrayWritable").collect()) - et = [(1, ()), - (2, (3.0, 4.0, 5.0)), - (3, (4.0, 5.0, 6.0))] - self.assertEqual(tuples, et) - - # with custom converters, primitive arrays can stay as arrays - arrays = sorted(self.sc.sequenceFile( - basepath + "/sftestdata/sfarray/", - "org.apache.hadoop.io.IntWritable", - "org.apache.spark.api.python.DoubleArrayWritable", - valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect()) - ea = [(1, array('d')), - (2, array('d', [3.0, 4.0, 5.0])), - (3, array('d', [4.0, 5.0, 6.0]))] - self.assertEqual(arrays, ea) - - clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", - "org.apache.hadoop.io.Text", - "org.apache.spark.api.python.TestWritable").collect()) - cname = u'org.apache.spark.api.python.TestWritable' - ec = [(u'1', {u'__class__': cname, u'double': 1.0, u'int': 1, u'str': u'test1'}), - (u'2', {u'__class__': cname, u'double': 2.3, u'int': 2, u'str': u'test2'}), - (u'3', {u'__class__': cname, u'double': 3.1, u'int': 3, u'str': u'test3'}), - (u'4', {u'__class__': cname, u'double': 4.2, u'int': 4, u'str': u'test4'}), - (u'5', {u'__class__': cname, u'double': 5.5, u'int': 5, u'str': u'test56'})] - self.assertEqual(clazz, ec) - - unbatched_clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", - "org.apache.hadoop.io.Text", - "org.apache.spark.api.python.TestWritable", - ).collect()) - self.assertEqual(unbatched_clazz, ec) - def test_oldhadoop(self): basepath = self.tempdir.name ints = sorted(self.sc.hadoopFile(basepath + "/sftestdata/sfint/", @@ -249,51 +151,6 @@ def setUp(self): def tearDown(self): shutil.rmtree(self.tempdir.name, ignore_errors=True) - @unittest.skipIf(sys.version >= "3", "serialize array of byte") - def test_sequencefiles(self): - basepath = self.tempdir.name - ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] - self.sc.parallelize(ei).saveAsSequenceFile(basepath + "/sfint/") - ints = sorted(self.sc.sequenceFile(basepath + "/sfint/").collect()) - self.assertEqual(ints, ei) - - ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')] - self.sc.parallelize(ed).saveAsSequenceFile(basepath + "/sfdouble/") - doubles = sorted(self.sc.sequenceFile(basepath + "/sfdouble/").collect()) - self.assertEqual(doubles, ed) - - ebs = [(1, bytearray(b'\x00\x07spam\x08')), (2, bytearray(b'\x00\x07spam\x08'))] - self.sc.parallelize(ebs).saveAsSequenceFile(basepath + "/sfbytes/") - bytes = sorted(self.sc.sequenceFile(basepath + "/sfbytes/").collect()) - self.assertEqual(bytes, ebs) - - et = [(u'1', u'aa'), - (u'2', u'bb'), - (u'3', u'cc')] - self.sc.parallelize(et).saveAsSequenceFile(basepath + "/sftext/") - text = sorted(self.sc.sequenceFile(basepath + "/sftext/").collect()) - self.assertEqual(text, et) - - eb = [(1, False), (1, True), (2, False), (2, False), (2, True), (3, True)] - self.sc.parallelize(eb).saveAsSequenceFile(basepath + "/sfbool/") - bools = sorted(self.sc.sequenceFile(basepath + "/sfbool/").collect()) - self.assertEqual(bools, eb) - - en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)] - self.sc.parallelize(en).saveAsSequenceFile(basepath + "/sfnull/") - nulls = sorted(self.sc.sequenceFile(basepath + "/sfnull/").collect()) - self.assertEqual(nulls, en) - - em = [(1, {}), - (1, {3.0: u'bb'}), - (2, {1.0: u'aa'}), - (2, {1.0: u'cc'}), - (3, {2.0: u'dd'})] - self.sc.parallelize(em).saveAsSequenceFile(basepath + "/sfmap/") - maps = self.sc.sequenceFile(basepath + "/sfmap/").collect() - for v in maps: - self.assertTrue(v, em) - def test_oldhadoop(self): basepath = self.tempdir.name dict_data = [(1, {}), @@ -361,46 +218,6 @@ def test_newhadoop(self): conf=input_conf).collect()) self.assertEqual(new_dataset, data) - @unittest.skipIf(sys.version >= "3", "serialize of array") - def test_newhadoop_with_array(self): - basepath = self.tempdir.name - # use custom ArrayWritable types and converters to handle arrays - array_data = [(1, array('d')), - (1, array('d', [1.0, 2.0, 3.0])), - (2, array('d', [3.0, 4.0, 5.0]))] - self.sc.parallelize(array_data).saveAsNewAPIHadoopFile( - basepath + "/newhadoop/", - "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", - "org.apache.hadoop.io.IntWritable", - "org.apache.spark.api.python.DoubleArrayWritable", - valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter") - result = sorted(self.sc.newAPIHadoopFile( - basepath + "/newhadoop/", - "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", - "org.apache.hadoop.io.IntWritable", - "org.apache.spark.api.python.DoubleArrayWritable", - valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect()) - self.assertEqual(result, array_data) - - conf = { - "mapreduce.job.outputformat.class": - "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", - "mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable", - "mapreduce.job.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable", - "mapreduce.output.fileoutputformat.outputdir": basepath + "/newdataset/" - } - self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset( - conf, - valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter") - input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/newdataset/"} - new_dataset = sorted(self.sc.newAPIHadoopRDD( - "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", - "org.apache.hadoop.io.IntWritable", - "org.apache.spark.api.python.DoubleArrayWritable", - valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter", - conf=input_conf).collect()) - self.assertEqual(new_dataset, array_data) - def test_newolderror(self): basepath = self.tempdir.name rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x)) diff --git a/python/pyspark/tests/test_shuffle.py b/python/pyspark/tests/test_shuffle.py index d50ba632d6cd4..434414618e59d 100644 --- a/python/pyspark/tests/test_shuffle.py +++ b/python/pyspark/tests/test_shuffle.py @@ -23,15 +23,12 @@ from pyspark import shuffle, PickleSerializer, SparkConf, SparkContext from pyspark.shuffle import Aggregator, ExternalMerger, ExternalSorter -if sys.version_info[0] >= 3: - xrange = range - class MergerTests(unittest.TestCase): def setUp(self): self.N = 1 << 12 - self.l = [i for i in xrange(self.N)] + self.l = [i for i in range(self.N)] self.data = list(zip(self.l, self.l)) self.agg = Aggregator(lambda x: [x], lambda x, y: x.append(y) or x, @@ -42,26 +39,26 @@ def test_small_dataset(self): m.mergeValues(self.data) self.assertEqual(m.spills, 0) self.assertEqual(sum(sum(v) for k, v in m.items()), - sum(xrange(self.N))) + sum(range(self.N))) m = ExternalMerger(self.agg, 1000) m.mergeCombiners(map(lambda x_y1: (x_y1[0], [x_y1[1]]), self.data)) self.assertEqual(m.spills, 0) self.assertEqual(sum(sum(v) for k, v in m.items()), - sum(xrange(self.N))) + sum(range(self.N))) def test_medium_dataset(self): m = ExternalMerger(self.agg, 20) m.mergeValues(self.data) self.assertTrue(m.spills >= 1) self.assertEqual(sum(sum(v) for k, v in m.items()), - sum(xrange(self.N))) + sum(range(self.N))) m = ExternalMerger(self.agg, 10) m.mergeCombiners(map(lambda x_y2: (x_y2[0], [x_y2[1]]), self.data * 3)) self.assertTrue(m.spills >= 1) self.assertEqual(sum(sum(v) for k, v in m.items()), - sum(xrange(self.N)) * 3) + sum(range(self.N)) * 3) def test_huge_dataset(self): m = ExternalMerger(self.agg, 5, partitions=3) diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index 90e4bcdfadc03..8c2bedbe4e212 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -26,9 +26,6 @@ from pyspark import SparkConf, SparkContext, TaskContext, BarrierTaskContext from pyspark.testing.utils import PySparkTestCase, SPARK_HOME -if sys.version_info[0] >= 3: - xrange = range - class TaskContextTests(PySparkTestCase): @@ -251,9 +248,9 @@ def context_barrier(x): def test_task_context_correct_with_python_worker_reuse(self): """Verify the task context correct when reused python worker""" # start a normal job first to start all workers and get all worker pids - worker_pids = self.sc.parallelize(xrange(2), 2).map(lambda x: os.getpid()).collect() + worker_pids = self.sc.parallelize(range(2), 2).map(lambda x: os.getpid()).collect() # the worker will reuse in this barrier job - rdd = self.sc.parallelize(xrange(10), 2) + rdd = self.sc.parallelize(range(10), 2) def context(iterator): tp = TaskContext.get().partitionId() diff --git a/python/pyspark/tests/test_util.py b/python/pyspark/tests/test_util.py index 81bfb66e7019d..511d62a51f3df 100644 --- a/python/pyspark/tests/test_util.py +++ b/python/pyspark/tests/test_util.py @@ -61,14 +61,12 @@ def set(self, x=None, other=None, other_x=None): class UtilTests(PySparkTestCase): - def test_py4j_exception_message(self): - from pyspark.util import _exception_message - + def test_py4j_str(self): with self.assertRaises(Py4JJavaError) as context: # This attempts java.lang.String(null) which throws an NPE. self.sc._jvm.java.lang.String(None) - self.assertTrue('NullPointerException' in _exception_message(context.exception)) + self.assertTrue('NullPointerException' in str(context.exception)) def test_parsing_version_string(self): from pyspark.util import VersionUtils diff --git a/python/pyspark/tests/test_worker.py b/python/pyspark/tests/test_worker.py index dba9298ee161a..3b1848dcfdee9 100644 --- a/python/pyspark/tests/test_worker.py +++ b/python/pyspark/tests/test_worker.py @@ -32,9 +32,6 @@ from pyspark import SparkConf, SparkContext from pyspark.testing.utils import ReusedPySparkTestCase, PySparkTestCase, QuietTest -if sys.version_info[0] >= 3: - xrange = range - class WorkerTests(ReusedPySparkTestCase): def test_cancel_task(self): @@ -88,13 +85,13 @@ def run(): self.fail("daemon had been killed") # run a normal job - rdd = self.sc.parallelize(xrange(100), 1) + rdd = self.sc.parallelize(range(100), 1) self.assertEqual(100, rdd.map(str).count()) def test_after_exception(self): def raise_exception(_): raise Exception() - rdd = self.sc.parallelize(xrange(100), 1) + rdd = self.sc.parallelize(range(100), 1) with QuietTest(self.sc): self.assertRaises(Exception, lambda: rdd.foreach(raise_exception)) self.assertEqual(100, rdd.map(str).count()) @@ -110,22 +107,22 @@ def test_after_jvm_exception(self): with QuietTest(self.sc): self.assertRaises(Exception, lambda: filtered_data.count()) - rdd = self.sc.parallelize(xrange(100), 1) + rdd = self.sc.parallelize(range(100), 1) self.assertEqual(100, rdd.map(str).count()) def test_accumulator_when_reuse_worker(self): from pyspark.accumulators import INT_ACCUMULATOR_PARAM acc1 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM) - self.sc.parallelize(xrange(100), 20).foreach(lambda x: acc1.add(x)) + self.sc.parallelize(range(100), 20).foreach(lambda x: acc1.add(x)) self.assertEqual(sum(range(100)), acc1.value) acc2 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM) - self.sc.parallelize(xrange(100), 20).foreach(lambda x: acc2.add(x)) + self.sc.parallelize(range(100), 20).foreach(lambda x: acc2.add(x)) self.assertEqual(sum(range(100)), acc2.value) self.assertEqual(sum(range(100)), acc1.value) def test_reuse_worker_after_take(self): - rdd = self.sc.parallelize(xrange(100000), 1) + rdd = self.sc.parallelize(range(100000), 1) self.assertEqual(0, rdd.first()) def count(): @@ -160,17 +157,13 @@ def f(): self.sc.parallelize([1]).map(lambda x: f()).count() except Py4JJavaError as e: - if sys.version_info.major < 3: - # we have to use unicode here to avoid UnicodeDecodeError - self.assertRegexpMatches(unicode(e).encode("utf-8"), "exception with 中") - else: - self.assertRegexpMatches(str(e), "exception with 中") + self.assertRegexpMatches(str(e), "exception with 中") class WorkerReuseTest(PySparkTestCase): - def test_reuse_worker_of_parallelize_xrange(self): - rdd = self.sc.parallelize(xrange(20), 8) + def test_reuse_worker_of_parallelize_range(self): + rdd = self.sc.parallelize(range(20), 8) previous_pids = rdd.map(lambda x: os.getpid()).collect() current_pids = rdd.map(lambda x: os.getpid()).collect() for pid in current_pids: @@ -189,7 +182,7 @@ def setUp(self): self.sc = SparkContext('local[4]', class_name, conf=conf) def test_memory_limit(self): - rdd = self.sc.parallelize(xrange(1), 1) + rdd = self.sc.parallelize(range(1), 1) def getrlimit(): import resource diff --git a/python/pyspark/util.py b/python/pyspark/util.py index d9429372a6bfc..c003586e9c03b 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -19,52 +19,10 @@ import re import sys import traceback -import os -import warnings -import inspect -from py4j.protocol import Py4JJavaError __all__ = [] -def _exception_message(excp): - """Return the message from an exception as either a str or unicode object. Supports both - Python 2 and Python 3. - - >>> msg = "Exception message" - >>> excp = Exception(msg) - >>> msg == _exception_message(excp) - True - - >>> msg = u"unicöde" - >>> excp = Exception(msg) - >>> msg == _exception_message(excp) - True - """ - if isinstance(excp, Py4JJavaError): - # 'Py4JJavaError' doesn't contain the stack trace available on the Java side in 'message' - # attribute in Python 2. We should call 'str' function on this exception in general but - # 'Py4JJavaError' has an issue about addressing non-ascii strings. So, here we work - # around by the direct call, '__str__()'. Please see SPARK-23517. - return excp.__str__() - if hasattr(excp, "message"): - return excp.message - return str(excp) - - -def _get_argspec(f): - """ - Get argspec of a function. Supports both Python 2 and Python 3. - """ - if sys.version_info[0] < 3: - argspec = inspect.getargspec(f) - else: - # `getargspec` is deprecated since python3.0 (incompatible with function annotations). - # See SPARK-23569. - argspec = inspect.getfullargspec(f) - return argspec - - def print_exec(stream): ei = sys.exc_info() traceback.print_exception(ei[0], ei[1], ei[2], None, stream) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 5f4a8a2d2db1f..9b54affb137f5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -18,11 +18,11 @@ """ Worker that receives input from Piped RDD. """ -from __future__ import print_function -from __future__ import absolute_import import os import sys import time +from inspect import getfullargspec +import importlib # 'resource' is a Unix specific module. has_resource_module = True try: @@ -44,14 +44,9 @@ from pyspark.sql.pandas.serializers import ArrowStreamPandasUDFSerializer, CogroupUDFSerializer from pyspark.sql.pandas.types import to_arrow_type from pyspark.sql.types import StructType -from pyspark.util import _get_argspec, fail_on_stopiteration +from pyspark.util import fail_on_stopiteration from pyspark import shuffle -if sys.version >= '3': - basestring = str -else: - from itertools import imap as map # use iterator map by default - pickleSer = PickleSerializer() utf8_deserializer = UTF8Deserializer() @@ -272,10 +267,10 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index): elif eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF: return arg_offsets, wrap_pandas_iter_udf(func, return_type) elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF: - argspec = _get_argspec(chained_func) # signature was lost when wrapping it + argspec = getfullargspec(chained_func) # signature was lost when wrapping it return arg_offsets, wrap_grouped_map_pandas_udf(func, return_type, argspec) elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: - argspec = _get_argspec(chained_func) # signature was lost when wrapping it + argspec = getfullargspec(chained_func) # signature was lost when wrapping it return arg_offsets, wrap_cogrouped_map_pandas_udf(func, return_type, argspec) elif eval_type == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: return arg_offsets, wrap_grouped_agg_pandas_udf(func, return_type) @@ -342,11 +337,13 @@ def read_udfs(pickleSer, infile, eval_type): pickleSer, infile, eval_type, runner_conf, udf_index=0) def func(_, iterator): - num_input_rows = [0] # TODO(SPARK-29909): Use nonlocal after we drop Python 2. + num_input_rows = 0 def map_batch(batch): + nonlocal num_input_rows + udf_args = [batch[offset] for offset in arg_offsets] - num_input_rows[0] += len(udf_args[0]) + num_input_rows += len(udf_args[0]) if len(udf_args) == 1: return udf_args[0] else: @@ -363,7 +360,7 @@ def map_batch(batch): # by consuming the input iterator in user side. Therefore, # it's very unlikely the output length is higher than # input length. - assert is_map_iter or num_output_rows <= num_input_rows[0], \ + assert is_map_iter or num_output_rows <= num_input_rows, \ "Pandas SCALAR_ITER UDF outputted more rows than input rows." yield (result_batch, result_type) @@ -376,11 +373,11 @@ def map_batch(batch): raise RuntimeError("pandas iterator UDF should exhaust the input " "iterator.") - if num_output_rows != num_input_rows[0]: + if num_output_rows != num_input_rows: raise RuntimeError( "The length of output in Scalar iterator pandas UDF should be " "the same with the input's; however, the length of output was %d and the " - "length of input was %d." % (num_output_rows, num_input_rows[0])) + "length of input was %d." % (num_output_rows, num_input_rows)) # profiling is not supported for UDF return func, None, ser, ser @@ -548,9 +545,8 @@ def main(infile, outfile): for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) add_path(os.path.join(spark_files_dir, filename)) - if sys.version > '3': - import importlib - importlib.invalidate_caches() + + importlib.invalidate_caches() # fetch names and values of broadcast variables needs_broadcast_decryption_server = read_bool(infile) diff --git a/python/run-tests.py b/python/run-tests.py index 42510c7642264..23076eab1c3e4 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -28,10 +28,7 @@ from threading import Thread, Lock import time import uuid -if sys.version < '3': - import Queue -else: - import queue as Queue +import queue as Queue from multiprocessing import Manager @@ -75,7 +72,6 @@ def run_individual_python_test(target_dir, test_name, pyspark_python): 'SPARK_PREPEND_CLASSES': '1', 'PYSPARK_PYTHON': which(pyspark_python), 'PYSPARK_DRIVER_PYTHON': which(pyspark_python), - 'PYSPARK_ROW_FIELD_SORTING_ENABLED': 'true' }) # Create a unique temp directory under 'target/' for each run. The TMPDIR variable is @@ -161,7 +157,8 @@ def run_individual_python_test(target_dir, test_name, pyspark_python): def get_default_python_executables(): - python_execs = [x for x in ["python3.6", "python2.7", "pypy3", "pypy"] if which(x)] + # TODO(SPARK-32278): install PyPy3 in Jenkins to test + python_execs = [x for x in ["python3.6", "python3.8", "pypy3"] if which(x)] if "python3.6" not in python_execs: p = which("python3") diff --git a/python/setup.py b/python/setup.py index afbd601b04a94..c456a32fea87c 100755 --- a/python/setup.py +++ b/python/setup.py @@ -16,18 +16,12 @@ # See the License for the specific language governing permissions and # limitations under the License. -from __future__ import print_function import glob import os import sys from setuptools import setup from shutil import copyfile, copytree, rmtree -if sys.version_info < (2, 7): - print("Python versions prior to 2.7 are not supported for pip installed PySpark.", - file=sys.stderr) - sys.exit(-1) - try: exec(open('pyspark/version.py').read()) except IOError: @@ -217,13 +211,10 @@ def _supports_symlinks(): 'pyarrow>=%s' % _minimum_pyarrow_version, ] }, + python_requires='>=3.6', classifiers=[ 'Development Status :: 5 - Production/Stable', 'License :: OSI Approved :: Apache Software License', - 'Programming Language :: Python :: 2.7', - 'Programming Language :: Python :: 3', - 'Programming Language :: Python :: 3.4', - 'Programming Language :: Python :: 3.5', 'Programming Language :: Python :: 3.6', 'Programming Language :: Python :: 3.7', 'Programming Language :: Python :: 3.8', diff --git a/resource-managers/kubernetes/integration-tests/tests/pyfiles.py b/resource-managers/kubernetes/integration-tests/tests/pyfiles.py index ba55b75803276..51c0160554866 100644 --- a/resource-managers/kubernetes/integration-tests/tests/pyfiles.py +++ b/resource-managers/kubernetes/integration-tests/tests/pyfiles.py @@ -14,9 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # - -from __future__ import print_function - import sys from pyspark.sql import SparkSession diff --git a/resource-managers/kubernetes/integration-tests/tests/worker_memory_check.py b/resource-managers/kubernetes/integration-tests/tests/worker_memory_check.py index d312a29f388e4..74559a0b54402 100644 --- a/resource-managers/kubernetes/integration-tests/tests/worker_memory_check.py +++ b/resource-managers/kubernetes/integration-tests/tests/worker_memory_check.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - import resource import sys diff --git a/sql/hive/src/test/resources/data/scripts/cat.py b/sql/hive/src/test/resources/data/scripts/cat.py index aea0362f899fa..420d9f832a184 100644 --- a/sql/hive/src/test/resources/data/scripts/cat.py +++ b/sql/hive/src/test/resources/data/scripts/cat.py @@ -16,7 +16,6 @@ # specific language governing permissions and limitations # under the License. # -from __future__ import print_function import sys import os diff --git a/sql/hive/src/test/resources/data/scripts/dumpdata_script.py b/sql/hive/src/test/resources/data/scripts/dumpdata_script.py index 5b360208d36f6..f724fdc85b177 100644 --- a/sql/hive/src/test/resources/data/scripts/dumpdata_script.py +++ b/sql/hive/src/test/resources/data/scripts/dumpdata_script.py @@ -18,12 +18,9 @@ # import sys -if sys.version_info[0] >= 3: - xrange = range - -for i in xrange(50): - for j in xrange(5): - for k in xrange(20022): +for i in range(50): + for j in range(5): + for k in range(20022): print(20000 * i + k) for line in sys.stdin: From 24be81689cee76e03cd5136dfd089123bbff4595 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Tue, 14 Jul 2020 04:26:29 +0000 Subject: [PATCH 185/384] [SPARK-32241][SQL] Remove empty children of union ### What changes were proposed in this pull request? This PR removes the empty child relations of a `Union`. E.g. the query `SELECT c FROM t UNION ALL SELECT c FROM t WHERE false` has the following plan before this PR: ``` == Physical Plan == Union :- *(1) Project [value#219 AS c#222] : +- *(1) LocalTableScan [value#219] +- LocalTableScan , [c#224] ``` and after this PR: ``` == Physical Plan == *(1) Project [value#219 AS c#222] +- *(1) LocalTableScan [value#219] ``` ### Why are the changes needed? To have a simpler plan. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added new UTs. Closes #29053 from peter-toth/SPARK-32241-remove-empty-children-of-union. Authored-by: Peter Toth Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/Optimizer.scala | 10 ++++- .../optimizer/PropagateEmptyRelation.scala | 22 ++++++++++- .../PropagateEmptyRelationSuite.scala | 38 ++++++++++++++++++- 3 files changed, 65 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 1b141572cc7f9..d7e8571f6ce43 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 @@ -161,7 +161,10 @@ abstract class Optimizer(catalogManager: CatalogManager) // LocalRelation and does not trigger many rules. Batch("LocalRelation early", fixedPoint, ConvertToLocalRelation, - PropagateEmptyRelation) :: + PropagateEmptyRelation, + // PropagateEmptyRelation can change the nullability of an attribute from nullable to + // non-nullable when an empty relation child of a Union is removed + UpdateAttributeNullability) :: Batch("Pullup Correlated Expressions", Once, PullupCorrelatedPredicates) :: // Subquery batch applies the optimizer rules recursively. Therefore, it makes no sense @@ -198,7 +201,10 @@ abstract class Optimizer(catalogManager: CatalogManager) ReassignLambdaVariableID) :+ Batch("LocalRelation", fixedPoint, ConvertToLocalRelation, - PropagateEmptyRelation) :+ + PropagateEmptyRelation, + // PropagateEmptyRelation can change the nullability of an attribute from nullable to + // non-nullable when an empty relation child of a Union is removed + UpdateAttributeNullability) :+ // The following batch should be executed after batch "Join Reorder" and "LocalRelation". Batch("Check Cartesian Products", Once, CheckCartesianProducts) :+ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala index b19e13870aa65..0299646150ff3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala @@ -50,8 +50,26 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper wit override def conf: SQLConf = SQLConf.get def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case p: Union if p.children.forall(isEmptyLocalRelation) => - empty(p) + case p @ Union(children) if children.exists(isEmptyLocalRelation) => + val newChildren = children.filterNot(isEmptyLocalRelation) + if (newChildren.isEmpty) { + empty(p) + } else { + val newPlan = if (newChildren.size > 1) Union(newChildren) else newChildren.head + val outputs = newPlan.output.zip(p.output) + // the original Union may produce different output attributes than the new one so we alias + // them if needed + if (outputs.forall { case (newAttr, oldAttr) => newAttr.exprId == oldAttr.exprId }) { + newPlan + } else { + val outputAliases = outputs.map { case (newAttr, oldAttr) => + val newExplicitMetadata = + if (oldAttr.metadata != newAttr.metadata) Some(oldAttr.metadata) else None + Alias(newAttr, oldAttr.name)(oldAttr.exprId, explicitMetadata = newExplicitMetadata) + } + Project(outputAliases, newPlan) + } + } // Joins on empty LocalRelations generated from streaming sources are not eliminated // as stateful streaming joins need to perform other state management operations other than diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala index 9c7d4c7d8d233..dc323d4e5c77c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.{IntegerType, MetadataBuilder, StructType} class PropagateEmptyRelationSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { @@ -55,6 +55,9 @@ class PropagateEmptyRelationSuite extends PlanTest { val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = Seq(Row(1))) val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = Seq(Row(1))) + val metadata = new MetadataBuilder().putLong("test", 1).build() + val testRelation3 = + LocalRelation.fromExternalRows(Seq('c.int.notNull.withMetadata(metadata)), data = Seq(Row(1))) test("propagate empty relation through Union") { val query = testRelation1 @@ -67,6 +70,39 @@ class PropagateEmptyRelationSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("SPARK-32241: remove empty relation children from Union") { + val query = testRelation1.union(testRelation2.where(false)) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation1 + comparePlans(optimized, correctAnswer) + + val query2 = testRelation1.where(false).union(testRelation2) + val optimized2 = Optimize.execute(query2.analyze) + val correctAnswer2 = testRelation2.select('b.as('a)).analyze + comparePlans(optimized2, correctAnswer2) + + val query3 = testRelation1.union(testRelation2.where(false)).union(testRelation3) + val optimized3 = Optimize.execute(query3.analyze) + val correctAnswer3 = testRelation1.union(testRelation3) + comparePlans(optimized3, correctAnswer3) + + val query4 = testRelation1.where(false).union(testRelation2).union(testRelation3) + val optimized4 = Optimize.execute(query4.analyze) + val correctAnswer4 = testRelation2.union(testRelation3).select('b.as('a)).analyze + comparePlans(optimized4, correctAnswer4) + + // Nullability can change from nullable to non-nullable + val query5 = testRelation1.where(false).union(testRelation3) + val optimized5 = Optimize.execute(query5.analyze) + assert(query5.output.head.nullable, "Original output should be nullable") + assert(!optimized5.output.head.nullable, "New output should be non-nullable") + + // Keep metadata + val query6 = testRelation3.where(false).union(testRelation1) + val optimized6 = Optimize.execute(query6.analyze) + assert(optimized6.output.head.metadata == metadata, "New output should keep metadata") + } + test("propagate empty relation through Join") { // Testcases are tuples of (left predicate, right predicate, joinType, correct answer) // Note that `None` is used to compare with OptimizeWithoutPropagateEmptyRelation. From cc9371d885867b2cbc5d61d90083d23ce017f7a7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 14 Jul 2020 05:51:59 +0000 Subject: [PATCH 186/384] [SPARK-32258][SQL] Not duplicate normalization on children for float/double If/CaseWhen/Coalesce ### What changes were proposed in this pull request? This is followup to #29061. See https://github.com/apache/spark/pull/29061#discussion_r453458611. Basically this moves If/CaseWhen/Coalesce case patterns after float/double case so we don't duplicate normalization on children for float/double If/CaseWhen/Coalesce. ### Why are the changes needed? Simplify expression tree. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Modify unit tests. Closes #29091 from viirya/SPARK-32258-followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../optimizer/NormalizeFloatingNumbers.scala | 6 ++--- .../NormalizeFloatingPointNumbersSuite.scala | 26 +++++++++---------- 2 files changed, 15 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala index 98c78c6312222..10f846cf910f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala @@ -116,6 +116,9 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { case CreateMap(children, useStringTypeWhenEmpty) => CreateMap(children.map(normalize), useStringTypeWhenEmpty) + case _ if expr.dataType == FloatType || expr.dataType == DoubleType => + KnownFloatingPointNormalized(NormalizeNaNAndZero(expr)) + case If(cond, trueValue, falseValue) => If(cond, normalize(trueValue), normalize(falseValue)) @@ -125,9 +128,6 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { case Coalesce(children) => Coalesce(children.map(normalize)) - case _ if expr.dataType == FloatType || expr.dataType == DoubleType => - KnownFloatingPointNormalized(NormalizeNaNAndZero(expr)) - case _ if expr.dataType.isInstanceOf[StructType] => val fields = expr.dataType.asInstanceOf[StructType].fields.indices.map { i => normalize(GetStructField(expr, i)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala index 3f6bdd206535b..bb9919f94eef2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala @@ -85,25 +85,23 @@ class NormalizeFloatingPointNumbersSuite extends PlanTest { val optimized = Optimize.execute(query) val doubleOptimized = Optimize.execute(optimized) val joinCond = IsNull(a) === IsNull(b) && - coalesce(KnownFloatingPointNormalized(NormalizeNaNAndZero(a)), - KnownFloatingPointNormalized(NormalizeNaNAndZero(0.0))) === - coalesce(KnownFloatingPointNormalized(NormalizeNaNAndZero(b)), - KnownFloatingPointNormalized(NormalizeNaNAndZero(0.0))) + KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(a, 0.0))) === + KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(b, 0.0))) val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) comparePlans(doubleOptimized, correctAnswer) } test("SPARK-32258: normalize the children of If") { - val cond = If(a > 0.1D, a, a + 0.2D) === b + val cond = If(a > 0.1D, namedStruct("a", a), namedStruct("a", a + 0.2D)) === namedStruct("a", b) val query = testRelation1.join(testRelation2, condition = Some(cond)) val optimized = Optimize.execute(query) val doubleOptimized = Optimize.execute(optimized) val joinCond = If(a > 0.1D, - KnownFloatingPointNormalized(NormalizeNaNAndZero(a)), - KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.2D))) === - KnownFloatingPointNormalized(NormalizeNaNAndZero(b)) + namedStruct("a", KnownFloatingPointNormalized(NormalizeNaNAndZero(a))), + namedStruct("a", KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.2D)))) === + namedStruct("a", KnownFloatingPointNormalized(NormalizeNaNAndZero(b))) val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) comparePlans(doubleOptimized, correctAnswer) @@ -111,17 +109,17 @@ class NormalizeFloatingPointNumbersSuite extends PlanTest { test("SPARK-32258: normalize the children of CaseWhen") { val cond = CaseWhen( - Seq((a > 0.1D, a), (a > 0.2D, a + 0.2D)), - Some(a + 0.3D)) === b + Seq((a > 0.1D, namedStruct("a", a)), (a > 0.2D, namedStruct("a", a + 0.2D))), + Some(namedStruct("a", a + 0.3D))) === namedStruct("a", b) val query = testRelation1.join(testRelation2, condition = Some(cond)) val optimized = Optimize.execute(query) val doubleOptimized = Optimize.execute(optimized) val joinCond = CaseWhen( - Seq((a > 0.1D, KnownFloatingPointNormalized(NormalizeNaNAndZero(a))), - (a > 0.2D, KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.2D)))), - Some(KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.3D)))) === - KnownFloatingPointNormalized(NormalizeNaNAndZero(b)) + Seq((a > 0.1D, namedStruct("a", KnownFloatingPointNormalized(NormalizeNaNAndZero(a)))), + (a > 0.2D, namedStruct("a", KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.2D))))), + Some(namedStruct("a", KnownFloatingPointNormalized(NormalizeNaNAndZero(a + 0.3D))))) === + namedStruct("a", KnownFloatingPointNormalized(NormalizeNaNAndZero(b))) val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) comparePlans(doubleOptimized, correctAnswer) From d6a68e0b67ff7de58073c176dd097070e88ac831 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 14 Jul 2020 02:06:50 -0700 Subject: [PATCH 187/384] [SPARK-29292][STREAMING][SQL][BUILD] Get streaming, catalyst, sql compiling for Scala 2.13 ### What changes were proposed in this pull request? Continuation of https://github.com/apache/spark/pull/28971 which lets streaming, catalyst and sql compile for 2.13. Same idea. ### Why are the changes needed? Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12) Closes #29078 from srowen/SPARK-29292.2. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../streaming/kafka010/ConsumerStrategy.scala | 6 +- .../kafka010/DirectKafkaInputDStream.scala | 3 +- .../kafka010/JavaConsumerStrategySuite.java | 13 +- .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../catalyst/analysis/CTESubstitution.scala | 8 +- .../expressions/EquivalentExpressions.scala | 2 +- .../expressions/codegen/CodeGenerator.scala | 5 +- .../expressions/objects/objects.scala | 2 +- .../sql/catalyst/expressions/package.scala | 2 +- .../sql/catalyst/expressions/subquery.scala | 2 +- .../optimizer/CostBasedJoinReorder.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 4 +- .../sql/catalyst/optimizer/expressions.scala | 2 +- .../sql/catalyst/optimizer/subquery.scala | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 115 +++++++++--------- .../statsEstimation/EstimationUtils.scala | 2 +- .../statsEstimation/JoinEstimation.scala | 2 +- .../spark/sql/catalyst/trees/TreeNode.scala | 6 +- .../sql/catalyst/util/GenericArrayData.scala | 2 +- .../sql/catalyst/util/QuantileSummaries.scala | 2 +- .../apache/spark/sql/types/StructType.scala | 6 +- .../apache/spark/sql/util/ArrowUtils.scala | 4 +- .../spark/sql/RandomDataGenerator.scala | 4 +- .../optimizer/ColumnPruningSuite.scala | 2 +- .../sql/catalyst/util/ComplexDataSuite.scala | 2 - .../util/TimestampFormatterSuite.scala | 4 +- .../DataTypeWriteCompatibilitySuite.scala | 2 +- .../spark/sql/RelationalGroupedDataset.scala | 2 +- .../org/apache/spark/sql/SparkSession.scala | 4 +- .../spark/sql/SparkSessionExtensions.scala | 12 +- .../execution/AggregatingAccumulator.scala | 14 +-- .../spark/sql/execution/SparkSqlParser.scala | 7 +- .../sql/execution/WholeStageCodegenExec.scala | 2 +- .../adaptive/AdaptiveSparkPlanExec.scala | 2 +- .../adaptive/AdaptiveSparkPlanHelper.scala | 6 +- .../adaptive/CustomShuffleReaderExec.scala | 2 +- .../adaptive/OptimizeSkewedJoin.scala | 4 +- .../adaptive/ShufflePartitionsUtil.scala | 2 +- .../execution/columnar/InMemoryRelation.scala | 2 +- .../command/AnalyzePartitionCommand.scala | 2 +- .../spark/sql/execution/command/ddl.scala | 4 +- .../spark/sql/execution/command/tables.scala | 6 +- .../execution/datasources/DataSource.scala | 2 +- .../execution/datasources/FilePartition.scala | 2 +- .../datasources/HadoopFileLinesReader.scala | 8 +- .../HadoopFileWholeTextReader.scala | 8 +- .../datasources/InMemoryFileIndex.scala | 2 +- .../datasources/PartitioningUtils.scala | 4 +- .../datasources/parquet/ParquetFilters.scala | 6 +- .../parquet/ParquetSchemaConverter.scala | 2 +- .../v2/DescribeNamespaceExec.scala | 2 +- .../datasources/v2/DescribeTableExec.scala | 2 +- .../datasources/v2/PushDownUtils.scala | 2 +- .../datasources/v2/ShowNamespacesExec.scala | 2 +- .../datasources/v2/ShowTablesExec.scala | 2 +- .../datasources/v2/V2SessionCatalog.scala | 2 +- .../exchange/EnsureRequirements.scala | 2 +- .../python/AggregateInPandasExec.scala | 4 +- .../sql/execution/python/EvalPythonExec.scala | 4 +- .../execution/python/ExtractPythonUDFs.scala | 6 +- .../execution/python/PandasGroupUtils.scala | 2 +- .../streaming/CompactibleFileStreamLog.scala | 2 +- .../streaming/ContinuousRecordEndpoint.scala | 4 +- .../streaming/FileStreamSource.scala | 4 +- .../ManifestFileCommitProtocol.scala | 2 +- .../streaming/ProgressReporter.scala | 11 +- .../sql/execution/streaming/memory.scala | 2 +- .../execution/streaming/sources/memory.scala | 6 +- .../streaming/statefulOperators.scala | 2 +- .../sql/execution/ui/AllExecutionsPage.scala | 6 +- .../execution/ui/SQLAppStatusListener.scala | 4 +- .../sql/execution/ui/SparkPlanGraph.scala | 2 +- .../execution/window/AggregateProcessor.scala | 7 +- .../sql/DataFrameSetOperationsSuite.scala | 5 +- .../spark/sql/DatasetPrimitiveSuite.scala | 10 -- .../org/apache/spark/sql/JoinSuite.scala | 10 +- .../org/apache/spark/sql/QueryTest.scala | 2 +- .../spark/sql/ShowCreateTableSuite.scala | 2 +- .../org/apache/spark/sql/SubquerySuite.scala | 2 +- .../benchmark/IntervalBenchmark.scala | 4 +- .../metric/SQLMetricsTestUtils.scala | 4 +- .../sources/ForeachWriterSuite.scala | 2 +- .../ui/SQLAppStatusListenerSuite.scala | 39 +++--- .../spark/sql/streaming/StreamTest.scala | 2 +- .../StreamingQueryListenerSuite.scala | 6 +- ...StreamingQueryStatusAndProgressSuite.scala | 6 +- .../apache/spark/streaming/DStreamGraph.scala | 2 +- .../api/java/JavaStreamingContext.scala | 6 +- .../java/JavaStreamingListenerWrapper.scala | 4 +- .../streaming/dstream/UnionDStream.scala | 2 +- .../spark/streaming/rdd/MapWithStateRDD.scala | 2 +- .../scheduler/ReceivedBlockTracker.scala | 4 +- .../scheduler/ReceiverSchedulingPolicy.scala | 4 +- .../streaming/scheduler/ReceiverTracker.scala | 2 +- .../streaming/util/BatchedWriteAheadLog.scala | 2 +- .../util/FileBasedWriteAheadLog.scala | 2 +- .../spark/streaming/DStreamScopeSuite.scala | 2 +- .../spark/streaming/JavaTestUtils.scala | 2 +- .../spark/streaming/MasterFailureTest.scala | 2 +- .../streaming/util/WriteAheadLogSuite.scala | 4 +- 100 files changed, 269 insertions(+), 266 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index 3e32b592b3a3a..ab6550ddf2fb3 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -241,7 +241,7 @@ object ConsumerStrategies { new Subscribe[K, V]( new ju.ArrayList(topics.asJavaCollection), new ju.HashMap[String, Object](kafkaParams.asJava), - new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).asJava)) + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).toMap.asJava)) } /** @@ -320,7 +320,7 @@ object ConsumerStrategies { new SubscribePattern[K, V]( pattern, new ju.HashMap[String, Object](kafkaParams.asJava), - new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).asJava)) + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).toMap.asJava)) } /** @@ -404,7 +404,7 @@ object ConsumerStrategies { new Assign[K, V]( new ju.ArrayList(topicPartitions.asJavaCollection), new ju.HashMap[String, Object](kafkaParams.asJava), - new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).asJava)) + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).toMap.asJava)) } /** diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index a449a8bb7213e..fcdc92580ba35 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -70,7 +70,8 @@ private[spark] class DirectKafkaInputDStream[K, V]( @transient private var kc: Consumer[K, V] = null def consumer(): Consumer[K, V] = this.synchronized { if (null == kc) { - kc = consumerStrategy.onStart(currentOffsets.mapValues(l => java.lang.Long.valueOf(l)).asJava) + kc = consumerStrategy.onStart( + currentOffsets.mapValues(l => java.lang.Long.valueOf(l)).toMap.asJava) } kc } diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java index dc364aca9bd3b..3d6e5ebe978e8 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -48,15 +48,12 @@ public void testConsumerStrategyConstructors() { JavaConverters.mapAsScalaMapConverter(kafkaParams).asScala(); final Map offsets = new HashMap<>(); offsets.put(tp1, 23L); + final Map dummyOffsets = new HashMap<>(); + for (Map.Entry kv : offsets.entrySet()) { + dummyOffsets.put(kv.getKey(), kv.getValue()); + } final scala.collection.Map sOffsets = - JavaConverters.mapAsScalaMapConverter(offsets).asScala().mapValues( - new scala.runtime.AbstractFunction1() { - @Override - public Object apply(Long x) { - return (Object) x; - } - } - ); + JavaConverters.mapAsScalaMap(dummyOffsets); final ConsumerStrategy sub1 = ConsumerStrategies.Subscribe(sTopics, sKafkaParams, sOffsets); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d669509d18455..023ef2ee17473 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1050,7 +1050,7 @@ class Analyzer( val partCols = partitionColumnNames(r.table) validatePartitionSpec(partCols, i.partitionSpec) - val staticPartitions = i.partitionSpec.filter(_._2.isDefined).mapValues(_.get) + val staticPartitions = i.partitionSpec.filter(_._2.isDefined).mapValues(_.get).toMap val query = addStaticPartitionColumns(r, i.query, staticPartitions) if (!i.overwrite) { @@ -2238,7 +2238,7 @@ class Analyzer( } } if (aggregateExpressions.nonEmpty) { - Some(aggregateExpressions, transformedAggregateFilter) + Some(aggregateExpressions.toSeq, transformedAggregateFilter) } else { None } @@ -2677,7 +2677,7 @@ class Analyzer( val windowOps = groupedWindowExpressions.foldLeft(child) { case (last, ((partitionSpec, orderSpec, _), windowExpressions)) => - Window(windowExpressions, partitionSpec, orderSpec, last) + Window(windowExpressions.toSeq, partitionSpec, orderSpec, last) } // Finally, we create a Project to output windowOps's output diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index 9f0eff5017f38..623cd131bf8da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -72,10 +72,10 @@ object CTESubstitution extends Rule[LogicalPlan] { } // CTE relation is defined as `SubqueryAlias`. Here we skip it and check the child // directly, so that `startOfQuery` is set correctly. - assertNoNameConflictsInCTE(relation.child, newNames) + assertNoNameConflictsInCTE(relation.child, newNames.toSeq) newNames += name } - assertNoNameConflictsInCTE(child, newNames, startOfQuery = false) + assertNoNameConflictsInCTE(child, newNames.toSeq, startOfQuery = false) case other => other.subqueries.foreach(assertNoNameConflictsInCTE(_, outerCTERelationNames)) @@ -162,9 +162,9 @@ object CTESubstitution extends Rule[LogicalPlan] { traverseAndSubstituteCTE(relation) } // CTE definition can reference a previous one - resolvedCTERelations += (name -> substituteCTE(innerCTEResolved, resolvedCTERelations)) + resolvedCTERelations += (name -> substituteCTE(innerCTEResolved, resolvedCTERelations.toSeq)) } - resolvedCTERelations + resolvedCTERelations.toSeq } private def substituteCTE( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala index a32052ce121df..458c48df6d0c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala @@ -106,7 +106,7 @@ class EquivalentExpressions { * an empty collection if there are none. */ def getEquivalentExprs(e: Expression): Seq[Expression] = { - equivalenceMap.getOrElse(Expr(e), Seq.empty) + equivalenceMap.getOrElse(Expr(e), Seq.empty).toSeq } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 817dd948f1a6a..9c20916790c21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -375,7 +375,7 @@ class CodegenContext extends Logging { // The generated initialization code may exceed 64kb function size limit in JVM if there are too // many mutable states, so split it into multiple functions. - splitExpressions(expressions = initCodes, funcName = "init", arguments = Nil) + splitExpressions(expressions = initCodes.toSeq, funcName = "init", arguments = Nil) } /** @@ -927,6 +927,7 @@ class CodegenContext extends Logging { length += CodeFormatter.stripExtraNewLinesAndComments(code).length } blocks += blockBuilder.toString() + blocks.toSeq } /** @@ -1002,7 +1003,7 @@ class CodegenContext extends Logging { def subexprFunctionsCode: String = { // Whole-stage codegen's subexpression elimination is handled in another code path assert(currentVars == null || subexprFunctions.isEmpty) - splitExpressions(subexprFunctions, "subexprFunc_split", Seq("InternalRow" -> INPUT_ROW)) + splitExpressions(subexprFunctions.toSeq, "subexprFunc_split", Seq("InternalRow" -> INPUT_ROW)) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index ab2f66b1a53e4..361bcd492965b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -741,7 +741,7 @@ case class MapObjects private( case ObjectType(cls) if cls.isArray => _.asInstanceOf[Array[_]].toSeq case ObjectType(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) => - _.asInstanceOf[java.util.List[_]].asScala + _.asInstanceOf[java.util.List[_]].asScala.toSeq case ObjectType(cls) if cls == classOf[Object] => (inputCollection) => { if (inputCollection.getClass.isArray) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 8bf1f19844556..d950fef3b26a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -124,7 +124,7 @@ package object expressions { } private def unique[T](m: Map[T, Seq[Attribute]]): Map[T, Seq[Attribute]] = { - m.mapValues(_.distinct).map(identity) + m.mapValues(_.distinct).toMap } /** Map to use for direct case insensitive attribute lookups. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index f46a1c6836fcf..ff8856708c6d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -186,7 +186,7 @@ object SubExprUtils extends PredicateHelper { e } } - outerExpressions + outerExpressions.toSeq } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index f92d8f5b8e534..c450ea891a612 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -162,7 +162,7 @@ object JoinReorderDP extends PredicateHelper with Logging { val topOutputSet = AttributeSet(output) while (foundPlans.size < items.length) { // Build plans for the next level. - foundPlans += searchLevel(foundPlans, conf, conditions, topOutputSet, filters) + foundPlans += searchLevel(foundPlans.toSeq, conf, conditions, topOutputSet, filters) } val durationInMs = (System.nanoTime() - startTime) / (1000 * 1000) 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 d7e8571f6ce43..e481cdbd5fdf4 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 @@ -425,7 +425,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { // Create the attribute mapping. Note that the currentNextAttrPairs can contain duplicate // keys in case of Union (this is caused by the PushProjectionThroughUnion rule); in this // case we use the first mapping (which should be provided by the first child). - val mapping = AttributeMap(currentNextAttrPairs) + val mapping = AttributeMap(currentNextAttrPairs.toSeq) // Create a an expression cleaning function for nodes that can actually produce redundant // aliases, use identity otherwise. @@ -940,7 +940,7 @@ object CombineUnions extends Rule[LogicalPlan] { flattened += child } } - Union(flattened) + Union(flattened.toSeq) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index bd400f86ea2c1..759ce5718ed2a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -765,7 +765,7 @@ object CombineConcats extends Rule[LogicalPlan] { flattened += child } } - Concat(flattened) + Concat(flattened.toSeq) } private def hasNestedConcats(concat: Concat): Boolean = concat.children.exists { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 6fdd2110ab12a..7b696912aa465 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -478,11 +478,11 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { while (true) { bottomPart match { case havingPart @ Filter(_, aggPart: Aggregate) => - return (topPart, Option(havingPart), aggPart) + return (topPart.toSeq, Option(havingPart), aggPart) case aggPart: Aggregate => // No HAVING clause - return (topPart, None, aggPart) + return (topPart.toSeq, None, aggPart) case p @ Project(_, child) => topPart += p 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 f8261c293782d..29621e11e534c 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 @@ -141,7 +141,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging s"CTE definition can't have duplicate names: ${duplicates.mkString("'", "', '", "'")}.", ctx) } - With(plan, ctes) + With(plan, ctes.toSeq) } /** @@ -182,7 +182,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (selects.length == 1) { selects.head } else { - Union(selects) + Union(selects.toSeq) } } @@ -229,7 +229,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (inserts.length == 1) { inserts.head } else { - Union(inserts) + Union(inserts.toSeq) } } @@ -389,7 +389,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging assignCtx.assignment().asScala.map { assign => Assignment(UnresolvedAttribute(visitMultipartIdentifier(assign.key)), expression(assign.value)) - } + }.toSeq } override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) { @@ -444,7 +444,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging throw new ParseException("The number of inserted values cannot match the fields.", clause.notMatchedAction()) } - InsertAction(condition, columns.zip(values).map(kv => Assignment(kv._1, kv._2))) + InsertAction(condition, columns.zip(values).map(kv => Assignment(kv._1, kv._2)).toSeq) } } else { // It should not be here. @@ -473,8 +473,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging aliasedTarget, aliasedSource, mergeCondition, - matchedActions, - notMatchedActions) + matchedActions.toSeq, + notMatchedActions.toSeq) } /** @@ -490,7 +490,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Before calling `toMap`, we check duplicated keys to avoid silently ignore partition values // in partition spec like PARTITION(a='1', b='2', a='3'). The real semantical check for // partition columns will be done in analyzer. - checkDuplicateKeys(parts, ctx) + checkDuplicateKeys(parts.toSeq, ctx) parts.toMap } @@ -530,17 +530,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val withOrder = if ( !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) { // ORDER BY ... - Sort(order.asScala.map(visitSortItem), global = true, query) + Sort(order.asScala.map(visitSortItem).toSeq, global = true, query) } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) { // SORT BY ... - Sort(sort.asScala.map(visitSortItem), global = false, query) + Sort(sort.asScala.map(visitSortItem).toSeq, global = false, query) } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) { // DISTRIBUTE BY ... withRepartitionByExpression(ctx, expressionList(distributeBy), query) } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) { // SORT BY ... DISTRIBUTE BY ... Sort( - sort.asScala.map(visitSortItem), + sort.asScala.map(visitSortItem).toSeq, global = false, withRepartitionByExpression(ctx, expressionList(distributeBy), query)) } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) { @@ -841,7 +841,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Note that mapValues creates a view instead of materialized map. We force materialization by // mapping over identity. - WithWindowDefinition(windowMapView.map(identity), query) + WithWindowDefinition(windowMapView.map(identity).toMap, query) } /** @@ -856,8 +856,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (ctx.GROUPING != null) { // GROUP BY .... GROUPING SETS (...) val selectedGroupByExprs = - ctx.groupingSet.asScala.map(_.expression.asScala.map(e => expression(e))) - GroupingSets(selectedGroupByExprs, groupByExpressions, query, selectExpressions) + ctx.groupingSet.asScala.map(_.expression.asScala.map(e => expression(e)).toSeq) + GroupingSets(selectedGroupByExprs.toSeq, groupByExpressions, query, selectExpressions) } else { // GROUP BY .... (WITH CUBE | WITH ROLLUP)? val mappedGroupByExpressions = if (ctx.CUBE != null) { @@ -878,8 +878,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx: HintContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { var plan = query - ctx.hintStatements.asScala.reverse.foreach { case stmt => - plan = UnresolvedHint(stmt.hintName.getText, stmt.parameters.asScala.map(expression), plan) + ctx.hintStatements.asScala.reverse.foreach { stmt => + plan = UnresolvedHint(stmt.hintName.getText, + stmt.parameters.asScala.map(expression).toSeq, plan) } plan } @@ -898,10 +899,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } else { CreateStruct( ctx.pivotColumn.identifiers.asScala.map( - identifier => UnresolvedAttribute.quoted(identifier.getText))) + identifier => UnresolvedAttribute.quoted(identifier.getText)).toSeq) } val pivotValues = ctx.pivotValues.asScala.map(visitPivotValue) - Pivot(None, pivotColumn, pivotValues, aggregates, query) + Pivot(None, pivotColumn, pivotValues.toSeq, aggregates, query) } /** @@ -930,7 +931,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // scalastyle:off caselocale Some(ctx.tblName.getText.toLowerCase), // scalastyle:on caselocale - ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply), + ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply).toSeq, query) } @@ -1081,7 +1082,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } val tvf = UnresolvedTableValuedFunction( - func.funcName.getText, func.expression.asScala.map(expression), aliases) + func.funcName.getText, func.expression.asScala.map(expression).toSeq, aliases) tvf.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) } @@ -1106,7 +1107,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Seq.tabulate(rows.head.size)(i => s"col${i + 1}") } - val table = UnresolvedInlineTable(aliases, rows) + val table = UnresolvedInlineTable(aliases, rows.toSeq) table.optionalMap(ctx.tableAlias.strictIdentifier)(aliasPlan) } @@ -1180,7 +1181,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a Sequence of Strings for an identifier list. */ override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) { - ctx.ident.asScala.map(_.getText) + ctx.ident.asScala.map(_.getText).toSeq } /* ******************************************************************************************** @@ -1205,10 +1206,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging /** * Create a multi-part identifier. */ - override def visitMultipartIdentifier( - ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { - ctx.parts.asScala.map(_.getText) - } + override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = + withOrigin(ctx) { + ctx.parts.asScala.map(_.getText).toSeq + } /* ******************************************************************************************** * Expression parsing @@ -1223,7 +1224,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create sequence of expressions from the given sequence of contexts. */ private def expressionList(trees: java.util.List[ExpressionContext]): Seq[Expression] = { - trees.asScala.map(expression) + trees.asScala.map(expression).toSeq } /** @@ -1231,7 +1232,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Both un-targeted (global) and targeted aliases are supported. */ override def visitStar(ctx: StarContext): Expression = withOrigin(ctx) { - UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText))) + UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText).toSeq)) } /** @@ -1387,7 +1388,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (expressions.isEmpty) { throw new ParseException("Expected something between '(' and ')'.", ctx) } else { - expressions.asScala.map(expression).map(p => invertIfNotDefined(new Like(e, p))) + expressions.asScala.map(expression).map(p => invertIfNotDefined(new Like(e, p))).toSeq } } @@ -1401,7 +1402,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case SqlBaseParser.IN if ctx.query != null => invertIfNotDefined(InSubquery(getValueExpressions(e), ListQuery(plan(ctx.query)))) case SqlBaseParser.IN => - invertIfNotDefined(In(e, ctx.expression.asScala.map(expression))) + invertIfNotDefined(In(e, ctx.expression.asScala.map(expression).toSeq)) case SqlBaseParser.LIKE => Option(ctx.quantifier).map(_.getType) match { case Some(SqlBaseParser.ANY) | Some(SqlBaseParser.SOME) => @@ -1526,7 +1527,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a [[CreateStruct]] expression. */ override def visitStruct(ctx: StructContext): Expression = withOrigin(ctx) { - CreateStruct.create(ctx.argument.asScala.map(expression)) + CreateStruct.create(ctx.argument.asScala.map(expression).toSeq) } /** @@ -1617,7 +1618,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Transform COUNT(*) into COUNT(1). Seq(Literal(1)) case expressions => - expressions + expressions.toSeq } val filter = Option(ctx.where).map(expression(_)) val function = UnresolvedFunction( @@ -1639,14 +1640,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * This is used in CREATE FUNCTION, DROP FUNCTION, SHOWFUNCTIONS. */ protected def visitFunctionName(ctx: MultipartIdentifierContext): FunctionIdentifier = { - visitFunctionName(ctx, ctx.parts.asScala.map(_.getText)) + visitFunctionName(ctx, ctx.parts.asScala.map(_.getText).toSeq) } /** * Create a function database (optional) and name pair. */ protected def visitFunctionName(ctx: QualifiedNameContext): FunctionIdentifier = { - visitFunctionName(ctx, ctx.identifier().asScala.map(_.getText)) + visitFunctionName(ctx, ctx.identifier().asScala.map(_.getText).toSeq) } /** @@ -1682,7 +1683,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val function = expression(ctx.expression).transformUp { case a: UnresolvedAttribute => UnresolvedNamedLambdaVariable(a.nameParts) } - LambdaFunction(function, arguments) + LambdaFunction(function, arguments.toSeq) } /** @@ -1714,8 +1715,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } WindowSpecDefinition( - partition, - order, + partition.toSeq, + order.toSeq, frameSpecOption.getOrElse(UnspecifiedFrame)) } @@ -1747,7 +1748,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a [[CreateStruct]] expression. */ override def visitRowConstructor(ctx: RowConstructorContext): Expression = withOrigin(ctx) { - CreateStruct(ctx.namedExpression().asScala.map(expression)) + CreateStruct(ctx.namedExpression().asScala.map(expression).toSeq) } /** @@ -1773,7 +1774,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val branches = ctx.whenClause.asScala.map { wCtx => (EqualTo(e, expression(wCtx.condition)), expression(wCtx.result)) } - CaseWhen(branches, Option(ctx.elseExpression).map(expression)) + CaseWhen(branches.toSeq, Option(ctx.elseExpression).map(expression)) } /** @@ -1792,7 +1793,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val branches = ctx.whenClause.asScala.map { wCtx => (expression(wCtx.condition), expression(wCtx.result)) } - CaseWhen(branches, Option(ctx.elseExpression).map(expression)) + CaseWhen(branches.toSeq, Option(ctx.elseExpression).map(expression)) } /** @@ -2245,7 +2246,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a [[StructType]] from a number of column definitions. */ override def visitColTypeList(ctx: ColTypeListContext): Seq[StructField] = withOrigin(ctx) { - ctx.colType().asScala.map(visitColType) + ctx.colType().asScala.map(visitColType).toSeq } /** @@ -2286,7 +2287,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitComplexColTypeList( ctx: ComplexColTypeListContext): Seq[StructField] = withOrigin(ctx) { - ctx.complexColType().asScala.map(visitComplexColType) + ctx.complexColType().asScala.map(visitComplexColType).toSeq } /** @@ -2362,7 +2363,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging key -> value } // Check for duplicate property names. - checkDuplicateKeys(properties, ctx) + checkDuplicateKeys(properties.toSeq, ctx) properties.toMap } @@ -2443,7 +2444,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (temporary && ifNotExists) { operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) } - val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText) + val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText).toSeq (multipartIdentifier, temporary, ifNotExists, ctx.EXTERNAL != null) } @@ -2452,7 +2453,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitReplaceTableHeader( ctx: ReplaceTableHeaderContext): TableHeader = withOrigin(ctx) { - val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText) + val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText).toSeq (multipartIdentifier, false, false, false) } @@ -2460,7 +2461,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Parse a qualified name to a multipart name. */ override def visitQualifiedName(ctx: QualifiedNameContext): Seq[String] = withOrigin(ctx) { - ctx.identifier.asScala.map(_.getText) + ctx.identifier.asScala.map(_.getText).toSeq } /** @@ -2498,7 +2499,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging IdentityTransform(FieldReference(typedVisit[Seq[String]](identityCtx.qualifiedName))) case applyCtx: ApplyTransformContext => - val arguments = applyCtx.argument.asScala.map(visitTransformArgument) + val arguments = applyCtx.argument.asScala.map(visitTransformArgument).toSeq applyCtx.identifier.getText match { case "bucket" => @@ -2515,7 +2516,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val fields = arguments.tail.map(arg => getFieldReference(applyCtx, arg)) - BucketTransform(LiteralValue(numBuckets, IntegerType), fields) + BucketTransform(LiteralValue(numBuckets, IntegerType), fields.toSeq) case "years" => YearsTransform(getSingleFieldReference(applyCtx, arguments)) @@ -2532,7 +2533,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case name => ApplyTransform(name, arguments) } - } + }.toSeq } /** @@ -2956,7 +2957,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) { AlterTableAddColumnsStatement( visitMultipartIdentifier(ctx.multipartIdentifier), - ctx.columns.qualifiedColTypeWithPosition.asScala.map(typedVisit[QualifiedColType]) + ctx.columns.qualifiedColTypeWithPosition.asScala.map(typedVisit[QualifiedColType]).toSeq ) } @@ -2972,7 +2973,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx: RenameTableColumnContext): LogicalPlan = withOrigin(ctx) { AlterTableRenameColumnStatement( visitMultipartIdentifier(ctx.table), - ctx.from.parts.asScala.map(_.getText), + ctx.from.parts.asScala.map(_.getText).toSeq, ctx.to.getText) } @@ -3084,7 +3085,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Column position is not supported in Hive-style REPLACE COLUMNS") } typedVisit[QualifiedColType](colType) - } + }.toSeq ) } @@ -3102,7 +3103,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val columnsToDrop = ctx.columns.multipartIdentifier.asScala.map(typedVisit[Seq[String]]) AlterTableDropColumnsStatement( visitMultipartIdentifier(ctx.multipartIdentifier), - columnsToDrop) + columnsToDrop.toSeq) } /** @@ -3175,7 +3176,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } else { DescribeColumnStatement( visitMultipartIdentifier(ctx.multipartIdentifier()), - ctx.describeColName.nameParts.asScala.map(_.getText), + ctx.describeColName.nameParts.asScala.map(_.getText).toSeq, isExtended) } } else { @@ -3411,7 +3412,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } AlterTableAddPartitionStatement( visitMultipartIdentifier(ctx.multipartIdentifier), - specsAndLocs, + specsAndLocs.toSeq, ctx.EXISTS != null) } @@ -3451,7 +3452,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } AlterTableDropPartitionStatement( visitMultipartIdentifier(ctx.multipartIdentifier), - ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec), + ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec).toSeq, ifExists = ctx.EXISTS != null, purge = ctx.PURGE != null, retainData = false) @@ -3646,7 +3647,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging CreateFunctionStatement( functionIdentifier, string(ctx.className), - resources, + resources.toSeq, ctx.TEMPORARY != null, ctx.EXISTS != null, ctx.REPLACE != null) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala index e1dbef9ebeede..967ccedeeeacb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala @@ -344,7 +344,7 @@ object EstimationUtils { } } } - overlappedRanges + overlappedRanges.toSeq } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala index 19a0d1279cc32..777a4c8291223 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala @@ -323,7 +323,7 @@ case class JoinEstimation(join: Join) extends Logging { outputAttrStats += a -> newColStat } } - outputAttrStats + outputAttrStats.toSeq } private def extractJoinKeysWithColStats( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index c4a106702a515..6cd062da2b94a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -185,7 +185,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def map[A](f: BaseType => A): Seq[A] = { val ret = new collection.mutable.ArrayBuffer[A]() foreach(ret += f(_)) - ret + ret.toSeq } /** @@ -195,7 +195,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def flatMap[A](f: BaseType => TraversableOnce[A]): Seq[A] = { val ret = new collection.mutable.ArrayBuffer[A]() foreach(ret ++= f(_)) - ret + ret.toSeq } /** @@ -206,7 +206,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { val ret = new collection.mutable.ArrayBuffer[B]() val lifted = pf.lift foreach(node => lifted(node).foreach(ret.+=)) - ret + ret.toSeq } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index 1f88a700847de..711ef265c6cf7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -26,7 +26,7 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(seq: Seq[Any]) = this(seq.toArray) - def this(list: java.util.List[Any]) = this(list.asScala) + def this(list: java.util.List[Any]) = this(list.asScala.toSeq) // TODO: This is boxing. We should specialize. def this(primitiveArray: Array[Int]) = this(primitiveArray.toSeq) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala index 3a0490d07733d..2797a40614504 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala @@ -223,7 +223,7 @@ class QuantileSummaries( otherIdx += 1 } - val comp = compressImmut(mergedSampled, 2 * mergedRelativeError * mergedCount) + val comp = compressImmut(mergedSampled.toIndexedSeq, 2 * mergedRelativeError * mergedCount) new QuantileSummaries(other.compressThreshold, mergedRelativeError, comp, mergedCount, true) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index bd2c1d5c26299..b14fb04cc4539 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -423,7 +423,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum override def simpleString: String = { - val fieldTypes = fields.view.map(field => s"${field.name}:${field.dataType.simpleString}") + val fieldTypes = fields.view.map(field => s"${field.name}:${field.dataType.simpleString}").toSeq truncatedString( fieldTypes, "struct<", ",", ">", @@ -542,7 +542,7 @@ object StructType extends AbstractDataType { def apply(fields: java.util.List[StructField]): StructType = { import scala.collection.JavaConverters._ - StructType(fields.asScala) + StructType(fields.asScala.toSeq) } private[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = @@ -606,7 +606,7 @@ object StructType extends AbstractDataType { newFields += f } - StructType(newFields) + StructType(newFields.toSeq) case (DecimalType.Fixed(leftPrecision, leftScale), DecimalType.Fixed(rightPrecision, rightScale)) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index 003ce850c926e..c3bc67d76138a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -121,7 +121,7 @@ private[sql] object ArrowUtils { val dt = fromArrowField(child) StructField(child.getName, dt, child.isNullable) } - StructType(fields) + StructType(fields.toSeq) case arrowType => fromArrowType(arrowType) } } @@ -137,7 +137,7 @@ private[sql] object ArrowUtils { StructType(schema.getFields.asScala.map { field => val dt = fromArrowField(field) StructField(field.getName, dt, field.isNullable) - }) + }.toSeq) } /** Return Map with conf settings to be used in ArrowPythonRunner */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala index 6a5bdc4f6fc3d..9fb8b0f351d51 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala @@ -136,7 +136,7 @@ object RandomDataGenerator { } i += 1 } - StructType(fields) + StructType(fields.toSeq) } /** @@ -372,6 +372,6 @@ object RandomDataGenerator { fields += gen() } } - Row.fromSeq(fields) + Row.fromSeq(fields.toSeq) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala index 5be37318ae6eb..bfa415afeab93 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala @@ -127,7 +127,7 @@ class ColumnPruningSuite extends PlanTest { val optimized = Optimize.execute(query) - val aliases = NestedColumnAliasingSuite.collectGeneratedAliases(optimized) + val aliases = NestedColumnAliasingSuite.collectGeneratedAliases(optimized).toSeq val selectedFields = UnresolvedAttribute("a") +: aliasedExprs(aliases) val finalSelectedExprs = Seq(UnresolvedAttribute("a"), $"${aliases(0)}".as("c.d")) ++ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala index 229e32479082c..f921f06537080 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.util -import scala.collection._ - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow, SpecificInternalRow, UnsafeMapData, UnsafeProjection} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala index 51286986b835c..79c06cf8313b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala @@ -288,14 +288,14 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite { withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> zoneId.getId) { withDefaultTimeZone(zoneId) { withClue(s"zoneId = ${zoneId.getId}") { - val formatters = LegacyDateFormats.values.map { legacyFormat => + val formatters = LegacyDateFormats.values.toSeq.map { legacyFormat => TimestampFormatter( TimestampFormatter.defaultPattern, zoneId, TimestampFormatter.defaultLocale, legacyFormat, isParsing = false) - }.toSeq :+ TimestampFormatter.getFractionFormatter(zoneId) + } :+ TimestampFormatter.getFractionFormatter(zoneId) formatters.foreach { formatter => assert(microsToInstant(formatter.parse("1000-01-01 01:02:03")) .atZone(zoneId) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala index 1a262d646ca10..9fa016146bbd3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala @@ -502,6 +502,6 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { DataType.canWrite(writeType, readType, byName, analysis.caseSensitiveResolution, name, storeAssignmentPolicy, errMsg => errs += errMsg) === false, desc) assert(errs.size === numErrs, s"Should produce $numErrs error messages") - checkErrors(errs) + checkErrors(errs.toSeq) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index c37d8eaa294bf..611c03e7b208e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -479,7 +479,7 @@ class RelationalGroupedDataset protected[sql]( * @since 2.4.0 */ def pivot(pivotColumn: Column, values: java.util.List[Any]): RelationalGroupedDataset = { - pivot(pivotColumn, values.asScala) + pivot(pivotColumn, values.asScala.toSeq) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 9278eeeefe608..08b0a1c6a60a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -372,7 +372,7 @@ class SparkSession private( */ @DeveloperApi def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = withActive { - Dataset.ofRows(self, LocalRelation.fromExternalRows(schema.toAttributes, rows.asScala)) + Dataset.ofRows(self, LocalRelation.fromExternalRows(schema.toAttributes, rows.asScala.toSeq)) } /** @@ -495,7 +495,7 @@ class SparkSession private( * @since 2.0.0 */ def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { - createDataset(data.asScala) + createDataset(data.asScala.toSeq) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index 1c2bf9e7c2a57..ff706b5061f0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -103,7 +103,7 @@ class SparkSessionExtensions { * Build the override rules for columnar execution. */ private[sql] def buildColumnarRules(session: SparkSession): Seq[ColumnarRule] = { - columnarRuleBuilders.map(_.apply(session)) + columnarRuleBuilders.map(_.apply(session)).toSeq } /** @@ -119,7 +119,7 @@ class SparkSessionExtensions { * Build the analyzer resolution `Rule`s using the given [[SparkSession]]. */ private[sql] def buildResolutionRules(session: SparkSession): Seq[Rule[LogicalPlan]] = { - resolutionRuleBuilders.map(_.apply(session)) + resolutionRuleBuilders.map(_.apply(session)).toSeq } /** @@ -136,7 +136,7 @@ class SparkSessionExtensions { * Build the analyzer post-hoc resolution `Rule`s using the given [[SparkSession]]. */ private[sql] def buildPostHocResolutionRules(session: SparkSession): Seq[Rule[LogicalPlan]] = { - postHocResolutionRuleBuilders.map(_.apply(session)) + postHocResolutionRuleBuilders.map(_.apply(session)).toSeq } /** @@ -153,7 +153,7 @@ class SparkSessionExtensions { * Build the check analysis `Rule`s using the given [[SparkSession]]. */ private[sql] def buildCheckRules(session: SparkSession): Seq[LogicalPlan => Unit] = { - checkRuleBuilders.map(_.apply(session)) + checkRuleBuilders.map(_.apply(session)).toSeq } /** @@ -168,7 +168,7 @@ class SparkSessionExtensions { private[this] val optimizerRules = mutable.Buffer.empty[RuleBuilder] private[sql] def buildOptimizerRules(session: SparkSession): Seq[Rule[LogicalPlan]] = { - optimizerRules.map(_.apply(session)) + optimizerRules.map(_.apply(session)).toSeq } /** @@ -184,7 +184,7 @@ class SparkSessionExtensions { private[this] val plannerStrategyBuilders = mutable.Buffer.empty[StrategyBuilder] private[sql] def buildPlannerStrategies(session: SparkSession): Seq[Strategy] = { - plannerStrategyBuilders.map(_.apply(session)) + plannerStrategyBuilders.map(_.apply(session)).toSeq } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala index 9807b5dbe9348..94e159c562e31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala @@ -257,16 +257,16 @@ object AggregatingAccumulator { imperative }) - val updateAttrSeq: AttributeSeq = aggBufferAttributes ++ inputAttributes - val mergeAttrSeq: AttributeSeq = aggBufferAttributes ++ inputAggBufferAttributes - val aggBufferAttributesSeq: AttributeSeq = aggBufferAttributes + val updateAttrSeq: AttributeSeq = (aggBufferAttributes ++ inputAttributes).toSeq + val mergeAttrSeq: AttributeSeq = (aggBufferAttributes ++ inputAggBufferAttributes).toSeq + val aggBufferAttributesSeq: AttributeSeq = aggBufferAttributes.toSeq // Create the accumulator. new AggregatingAccumulator( - aggBufferAttributes.map(_.dataType), - initialValues, - updateExpressions.map(BindReferences.bindReference(_, updateAttrSeq)), - mergeExpressions.map(BindReferences.bindReference(_, mergeAttrSeq)), + aggBufferAttributes.map(_.dataType).toSeq, + initialValues.toSeq, + updateExpressions.map(BindReferences.bindReference(_, updateAttrSeq)).toSeq, + mergeExpressions.map(BindReferences.bindReference(_, mergeAttrSeq)).toSeq, resultExpressions.map(BindReferences.bindReference(_, aggBufferAttributesSeq)), imperatives.toArray, typedImperatives.toArray, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3a2c673229c20..363282ea95997 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -229,14 +229,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { */ override def visitNestedConstantList( ctx: NestedConstantListContext): Seq[Seq[String]] = withOrigin(ctx) { - ctx.constantList.asScala.map(visitConstantList) + ctx.constantList.asScala.map(visitConstantList).toSeq } /** * Convert a constants list into a String sequence. */ override def visitConstantList(ctx: ConstantListContext): Seq[String] = withOrigin(ctx) { - ctx.constant.asScala.map(visitStringConstant) + ctx.constant.asScala.map(visitStringConstant).toSeq } /** @@ -355,7 +355,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { // Storage format val defaultStorage = HiveSerDe.getDefaultStorage(conf) - validateRowFormatFileFormat(ctx.rowFormat.asScala, ctx.createFileFormat.asScala, ctx) + validateRowFormatFileFormat( + ctx.rowFormat.asScala.toSeq, ctx.createFileFormat.asScala.toSeq, ctx) val fileStorage = ctx.createFileFormat.asScala.headOption.map(visitCreateFileFormat) .getOrElse(CatalogStorageFormat.empty) val rowStorage = ctx.rowFormat.asScala.headOption.map(visitRowFormat) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 0244542054611..558d990e8c4bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -263,7 +263,7 @@ trait CodegenSupport extends SparkPlan { paramVars += ExprCode(paramIsNull, JavaCode.variable(paramName, attributes(i).dataType)) } - (arguments, parameters, paramVars) + (arguments.toSeq, parameters.toSeq, paramVars.toSeq) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index bc924e6978ddc..112090640040a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -196,7 +196,7 @@ case class AdaptiveSparkPlanExec( // In case of errors, we cancel all running stages and throw exception. if (errors.nonEmpty) { - cleanUpAndThrowException(errors, None) + cleanUpAndThrowException(errors.toSeq, None) } // Try re-optimizing and re-planning. Adopt the new plan if its cost is equal to or less diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala index 3cf6a13a4a892..8d7a2c95081c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala @@ -65,7 +65,7 @@ trait AdaptiveSparkPlanHelper { def mapPlans[A](p: SparkPlan)(f: SparkPlan => A): Seq[A] = { val ret = new collection.mutable.ArrayBuffer[A]() foreach(p)(ret += f(_)) - ret + ret.toSeq } /** @@ -75,7 +75,7 @@ trait AdaptiveSparkPlanHelper { def flatMap[A](p: SparkPlan)(f: SparkPlan => TraversableOnce[A]): Seq[A] = { val ret = new collection.mutable.ArrayBuffer[A]() foreach(p)(ret ++= f(_)) - ret + ret.toSeq } /** @@ -86,7 +86,7 @@ trait AdaptiveSparkPlanHelper { val ret = new collection.mutable.ArrayBuffer[B]() val lifted = pf.lift foreach(p)(node => lifted(node).foreach(ret.+=)) - ret + ret.toSeq } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala index b2633c774f532..af18ee065aa86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala @@ -146,7 +146,7 @@ case class CustomShuffleReaderExec private( partitionDataSizeMetrics.set(dataSizes.sum) } - SQLMetrics.postDriverMetricsUpdatedByValue(sparkContext, executionId, driverAccumUpdates) + SQLMetrics.postDriverMetricsUpdatedByValue(sparkContext, executionId, driverAccumUpdates.toSeq) } @transient override lazy val metrics: Map[String, SQLMetric] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index 396c9c9d6b4e5..627f0600f2383 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -241,8 +241,8 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { logDebug(s"number of skewed partitions: left $numSkewedLeft, right $numSkewedRight") if (numSkewedLeft > 0 || numSkewedRight > 0) { - val newLeft = CustomShuffleReaderExec(left.shuffleStage, leftSidePartitions) - val newRight = CustomShuffleReaderExec(right.shuffleStage, rightSidePartitions) + val newLeft = CustomShuffleReaderExec(left.shuffleStage, leftSidePartitions.toSeq) + val newRight = CustomShuffleReaderExec(right.shuffleStage, rightSidePartitions.toSeq) smj.copy( left = s1.copy(child = newLeft), right = s2.copy(child = newRight), isSkewJoin = true) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala index d6e44b780d772..83fdafbadcb60 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala @@ -121,7 +121,7 @@ object ShufflePartitionsUtil extends Logging { i += 1 } createPartitionSpec() - partitionSpecs + partitionSpecs.toSeq } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index c8fa07941af87..cf9f3ddeb42a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -123,7 +123,7 @@ case class CachedRDDBuilder( rowCountStats.add(rowCount) val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics)) + columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq) CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala index 33b29bde93ee5..fc62dce5002b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala @@ -69,7 +69,7 @@ case class AnalyzePartitionCommand( if (filteredSpec.isEmpty) { None } else { - Some(filteredSpec) + Some(filteredSpec.toMap) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 47b213fc2d83b..d550fe270c753 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -650,7 +650,7 @@ case class AlterTableRecoverPartitionsCommand( val pathFilter = getPathFilter(hadoopConf) val evalPool = ThreadUtils.newForkJoinPool("AlterTableRecoverPartitionsCommand", 8) - val partitionSpecsAndLocs: Seq[(TablePartitionSpec, Path)] = + val partitionSpecsAndLocs: GenSeq[(TablePartitionSpec, Path)] = try { scanPartitions(spark, fs, pathFilter, root, Map(), table.partitionColumnNames, threshold, spark.sessionState.conf.resolver, new ForkJoinTaskSupport(evalPool)).seq @@ -697,7 +697,7 @@ case class AlterTableRecoverPartitionsCommand( // parallelize the list of partitions here, then we can have better parallelism later. val parArray = new ParVector(statuses.toVector) parArray.tasksupport = evalTaskSupport - parArray + parArray.seq } else { statuses } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index fc8cc11bb1067..7aebdddf1d59c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -657,7 +657,7 @@ case class DescribeTableCommand( } } - result + result.toSeq } private def describePartitionInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { @@ -740,7 +740,7 @@ case class DescribeQueryCommand(queryText: String, plan: LogicalPlan) val result = new ArrayBuffer[Row] val queryExecution = sparkSession.sessionState.executePlan(plan) describeSchema(queryExecution.analyzed.schema, result, header = false) - result + result.toSeq } } @@ -815,7 +815,7 @@ case class DescribeColumnCommand( } yield histogramDescription(hist) buffer ++= histDesc.getOrElse(Seq(Row("histogram", "NULL"))) } - buffer + buffer.toSeq } private def histogramDescription(histogram: Histogram): Seq[Row] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 07d7c4e97a095..db564485be883 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -811,7 +811,7 @@ object DataSource extends Logging { val path = CaseInsensitiveMap(options).get("path") val optionsWithoutPath = options.filterKeys(_.toLowerCase(Locale.ROOT) != "path") CatalogStorageFormat.empty.copy( - locationUri = path.map(CatalogUtils.stringToURI), properties = optionsWithoutPath) + locationUri = path.map(CatalogUtils.stringToURI), properties = optionsWithoutPath.toMap) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala index 095940772ae78..864130bbd87b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala @@ -80,7 +80,7 @@ object FilePartition extends Logging { currentFiles += file } closePartition() - partitions + partitions.toSeq } def maxSplitBytes( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala index 57082b40e1132..b5e276bd421a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala @@ -46,7 +46,7 @@ class HadoopFileLinesReader( def this(file: PartitionedFile, conf: Configuration) = this(file, None, conf) - private val iterator = { + private val _iterator = { val fileSplit = new FileSplit( new Path(new URI(file.filePath)), file.start, @@ -66,9 +66,9 @@ class HadoopFileLinesReader( new RecordReaderIterator(reader) } - override def hasNext: Boolean = iterator.hasNext + override def hasNext: Boolean = _iterator.hasNext - override def next(): Text = iterator.next() + override def next(): Text = _iterator.next() - override def close(): Unit = iterator.close() + override def close(): Unit = _iterator.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.scala index 0e6d803f02d4d..a48001f04a9bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.scala @@ -35,7 +35,7 @@ import org.apache.spark.input.WholeTextFileRecordReader */ class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration) extends Iterator[Text] with Closeable { - private val iterator = { + private val _iterator = { val fileSplit = new CombineFileSplit( Array(new Path(new URI(file.filePath))), Array(file.start), @@ -50,9 +50,9 @@ class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration) new RecordReaderIterator(reader) } - override def hasNext: Boolean = iterator.hasNext + override def hasNext: Boolean = _iterator.hasNext - override def next(): Text = iterator.next() + override def next(): Text = _iterator.next() - override def close(): Unit = iterator.close() + override def close(): Unit = _iterator.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index 84160f35540df..a488ed16a835a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -133,7 +133,7 @@ class InMemoryFileIndex( } val filter = FileInputFormat.getInputPathFilter(new JobConf(hadoopConf, this.getClass)) val discovered = InMemoryFileIndex.bulkListLeafFiles( - pathsToFetch, hadoopConf, filter, sparkSession, areRootPaths = true) + pathsToFetch.toSeq, hadoopConf, filter, sparkSession, areRootPaths = true) discovered.foreach { case (path, leafFiles) => HiveCatalogMetrics.incrementFilesDiscovered(leafFiles.size) fileStatusCache.putLeafFiles(path, leafFiles.toArray) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 5846d46e146ed..4087efc486a4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -273,7 +273,7 @@ object PartitioningUtils { (None, Some(path)) } else { val (columnNames, values) = columns.reverse.unzip - (Some(PartitionValues(columnNames, values)), Some(currentPath)) + (Some(PartitionValues(columnNames.toSeq, values.toSeq)), Some(currentPath)) } } @@ -420,7 +420,7 @@ object PartitioningUtils { val distinctPartColNames = pathWithPartitionValues.map(_._2.columnNames).distinct def groupByKey[K, V](seq: Seq[(K, V)]): Map[K, Iterable[V]] = - seq.groupBy { case (key, _) => key }.mapValues(_.map { case (_, value) => value }) + seq.groupBy { case (key, _) => key }.mapValues(_.map { case (_, value) => value }).toMap val partColNamesToPaths = groupByKey(pathWithPartitionValues.map { case (path, partValues) => partValues.columnNames -> path diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 39bbc60200b86..73910c3943e9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -68,14 +68,14 @@ class ParquetFilters( // When g is a `Map`, `g.getOriginalType` is `MAP`. // When g is a `List`, `g.getOriginalType` is `LIST`. case g: GroupType if g.getOriginalType == null => - getPrimitiveFields(g.getFields.asScala, parentFieldNames :+ g.getName) + getPrimitiveFields(g.getFields.asScala.toSeq, parentFieldNames :+ g.getName) // Parquet only supports push-down for primitive types; as a result, Map and List types // are removed. case _ => None } } - val primitiveFields = getPrimitiveFields(schema.getFields.asScala).map { field => + val primitiveFields = getPrimitiveFields(schema.getFields.asScala.toSeq).map { field => import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper (field.fieldNames.toSeq.quoted, field) } @@ -90,7 +90,7 @@ class ParquetFilters( .groupBy(_._1.toLowerCase(Locale.ROOT)) .filter(_._2.size == 1) .mapValues(_.head._2) - CaseInsensitiveMap(dedupPrimitiveFields) + CaseInsensitiveMap(dedupPrimitiveFields.toMap) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 8ce8a86d2f026..2eb205db8ccdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -79,7 +79,7 @@ class ParquetToSparkSchemaConverter( } } - StructType(fields) + StructType(fields.toSeq) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala index b4a14c6face31..e273abf90e3bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala @@ -55,7 +55,7 @@ case class DescribeNamespaceExec( rows += toCatalystRow("Properties", properties.toSeq.mkString("(", ",", ")")) } } - rows + rows.toSeq } private def toCatalystRow(strs: String*): InternalRow = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index bc6bb175f979e..81b1c81499c74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -43,7 +43,7 @@ case class DescribeTableExec( if (isExtended) { addTableDetails(rows) } - rows + rows.toSeq } private def addTableDetails(rows: ArrayBuffer[InternalRow]): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala index 1a6f03f54f2e9..7f6ae20d5cd0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala @@ -63,7 +63,7 @@ object PushDownUtils extends PredicateHelper { val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) } - (r.pushedFilters(), untranslatableExprs ++ postScanFilters) + (r.pushedFilters(), (untranslatableExprs ++ postScanFilters).toSeq) case _ => (Nil, filters) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala index 9188f4eb60d56..ceeed0f840700 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala @@ -52,6 +52,6 @@ case class ShowNamespacesExec( } } - rows + rows.toSeq } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 820f5ae8f1b12..5ba01deae9513 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -49,6 +49,6 @@ case class ShowTablesExec( } } - rows + rows.toSeq } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 2ed33b867183b..df3f231f7d0ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -288,7 +288,7 @@ private[sql] object V2SessionCatalog { s"SessionCatalog does not support partition transform: $transform") } - (identityCols, bucketSpec) + (identityCols.toSeq, bucketSpec) } private def toCatalogDatabase( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 3242ac21ab324..186bac6f43332 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -162,7 +162,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { return (leftKeys, rightKeys) } } - (leftKeysBuffer, rightKeysBuffer) + (leftKeysBuffer.toSeq, rightKeysBuffer.toSeq) } private def reorderJoinKeys( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala index fcbd0b19515b1..dadf1129c34b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala @@ -103,11 +103,11 @@ case class AggregateInPandasExec( // Schema of input rows to the python runner val aggInputSchema = StructType(dataTypes.zipWithIndex.map { case (dt, i) => StructField(s"_$i", dt) - }) + }.toSeq) // Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { - val prunedProj = UnsafeProjection.create(allInputs, child.output) + val prunedProj = UnsafeProjection.create(allInputs.toSeq, child.output) val grouped = if (groupingExpressions.isEmpty) { // Use an empty unsafe row as a place holder for the grouping key diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala index 96e3bb721a822..298d63478b63e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala @@ -114,10 +114,10 @@ trait EvalPythonExec extends UnaryExecNode { } }.toArray }.toArray - val projection = MutableProjection.create(allInputs, child.output) + val projection = MutableProjection.create(allInputs.toSeq, child.output) val schema = StructType(dataTypes.zipWithIndex.map { case (dt, i) => StructField(s"_$i", dt) - }) + }.toSeq) // Add rows to queue to join later with the result. val projectedRowIter = iter.map { inputRow => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index 7bc8b95cfb03b..1c88056cb50c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -72,7 +72,7 @@ object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] { } } // There is no Python UDF over aggregate expression - Project(projList, agg.copy(aggregateExpressions = aggExpr)) + Project(projList.toSeq, agg.copy(aggregateExpressions = aggExpr.toSeq)) } def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { @@ -134,9 +134,9 @@ object ExtractGroupingPythonUDFFromAggregate extends Rule[LogicalPlan] { }.asInstanceOf[NamedExpression] } agg.copy( - groupingExpressions = groupingExpr, + groupingExpressions = groupingExpr.toSeq, aggregateExpressions = aggExpr, - child = Project(projList ++ agg.child.output, agg.child)) + child = Project((projList ++ agg.child.output).toSeq, agg.child)) } def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PandasGroupUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PandasGroupUtils.scala index 68ce991a8ae7f..2da0000dad4ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PandasGroupUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PandasGroupUtils.scala @@ -118,6 +118,6 @@ private[python] object PandasGroupUtils { // Attributes after deduplication val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes - (dedupAttributes, argOffsets) + (dedupAttributes.toSeq, argOffsets) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala index e8ae0eaf0ea48..29537cc0e573f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala @@ -347,7 +347,7 @@ object CompactibleFileStreamLog { } else if (defaultInterval < (latestCompactBatchId + 1) / 2) { // Find the first divisor >= default compact interval def properDivisors(min: Int, n: Int) = - (min to n/2).view.filter(i => n % i == 0) :+ n + (min to n/2).view.filter(i => n % i == 0).toSeq :+ n properDivisors(defaultInterval, latestCompactBatchId + 1).head } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala index 985a5fa6063ef..11bdfee460e66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.streaming +import scala.collection.mutable + import org.apache.spark.SparkEnv import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.sql.catalyst.expressions.UnsafeRow @@ -33,7 +35,7 @@ case class GetRecord(offset: ContinuousRecordPartitionOffset) * to the number of partitions. * @param lock a lock object for locking the buckets for read */ -class ContinuousRecordEndpoint(buckets: Seq[Seq[UnsafeRow]], lock: Object) +class ContinuousRecordEndpoint(buckets: Seq[mutable.Seq[UnsafeRow]], lock: Object) extends ThreadSafeRpcEndpoint { private var startOffsets: Seq[Int] = List.fill(buckets.size)(0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index e8ce8e1487093..f2557696485b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -482,7 +482,7 @@ object FileStreamSource { } private def buildSourceGlobFilters(sourcePath: Path): Seq[GlobFilter] = { - val filters = new scala.collection.mutable.MutableList[GlobFilter]() + val filters = new scala.collection.mutable.ArrayBuffer[GlobFilter]() var currentPath = sourcePath while (!currentPath.isRoot) { @@ -490,7 +490,7 @@ object FileStreamSource { currentPath = currentPath.getParent } - filters.toList + filters.toSeq } override protected def cleanTask(entry: FileEntry): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index f6cc8116c6c4c..de8a8cd7d3b58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -139,7 +139,7 @@ class ManifestFileCommitProtocol(jobId: String, path: String) if (addedFiles.nonEmpty) { val fs = new Path(addedFiles.head).getFileSystem(taskContext.getConfiguration) val statuses: Seq[SinkFileStatus] = - addedFiles.map(f => SinkFileStatus(fs.getFileStatus(new Path(f)))) + addedFiles.map(f => SinkFileStatus(fs.getFileStatus(new Path(f)))).toSeq new TaskCommitMessage(statuses) } else { new TaskCommitMessage(Seq.empty[SinkFileStatus]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 2c737206dd2d9..fe3f0e95b383c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -127,8 +127,8 @@ trait ProgressReporter extends Logging { * `committedOffsets` in `StreamExecution` to make sure that the correct range is recorded. */ protected def recordTriggerOffsets(from: StreamProgress, to: StreamProgress): Unit = { - currentTriggerStartOffsets = from.mapValues(_.json) - currentTriggerEndOffsets = to.mapValues(_.json) + currentTriggerStartOffsets = from.mapValues(_.json).toMap + currentTriggerEndOffsets = to.mapValues(_.json).toMap } private def updateProgress(newProgress: StreamingQueryProgress): Unit = { @@ -192,7 +192,8 @@ trait ProgressReporter extends Logging { timestamp = formatTimestamp(currentTriggerStartTimestamp), batchId = currentBatchId, batchDuration = processingTimeMills, - durationMs = new java.util.HashMap(currentDurationsMs.toMap.mapValues(long2Long).asJava), + durationMs = + new java.util.HashMap(currentDurationsMs.toMap.mapValues(long2Long).toMap.asJava), eventTime = new java.util.HashMap(executionStats.eventTimeStats.asJava), stateOperators = executionStats.stateOperators.toArray, sources = sourceProgress.toArray, @@ -255,14 +256,14 @@ trait ProgressReporter extends Logging { "avg" -> stats.avg.toLong).mapValues(formatTimestamp) }.headOption.getOrElse(Map.empty) ++ watermarkTimestamp - ExecutionStats(numInputRows, stateOperators, eventTimeStats) + ExecutionStats(numInputRows, stateOperators, eventTimeStats.toMap) } /** Extract number of input sources for each streaming source in plan */ private def extractSourceToNumInputRows(): Map[SparkDataStream, Long] = { def sumRows(tuples: Seq[(SparkDataStream, Long)]): Map[SparkDataStream, Long] = { - tuples.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source + tuples.groupBy(_._1).mapValues(_.map(_._2).sum).toMap // sum up rows for each source } val onlyDataSourceV2Sources = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index e5b9e68d71026..9adb9af7318d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -215,7 +215,7 @@ case class MemoryStream[A : Encoder]( batches.slice(sliceStart, sliceEnd) } - logDebug(generateDebugString(newBlocks.flatten, startOrdinal, endOrdinal)) + logDebug(generateDebugString(newBlocks.flatten.toSeq, startOrdinal, endOrdinal)) numPartitions match { case Some(numParts) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala index 03ebbb9f1b376..24ff9c2e8384d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala @@ -80,7 +80,7 @@ class MemorySink extends Table with SupportsWrite with Logging { /** Returns all rows that are stored in this [[Sink]]. */ def allData: Seq[Row] = synchronized { - batches.flatMap(_.data) + batches.flatMap(_.data).toSeq } def latestBatchId: Option[Long] = synchronized { @@ -92,7 +92,7 @@ class MemorySink extends Table with SupportsWrite with Logging { } def dataSinceBatch(sinceBatchId: Long): Seq[Row] = synchronized { - batches.filter(_.batchId > sinceBatchId).flatMap(_.data) + batches.filter(_.batchId > sinceBatchId).flatMap(_.data).toSeq } def toDebugString: String = synchronized { @@ -183,7 +183,7 @@ class MemoryDataWriter(partition: Int, schema: StructType) } override def commit(): MemoryWriterCommitMessage = { - val msg = MemoryWriterCommitMessage(partition, data.clone()) + val msg = MemoryWriterCommitMessage(partition, data.clone().toSeq) data.clear() msg } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index a9c01e69b9b13..497b13793a67b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -97,7 +97,7 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => .map(entry => entry._1 -> longMetric(entry._1).value) val javaConvertedCustomMetrics: java.util.HashMap[String, java.lang.Long] = - new java.util.HashMap(customMetrics.mapValues(long2Long).asJava) + new java.util.HashMap(customMetrics.mapValues(long2Long).toMap.asJava) new StateOperatorProgress( numRowsTotal = longMetric("numTotalStateRows").value, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala index 33539c01ee5dd..ff229c2bea7ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala @@ -57,7 +57,7 @@ private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with L if (running.nonEmpty) { val runningPageTable = - executionsTable(request, "running", running, currentTime, true, true, true) + executionsTable(request, "running", running.toSeq, currentTime, true, true, true) _content ++= val rng = new org.apache.spark.util.random.XORShiftRandom(7 + index) data.filter(_.getInt(0) < rng.nextDouble() * 10) - } - } + }.toSeq val union = df1.union(df2) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index 124b58483d24f..2be86b9ad6208 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -223,16 +223,6 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSparkSession { checkDataset(Seq(Queue(true)).toDS(), Queue(true)) checkDataset(Seq(Queue("test")).toDS(), Queue("test")) checkDataset(Seq(Queue(Tuple1(1))).toDS(), Queue(Tuple1(1))) - - checkDataset(Seq(ArrayBuffer(1)).toDS(), ArrayBuffer(1)) - checkDataset(Seq(ArrayBuffer(1.toLong)).toDS(), ArrayBuffer(1.toLong)) - checkDataset(Seq(ArrayBuffer(1.toDouble)).toDS(), ArrayBuffer(1.toDouble)) - checkDataset(Seq(ArrayBuffer(1.toFloat)).toDS(), ArrayBuffer(1.toFloat)) - checkDataset(Seq(ArrayBuffer(1.toByte)).toDS(), ArrayBuffer(1.toByte)) - checkDataset(Seq(ArrayBuffer(1.toShort)).toDS(), ArrayBuffer(1.toShort)) - checkDataset(Seq(ArrayBuffer(true)).toDS(), ArrayBuffer(true)) - checkDataset(Seq(ArrayBuffer("test")).toDS(), ArrayBuffer("test")) - checkDataset(Seq(ArrayBuffer(Tuple1(1))).toDS(), ArrayBuffer(Tuple1(1))) } test("sequence and product combinations") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index fe6775cc7f9b9..f24da6df67ca0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -712,7 +712,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |ON | big.key = small.a """.stripMargin), - expected + expected.toSeq ) } @@ -729,7 +729,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |ON | big.key = small.a """.stripMargin), - expected + expected.toSeq ) } } @@ -770,7 +770,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |ON | big.key = small.a """.stripMargin), - expected + expected.toSeq ) } @@ -787,7 +787,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |ON | big.key = small.a """.stripMargin), - expected + expected.toSeq ) } @@ -806,7 +806,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |ON | big.key = small.a """.stripMargin), - expected + expected.toSeq ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index e52d2262a6bf8..8469216901b05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -418,7 +418,7 @@ object QueryTest extends Assertions { } def checkAnswer(df: DataFrame, expectedAnswer: java.util.List[Row]): Unit = { - getErrorMessageInCheckAnswer(df, expectedAnswer.asScala) match { + getErrorMessageInCheckAnswer(df, expectedAnswer.asScala.toSeq) match { case Some(errorMessage) => Assert.fail(errorMessage) case None => } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala index 4e85f739b95a2..1106a787cc9a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala @@ -238,7 +238,7 @@ abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { table.copy( createTime = 0L, lastAccessTime = 0L, - properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)).toMap, stats = None, ignoredProperties = Map.empty ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 347bc735a8b76..2bb9aa55e4579 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -992,7 +992,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark subqueryExpressions ++= (getSubqueryExpressions(s.plan) :+ s) s } - subqueryExpressions + subqueryExpressions.toSeq } private def getNumSorts(plan: LogicalPlan): Int = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala index 96ad453aeb2d7..a9696e6718de8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala @@ -109,11 +109,11 @@ object IntervalBenchmark extends SqlBasedBenchmark { // The first 2 cases are used to show the overhead of preparing the interval string. addCase(benchmark, cardinality, "prepare string w/ interval", buildString(true, timeUnits)) addCase(benchmark, cardinality, "prepare string w/o interval", buildString(false, timeUnits)) - addCase(benchmark, cardinality, intervalToTest) // Only years + addCase(benchmark, cardinality, intervalToTest.toSeq) // Only years for (unit <- timeUnits) { intervalToTest.append(unit) - addCase(benchmark, cardinality, intervalToTest) + addCase(benchmark, cardinality, intervalToTest.toSeq) } benchmark.run() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index 8bbf81efff316..ce726046c3215 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -220,9 +220,9 @@ trait SQLMetricsTestUtils extends SQLTestUtils { (nodeName, nodeMetrics.mapValues(expectedMetricValue => (actualMetricValue: Any) => { actualMetricValue.toString.matches(expectedMetricValue.toString) - })) + }).toMap) } - testSparkPlanMetricsWithPredicates(df, expectedNumOfJobs, expectedMetricsPredicates, + testSparkPlanMetricsWithPredicates(df, expectedNumOfJobs, expectedMetricsPredicates.toMap, enableWholeStage) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index e87bd11f0dca5..0fe339b93047a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -333,6 +333,6 @@ class TestForeachWriter extends ForeachWriter[Int] { override def close(errorOrNull: Throwable): Unit = { events += ForeachWriterSuite.Close(error = Option(errorOrNull)) - ForeachWriterSuite.addEvents(events) + ForeachWriterSuite.addEvents(events.toSeq) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 4d5cd109b7c24..b033761498ea7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -209,21 +209,24 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils (1L, 0, 0, createAccumulatorInfos(accumulatorUpdates)) ))) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 2).toMap) // Driver accumulator updates don't belong to this execution should be filtered and no // exception will be thrown. listener.onOtherEvent(SparkListenerDriverAccumUpdates(0, Seq((999L, 2L)))) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 2).toMap) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, accum updates) (0L, 0, 0, createAccumulatorInfos(accumulatorUpdates)), - (1L, 0, 0, createAccumulatorInfos(accumulatorUpdates.mapValues(_ * 2))) + (1L, 0, 0, createAccumulatorInfos(accumulatorUpdates.mapValues(_ * 2).toMap)) ))) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 3)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 3).toMap) // Retrying a stage should reset the metrics listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 1))) @@ -236,7 +239,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils (1L, 0, 1, createAccumulatorInfos(accumulatorUpdates)) ))) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 2).toMap) // Ignore the task end for the first attempt listener.onTaskEnd(SparkListenerTaskEnd( @@ -244,11 +248,12 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils stageAttemptId = 0, taskType = "", reason = null, - createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 100)), + createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 100).toMap), new ExecutorMetrics, null)) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 2).toMap) // Finish two tasks listener.onTaskEnd(SparkListenerTaskEnd( @@ -256,7 +261,7 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils stageAttemptId = 1, taskType = "", reason = null, - createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 2)), + createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 2).toMap), new ExecutorMetrics, null)) listener.onTaskEnd(SparkListenerTaskEnd( @@ -264,11 +269,12 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils stageAttemptId = 1, taskType = "", reason = null, - createTaskInfo(1, 0, accums = accumulatorUpdates.mapValues(_ * 3)), + createTaskInfo(1, 0, accums = accumulatorUpdates.mapValues(_ * 3).toMap), new ExecutorMetrics, null)) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 5)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 5).toMap) // Summit a new stage listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(1, 0))) @@ -281,7 +287,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils (1L, 1, 0, createAccumulatorInfos(accumulatorUpdates)) ))) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 7)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 7).toMap) // Finish two tasks listener.onTaskEnd(SparkListenerTaskEnd( @@ -289,7 +296,7 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils stageAttemptId = 0, taskType = "", reason = null, - createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 3)), + createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 3).toMap), new ExecutorMetrics, null)) listener.onTaskEnd(SparkListenerTaskEnd( @@ -297,11 +304,12 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils stageAttemptId = 0, taskType = "", reason = null, - createTaskInfo(1, 0, accums = accumulatorUpdates.mapValues(_ * 3)), + createTaskInfo(1, 0, accums = accumulatorUpdates.mapValues(_ * 3).toMap), new ExecutorMetrics, null)) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 11)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 11).toMap) assertJobs(statusStore.execution(executionId), running = Seq(0)) @@ -315,7 +323,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils assertJobs(statusStore.execution(executionId), completed = Seq(0)) - checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 11)) + checkAnswer(statusStore.executionMetrics(executionId), + accumulatorUpdates.mapValues(_ * 11).toMap) } test("control a plan explain mode in listeners via SQLConf") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 8d5439534b513..5e401f5136019 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -873,7 +873,7 @@ trait StreamTest extends QueryTest with SharedSparkSession with TimeLimits with } if(!running) { actions += StartStream() } addCheck() - testStream(ds)(actions: _*) + testStream(ds)(actions.toSeq: _*) } object AwaitTerminationTester { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 6e08b88f538df..26158f4d639ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -323,7 +323,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { actions += AssertOnQuery { q => q.recentProgress.size > 1 && q.recentProgress.size <= 11 } - testStream(input.toDS)(actions: _*) + testStream(input.toDS)(actions.toSeq: _*) spark.sparkContext.listenerBus.waitUntilEmpty() // 11 is the max value of the possible numbers of events. assert(numProgressEvent > 1 && numProgressEvent <= 11) @@ -559,11 +559,11 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { private val _progressEvents = new mutable.Queue[StreamingQueryProgress] def progressEvents: Seq[StreamingQueryProgress] = _progressEvents.synchronized { - _progressEvents.filter(_.numInputRows > 0) + _progressEvents.filter(_.numInputRows > 0).toSeq } def allProgressEvents: Seq[StreamingQueryProgress] = _progressEvents.synchronized { - _progressEvents.clone() + _progressEvents.clone().toSeq } def reset(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index 98e2342c78e56..ec61102804ea3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -316,7 +316,7 @@ object StreamingQueryStatusAndProgressSuite { timestamp = "2016-12-05T20:54:20.827Z", batchId = 2L, batchDuration = 0L, - durationMs = new java.util.HashMap(Map("total" -> 0L).mapValues(long2Long).asJava), + durationMs = new java.util.HashMap(Map("total" -> 0L).mapValues(long2Long).toMap.asJava), eventTime = new java.util.HashMap(Map( "max" -> "2016-12-05T20:54:20.827Z", "min" -> "2016-12-05T20:54:20.827Z", @@ -326,7 +326,7 @@ object StreamingQueryStatusAndProgressSuite { numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 3, numRowsDroppedByWatermark = 0, customMetrics = new java.util.HashMap(Map("stateOnCurrentVersionSizeBytes" -> 2L, "loadedMapCacheHitCount" -> 1L, "loadedMapCacheMissCount" -> 0L) - .mapValues(long2Long).asJava) + .mapValues(long2Long).toMap.asJava) )), sources = Array( new SourceProgress( @@ -351,7 +351,7 @@ object StreamingQueryStatusAndProgressSuite { timestamp = "2016-12-05T20:54:20.827Z", batchId = 2L, batchDuration = 0L, - durationMs = new java.util.HashMap(Map("total" -> 0L).mapValues(long2Long).asJava), + durationMs = new java.util.HashMap(Map("total" -> 0L).mapValues(long2Long).toMap.asJava), // empty maps should be handled correctly eventTime = new java.util.HashMap(Map.empty[String, String].asJava), stateOperators = Array(new StateOperatorProgress( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index 683db21d3f0e1..37cc1b8a6d2ab 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -123,7 +123,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { val jobOption = outputStream.generateJob(time) jobOption.foreach(_.setCallSite(outputStream.creationSite)) jobOption - } + }.toSeq } logDebug("Generated " + jobs.length + " jobs for time " + time) jobs diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 7850285be83af..af3f5a060f54b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -366,7 +366,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] - sQueue.enqueue(queue.asScala.map(_.rdd).toSeq: _*) + sQueue ++= queue.asScala.map(_.rdd) ssc.queueStream(sQueue) } @@ -390,7 +390,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] - sQueue.enqueue(queue.asScala.map(_.rdd).toSeq: _*) + sQueue ++= queue.asScala.map(_.rdd) ssc.queueStream(sQueue, oneAtATime) } @@ -415,7 +415,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] - sQueue.enqueue(queue.asScala.map(_.rdd).toSeq: _*) + sQueue ++= queue.asScala.map(_.rdd) ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapper.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapper.scala index ee8370d262609..7555e2f57fccb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapper.scala @@ -65,7 +65,7 @@ private[streaming] class JavaStreamingListenerWrapper(javaStreamingListener: Jav private def toJavaBatchInfo(batchInfo: BatchInfo): JavaBatchInfo = { JavaBatchInfo( batchInfo.batchTime, - batchInfo.streamIdToInputInfo.mapValues(toJavaStreamInputInfo(_)).asJava, + batchInfo.streamIdToInputInfo.mapValues(toJavaStreamInputInfo).toMap.asJava, batchInfo.submissionTime, batchInfo.processingStartTime.getOrElse(-1), batchInfo.processingEndTime.getOrElse(-1), @@ -73,7 +73,7 @@ private[streaming] class JavaStreamingListenerWrapper(javaStreamingListener: Jav batchInfo.processingDelay.getOrElse(-1), batchInfo.totalDelay.getOrElse(-1), batchInfo.numRecords, - batchInfo.outputOperationInfos.mapValues(toJavaOutputOperationInfo(_)).asJava + batchInfo.outputOperationInfos.mapValues(toJavaOutputOperationInfo).toMap.asJava ) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala index d46c0a01e05d9..2f4536ec6f0c2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala @@ -45,7 +45,7 @@ class UnionDStream[T: ClassTag](parents: Array[DStream[T]]) s" time $validTime") } if (rdds.nonEmpty) { - Some(ssc.sc.union(rdds)) + Some(ssc.sc.union(rdds.toSeq)) } else { None } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala index 8da5a5f8193cf..662312b7b0db8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala @@ -75,7 +75,7 @@ private[streaming] object MapWithStateRDDRecord { } } - MapWithStateRDDRecord(newStateMap, mappedData) + MapWithStateRDDRecord(newStateMap, mappedData.toSeq) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 6c71b18b46213..d038021e93e73 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -116,7 +116,9 @@ private[streaming] class ReceivedBlockTracker( // a few thousand elements. So we explicitly allocate a collection for serialization which // we know doesn't have this issue. (See SPARK-26734). val streamIdToBlocks = streamIds.map { streamId => - (streamId, mutable.ArrayBuffer(getReceivedBlockQueue(streamId).clone(): _*)) + val blocks = mutable.ArrayBuffer[ReceivedBlockInfo]() + blocks ++= getReceivedBlockQueue(streamId).clone() + (streamId, blocks.toSeq) }.toMap val allocatedBlocks = AllocatedBlocks(streamIdToBlocks) if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index 4105171a3db24..0569abab1f36d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -135,7 +135,7 @@ private[streaming] class ReceiverSchedulingPolicy { leastScheduledExecutors += executor } - receivers.map(_.streamId).zip(scheduledLocations).toMap + receivers.map(_.streamId).zip(scheduledLocations.map(_.toSeq)).toMap } /** @@ -183,7 +183,7 @@ private[streaming] class ReceiverSchedulingPolicy { val executorWeights: Map[ExecutorCacheTaskLocation, Double] = { receiverTrackingInfoMap.values.flatMap(convertReceiverTrackingInfoToExecutorWeights) - .groupBy(_._1).mapValues(_.map(_._2).sum) // Sum weights for each executor + .groupBy(_._1).mapValues(_.map(_._2).sum).toMap // Sum weights for each executor } val idleExecutors = executors.toSet -- executorWeights.keys diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 51c4b0fd4a2d8..342a0a43b5068 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -248,7 +248,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false _.runningExecutor.map { _.executorId } - } + }.toMap } else { Map.empty } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 31e4c6b59a64a..d0a3517af70b9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -170,7 +170,7 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp // We take the latest record for the timestamp. Please refer to the class Javadoc for // detailed explanation val time = sortedByTime.last.time - segment = wrappedLog.write(aggregate(sortedByTime), time) + segment = wrappedLog.write(aggregate(sortedByTime.toSeq), time) } buffer.foreach(_.promise.success(segment)) } catch { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index d33f83c819086..6f3ee5cbaec63 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -146,7 +146,7 @@ private[streaming] class FileBasedWriteAheadLog( } else { // For performance gains, it makes sense to parallelize the recovery if // closeFileAfterWrite = true - seqToParIterator(executionContext, logFilesToRead, readFile).asJava + seqToParIterator(executionContext, logFilesToRead.toSeq, readFile).asJava } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala index 36036fcd44b04..541a6e2d48b51 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala @@ -190,7 +190,7 @@ class DStreamScopeSuite assertDefined(foreachBaseScope) assert(foreachBaseScope.get.name === "foreachRDD") - val rddScopes = generatedRDDs.map { _.scope } + val rddScopes = generatedRDDs.map { _.scope }.toSeq assertDefined(rddScopes: _*) rddScopes.zipWithIndex.foreach { case (rddScope, idx) => assert(rddScope.get.name === "reduceByKey") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/scala/org/apache/spark/streaming/JavaTestUtils.scala index 0c4a64ccc513f..42a5aaba5178f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/JavaTestUtils.scala @@ -36,7 +36,7 @@ trait JavaTestBase extends TestSuiteBase { ssc: JavaStreamingContext, data: JList[JList[T]], numPartitions: Int): JavaDStream[T] = { - val seqData = data.asScala.map(_.asScala) + val seqData = data.asScala.map(_.asScala.toSeq).toSeq implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index d0a5ababc7cac..9d735a32f7090 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -284,7 +284,7 @@ object MasterFailureTest extends Logging { }) } } - mergedOutput + mergedOutput.toSeq } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index bb60d6fa7bf78..60e04403937a2 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -612,7 +612,7 @@ object WriteAheadLogSuite { } } writer.close() - segments + segments.toSeq } /** @@ -685,7 +685,7 @@ object WriteAheadLogSuite { } finally { reader.close() } - buffer + buffer.toSeq } /** Read all the data from a log file using reader class and return the list of byte buffers. */ From a47b69a88a271e423271709ee491e2de57c5581b Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 14 Jul 2020 12:19:01 -0700 Subject: [PATCH 188/384] [SPARK-32307][SQL] ScalaUDF's canonicalized expression should exclude inputEncoders ### What changes were proposed in this pull request? Override `canonicalized` to empty the `inputEncoders` for the canonicalized `ScalaUDF`. ### Why are the changes needed? The following fails on `branch-3.0` currently, not on Apache Spark 3.0.0 release. ```scala spark.udf.register("key", udf((m: Map[String, String]) => m.keys.head.toInt)) Seq(Map("1" -> "one", "2" -> "two")).toDF("a").createOrReplaceTempView("t") checkAnswer(sql("SELECT key(a) AS k FROM t GROUP BY key(a)"), Row(1) :: Nil) [info] org.apache.spark.sql.AnalysisException: expression 't.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;; [info] Aggregate [UDF(a#6)], [UDF(a#6) AS k#8] [info] +- SubqueryAlias t [info] +- Project [value#3 AS a#6] [info] +- LocalRelation [value#3] [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:49) [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:48) [info] at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:130) [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:257) [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10(CheckAnalysis.scala:259) [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10$adapted(CheckAnalysis.scala:259) [info] at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) [info] at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) [info] at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:259) [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10(CheckAnalysis.scala:259) [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10$adapted(CheckAnalysis.scala:259) [info] at scala.collection.immutable.List.foreach(List.scala:392) [info] at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:259) ... ``` We use the rule`ResolveEncodersInUDF` to resolve `inputEncoders` and the original`ScalaUDF` instance will be updated to a new `ScalaUDF` instance with the resolved encoders at the end. Note, during encoder resolving, types like `map`, `array` will be resolved to new expression(e.g. `MapObjects`, `CatalystToExternalMap`). However, `ExpressionEncoder` can't be canonicalized. Thus, the canonicalized `ScalaUDF`s become different even if their original `ScalaUDF`s are the same. Finally, it fails the `checkValidAggregateExpression` when this `ScalaUDF` is used as a group expression. ### Does this PR introduce _any_ user-facing change? Yes, users will not hit the exception after this fix. ### How was this patch tested? Added tests. Closes #29106 from Ngone51/spark-32307. Authored-by: yi.wu Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/ScalaUDF.scala | 6 ++++++ .../test/scala/org/apache/spark/sql/UDFSuite.scala | 12 ++++++++++++ 2 files changed, 18 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 44ee06ae011af..6e2bd96784b94 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -59,6 +59,12 @@ case class ScalaUDF( override def toString: String = s"${udfName.getOrElse("UDF")}(${children.mkString(", ")})" + override lazy val canonicalized: Expression = { + // SPARK-32307: `ExpressionEncoder` can't be canonicalized, and technically we don't + // need it to identify a `ScalaUDF`. + Canonicalize.execute(copy(children = children.map(_.canonicalized), inputEncoders = Nil)) + } + /** * The analyzer should be aware of Scala primitive types so as to make the * UDF return null if there is any null input value of these types. On the diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 05a33f9aa17bb..f0d5a61ad8006 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -775,4 +775,16 @@ class UDFSuite extends QueryTest with SharedSparkSession { } assert(e2.getMessage.contains("UDFSuite$MalformedClassObject$MalformedPrimitiveFunction")) } + + test("SPARK-32307: Aggression that use map type input UDF as group expression") { + spark.udf.register("key", udf((m: Map[String, String]) => m.keys.head.toInt)) + Seq(Map("1" -> "one", "2" -> "two")).toDF("a").createOrReplaceTempView("t") + checkAnswer(sql("SELECT key(a) AS k FROM t GROUP BY key(a)"), Row(1) :: Nil) + } + + test("SPARK-32307: Aggression that use array type input UDF as group expression") { + spark.udf.register("key", udf((m: Array[Int]) => m.head)) + Seq(Array(1)).toDF("a").createOrReplaceTempView("t") + checkAnswer(sql("SELECT key(a) AS k FROM t GROUP BY key(a)"), Row(1) :: Nil) + } } From 2a0faca830a418ffed8da0c1962defc081a26aa2 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 14 Jul 2020 12:29:56 -0700 Subject: [PATCH 189/384] [SPARK-32309][PYSPARK] Import missing sys import # What changes were proposed in this pull request? While seeing if we can use mypy for checking the Python types, I've stumbled across this missing import: https://github.com/apache/spark/blob/34fa913311bc1730015f1af111ff4a03c2bad9f6/python/pyspark/ml/feature.py#L5773-L5774 ### Why are the changes needed? The `import` is required because it's used. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual. Closes #29108 from Fokko/SPARK-32309. Authored-by: Fokko Driesprong Signed-off-by: Dongjoon Hyun --- python/pyspark/ml/feature.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index c52ea62686ab9..a319dace6869a 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -5736,6 +5736,7 @@ def selectedFeatures(self): if __name__ == "__main__": import doctest + import sys import tempfile import pyspark.ml.feature From 5e0cb3ee16dde3666af3bf5f2b152c7d0dfe9d7b Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 14 Jul 2020 12:40:47 -0700 Subject: [PATCH 190/384] [SPARK-32305][BUILD] Make `mvn clean` remove `metastore_db` and `spark-warehouse` ### What changes were proposed in this pull request? Add additional configuration to `maven-clean-plugin` to ensure cleanup `metastore_db` and `spark-warehouse` directory when execute `mvn clean` command. ### Why are the changes needed? Now Spark support two version of build-in hive and there are some test generated meta data not in target dir like `metastore_db`, they don't clean up automatically when we run `mvn clean` command. So if we run `mvn clean test -pl sql/hive -am -Phadoop-2.7 -Phive -Phive-1.2 ` , the `metastore_db` dir will created and meta data will remains after test complete. Then we need manual cleanup `metastore_db` directory to ensure `mvn clean test -pl sql/hive -am -Phadoop-2.7 -Phive` command use hive2.3 profile can succeed because the residual metastore data is not compatible. `spark-warehouse` will also cause test failure in some data residual scenarios because test case thinks that meta data should not exist. This pr is used to simplify manual cleanup `metastore_db` and `spark-warehouse` directory operation. ### How was this patch tested? Manual execute `mvn clean test -pl sql/hive -am -Phadoop-2.7 -Phive -Phive-1.2`, then execute `mvn clean test -pl sql/hive -am -Phadoop-2.7 -Phive`, both commands should succeed. Closes #29103 from LuciferYang/add-clean-directory. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 279d76d42ce12..cfcb55b27fa99 100644 --- a/pom.xml +++ b/pom.xml @@ -2594,6 +2594,12 @@ lib_managed + + metastore_db + + + spark-warehouse + From c602d79f89a133d6cbc9cc8d95cb09510cbd9c30 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 14 Jul 2020 12:46:11 -0700 Subject: [PATCH 191/384] [SPARK-32311][PYSPARK][TESTS] Remove duplicate import ### What changes were proposed in this pull request? `datetime` is already imported a few lines below :) https://github.com/apache/spark/blob/ce27cc54c1b2e533cd91e31f2414f3e0a172c328/python/pyspark/sql/tests/test_pandas_udf_scalar.py#L24 ### Why are the changes needed? This is the last instance of the duplicate import. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual. Closes #29109 from Fokko/SPARK-32311. Authored-by: Fokko Driesprong Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_pandas_udf_scalar.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index 75e2a0929ed29..448e409b0c377 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import datetime import os import random import shutil From 90b0c26b222dcb8f207f152494604aac090eb940 Mon Sep 17 00:00:00 2001 From: Baohe Zhang Date: Wed, 15 Jul 2020 07:51:13 +0900 Subject: [PATCH 192/384] [SPARK-31608][CORE][WEBUI] Add a new type of KVStore to make loading UI faster ### What changes were proposed in this pull request? Add a new class HybridStore to make the history server faster when loading event files. When rebuilding the application state from event logs, HybridStore will write data to InMemoryStore at first and use a background thread to dump data to LevelDB once the writing to InMemoryStore is completed. HybridStore is to make content serving faster by using more memory. It's only safe to enable it when the cluster is not having a heavy load. ### Why are the changes needed? HybridStore can greatly reduce the event logs loading time, especially for large log files. In general, it has 4x - 6x UI loading speed improvement for large log files. The detailed result is shown in comments. ### Does this PR introduce any user-facing change? This PR adds new configs `spark.history.store.hybridStore.enabled` and `spark.history.store.hybridStore.maxMemoryUsage`. ### How was this patch tested? A test suite for HybridStore is added. I also manually tested it on 3.1.0 on mac os. This is a follow-up for the work done by Hieu Huynh in 2019. Closes #28412 from baohe-zhang/SPARK-31608. Authored-by: Baohe Zhang Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../deploy/history/FsHistoryProvider.scala | 99 ++++++++++ .../history/HistoryServerMemoryManager.scala | 85 ++++++++ .../spark/deploy/history/HybridStore.scala | 185 ++++++++++++++++++ .../spark/internal/config/History.scala | 16 ++ docs/monitoring.md | 19 ++ 5 files changed, 404 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServerMemoryManager.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 25ea75acc37d3..891c5bffa52ed 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -128,6 +128,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val storePath = conf.get(LOCAL_STORE_DIR).map(new File(_)) private val fastInProgressParsing = conf.get(FAST_IN_PROGRESS_PARSING) + private val hybridStoreEnabled = conf.get(History.HYBRID_STORE_ENABLED) + // Visible for testing. private[history] val listing: KVStore = storePath.map { path => val dbPath = Files.createDirectories(new File(path, "listing.ldb").toPath()).toFile() @@ -158,6 +160,11 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) new HistoryServerDiskManager(conf, path, listing, clock) } + private var memoryManager: HistoryServerMemoryManager = null + if (hybridStoreEnabled) { + memoryManager = new HistoryServerMemoryManager(conf) + } + private val fileCompactor = new EventLogFileCompactor(conf, hadoopConf, fs, conf.get(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN), conf.get(EVENT_LOG_COMPACTION_SCORE_THRESHOLD)) @@ -262,6 +269,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private def startPolling(): Unit = { diskManager.foreach(_.initialize()) + if (memoryManager != null) { + memoryManager.initialize() + } // Validate the log directory. val path = new Path(logDir) @@ -1167,6 +1177,95 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // At this point the disk data either does not exist or was deleted because it failed to // load, so the event log needs to be replayed. + // If the hybrid store is enabled, try it first and fail back to leveldb store. + if (hybridStoreEnabled) { + try { + return createHybridStore(dm, appId, attempt, metadata) + } catch { + case e: Exception => + logInfo(s"Failed to create HybridStore for $appId/${attempt.info.attemptId}." + + " Using LevelDB.", e) + } + } + + createLevelDBStore(dm, appId, attempt, metadata) + } + + private def createHybridStore( + dm: HistoryServerDiskManager, + appId: String, + attempt: AttemptInfoWrapper, + metadata: AppStatusStoreMetadata): KVStore = { + var retried = false + var hybridStore: HybridStore = null + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastIndex) + + // Use InMemoryStore to rebuild app store + while (hybridStore == null) { + // A RuntimeException will be thrown if the heap memory is not sufficient + memoryManager.lease(appId, attempt.info.attemptId, reader.totalSize, + reader.compressionCodec) + var store: HybridStore = null + try { + store = new HybridStore() + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) + hybridStore = store + } catch { + case _: IOException if !retried => + // compaction may touch the file(s) which app rebuild wants to read + // compaction wouldn't run in short interval, so try again... + logWarning(s"Exception occurred while rebuilding log path ${attempt.logPath} - " + + "trying again...") + store.close() + memoryManager.release(appId, attempt.info.attemptId) + retried = true + case e: Exception => + store.close() + memoryManager.release(appId, attempt.info.attemptId) + throw e + } + } + + // Create a LevelDB and start a background thread to dump data to LevelDB + var lease: dm.Lease = null + try { + logInfo(s"Leasing disk manager space for app $appId / ${attempt.info.attemptId}...") + lease = dm.lease(reader.totalSize, reader.compressionCodec.isDefined) + val levelDB = KVUtils.open(lease.tmpPath, metadata) + hybridStore.setLevelDB(levelDB) + hybridStore.switchToLevelDB(new HybridStore.SwitchToLevelDBListener { + override def onSwitchToLevelDBSuccess: Unit = { + logInfo(s"Completely switched to LevelDB for app $appId / ${attempt.info.attemptId}.") + levelDB.close() + val newStorePath = lease.commit(appId, attempt.info.attemptId) + hybridStore.setLevelDB(KVUtils.open(newStorePath, metadata)) + memoryManager.release(appId, attempt.info.attemptId) + } + override def onSwitchToLevelDBFail(e: Exception): Unit = { + logWarning(s"Failed to switch to LevelDB for app $appId / ${attempt.info.attemptId}", e) + levelDB.close() + lease.rollback() + } + }, appId, attempt.info.attemptId) + } catch { + case e: Exception => + hybridStore.close() + memoryManager.release(appId, attempt.info.attemptId) + if (lease != null) { + lease.rollback() + } + throw e + } + + hybridStore + } + + private def createLevelDBStore( + dm: HistoryServerDiskManager, + appId: String, + attempt: AttemptInfoWrapper, + metadata: AppStatusStoreMetadata): KVStore = { var retried = false var newStorePath: File = null while (newStorePath == null) { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerMemoryManager.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerMemoryManager.scala new file mode 100644 index 0000000000000..7fc0722233854 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerMemoryManager.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable.HashMap + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.History._ +import org.apache.spark.util.Utils + +/** + * A class used to keep track of in-memory store usage by the SHS. + */ +private class HistoryServerMemoryManager( + conf: SparkConf) extends Logging { + + private val maxUsage = conf.get(MAX_IN_MEMORY_STORE_USAGE) + private val currentUsage = new AtomicLong(0L) + private val active = new HashMap[(String, Option[String]), Long]() + + def initialize(): Unit = { + logInfo("Initialized memory manager: " + + s"current usage = ${Utils.bytesToString(currentUsage.get())}, " + + s"max usage = ${Utils.bytesToString(maxUsage)}") + } + + def lease( + appId: String, + attemptId: Option[String], + eventLogSize: Long, + codec: Option[String]): Unit = { + val memoryUsage = approximateMemoryUsage(eventLogSize, codec) + if (memoryUsage + currentUsage.get > maxUsage) { + throw new RuntimeException("Not enough memory to create hybrid store " + + s"for app $appId / $attemptId.") + } + active.synchronized { + active(appId -> attemptId) = memoryUsage + } + currentUsage.addAndGet(memoryUsage) + logInfo(s"Leasing ${Utils.bytesToString(memoryUsage)} memory usage for " + + s"app $appId / $attemptId") + } + + def release(appId: String, attemptId: Option[String]): Unit = { + val memoryUsage = active.synchronized { active.remove(appId -> attemptId) } + + memoryUsage match { + case Some(m) => + currentUsage.addAndGet(-m) + logInfo(s"Released ${Utils.bytesToString(m)} memory usage for " + + s"app $appId / $attemptId") + case None => + } + } + + private def approximateMemoryUsage(eventLogSize: Long, codec: Option[String]): Long = { + codec match { + case Some("zstd") => + eventLogSize * 10 + case Some(_) => + eventLogSize * 4 + case None => + eventLogSize / 2 + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala new file mode 100644 index 0000000000000..96db86f8e745a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.io.IOException +import java.util.Collection +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ + +import org.apache.spark.util.kvstore._ + +/** + * An implementation of KVStore that accelerates event logs loading. + * + * When rebuilding the application state from event logs, HybridStore will + * write data to InMemoryStore at first and use a background thread to dump + * data to LevelDB once the app store is restored. We don't expect write + * operations (except the case for caching) after calling switch to level DB. + */ + +private[history] class HybridStore extends KVStore { + + private val inMemoryStore = new InMemoryStore() + + private var levelDB: LevelDB = null + + // Flag to indicate whether we should use inMemoryStore or levelDB + private val shouldUseInMemoryStore = new AtomicBoolean(true) + + // Flag to indicate whether this hybrid store is closed, use this flag + // to avoid starting background thread after the store is closed + private val closed = new AtomicBoolean(false) + + // A background thread that dumps data from inMemoryStore to levelDB + private var backgroundThread: Thread = null + + // A hash map that stores all classes that had been writen to inMemoryStore + private val klassMap = new ConcurrentHashMap[Class[_], Boolean] + + override def getMetadata[T](klass: Class[T]): T = { + getStore().getMetadata(klass) + } + + override def setMetadata(value: Object): Unit = { + getStore().setMetadata(value) + } + + override def read[T](klass: Class[T], naturalKey: Object): T = { + getStore().read(klass, naturalKey) + } + + override def write(value: Object): Unit = { + getStore().write(value) + + if (backgroundThread == null) { + // New classes won't be dumped once the background thread is started + klassMap.putIfAbsent(value.getClass(), true) + } + } + + override def delete(klass: Class[_], naturalKey: Object): Unit = { + if (backgroundThread != null) { + throw new IllegalStateException("delete() shouldn't be called after " + + "the hybrid store begins switching to levelDB") + } + + getStore().delete(klass, naturalKey) + } + + override def view[T](klass: Class[T]): KVStoreView[T] = { + getStore().view(klass) + } + + override def count(klass: Class[_]): Long = { + getStore().count(klass) + } + + override def count(klass: Class[_], index: String, indexedValue: Object): Long = { + getStore().count(klass, index, indexedValue) + } + + override def close(): Unit = { + try { + closed.set(true) + if (backgroundThread != null && backgroundThread.isAlive()) { + // The background thread is still running, wait for it to finish + backgroundThread.join() + } + } finally { + inMemoryStore.close() + if (levelDB != null) { + levelDB.close() + } + } + } + + override def removeAllByIndexValues[T]( + klass: Class[T], + index: String, + indexValues: Collection[_]): Boolean = { + if (backgroundThread != null) { + throw new IllegalStateException("removeAllByIndexValues() shouldn't be " + + "called after the hybrid store begins switching to levelDB") + } + + getStore().removeAllByIndexValues(klass, index, indexValues) + } + + def setLevelDB(levelDB: LevelDB): Unit = { + this.levelDB = levelDB + } + + /** + * This method is called when the writing is done for inMemoryStore. A + * background thread will be created and be started to dump data in inMemoryStore + * to levelDB. Once the dumping is completed, the underlying kvstore will be + * switched to levelDB. + */ + def switchToLevelDB( + listener: HybridStore.SwitchToLevelDBListener, + appId: String, + attemptId: Option[String]): Unit = { + if (closed.get) { + return + } + + backgroundThread = new Thread(() => { + try { + for (klass <- klassMap.keys().asScala) { + val it = inMemoryStore.view(klass).closeableIterator() + while (it.hasNext()) { + levelDB.write(it.next()) + } + } + listener.onSwitchToLevelDBSuccess() + shouldUseInMemoryStore.set(false) + inMemoryStore.close() + } catch { + case e: Exception => + listener.onSwitchToLevelDBFail(e) + } + }) + backgroundThread.setDaemon(true) + backgroundThread.setName(s"hybridstore-$appId-$attemptId") + backgroundThread.start() + } + + /** + * This method return the store that we should use. + */ + private def getStore(): KVStore = { + if (shouldUseInMemoryStore.get) { + inMemoryStore + } else { + levelDB + } + } +} + +private[history] object HybridStore { + + trait SwitchToLevelDBListener { + + def onSwitchToLevelDBSuccess(): Unit + + def onSwitchToLevelDBFail(e: Exception): Unit + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index 581777de366ef..a6d1c044130f5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -195,4 +195,20 @@ private[spark] object History { .version("3.0.0") .booleanConf .createWithDefault(true) + + val HYBRID_STORE_ENABLED = ConfigBuilder("spark.history.store.hybridStore.enabled") + .doc("Whether to use HybridStore as the store when parsing event logs. " + + "HybridStore will first write data to an in-memory store and having a background thread " + + "that dumps data to a disk store after the writing to in-memory store is completed.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + + val MAX_IN_MEMORY_STORE_USAGE = ConfigBuilder("spark.history.store.hybridStore.maxMemoryUsage") + .doc("Maximum memory space that can be used to create HybridStore. The HybridStore co-uses " + + "the heap memory, so the heap memory should be increased through the memory option for SHS " + + "if the HybridStore is enabled.") + .version("3.1.0") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("2g") } diff --git a/docs/monitoring.md b/docs/monitoring.md index 32959b77c4773..81c386aa90dd8 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -373,6 +373,25 @@ Security options for the Spark History Server are covered more detail in the + + + + + + + + + + + +
Writable TypePython Type
Textunicode str
Textstr
IntWritableint
FloatWritablefloat
DoubleWritablefloat
3.0.0
spark.history.store.hybridStore.enabledfalse + Whether to use HybridStore as the store when parsing event logs. HybridStore will first write data + to an in-memory store and having a background thread that dumps data to a disk store after the writing + to in-memory store is completed. + 3.1.0
spark.history.store.hybridStore.maxMemoryUsage2g + Maximum memory space that can be used to create HybridStore. The HybridStore co-uses the heap memory, + so the heap memory should be increased through the memory option for SHS if the HybridStore is enabled. + 3.1.0
Note that in all of these UIs, the tables are sortable by clicking their headers, From 902e1342a324c9e1e01dc68817850d9241a58227 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 14 Jul 2020 16:43:16 -0700 Subject: [PATCH 193/384] [SPARK-32303][PYTHON][TESTS] Remove leftover from editable mode installation in PIP test ### What changes were proposed in this pull request? Currently the Jenkins PIP packaging test fails as below intermediately: ``` Installing dist into virtual env Processing ./python/dist/pyspark-3.1.0.dev0.tar.gz Collecting py4j==0.10.9 (from pyspark==3.1.0.dev0) Downloading https://files.pythonhosted.org/packages/9e/b6/6a4fb90cd235dc8e265a6a2067f2a2c99f0d91787f06aca4bcf7c23f3f80/py4j-0.10.9-py2.py3-none-any.whl (198kB) Installing collected packages: py4j, pyspark Found existing installation: py4j 0.10.9 Uninstalling py4j-0.10.9: Successfully uninstalled py4j-0.10.9 Found existing installation: pyspark 3.1.0.dev0 Exception: Traceback (most recent call last): File "/home/anaconda/envs/py36/lib/python3.6/site-packages/pip/_internal/cli/base_command.py", line 179, in main status = self.run(options, args) File "/home/anaconda/envs/py36/lib/python3.6/site-packages/pip/_internal/commands/install.py", line 393, in run use_user_site=options.use_user_site, File "/home/anaconda/envs/py36/lib/python3.6/site-packages/pip/_internal/req/__init__.py", line 50, in install_given_reqs auto_confirm=True File "/home/anaconda/envs/py36/lib/python3.6/site-packages/pip/_internal/req/req_install.py", line 816, in uninstall uninstalled_pathset = UninstallPathSet.from_dist(dist) File "/home/anaconda/envs/py36/lib/python3.6/site-packages/pip/_internal/req/req_uninstall.py", line 505, in from_dist '(at %s)' % (link_pointer, dist.project_name, dist.location) AssertionError: Egg-link /home/jenkins/workspace/SparkPullRequestBuilder3/python does not match installed ``` - https://github.com/apache/spark/pull/29099#issuecomment-658073453 (amp-jenkins-worker-04) - https://github.com/apache/spark/pull/29090#issuecomment-657819973 (amp-jenkins-worker-03) Seems like the previous installation of editable mode affects other PRs. This PR simply works around by removing the symbolic link from the previous editable installation. This is a common workaround up to my knowledge. ### Why are the changes needed? To recover the Jenkins build. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Jenkins build will test it out. Closes #29102 from HyukjinKwon/SPARK-32303. Lead-authored-by: HyukjinKwon Co-authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- dev/run-pip-tests | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 9ebe69dbd476c..5fd0be7476f29 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -96,6 +96,8 @@ for python in "${PYTHON_EXECS[@]}"; do cd "$FWDIR"/python # Delete the egg info file if it exists, this can cache the setup file. rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" + # Also, delete the symbolic link if exists. It can be left over from the previous editable mode installation. + python3 -c "from distutils.sysconfig import get_python_lib; import os; f = os.path.join(get_python_lib(), 'pyspark.egg-link'); os.unlink(f) if os.path.isfile(f) else 0" python3 setup.py sdist From 676d92ecceb3d46baa524c725b9f9a14450f1e9d Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 15 Jul 2020 08:44:48 +0900 Subject: [PATCH 194/384] [SPARK-32301][PYTHON][TESTS] Add a test case for toPandas to work with empty partitioned Spark DataFrame ### What changes were proposed in this pull request? This PR proposes to port the test case from https://github.com/apache/spark/pull/29098 to branch-3.0 and master. In the master and branch-3.0, this was fixed together at https://github.com/apache/spark/commit/ecaa495b1fe532c36e952ccac42f4715809476af but no partition case is not being tested. ### Why are the changes needed? To improve test coverage. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Unit test was forward-ported. Closes #29099 from HyukjinKwon/SPARK-32300-1. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/tests/test_arrow.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index 90fc983aec021..148df9b7d45b8 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -447,6 +447,13 @@ def test_createDataFrame_with_float_index(self): self.spark.createDataFrame( pd.DataFrame({'a': [1, 2, 3]}, index=[2., 3., 4.])).distinct().count(), 3) + def test_no_partition_toPandas(self): + # SPARK-32301: toPandas should work from a Spark DataFrame with no partitions + # Forward-ported from SPARK-32300. + pdf = self.spark.sparkContext.emptyRDD().toDF("col1 int").toPandas() + self.assertEqual(len(pdf), 0) + self.assertEqual(list(pdf.columns), ["col1"]) + @unittest.skipIf( not have_pandas or not have_pyarrow, From 03b5707b516187aaa8012049fce8b1cd0ac0fddd Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 15 Jul 2020 08:46:20 +0900 Subject: [PATCH 195/384] [MINOR][R] Match collectAsArrowToR with non-streaming collectAsArrowToPython ### What changes were proposed in this pull request? This PR proposes to port forward #29098 to `collectAsArrowToR`. `collectAsArrowToR` follows `collectAsArrowToPython` in branch-2.4 due to the limitation of ARROW-4512. SparkR vectorization currently cannot use streaming format. ### Why are the changes needed? For simplicity and consistency. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The same code is being tested in `collectAsArrowToPython` of branch-2.4. Closes #29100 from HyukjinKwon/minor-parts. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 3472b9fdec9d8..d5501326397c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -3587,7 +3587,7 @@ class Dataset[T] private[sql]( val numPartitions = arrowBatchRdd.partitions.length // Store collection results for worst case of 1 to N-1 partitions - val results = new Array[Array[Array[Byte]]](numPartitions - 1) + val results = new Array[Array[Array[Byte]]](Math.max(0, numPartitions - 1)) var lastIndex = -1 // index of last partition written // Handler to eagerly write partitions to Python in order From 6bdd710c4d4125b0801a93d57f53e05e301ebebd Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 14 Jul 2020 20:44:09 -0700 Subject: [PATCH 196/384] [SPARK-32316][TESTS][INFRA] Test PySpark with Python 3.8 in Github Actions ### What changes were proposed in this pull request? This PR aims to test PySpark with Python 3.8 in Github Actions. In the script side, it is already ready: https://github.com/apache/spark/blob/4ad9bfd53b84a6d2497668c73af6899bae14c187/python/run-tests.py#L161 This PR includes small related fixes together: 1. Install Python 3.8 2. Only install one Python implementation instead of installing many for SQL and Yarn test cases because they need one Python executable in their test cases that is higher than Python 2. 3. Do not install Python 2 which is not needed anymore after we dropped Python 2 at SPARK-32138 4. Remove a comment about installing PyPy3 on Jenkins - SPARK-32278. It is already installed. ### Why are the changes needed? Currently, only PyPy3 and Python 3.6 are being tested with PySpark in Github Actions. We should test the latest version of Python as well because some optimizations can be only enabled with Python 3.8+. See also https://github.com/apache/spark/pull/29114 ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Was not tested. Github Actions build in this PR will test it out. Closes #29116 from HyukjinKwon/test-python3.8-togehter. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 32 ++++++++++++++++++-------------- python/run-tests.py | 1 - 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 5cf00c6ed9e67..fe01b92036377 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -117,38 +117,42 @@ jobs: java-version: ${{ matrix.java }} # PySpark - name: Install PyPy3 - # SQL component also has Python related tests, for example, IntegratedUDFTestUtils. # Note that order of Python installations here matters because default python3 is # overridden by pypy3. uses: actions/setup-python@v2 - if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) + if: contains(matrix.modules, 'pyspark') with: python-version: pypy3 architecture: x64 - - name: Install Python 2.7 + - name: Install Python 3.6 uses: actions/setup-python@v2 - if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) + if: contains(matrix.modules, 'pyspark') with: - python-version: 2.7 + python-version: 3.6 architecture: x64 - - name: Install Python 3.6 + - name: Install Python 3.8 uses: actions/setup-python@v2 - # Yarn has a Python specific test too, for example, YarnClusterSuite. + # We should install one Python that is higher then 3+ for SQL and Yarn because: + # - SQL component also has Python related tests, for example, IntegratedUDFTestUtils. + # - Yarn has a Python specific test too, for example, YarnClusterSuite. if: contains(matrix.modules, 'yarn') || contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) with: - python-version: 3.6 + python-version: 3.8 architecture: x64 - - name: Install Python packages - if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) + - name: Install Python packages (Python 3.6 and PyPy3) + if: contains(matrix.modules, 'pyspark') # PyArrow is not supported in PyPy yet, see ARROW-2651. # TODO(SPARK-32247): scipy installation with PyPy fails for an unknown reason. run: | - python3 -m pip install numpy pyarrow pandas scipy - python3 -m pip list - python2 -m pip install numpy pyarrow pandas scipy - python2 -m pip list + python3.6 -m pip install numpy pyarrow pandas scipy + python3.6 -m pip list pypy3 -m pip install numpy pandas pypy3 -m pip list + - name: Install Python packages (Python 3.8) + if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) + run: | + python3.8 -m pip install numpy pyarrow pandas scipy + python3.8 -m pip list # SparkR - name: Install R 3.6 uses: r-lib/actions/setup-r@v1 diff --git a/python/run-tests.py b/python/run-tests.py index 23076eab1c3e4..357eb8f449beb 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -157,7 +157,6 @@ def run_individual_python_test(target_dir, test_name, pyspark_python): def get_default_python_executables(): - # TODO(SPARK-32278): install PyPy3 in Jenkins to test python_execs = [x for x in ["python3.6", "python3.8", "pypy3"] if which(x)] if "python3.6" not in python_execs: From af8e65fca989518cf65ec47f77eea2ce649bd6bb Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 14 Jul 2020 21:17:33 -0700 Subject: [PATCH 197/384] [SPARK-32276][SQL] Remove redundant sorts before repartition nodes ### What changes were proposed in this pull request? This PR removes redundant sorts before repartition nodes with shuffles and repartitionByExpression with deterministic expressions. ### Why are the changes needed? It looks like our `EliminateSorts` rule can be extended further to remove sorts before repartition nodes that shuffle data as such repartition operations change the ordering and distribution of data. That's why it seems safe to perform the following rewrites: - `Repartition -> Sort -> Scan` as `Repartition -> Scan` - `Repartition -> Project -> Sort -> Scan` as `Repartition -> Project -> Scan` We don't apply this optimization to coalesce as it uses `DefaultPartitionCoalescer` that may preserve the ordering of data if there is no locality info in the parent RDD. At the same time, there is no guarantee that will happen. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? More test cases. Closes #29089 from aokolnychyi/spark-32276. Authored-by: Anton Okolnychyi Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/Optimizer.scala | 8 + ...EliminateSortsBeforeRepartitionSuite.scala | 179 ++++++++++++++++++ 2 files changed, 187 insertions(+) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index e481cdbd5fdf4..29f539904e645 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 @@ -974,6 +974,10 @@ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { * and the Join conditions is deterministic * 5) if the Sort operator is within GroupBy separated by 0...n Project/Filter operators only, * and the aggregate function is order irrelevant + * 6) if the Sort operator is within RepartitionByExpression separated by 0...n Project/Filter + * operators and the repartition expressions are deterministic + * 7) if the Sort operator is within Repartition separated by 0...n Project/Filter operators + * and the repartition requires a shuffle */ object EliminateSorts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -987,6 +991,10 @@ object EliminateSorts extends Rule[LogicalPlan] { j.copy(left = recursiveRemoveSort(originLeft), right = recursiveRemoveSort(originRight)) case g @ Aggregate(_, aggs, originChild) if isOrderIrrelevantAggs(aggs) => g.copy(child = recursiveRemoveSort(originChild)) + case r: RepartitionByExpression if r.partitionExpressions.forall(_.deterministic) => + r.copy(child = recursiveRemoveSort(r.child)) + case r: Repartition if r.shuffle => + r.copy(child = recursiveRemoveSort(r.child)) } private def recursiveRemoveSort(plan: LogicalPlan): LogicalPlan = plan match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala new file mode 100644 index 0000000000000..ff5521f98affb --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class EliminateSortsBeforeRepartitionSuite extends PlanTest { + + val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) + val analyzer = new Analyzer(catalog, conf) + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Default", FixedPoint(10), + FoldablePropagation, + LimitPushDown) :: + Batch("Eliminate Sorts", Once, + EliminateSorts) :: + Batch("Collapse Project", Once, + CollapseProject) :: Nil + } + + def repartition(plan: LogicalPlan): LogicalPlan = plan.repartition(10) + def isOptimized: Boolean = true + + test("sortBy") { + val plan = testRelation.select('a, 'b).sortBy('a.asc, 'b.desc) + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + val correctPlan = if (isOptimized) { + repartition(testRelation.select('a, 'b)) + } else { + planWithRepartition + } + comparePlans(optimizedPlan, analyzer.execute(correctPlan)) + } + + test("sortBy with projection") { + val plan = testRelation.select('a, 'b) + .sortBy('a.asc, 'b.asc) + .select('a + 1 as "a", 'b + 2 as "b") + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + val correctPlan = if (isOptimized) { + repartition(testRelation.select('a + 1 as "a", 'b + 2 as "b")) + } else { + planWithRepartition + } + comparePlans(optimizedPlan, analyzer.execute(correctPlan)) + } + + test("sortBy with projection and filter") { + val plan = testRelation.sortBy('a.asc, 'b.asc) + .select('a, 'b) + .where('a === 10) + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + val correctPlan = if (isOptimized) { + repartition(testRelation.select('a, 'b).where('a === 10)) + } else { + planWithRepartition + } + comparePlans(optimizedPlan, analyzer.execute(correctPlan)) + } + + test("sortBy with limit") { + val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) + } + + test("sortBy with non-deterministic projection") { + val plan = testRelation.sortBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) + } + + test("orderBy") { + val plan = testRelation.select('a, 'b).orderBy('a.asc, 'b.asc) + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + val correctPlan = if (isOptimized) { + repartition(testRelation.select('a, 'b)) + } else { + planWithRepartition + } + comparePlans(optimizedPlan, analyzer.execute(correctPlan)) + } + + test("orderBy with projection") { + val plan = testRelation.select('a, 'b) + .orderBy('a.asc, 'b.asc) + .select('a + 1 as "a", 'b + 2 as "b") + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + val correctPlan = if (isOptimized) { + repartition(testRelation.select('a + 1 as "a", 'b + 2 as "b")) + } else { + planWithRepartition + } + comparePlans(optimizedPlan, analyzer.execute(correctPlan)) + } + + test("orderBy with projection and filter") { + val plan = testRelation.orderBy('a.asc, 'b.asc) + .select('a, 'b) + .where('a === 10) + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + val correctPlan = if (isOptimized) { + repartition(testRelation.select('a, 'b).where('a === 10)) + } else { + planWithRepartition + } + comparePlans(optimizedPlan, analyzer.execute(correctPlan)) + } + + test("orderBy with limit") { + val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) + } + + test("orderBy with non-deterministic projection") { + val plan = testRelation.orderBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + val planWithRepartition = repartition(plan) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) + } +} + +class EliminateSortsBeforeRepartitionByExprsSuite extends EliminateSortsBeforeRepartitionSuite { + override def repartition(plan: LogicalPlan): LogicalPlan = plan.distribute('a, 'b)(10) + override def isOptimized: Boolean = true + + test("sortBy before repartition with non-deterministic expressions") { + val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) + } + + test("orderBy before repartition with non-deterministic expressions") { + val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) + val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) + comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) + } +} + +class EliminateSortsBeforeCoalesceSuite extends EliminateSortsBeforeRepartitionSuite { + override def repartition(plan: LogicalPlan): LogicalPlan = plan.coalesce(1) + override def isOptimized: Boolean = false +} From 542aefb4c4dd5ca2734773ffe983ba740729d074 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 15 Jul 2020 13:40:43 +0900 Subject: [PATCH 198/384] [SPARK-31985][SS] Remove incomplete/undocumented stateful aggregation in continuous mode ### What changes were proposed in this pull request? This removes the undocumented and incomplete feature of "stateful aggregation" in continuous mode, which would reduce 1100+ lines of code. ### Why are the changes needed? The work for the feature had been stopped for over an year, and no one asked/requested for the availability of such feature in community. Current state for the feature is that it only works with `coalesce(1)` which force the query to read and process, and write in "a" task, which doesn't make sense in production. The remaining code increases the work on DSv2 changes as well - that's why I don't simply propose reverting relevant commits - the code path has been changed due to DSv2 evolution. ### Does this PR introduce _any_ user-facing change? Technically no, because it's never documented and can't be used in production in current shape. ### How was this patch tested? Existing tests. Closes #29077 from HeartSaVioR/SPARK-31985. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../UnsupportedOperationChecker.scala | 11 - .../datasources/v2/DataSourceV2Strategy.scala | 14 +- .../continuous/ContinuousCoalesceExec.scala | 45 -- .../continuous/ContinuousCoalesceRDD.scala | 137 ------ .../continuous/ContinuousExecution.scala | 4 - .../shuffle/ContinuousShuffleReadRDD.scala | 80 ---- .../shuffle/ContinuousShuffleReader.scala | 32 -- .../shuffle/ContinuousShuffleWriter.scala | 27 -- .../shuffle/RPCContinuousShuffleReader.scala | 138 ------ .../shuffle/RPCContinuousShuffleWriter.scala | 60 --- .../streaming/state/StateStoreRDD.scala | 14 +- .../shuffle/ContinuousShuffleSuite.scala | 423 ------------------ .../ContinuousAggregationSuite.scala | 134 ------ 13 files changed, 2 insertions(+), 1117 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceExec.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReader.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleWriter.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/RPCContinuousShuffleReader.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/RPCContinuousShuffleWriter.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleSuite.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 423f89fefa093..0c11830cf06dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -393,17 +393,6 @@ object UnsupportedOperationChecker extends Logging { _: DeserializeToObject | _: SerializeFromObject | _: SubqueryAlias | _: TypedFilter) => case node if node.nodeName == "StreamingRelationV2" => - case Repartition(1, false, _) => - case node: Aggregate => - val aboveSinglePartitionCoalesce = node.find { - case Repartition(1, false, _) => true - case _ => false - }.isDefined - - if (!aboveSinglePartitionCoalesce) { - throwError(s"In continuous processing mode, coalesce(1) must be called before " + - s"aggregate operation ${node.nodeName}.") - } case node => throwError(s"Continuous processing does not support ${node.nodeName} operations.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index cca80c0cb6d57..f289a867e5ec0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.connector.catalog.{CatalogV2Util, StagingTableCatalo import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, ProjectExec, RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy -import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} +import org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -218,18 +218,6 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil - case Repartition(1, false, child) => - val isContinuous = child.find { - case r: StreamingDataSourceV2Relation => r.stream.isInstanceOf[ContinuousStream] - case _ => false - }.isDefined - - if (isContinuous) { - ContinuousCoalesceExec(1, planLater(child)) :: Nil - } else { - Nil - } - case desc @ DescribeNamespace(ResolvedNamespace(catalog, ns), extended) => DescribeNamespaceExec(desc.output, catalog.asNamespaceCatalog, ns, extended) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceExec.scala deleted file mode 100644 index 4c621890c9793..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceExec.scala +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} -import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} - -/** - * Physical plan for coalescing a continuous processing plan. - * - * Currently, only coalesces to a single partition are supported. `numPartitions` must be 1. - */ -case class ContinuousCoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecNode { - override def output: Seq[Attribute] = child.output - - override def outputPartitioning: Partitioning = SinglePartition - - override def doExecute(): RDD[InternalRow] = { - assert(numPartitions == 1) - new ContinuousCoalesceRDD( - sparkContext, - numPartitions, - conf.continuousStreamingExecutorQueueSize, - sparkContext.getLocalProperty(ContinuousExecution.EPOCH_INTERVAL_KEY).toLong, - child.execute()) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala deleted file mode 100644 index 14046f6a99c24..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous - -import java.util.UUID - -import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.execution.streaming.continuous.shuffle._ -import org.apache.spark.util.ThreadUtils - -case class ContinuousCoalesceRDDPartition( - index: Int, - endpointName: String, - queueSize: Int, - numShuffleWriters: Int, - epochIntervalMs: Long) - extends Partition { - // Initialized only on the executor, and only once even as we call compute() multiple times. - lazy val (reader: ContinuousShuffleReader, endpoint) = { - val env = SparkEnv.get.rpcEnv - val receiver = new RPCContinuousShuffleReader( - queueSize, numShuffleWriters, epochIntervalMs, env) - val endpoint = env.setupEndpoint(endpointName, receiver) - - TaskContext.get().addTaskCompletionListener[Unit] { ctx => - env.stop(endpoint) - } - (receiver, endpoint) - } - // This flag will be flipped on the executors to indicate that the threads processing - // partitions of the write-side RDD have been started. These will run indefinitely - // asynchronously as epochs of the coalesce RDD complete on the read side. - private[continuous] var writersInitialized: Boolean = false -} - -/** - * RDD for continuous coalescing. Asynchronously writes all partitions of `prev` into a local - * continuous shuffle, and then reads them in the task thread using `reader`. - */ -class ContinuousCoalesceRDD( - context: SparkContext, - numPartitions: Int, - readerQueueSize: Int, - epochIntervalMs: Long, - prev: RDD[InternalRow]) - extends RDD[InternalRow](context, Nil) { - - // When we support more than 1 target partition, we'll need to figure out how to pass in the - // required partitioner. - private val outputPartitioner = new HashPartitioner(1) - - private val readerEndpointNames = (0 until numPartitions).map { i => - s"ContinuousCoalesceRDD-part$i-${UUID.randomUUID()}" - } - - override def getPartitions: Array[Partition] = { - (0 until numPartitions).map { partIndex => - ContinuousCoalesceRDDPartition( - partIndex, - readerEndpointNames(partIndex), - readerQueueSize, - prev.getNumPartitions, - epochIntervalMs) - }.toArray - } - - private lazy val threadPool = ThreadUtils.newDaemonFixedThreadPool( - prev.getNumPartitions, - this.name) - - override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { - val part = split.asInstanceOf[ContinuousCoalesceRDDPartition] - - if (!part.writersInitialized) { - val rpcEnv = SparkEnv.get.rpcEnv - - // trigger lazy initialization - part.endpoint - val endpointRefs = readerEndpointNames.map { endpointName => - rpcEnv.setupEndpointRef(rpcEnv.address, endpointName) - } - - val runnables = prev.partitions.map { prevSplit => - new Runnable() { - override def run(): Unit = { - TaskContext.setTaskContext(context) - - val writer: ContinuousShuffleWriter = new RPCContinuousShuffleWriter( - prevSplit.index, outputPartitioner, endpointRefs.toArray) - - EpochTracker.initializeCurrentEpoch( - context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong) - while (!context.isInterrupted() && !context.isCompleted()) { - writer.write(prev.compute(prevSplit, context).asInstanceOf[Iterator[UnsafeRow]]) - // Note that current epoch is a inheritable thread local but makes another instance, - // so each writer thread can properly increment its own epoch without affecting - // the main task thread. - EpochTracker.incrementCurrentEpoch() - } - } - } - } - - context.addTaskCompletionListener[Unit] { ctx => - threadPool.shutdownNow() - } - - part.writersInitialized = true - - runnables.foreach(threadPool.execute) - } - - part.reader.read() - } - - override def clearDependencies(): Unit = { - throw new IllegalStateException("Continuous RDDs cannot be checkpointed") - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index a109c2171f3d2..d225e65aabe11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -206,9 +206,6 @@ class ContinuousExecution( currentEpochCoordinatorId = epochCoordinatorId sparkSessionForQuery.sparkContext.setLocalProperty( ContinuousExecution.EPOCH_COORDINATOR_ID_KEY, epochCoordinatorId) - sparkSessionForQuery.sparkContext.setLocalProperty( - ContinuousExecution.EPOCH_INTERVAL_KEY, - trigger.asInstanceOf[ContinuousTrigger].intervalMs.toString) // Use the parent Spark session for the endpoint since it's where this query ID is registered. val epochEndpoint = EpochCoordinatorRef.create( @@ -436,5 +433,4 @@ class ContinuousExecution( object ContinuousExecution { val START_EPOCH_KEY = "__continuous_start_epoch" val EPOCH_COORDINATOR_ID_KEY = "__epoch_coordinator_id" - val EPOCH_INTERVAL_KEY = "__continuous_epoch_interval" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala deleted file mode 100644 index 9b13f6398d837..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous.shuffle - -import java.util.UUID - -import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} -import org.apache.spark.rdd.RDD -import org.apache.spark.rpc.RpcAddress -import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.util.NextIterator - -case class ContinuousShuffleReadPartition( - index: Int, - endpointName: String, - queueSize: Int, - numShuffleWriters: Int, - epochIntervalMs: Long) - extends Partition { - // Initialized only on the executor, and only once even as we call compute() multiple times. - lazy val (reader: ContinuousShuffleReader, endpoint) = { - val env = SparkEnv.get.rpcEnv - val receiver = new RPCContinuousShuffleReader( - queueSize, numShuffleWriters, epochIntervalMs, env) - val endpoint = env.setupEndpoint(endpointName, receiver) - - TaskContext.get().addTaskCompletionListener[Unit] { ctx => - env.stop(endpoint) - } - (receiver, endpoint) - } -} - -/** - * RDD at the map side of each continuous processing shuffle task. Upstream tasks send their - * shuffle output to the wrapped receivers in partitions of this RDD; each of the RDD's tasks - * poll from their receiver until an epoch marker is sent. - * - * @param sc the RDD context - * @param numPartitions the number of read partitions for this RDD - * @param queueSize the size of the row buffers to use - * @param numShuffleWriters the number of continuous shuffle writers feeding into this RDD - * @param epochIntervalMs the checkpoint interval of the streaming query - */ -class ContinuousShuffleReadRDD( - sc: SparkContext, - numPartitions: Int, - queueSize: Int = 1024, - numShuffleWriters: Int = 1, - epochIntervalMs: Long = 1000, - val endpointNames: Seq[String] = Seq(s"RPCContinuousShuffleReader-${UUID.randomUUID()}")) - extends RDD[UnsafeRow](sc, Nil) { - - override protected def getPartitions: Array[Partition] = { - (0 until numPartitions).map { partIndex => - ContinuousShuffleReadPartition( - partIndex, endpointNames(partIndex), queueSize, numShuffleWriters, epochIntervalMs) - }.toArray - } - - override def compute(split: Partition, context: TaskContext): Iterator[UnsafeRow] = { - split.asInstanceOf[ContinuousShuffleReadPartition].reader.read() - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReader.scala deleted file mode 100644 index 42631c90ebc55..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReader.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous.shuffle - -import org.apache.spark.sql.catalyst.expressions.UnsafeRow - -/** - * Trait for reading from a continuous processing shuffle. - */ -trait ContinuousShuffleReader { - /** - * Returns an iterator over the incoming rows in an epoch. Implementations should block waiting - * for new rows to arrive, and end the iterator once they've received epoch markers from all - * shuffle writers. - */ - def read(): Iterator[UnsafeRow] -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleWriter.scala deleted file mode 100644 index 47b1f78b24505..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleWriter.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous.shuffle - -import org.apache.spark.sql.catalyst.expressions.UnsafeRow - -/** - * Trait for writing to a continuous processing shuffle. - */ -trait ContinuousShuffleWriter { - def write(epoch: Iterator[UnsafeRow]): Unit -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/RPCContinuousShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/RPCContinuousShuffleReader.scala deleted file mode 100644 index 502ae0d4822e8..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/RPCContinuousShuffleReader.scala +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous.shuffle - -import java.util.concurrent._ -import java.util.concurrent.atomic.AtomicBoolean - -import org.apache.spark.internal.Logging -import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} -import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.util.NextIterator - -/** - * Messages for the RPCContinuousShuffleReader endpoint. Either an incoming row or an epoch marker. - * - * Each message comes tagged with writerId, identifying which writer the message is coming - * from. The receiver will only begin the next epoch once all writers have sent an epoch - * marker ending the current epoch. - */ -private[shuffle] sealed trait RPCContinuousShuffleMessage extends Serializable { - def writerId: Int -} -private[shuffle] case class ReceiverRow(writerId: Int, row: UnsafeRow) - extends RPCContinuousShuffleMessage -private[shuffle] case class ReceiverEpochMarker(writerId: Int) extends RPCContinuousShuffleMessage - -/** - * RPC endpoint for receiving rows into a continuous processing shuffle task. Continuous shuffle - * writers will send rows here, with continuous shuffle readers polling for new rows as needed. - * - * TODO: Support multiple source tasks. We need to output a single epoch marker once all - * source tasks have sent one. - */ -private[continuous] class RPCContinuousShuffleReader( - queueSize: Int, - numShuffleWriters: Int, - epochIntervalMs: Long, - override val rpcEnv: RpcEnv) - extends ThreadSafeRpcEndpoint with ContinuousShuffleReader with Logging { - // Note that this queue will be drained from the main task thread and populated in the RPC - // response thread. - private val queues = Array.fill(numShuffleWriters) { - new ArrayBlockingQueue[RPCContinuousShuffleMessage](queueSize) - } - - // Exposed for testing to determine if the endpoint gets stopped on task end. - private[shuffle] val stopped = new AtomicBoolean(false) - - override def onStop(): Unit = { - stopped.set(true) - } - - override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case r: RPCContinuousShuffleMessage => - // Note that this will block a thread the shared RPC handler pool! - // The TCP based shuffle handler (SPARK-24541) will avoid this problem. - queues(r.writerId).put(r) - context.reply(()) - } - - override def read(): Iterator[UnsafeRow] = { - new NextIterator[UnsafeRow] { - // An array of flags for whether each writer ID has gotten an epoch marker. - private val writerEpochMarkersReceived = Array.fill(numShuffleWriters)(false) - - private val executor = Executors.newFixedThreadPool(numShuffleWriters) - private val completion = new ExecutorCompletionService[RPCContinuousShuffleMessage](executor) - - private def completionTask(writerId: Int) = new Callable[RPCContinuousShuffleMessage] { - override def call(): RPCContinuousShuffleMessage = queues(writerId).take() - } - - // Initialize by submitting tasks to read the first row from each writer. - (0 until numShuffleWriters).foreach(writerId => completion.submit(completionTask(writerId))) - - /** - * In each call to getNext(), we pull the next row available in the completion queue, and then - * submit another task to read the next row from the writer which returned it. - * - * When a writer sends an epoch marker, we note that it's finished and don't submit another - * task for it in this epoch. The iterator is over once all writers have sent an epoch marker. - */ - override def getNext(): UnsafeRow = { - var nextRow: UnsafeRow = null - while (!finished && nextRow == null) { - completion.poll(epochIntervalMs, TimeUnit.MILLISECONDS) match { - case null => - // Try again if the poll didn't wait long enough to get a real result. - // But we should be getting at least an epoch marker every checkpoint interval. - val writerIdsUncommitted = writerEpochMarkersReceived.zipWithIndex.collect { - case (flag, idx) if !flag => idx - } - logWarning( - s"Completion service failed to make progress after $epochIntervalMs ms. Waiting " + - s"for writers ${writerIdsUncommitted.mkString(",")} to send epoch markers.") - - // The completion service guarantees this future will be available immediately. - case future => future.get() match { - case ReceiverRow(writerId, r) => - // Start reading the next element in the queue we just took from. - completion.submit(completionTask(writerId)) - nextRow = r - case ReceiverEpochMarker(writerId) => - // Don't read any more from this queue. If all the writers have sent epoch markers, - // the epoch is over; otherwise we need to loop again to poll from the remaining - // writers. - writerEpochMarkersReceived(writerId) = true - if (writerEpochMarkersReceived.forall(_ == true)) { - finished = true - } - } - } - } - - nextRow - } - - override def close(): Unit = { - executor.shutdownNow() - } - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/RPCContinuousShuffleWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/RPCContinuousShuffleWriter.scala deleted file mode 100644 index 1c6f3ddb395e6..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/RPCContinuousShuffleWriter.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous.shuffle - -import scala.concurrent.Future -import scala.concurrent.duration.Duration - -import org.apache.spark.Partitioner -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.util.ThreadUtils - -/** - * A [[ContinuousShuffleWriter]] sending data to [[RPCContinuousShuffleReader]] instances. - * - * @param writerId The partition ID of this writer. - * @param outputPartitioner The partitioner on the reader side of the shuffle. - * @param endpoints The [[RPCContinuousShuffleReader]] endpoints to write to. Indexed by - * partition ID within outputPartitioner. - */ -class RPCContinuousShuffleWriter( - writerId: Int, - outputPartitioner: Partitioner, - endpoints: Array[RpcEndpointRef]) extends ContinuousShuffleWriter { - - if (outputPartitioner.numPartitions != 1) { - throw new IllegalArgumentException("multiple readers not yet supported") - } - - if (outputPartitioner.numPartitions != endpoints.length) { - throw new IllegalArgumentException(s"partitioner size ${outputPartitioner.numPartitions} did " + - s"not match endpoint count ${endpoints.length}") - } - - def write(epoch: Iterator[UnsafeRow]): Unit = { - while (epoch.hasNext) { - val row = epoch.next() - endpoints(outputPartitioner.getPartition(row)).askSync[Unit](ReceiverRow(writerId, row)) - } - - val futures = endpoints.map(_.ask[Unit](ReceiverEpochMarker(writerId))).toSeq - implicit val ec = ThreadUtils.sameThread - ThreadUtils.awaitResult(Future.sequence(futures), Duration.Inf) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala index 0eb3dce1bbd27..90a53727aa317 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala @@ -74,20 +74,8 @@ class StateStoreRDD[T: ClassTag, U: ClassTag]( StateStoreId(checkpointLocation, operatorId, partition.index), queryRunId) - // If we're in continuous processing mode, we should get the store version for the current - // epoch rather than the one at planning time. - val isContinuous = Option(ctxt.getLocalProperty(StreamExecution.IS_CONTINUOUS_PROCESSING)) - .map(_.toBoolean).getOrElse(false) - val currentVersion = if (isContinuous) { - val epoch = EpochTracker.getCurrentEpoch - assert(epoch.isDefined, "Current epoch must be defined for continuous processing streams.") - epoch.get - } else { - storeVersion - } - store = StateStore.get( - storeProviderId, keySchema, valueSchema, indexOrdinal, currentVersion, + storeProviderId, keySchema, valueSchema, indexOrdinal, storeVersion, storeConf, hadoopConfBroadcast.value.value) val inputIter = dataRDD.iterator(partition, ctxt) storeUpdateFunction(store, inputIter) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleSuite.scala deleted file mode 100644 index 54ec4a8352c1b..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleSuite.scala +++ /dev/null @@ -1,423 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous.shuffle - -import java.util.UUID - -import scala.language.implicitConversions - -import org.apache.spark.{HashPartitioner, TaskContext, TaskContextImpl} -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} -import org.apache.spark.sql.streaming.StreamTest -import org.apache.spark.sql.types.{DataType, IntegerType, StringType} -import org.apache.spark.unsafe.types.UTF8String - -class ContinuousShuffleSuite extends StreamTest { - // In this unit test, we emulate that we're in the task thread where - // ContinuousShuffleReadRDD.compute() will be evaluated. This requires a task context - // thread local to be set. - var ctx: TaskContextImpl = _ - - override def beforeEach(): Unit = { - super.beforeEach() - ctx = TaskContext.empty() - TaskContext.setTaskContext(ctx) - } - - override def afterEach(): Unit = { - ctx.markTaskCompleted(None) - TaskContext.unset() - ctx = null - super.afterEach() - } - - private implicit def unsafeRow(value: Int) = { - UnsafeProjection.create(Array(IntegerType : DataType))( - new GenericInternalRow(Array(value: Any))) - } - - private def unsafeRow(value: String) = { - UnsafeProjection.create(Array(StringType : DataType))( - new GenericInternalRow(Array(UTF8String.fromString(value): Any))) - } - - private def send(endpoint: RpcEndpointRef, messages: RPCContinuousShuffleMessage*) = { - messages.foreach(endpoint.askSync[Unit](_)) - } - - private def readRDDEndpoint(rdd: ContinuousShuffleReadRDD) = { - rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - } - - private def readEpoch(rdd: ContinuousShuffleReadRDD) = { - rdd.compute(rdd.partitions(0), ctx).toSeq.map(_.getInt(0)) - } - - test("reader - one epoch") { - val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - send( - endpoint, - ReceiverRow(0, unsafeRow(111)), - ReceiverRow(0, unsafeRow(222)), - ReceiverRow(0, unsafeRow(333)), - ReceiverEpochMarker(0) - ) - - val iter = rdd.compute(rdd.partitions(0), ctx) - assert(iter.toSeq.map(_.getInt(0)) == Seq(111, 222, 333)) - } - - test("reader - multiple epochs") { - val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - send( - endpoint, - ReceiverRow(0, unsafeRow(111)), - ReceiverEpochMarker(0), - ReceiverRow(0, unsafeRow(222)), - ReceiverRow(0, unsafeRow(333)), - ReceiverEpochMarker(0) - ) - - val firstEpoch = rdd.compute(rdd.partitions(0), ctx) - assert(firstEpoch.toSeq.map(_.getInt(0)) == Seq(111)) - - val secondEpoch = rdd.compute(rdd.partitions(0), ctx) - assert(secondEpoch.toSeq.map(_.getInt(0)) == Seq(222, 333)) - } - - test("reader - empty epochs") { - val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - - send( - endpoint, - ReceiverEpochMarker(0), - ReceiverEpochMarker(0), - ReceiverRow(0, unsafeRow(111)), - ReceiverEpochMarker(0), - ReceiverEpochMarker(0), - ReceiverEpochMarker(0) - ) - - assert(rdd.compute(rdd.partitions(0), ctx).isEmpty) - assert(rdd.compute(rdd.partitions(0), ctx).isEmpty) - - val thirdEpoch = rdd.compute(rdd.partitions(0), ctx) - assert(thirdEpoch.toSeq.map(_.getInt(0)) == Seq(111)) - - assert(rdd.compute(rdd.partitions(0), ctx).isEmpty) - assert(rdd.compute(rdd.partitions(0), ctx).isEmpty) - } - - test("reader - multiple partitions") { - val rdd = new ContinuousShuffleReadRDD( - sparkContext, - numPartitions = 5, - endpointNames = Seq.fill(5)(s"endpt-${UUID.randomUUID()}")) - // Send all data before processing to ensure there's no crossover. - for (p <- rdd.partitions) { - val part = p.asInstanceOf[ContinuousShuffleReadPartition] - // Send index for identification. - send( - part.endpoint, - ReceiverRow(0, unsafeRow(part.index)), - ReceiverEpochMarker(0) - ) - } - - for (p <- rdd.partitions) { - val part = p.asInstanceOf[ContinuousShuffleReadPartition] - val iter = rdd.compute(part, ctx) - assert(iter.next().getInt(0) == part.index) - assert(!iter.hasNext) - } - } - - test("reader - blocks waiting for new rows") { - val rdd = new ContinuousShuffleReadRDD( - sparkContext, numPartitions = 1, epochIntervalMs = Long.MaxValue) - val epoch = rdd.compute(rdd.partitions(0), ctx) - - val readRowThread = new Thread { - override def run(): Unit = { - try { - epoch.next().getInt(0) - } catch { - case _: InterruptedException => // do nothing - expected at test ending - } - } - } - - try { - readRowThread.start() - eventually(timeout(streamingTimeout)) { - assert(readRowThread.getState == Thread.State.TIMED_WAITING) - } - } finally { - readRowThread.interrupt() - readRowThread.join() - } - } - - test("reader - multiple writers") { - val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1, numShuffleWriters = 3) - val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - send( - endpoint, - ReceiverRow(0, unsafeRow("writer0-row0")), - ReceiverRow(1, unsafeRow("writer1-row0")), - ReceiverRow(2, unsafeRow("writer2-row0")), - ReceiverEpochMarker(0), - ReceiverEpochMarker(1), - ReceiverEpochMarker(2) - ) - - val firstEpoch = rdd.compute(rdd.partitions(0), ctx) - assert(firstEpoch.toSeq.map(_.getUTF8String(0).toString).toSet == - Set("writer0-row0", "writer1-row0", "writer2-row0")) - } - - test("reader - epoch only ends when all writers send markers") { - val rdd = new ContinuousShuffleReadRDD( - sparkContext, numPartitions = 1, numShuffleWriters = 3, epochIntervalMs = Long.MaxValue) - val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - send( - endpoint, - ReceiverRow(0, unsafeRow("writer0-row0")), - ReceiverRow(1, unsafeRow("writer1-row0")), - ReceiverRow(2, unsafeRow("writer2-row0")), - ReceiverEpochMarker(0), - ReceiverEpochMarker(2) - ) - - val epoch = rdd.compute(rdd.partitions(0), ctx) - val rows = (0 until 3).map(_ => epoch.next()).toSet - assert(rows.map(_.getUTF8String(0).toString) == - Set("writer0-row0", "writer1-row0", "writer2-row0")) - - // After checking the right rows, block until we get an epoch marker indicating there's no next. - // (Also fail the assertion if for some reason we get a row.) - - val readEpochMarkerThread = new Thread { - override def run(): Unit = { - assert(!epoch.hasNext) - } - } - - readEpochMarkerThread.start() - eventually(timeout(streamingTimeout)) { - assert(readEpochMarkerThread.getState == Thread.State.TIMED_WAITING) - } - - // Send the last epoch marker - now the epoch should finish. - send(endpoint, ReceiverEpochMarker(1)) - eventually(timeout(streamingTimeout)) { - !readEpochMarkerThread.isAlive - } - - // Join to pick up assertion failures. - readEpochMarkerThread.join(streamingTimeout.toMillis) - } - - test("reader - writer epochs non aligned") { - val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1, numShuffleWriters = 3) - val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - // We send multiple epochs for 0, then multiple for 1, then multiple for 2. The receiver should - // collate them as though the markers were aligned in the first place. - send( - endpoint, - ReceiverRow(0, unsafeRow("writer0-row0")), - ReceiverEpochMarker(0), - ReceiverRow(0, unsafeRow("writer0-row1")), - ReceiverEpochMarker(0), - ReceiverEpochMarker(0), - - ReceiverEpochMarker(1), - ReceiverRow(1, unsafeRow("writer1-row0")), - ReceiverEpochMarker(1), - ReceiverRow(1, unsafeRow("writer1-row1")), - ReceiverEpochMarker(1), - - ReceiverEpochMarker(2), - ReceiverEpochMarker(2), - ReceiverRow(2, unsafeRow("writer2-row0")), - ReceiverEpochMarker(2) - ) - - val firstEpoch = rdd.compute(rdd.partitions(0), ctx).map(_.getUTF8String(0).toString).toSet - assert(firstEpoch == Set("writer0-row0")) - - val secondEpoch = rdd.compute(rdd.partitions(0), ctx).map(_.getUTF8String(0).toString).toSet - assert(secondEpoch == Set("writer0-row1", "writer1-row0")) - - val thirdEpoch = rdd.compute(rdd.partitions(0), ctx).map(_.getUTF8String(0).toString).toSet - assert(thirdEpoch == Set("writer1-row1", "writer2-row0")) - } - - test("one epoch") { - val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val writer = new RPCContinuousShuffleWriter( - 0, new HashPartitioner(1), Array(readRDDEndpoint(reader))) - - writer.write(Iterator(1, 2, 3)) - - assert(readEpoch(reader) == Seq(1, 2, 3)) - } - - test("multiple epochs") { - val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val writer = new RPCContinuousShuffleWriter( - 0, new HashPartitioner(1), Array(readRDDEndpoint(reader))) - - writer.write(Iterator(1, 2, 3)) - writer.write(Iterator(4, 5, 6)) - - assert(readEpoch(reader) == Seq(1, 2, 3)) - assert(readEpoch(reader) == Seq(4, 5, 6)) - } - - test("empty epochs") { - val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val writer = new RPCContinuousShuffleWriter( - 0, new HashPartitioner(1), Array(readRDDEndpoint(reader))) - - writer.write(Iterator()) - writer.write(Iterator(1, 2)) - writer.write(Iterator()) - writer.write(Iterator()) - writer.write(Iterator(3, 4)) - writer.write(Iterator()) - - assert(readEpoch(reader) == Seq()) - assert(readEpoch(reader) == Seq(1, 2)) - assert(readEpoch(reader) == Seq()) - assert(readEpoch(reader) == Seq()) - assert(readEpoch(reader) == Seq(3, 4)) - assert(readEpoch(reader) == Seq()) - } - - test("blocks waiting for writer") { - val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val writer = new RPCContinuousShuffleWriter( - 0, new HashPartitioner(1), Array(readRDDEndpoint(reader))) - - val readerEpoch = reader.compute(reader.partitions(0), ctx) - - val readRowThread = new Thread { - override def run(): Unit = { - assert(readerEpoch.toSeq.map(_.getInt(0)) == Seq(1)) - } - } - readRowThread.start() - - eventually(timeout(streamingTimeout)) { - assert(readRowThread.getState == Thread.State.TIMED_WAITING) - } - - // Once we write the epoch the thread should stop waiting and succeed. - writer.write(Iterator(1)) - readRowThread.join(streamingTimeout.toMillis) - } - - test("multiple writer partitions") { - val numWriterPartitions = 3 - - val reader = new ContinuousShuffleReadRDD( - sparkContext, numPartitions = 1, numShuffleWriters = numWriterPartitions) - val writers = (0 until 3).map { idx => - new RPCContinuousShuffleWriter(idx, new HashPartitioner(1), Array(readRDDEndpoint(reader))) - } - - writers(0).write(Iterator(1, 4, 7)) - writers(1).write(Iterator(2, 5)) - writers(2).write(Iterator(3, 6)) - - writers(0).write(Iterator(4, 7, 10)) - writers(1).write(Iterator(5, 8)) - writers(2).write(Iterator(6, 9)) - - // Since there are multiple asynchronous writers, the original row sequencing is not guaranteed. - // The epochs should be deterministically preserved, however. - assert(readEpoch(reader).toSet == Seq(1, 2, 3, 4, 5, 6, 7).toSet) - assert(readEpoch(reader).toSet == Seq(4, 5, 6, 7, 8, 9, 10).toSet) - } - - test("reader epoch only ends when all writer partitions write it") { - val numWriterPartitions = 3 - - val reader = new ContinuousShuffleReadRDD( - sparkContext, numPartitions = 1, numShuffleWriters = numWriterPartitions) - val writers = (0 until 3).map { idx => - new RPCContinuousShuffleWriter(idx, new HashPartitioner(1), Array(readRDDEndpoint(reader))) - } - - writers(1).write(Iterator()) - writers(2).write(Iterator()) - - val readerEpoch = reader.compute(reader.partitions(0), ctx) - - val readEpochMarkerThread = new Thread { - override def run(): Unit = { - assert(!readerEpoch.hasNext) - } - } - - readEpochMarkerThread.start() - eventually(timeout(streamingTimeout)) { - assert(readEpochMarkerThread.getState == Thread.State.TIMED_WAITING) - } - - writers(0).write(Iterator()) - readEpochMarkerThread.join(streamingTimeout.toMillis) - } - - test("receiver stopped with row last") { - val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - send( - endpoint, - ReceiverEpochMarker(0), - ReceiverRow(0, unsafeRow(111)) - ) - - ctx.markTaskCompleted(None) - val receiver = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].reader - eventually(timeout(streamingTimeout)) { - assert(receiver.asInstanceOf[RPCContinuousShuffleReader].stopped.get()) - } - } - - test("receiver stopped with marker last") { - val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) - val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint - send( - endpoint, - ReceiverRow(0, unsafeRow(111)), - ReceiverEpochMarker(0) - ) - - ctx.markTaskCompleted(None) - val receiver = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].reader - eventually(timeout(streamingTimeout)) { - assert(receiver.asInstanceOf[RPCContinuousShuffleReader].stopped.get()) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala deleted file mode 100644 index 3ec4750c59fc5..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.streaming.continuous - -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED -import org.apache.spark.sql.streaming.OutputMode - -class ContinuousAggregationSuite extends ContinuousSuiteBase { - import testImplicits._ - - test("not enabled") { - val ex = intercept[AnalysisException] { - val input = ContinuousMemoryStream.singlePartition[Int] - testStream(input.toDF().agg(max('value)), OutputMode.Complete)() - } - - assert(ex.getMessage.contains( - "In continuous processing mode, coalesce(1) must be called before aggregate operation")) - } - - test("basic") { - withSQLConf((UNSUPPORTED_OPERATION_CHECK_ENABLED.key, "false")) { - val input = ContinuousMemoryStream.singlePartition[Int] - - testStream(input.toDF().agg(max('value)), OutputMode.Complete)( - AddData(input, 0, 1, 2), - CheckAnswer(2), - StopStream, - AddData(input, 3, 4, 5), - StartStream(), - CheckAnswer(5), - AddData(input, -1, -2, -3), - CheckAnswer(5)) - } - } - - test("multiple partitions with coalesce") { - val input = ContinuousMemoryStream[Int] - - val df = input.toDF().coalesce(1).agg(max('value)) - - testStream(df, OutputMode.Complete)( - AddData(input, 0, 1, 2), - CheckAnswer(2), - StopStream, - AddData(input, 3, 4, 5), - StartStream(), - CheckAnswer(5), - AddData(input, -1, -2, -3), - CheckAnswer(5)) - } - - test("multiple partitions with coalesce - multiple transformations") { - val input = ContinuousMemoryStream[Int] - - // We use a barrier to make sure predicates both before and after coalesce work - val df = input.toDF() - .select('value as 'copy, 'value) - .where('copy =!= 1) - .logicalPlan - .coalesce(1) - .where('copy =!= 2) - .agg(max('value)) - - testStream(df, OutputMode.Complete)( - AddData(input, 0, 1, 2), - CheckAnswer(0), - StopStream, - AddData(input, 3, 4, 5), - StartStream(), - CheckAnswer(5), - AddData(input, -1, -2, -3), - CheckAnswer(5)) - } - - test("multiple partitions with multiple coalesce") { - val input = ContinuousMemoryStream[Int] - - val df = input.toDF() - .coalesce(1) - .logicalPlan - .coalesce(1) - .select('value as 'copy, 'value) - .agg(max('value)) - - testStream(df, OutputMode.Complete)( - AddData(input, 0, 1, 2), - CheckAnswer(2), - StopStream, - AddData(input, 3, 4, 5), - StartStream(), - CheckAnswer(5), - AddData(input, -1, -2, -3), - CheckAnswer(5)) - } - - test("repeated restart") { - withSQLConf((UNSUPPORTED_OPERATION_CHECK_ENABLED.key, "false")) { - val input = ContinuousMemoryStream.singlePartition[Int] - - testStream(input.toDF().agg(max('value)), OutputMode.Complete)( - AddData(input, 0, 1, 2), - CheckAnswer(2), - StopStream, - StartStream(), - StopStream, - StartStream(), - StopStream, - StartStream(), - AddData(input, 0), - CheckAnswer(2), - AddData(input, 5), - CheckAnswer(5)) - } - } -} From 2527fbc896dc8a26f5a281ed719fb59b5df8cd2f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 14 Jul 2020 22:14:31 -0700 Subject: [PATCH 199/384] Revert "[SPARK-32276][SQL] Remove redundant sorts before repartition nodes" This reverts commit af8e65fca989518cf65ec47f77eea2ce649bd6bb. --- .../sql/catalyst/optimizer/Optimizer.scala | 8 - ...EliminateSortsBeforeRepartitionSuite.scala | 179 ------------------ 2 files changed, 187 deletions(-) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 29f539904e645..e481cdbd5fdf4 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 @@ -974,10 +974,6 @@ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { * and the Join conditions is deterministic * 5) if the Sort operator is within GroupBy separated by 0...n Project/Filter operators only, * and the aggregate function is order irrelevant - * 6) if the Sort operator is within RepartitionByExpression separated by 0...n Project/Filter - * operators and the repartition expressions are deterministic - * 7) if the Sort operator is within Repartition separated by 0...n Project/Filter operators - * and the repartition requires a shuffle */ object EliminateSorts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -991,10 +987,6 @@ object EliminateSorts extends Rule[LogicalPlan] { j.copy(left = recursiveRemoveSort(originLeft), right = recursiveRemoveSort(originRight)) case g @ Aggregate(_, aggs, originChild) if isOrderIrrelevantAggs(aggs) => g.copy(child = recursiveRemoveSort(originChild)) - case r: RepartitionByExpression if r.partitionExpressions.forall(_.deterministic) => - r.copy(child = recursiveRemoveSort(r.child)) - case r: Repartition if r.shuffle => - r.copy(child = recursiveRemoveSort(r.child)) } private def recursiveRemoveSort(plan: LogicalPlan): LogicalPlan = plan match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala deleted file mode 100644 index ff5521f98affb..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.optimizer - -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} -import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} -import org.apache.spark.sql.catalyst.rules.RuleExecutor - -class EliminateSortsBeforeRepartitionSuite extends PlanTest { - - val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) - val analyzer = new Analyzer(catalog, conf) - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - - object Optimize extends RuleExecutor[LogicalPlan] { - val batches = - Batch("Default", FixedPoint(10), - FoldablePropagation, - LimitPushDown) :: - Batch("Eliminate Sorts", Once, - EliminateSorts) :: - Batch("Collapse Project", Once, - CollapseProject) :: Nil - } - - def repartition(plan: LogicalPlan): LogicalPlan = plan.repartition(10) - def isOptimized: Boolean = true - - test("sortBy") { - val plan = testRelation.select('a, 'b).sortBy('a.asc, 'b.desc) - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - val correctPlan = if (isOptimized) { - repartition(testRelation.select('a, 'b)) - } else { - planWithRepartition - } - comparePlans(optimizedPlan, analyzer.execute(correctPlan)) - } - - test("sortBy with projection") { - val plan = testRelation.select('a, 'b) - .sortBy('a.asc, 'b.asc) - .select('a + 1 as "a", 'b + 2 as "b") - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - val correctPlan = if (isOptimized) { - repartition(testRelation.select('a + 1 as "a", 'b + 2 as "b")) - } else { - planWithRepartition - } - comparePlans(optimizedPlan, analyzer.execute(correctPlan)) - } - - test("sortBy with projection and filter") { - val plan = testRelation.sortBy('a.asc, 'b.asc) - .select('a, 'b) - .where('a === 10) - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - val correctPlan = if (isOptimized) { - repartition(testRelation.select('a, 'b).where('a === 10)) - } else { - planWithRepartition - } - comparePlans(optimizedPlan, analyzer.execute(correctPlan)) - } - - test("sortBy with limit") { - val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) - } - - test("sortBy with non-deterministic projection") { - val plan = testRelation.sortBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) - } - - test("orderBy") { - val plan = testRelation.select('a, 'b).orderBy('a.asc, 'b.asc) - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - val correctPlan = if (isOptimized) { - repartition(testRelation.select('a, 'b)) - } else { - planWithRepartition - } - comparePlans(optimizedPlan, analyzer.execute(correctPlan)) - } - - test("orderBy with projection") { - val plan = testRelation.select('a, 'b) - .orderBy('a.asc, 'b.asc) - .select('a + 1 as "a", 'b + 2 as "b") - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - val correctPlan = if (isOptimized) { - repartition(testRelation.select('a + 1 as "a", 'b + 2 as "b")) - } else { - planWithRepartition - } - comparePlans(optimizedPlan, analyzer.execute(correctPlan)) - } - - test("orderBy with projection and filter") { - val plan = testRelation.orderBy('a.asc, 'b.asc) - .select('a, 'b) - .where('a === 10) - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - val correctPlan = if (isOptimized) { - repartition(testRelation.select('a, 'b).where('a === 10)) - } else { - planWithRepartition - } - comparePlans(optimizedPlan, analyzer.execute(correctPlan)) - } - - test("orderBy with limit") { - val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) - } - - test("orderBy with non-deterministic projection") { - val plan = testRelation.orderBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) - val planWithRepartition = repartition(plan) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) - } -} - -class EliminateSortsBeforeRepartitionByExprsSuite extends EliminateSortsBeforeRepartitionSuite { - override def repartition(plan: LogicalPlan): LogicalPlan = plan.distribute('a, 'b)(10) - override def isOptimized: Boolean = true - - test("sortBy before repartition with non-deterministic expressions") { - val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) - val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) - } - - test("orderBy before repartition with non-deterministic expressions") { - val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) - val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) - val optimizedPlan = Optimize.execute(analyzer.execute(planWithRepartition)) - comparePlans(optimizedPlan, analyzer.execute(planWithRepartition)) - } -} - -class EliminateSortsBeforeCoalesceSuite extends EliminateSortsBeforeRepartitionSuite { - override def repartition(plan: LogicalPlan): LogicalPlan = plan.coalesce(1) - override def isOptimized: Boolean = false -} From e4499932da03743cb05c6bcc5d0149728380383a Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Wed, 15 Jul 2020 01:28:39 -0700 Subject: [PATCH 200/384] [SPARK-31480][SQL] Improve the EXPLAIN FORMATTED's output for DSV2's Scan Node ### What changes were proposed in this pull request? Improve the EXPLAIN FORMATTED output of DSV2 Scan nodes (file based ones). **Before** ``` == Physical Plan == * Project (4) +- * Filter (3) +- * ColumnarToRow (2) +- BatchScan (1) (1) BatchScan Output [2]: [value#7, id#8] Arguments: [value#7, id#8], ParquetScan(org.apache.spark.sql.test.TestSparkSession17477bbb,Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml, __spark_hadoop_conf__.xml,org.apache.spark.sql.execution.datasources.InMemoryFileIndexa6c363ce,StructType(StructField(value,IntegerType,true)),StructType(StructField(value,IntegerType,true)),StructType(StructField(id,IntegerType,true)),[Lorg.apache.spark.sql.sources.Filter;40fee459,org.apache.spark.sql.util.CaseInsensitiveStringMapfeca1ec6,Vector(isnotnull(id#8), (id#8 > 1)),List(isnotnull(value#7), (value#7 > 2))) (2) ... (3) ... (4) ... ``` **After** ``` == Physical Plan == * Project (4) +- * Filter (3) +- * ColumnarToRow (2) +- BatchScan (1) (1) BatchScan Output [2]: [value#7, id#8] DataFilters: [isnotnull(value#7), (value#7 > 2)] Format: parquet Location: InMemoryFileIndex[....] PartitionFilters: [isnotnull(id#8), (id#8 > 1)] PushedFilers: [IsNotNull(id), IsNotNull(value), GreaterThan(id,1), GreaterThan(value,2)] ReadSchema: struct (2) ... (3) ... (4) ... ``` ### Why are the changes needed? The old format is not very readable. This improves the readability of the plan. ### Does this PR introduce any user-facing change? Yes. the explain output will be different. ### How was this patch tested? Added a test case in ExplainSuite. Closes #28425 from dilipbiswal/dkb_dsv2_explain. Lead-authored-by: Dilip Biswal Co-authored-by: Dilip Biswal Signed-off-by: Dilip Biswal --- .../org/apache/spark/sql/avro/AvroSuite.scala | 33 +++++++++++- .../internal/connector/SupportsMetadata.scala | 26 ++++++++++ .../v2/DataSourceV2ScanExecBase.scala | 30 ++++++++++- .../execution/datasources/v2/FileScan.scala | 27 ++++++---- .../datasources/v2/csv/CSVScan.scala | 4 ++ .../datasources/v2/orc/OrcScan.scala | 4 ++ .../datasources/v2/parquet/ParquetScan.scala | 4 ++ .../org/apache/spark/sql/ExplainSuite.scala | 50 ++++++++++++++++++- 8 files changed, 163 insertions(+), 15 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SupportsMetadata.scala diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index e2ae489446d85..83a7ef0061fb2 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -39,9 +39,10 @@ import org.apache.spark.sql.TestingUDT.IntervalData import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.Filter import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, LA, UTC} -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{FormattedMode, SparkPlan} import org.apache.spark.sql.execution.datasources.{DataSource, FilePartition} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._ import org.apache.spark.sql.test.SharedSparkSession @@ -1808,7 +1809,7 @@ class AvroV1Suite extends AvroSuite { .set(SQLConf.USE_V1_SOURCE_LIST, "avro") } -class AvroV2Suite extends AvroSuite { +class AvroV2Suite extends AvroSuite with ExplainSuiteHelper { import testImplicits._ override protected def sparkConf: SparkConf = @@ -1907,4 +1908,32 @@ class AvroV2Suite extends AvroSuite { assert(scan1.sameResult(scan2)) } } + + test("explain formatted on an avro data source v2") { + withTempDir { dir => + val basePath = dir.getCanonicalPath + "/avro" + val expected_plan_fragment = + s""" + |\\(1\\) BatchScan + |Output \\[2\\]: \\[value#xL, id#x\\] + |DataFilters: \\[isnotnull\\(value#xL\\), \\(value#xL > 2\\)\\] + |Format: avro + |Location: InMemoryFileIndex\\[.*\\] + |PartitionFilters: \\[isnotnull\\(id#x\\), \\(id#x > 1\\)\\] + |ReadSchema: struct\\ + |""".stripMargin.trim + spark.range(10) + .select(col("id"), col("id").as("value")) + .write.option("header", true) + .partitionBy("id") + .format("avro") + .save(basePath) + val df = spark + .read + .format("avro") + .load(basePath).where($"id" > 1 && $"value" > 2) + val normalizedOutput = getNormalizedExplain(df, FormattedMode) + assert(expected_plan_fragment.r.findAllMatchIn(normalizedOutput).length == 1) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SupportsMetadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SupportsMetadata.scala new file mode 100644 index 0000000000000..b2cb19b009141 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SupportsMetadata.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.internal.connector + +/** + * A mix-in interface for {@link FileScan}. This can be used to report metadata + * for a file based scan operator. This is currently used for supporting formatted + * explain. + */ +trait SupportsMetadata { + def getMetaData(): Map[String, String] +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala index 211f61279ddd5..083c6bc7999bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan, SupportsReportPartitioning} -import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.{ExplainUtils, LeafExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.internal.connector.SupportsMetadata import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -43,7 +44,32 @@ trait DataSourceV2ScanExecBase extends LeafExecNode { override def simpleString(maxFields: Int): String = { val result = s"$nodeName${truncatedString(output, "[", ", ", "]", maxFields)} ${scan.description()}" - Utils.redact(sqlContext.sessionState.conf.stringRedactionPattern, result) + redact(result) + } + + /** + * Shorthand for calling redact() without specifying redacting rules + */ + protected def redact(text: String): String = { + Utils.redact(sqlContext.sessionState.conf.stringRedactionPattern, text) + } + + override def verboseStringWithOperatorId(): String = { + val metaDataStr = scan match { + case s: SupportsMetadata => + s.getMetaData().toSeq.sorted.flatMap { + case (_, value) if value.isEmpty || value.equals("[]") => None + case (key, value) => Some(s"$key: ${redact(value)}") + case _ => None + } + case _ => + Seq(scan.description()) + } + s""" + |$formattedNodeName + |${ExplainUtils.generateFieldString("Output", output)} + |${metaDataStr.mkString("\n")} + |""".stripMargin } override def outputPartitioning: physical.Partitioning = scan match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 7e8e0ed2dc675..f090d7861b629 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -29,11 +29,13 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.connector.read.{Batch, InputPartition, Scan, Statistics, SupportsReportStatistics} import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.internal.connector.SupportsMetadata import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -trait FileScan extends Scan with Batch with SupportsReportStatistics with Logging { +trait FileScan extends Scan + with Batch with SupportsReportStatistics with SupportsMetadata with Logging { /** * Returns whether a file with `path` could be split or not. */ @@ -93,23 +95,28 @@ trait FileScan extends Scan with Batch with SupportsReportStatistics with Loggin override def hashCode(): Int = getClass.hashCode() + val maxMetadataValueLength = 100 + override def description(): String = { - val maxMetadataValueLength = 100 + val metadataStr = getMetaData().toSeq.sorted.map { + case (key, value) => + val redactedValue = + Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, value) + key + ": " + StringUtils.abbreviate(redactedValue, maxMetadataValueLength) + }.mkString(", ") + s"${this.getClass.getSimpleName} $metadataStr" + } + + override def getMetaData(): Map[String, String] = { val locationDesc = fileIndex.getClass.getSimpleName + Utils.buildLocationMetadata(fileIndex.rootPaths, maxMetadataValueLength) - val metadata: Map[String, String] = Map( + Map( + "Format" -> s"${this.getClass.getSimpleName.replace("Scan", "").toLowerCase(Locale.ROOT)}", "ReadSchema" -> readDataSchema.catalogString, "PartitionFilters" -> seqToString(partitionFilters), "DataFilters" -> seqToString(dataFilters), "Location" -> locationDesc) - val metadataStr = metadata.toSeq.sorted.map { - case (key, value) => - val redactedValue = - Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, value) - key + ": " + StringUtils.abbreviate(redactedValue, maxMetadataValueLength) - }.mkString(", ") - s"${this.getClass.getSimpleName} $metadataStr" } protected def partitions: Seq[FilePartition] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index 4f510322815ef..efb21e1c1e597 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -107,4 +107,8 @@ case class CSVScan( override def description(): String = { super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") } + + override def getMetaData(): Map[String, String] = { + super.getMetaData() ++ Map("PushedFilers" -> seqToString(pushedFilters)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index 62894fa7a2538..38b8ced51a141 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -65,6 +65,10 @@ case class OrcScan( super.description() + ", PushedFilters: " + seqToString(pushedFilters) } + override def getMetaData(): Map[String, String] = { + super.getMetaData() ++ Map("PushedFilers" -> seqToString(pushedFilters)) + } + override def withFilters( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala index bb315262a8211..c9c1e28a36960 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala @@ -93,6 +93,10 @@ case class ParquetScan( super.description() + ", PushedFilters: " + seqToString(pushedFilters) } + override def getMetaData(): Map[String, String] = { + super.getMetaData() ++ Map("PushedFilers" -> seqToString(pushedFilters)) + } + override def withFilters( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index 1ad97185a564a..70303792fdf1a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, E import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} trait ExplainSuiteHelper extends QueryTest with SharedSparkSession { @@ -360,6 +360,54 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } } + + test("Explain formatted output for scan operator for datasource V2") { + withTempDir { dir => + Seq("parquet", "orc", "csv", "json").foreach { fmt => + val basePath = dir.getCanonicalPath + "/" + fmt + val pushFilterMaps = Map ( + "parquet" -> + "|PushedFilers: \\[.*\\(id\\), .*\\(value\\), .*\\(id,1\\), .*\\(value,2\\)\\]", + "orc" -> + "|PushedFilers: \\[.*\\(id\\), .*\\(value\\), .*\\(id,1\\), .*\\(value,2\\)\\]", + "csv" -> + "|PushedFilers: \\[IsNotNull\\(value\\), GreaterThan\\(value,2\\)\\]", + "json" -> + "|remove_marker" + ) + val expected_plan_fragment1 = + s""" + |\\(1\\) BatchScan + |Output \\[2\\]: \\[value#x, id#x\\] + |DataFilters: \\[isnotnull\\(value#x\\), \\(value#x > 2\\)\\] + |Format: $fmt + |Location: InMemoryFileIndex\\[.*\\] + |PartitionFilters: \\[isnotnull\\(id#x\\), \\(id#x > 1\\)\\] + ${pushFilterMaps.get(fmt).get} + |ReadSchema: struct\\ + |""".stripMargin.replaceAll("\nremove_marker", "").trim + + spark.range(10) + .select(col("id"), col("id").as("value")) + .write.option("header", true) + .partitionBy("id") + .format(fmt) + .save(basePath) + val readSchema = + StructType(Seq(StructField("id", IntegerType), StructField("value", IntegerType))) + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + val df = spark + .read + .schema(readSchema) + .option("header", true) + .format(fmt) + .load(basePath).where($"id" > 1 && $"value" > 2) + val normalizedOutput = getNormalizedExplain(df, FormattedMode) + assert(expected_plan_fragment1.r.findAllMatchIn(normalizedOutput).length == 1) + } + } + } + } } class ExplainSuiteAE extends ExplainSuiteHelper with EnableAdaptiveExecutionSuite { From 8950dcbb1cafccc2ba8bbf030ab7ac86cfe203a4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 15 Jul 2020 07:43:56 -0700 Subject: [PATCH 201/384] [SPARK-32318][SQL][TESTS] Add a test case to EliminateSortsSuite for ORDER BY in DISTRIBUTE BY ### What changes were proposed in this pull request? This PR aims to add a test case to EliminateSortsSuite to protect a valid use case which is using ORDER BY in DISTRIBUTE BY statement. ### Why are the changes needed? ```scala scala> scala.util.Random.shuffle((1 to 100000).map(x => (x % 2, x))).toDF("a", "b").repartition(2).createOrReplaceTempView("t") scala> sql("select * from (select * from t order by b) distribute by a").write.orc("/tmp/master") $ ls -al /tmp/master/ total 56 drwxr-xr-x 10 dongjoon wheel 320 Jul 14 22:12 ./ drwxrwxrwt 15 root wheel 480 Jul 14 22:12 ../ -rw-r--r-- 1 dongjoon wheel 8 Jul 14 22:12 ._SUCCESS.crc -rw-r--r-- 1 dongjoon wheel 12 Jul 14 22:12 .part-00000-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc.crc -rw-r--r-- 1 dongjoon wheel 16 Jul 14 22:12 .part-00043-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc.crc -rw-r--r-- 1 dongjoon wheel 16 Jul 14 22:12 .part-00191-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc.crc -rw-r--r-- 1 dongjoon wheel 0 Jul 14 22:12 _SUCCESS -rw-r--r-- 1 dongjoon wheel 119 Jul 14 22:12 part-00000-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc -rw-r--r-- 1 dongjoon wheel 932 Jul 14 22:12 part-00043-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc -rw-r--r-- 1 dongjoon wheel 939 Jul 14 22:12 part-00191-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc ``` The following was found during SPARK-32276. If Spark optimizer removes the inner `ORDER BY`, the file size increases. ```scala scala> scala.util.Random.shuffle((1 to 100000).map(x => (x % 2, x))).toDF("a", "b").repartition(2).createOrReplaceTempView("t") scala> sql("select * from (select * from t order by b) distribute by a").write.orc("/tmp/SPARK-32276") $ ls -al /tmp/SPARK-32276/ total 632 drwxr-xr-x 10 dongjoon wheel 320 Jul 14 22:08 ./ drwxrwxrwt 14 root wheel 448 Jul 14 22:08 ../ -rw-r--r-- 1 dongjoon wheel 8 Jul 14 22:08 ._SUCCESS.crc -rw-r--r-- 1 dongjoon wheel 12 Jul 14 22:08 .part-00000-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc.crc -rw-r--r-- 1 dongjoon wheel 1188 Jul 14 22:08 .part-00043-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc.crc -rw-r--r-- 1 dongjoon wheel 1188 Jul 14 22:08 .part-00191-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc.crc -rw-r--r-- 1 dongjoon wheel 0 Jul 14 22:08 _SUCCESS -rw-r--r-- 1 dongjoon wheel 119 Jul 14 22:08 part-00000-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc -rw-r--r-- 1 dongjoon wheel 150735 Jul 14 22:08 part-00043-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc -rw-r--r-- 1 dongjoon wheel 150741 Jul 14 22:08 part-00191-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc ``` ### Does this PR introduce _any_ user-facing change? No. This only improves the test coverage. ### How was this patch tested? Pass the GitHub Action or Jenkins. Closes #29118 from dongjoon-hyun/SPARK-32318. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/EliminateSortsSuite.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala index d7eb048ba8705..e2b599a7c090c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala @@ -284,6 +284,15 @@ class EliminateSortsSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("SPARK-32318: should not remove orderBy in distribute statement") { + val projectPlan = testRelation.select('a, 'b) + val orderByPlan = projectPlan.orderBy('b.desc) + val distributedPlan = orderByPlan.distribute('a)(1) + val optimized = Optimize.execute(distributedPlan.analyze) + val correctAnswer = distributedPlan.analyze + comparePlans(optimized, correctAnswer) + } + test("should not remove orderBy in left join clause if there is an outer limit") { val projectPlan = testRelation.select('a, 'b) val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc) From cf22d947fb8f37aa4d394b6633d6f08dbbf6dc1c Mon Sep 17 00:00:00 2001 From: Erik Krogen Date: Wed, 15 Jul 2020 11:40:55 -0500 Subject: [PATCH 202/384] [SPARK-32036] Replace references to blacklist/whitelist language with more appropriate terminology, excluding the blacklisting feature ### What changes were proposed in this pull request? This PR will remove references to these "blacklist" and "whitelist" terms besides the blacklisting feature as a whole, which can be handled in a separate JIRA/PR. This touches quite a few files, but the changes are straightforward (variable/method/etc. name changes) and most quite self-contained. ### Why are the changes needed? As per discussion on the Spark dev list, it will be beneficial to remove references to problematic language that can alienate potential community members. One such reference is "blacklist" and "whitelist". While it seems to me that there is some valid debate as to whether these terms have racist origins, the cultural connotations are inescapable in today's world. ### Does this PR introduce _any_ user-facing change? In the test file `HiveQueryFileTest`, a developer has the ability to specify the system property `spark.hive.whitelist` to specify a list of Hive query files that should be tested. This system property has been renamed to `spark.hive.includelist`. The old property has been kept for compatibility, but will log a warning if used. I am open to feedback from others on whether keeping a deprecated property here is unnecessary given that this is just for developers running tests. ### How was this patch tested? Existing tests should be suitable since no behavior changes are expected as a result of this PR. Closes #28874 from xkrogen/xkrogen-SPARK-32036-rename-blacklists. Authored-by: Erik Krogen Signed-off-by: Thomas Graves --- R/pkg/tests/fulltests/test_context.R | 2 +- R/pkg/tests/fulltests/test_sparkSQL.R | 8 +-- R/pkg/tests/run-all.R | 4 +- .../org/apache/spark/network/crypto/README.md | 2 +- .../deploy/history/FsHistoryProvider.scala | 29 +++++------ .../deploy/rest/RestSubmissionClient.scala | 4 +- .../scheduler/OutputCommitCoordinator.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 4 +- .../scala/org/apache/spark/ThreadAudit.scala | 4 +- .../spark/deploy/SparkSubmitSuite.scala | 22 ++++---- .../history/FsHistoryProviderSuite.scala | 8 +-- .../org/apache/spark/ui/UISeleniumSuite.scala | 14 ++++-- dev/sparktestsupport/modules.py | 10 ++-- docs/streaming-programming-guide.md | 50 +++++++++---------- .../JavaRecoverableNetworkWordCount.java | 20 ++++---- .../recoverable_network_wordcount.py | 16 +++--- .../RecoverableNetworkWordCount.scala | 16 +++--- .../org/apache/spark/util/DockerUtils.scala | 6 +-- project/SparkBuild.scala | 4 +- python/pylintrc | 2 +- python/pyspark/cloudpickle.py | 6 +-- python/pyspark/sql/functions.py | 4 +- python/pyspark/sql/pandas/typehints.py | 4 +- python/run-tests.py | 2 +- .../cluster/mesos/MesosClusterScheduler.scala | 4 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../spark/sql/catalyst/json/JSONOptions.scala | 10 ++-- .../sql/catalyst/optimizer/Optimizer.scala | 34 ++++++------- .../sql/catalyst/optimizer/expressions.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 2 +- .../sql/catalyst/rules/RuleExecutor.scala | 6 +-- .../optimizer/FilterPushdownSuite.scala | 2 +- .../PullupCorrelatedPredicatesSuite.scala | 2 +- .../datasources/json/JsonOutputWriter.scala | 2 +- .../inputs/{blacklist.sql => ignored.sql} | 2 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 6 +-- .../apache/spark/sql/TPCDSQuerySuite.scala | 4 +- .../datasources/json/JsonSuite.scala | 2 +- .../ThriftServerQueryTestSuite.scala | 4 +- .../execution/HiveCompatibilitySuite.scala | 16 +++--- .../HiveWindowFunctionQuerySuite.scala | 8 +-- .../add_partition_no_includelist.q | 7 +++ .../add_partition_no_whitelist.q | 7 --- .../add_partition_with_includelist.q | 7 +++ .../add_partition_with_whitelist.q | 9 ---- .../alter_partition_with_includelist.q | 9 ++++ .../alter_partition_with_whitelist.q | 9 ---- .../execution/AggregationQuerySuite.scala | 2 +- .../hive/execution/HiveQueryFileTest.scala | 39 +++++++++------ ...iveHadoopDelegationTokenManagerSuite.scala | 4 +- .../util/FileBasedWriteAheadLog.scala | 4 +- 52 files changed, 231 insertions(+), 219 deletions(-) rename sql/core/src/test/resources/sql-tests/inputs/{blacklist.sql => ignored.sql} (75%) create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_includelist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_includelist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_includelist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q diff --git a/R/pkg/tests/fulltests/test_context.R b/R/pkg/tests/fulltests/test_context.R index f86872d727a1d..1add5a9fdde44 100644 --- a/R/pkg/tests/fulltests/test_context.R +++ b/R/pkg/tests/fulltests/test_context.R @@ -139,7 +139,7 @@ test_that("utility function can be called", { expect_true(TRUE) }) -test_that("getClientModeSparkSubmitOpts() returns spark-submit args from whitelist", { +test_that("getClientModeSparkSubmitOpts() returns spark-submit args from allowList", { e <- new.env() e[["spark.driver.memory"]] <- "512m" ops <- getClientModeSparkSubmitOpts("sparkrmain", e) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 611d9057c0f13..e008bc5bbd7d9 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -3921,14 +3921,14 @@ test_that("No extra files are created in SPARK_HOME by starting session and maki # before creating a SparkSession with enableHiveSupport = T at the top of this test file # (filesBefore). The test here is to compare that (filesBefore) against the list of files before # any test is run in run-all.R (sparkRFilesBefore). - # sparkRWhitelistSQLDirs is also defined in run-all.R, and should contain only 2 whitelisted dirs, + # sparkRAllowedSQLDirs is also defined in run-all.R, and should contain only 2 allowed dirs, # here allow the first value, spark-warehouse, in the diff, everything else should be exactly the # same as before any test is run. - compare_list(sparkRFilesBefore, setdiff(filesBefore, sparkRWhitelistSQLDirs[[1]])) + compare_list(sparkRFilesBefore, setdiff(filesBefore, sparkRAllowedSQLDirs[[1]])) # third, ensure only spark-warehouse and metastore_db are created when enableHiveSupport = T # note: as the note above, after running all tests in this file while enableHiveSupport = T, we - # check the list of files again. This time we allow both whitelisted dirs to be in the diff. - compare_list(sparkRFilesBefore, setdiff(filesAfter, sparkRWhitelistSQLDirs)) + # check the list of files again. This time we allow both dirs to be in the diff. + compare_list(sparkRFilesBefore, setdiff(filesAfter, sparkRAllowedSQLDirs)) }) unlink(parquetPath) diff --git a/R/pkg/tests/run-all.R b/R/pkg/tests/run-all.R index bf02ecdad66ff..a46924a5d20e3 100644 --- a/R/pkg/tests/run-all.R +++ b/R/pkg/tests/run-all.R @@ -35,8 +35,8 @@ if (identical(Sys.getenv("NOT_CRAN"), "true")) { install.spark(overwrite = TRUE) sparkRDir <- file.path(Sys.getenv("SPARK_HOME"), "R") - sparkRWhitelistSQLDirs <- c("spark-warehouse", "metastore_db") - invisible(lapply(sparkRWhitelistSQLDirs, + sparkRAllowedSQLDirs <- c("spark-warehouse", "metastore_db") + invisible(lapply(sparkRAllowedSQLDirs, function(x) { unlink(file.path(sparkRDir, x), recursive = TRUE, force = TRUE)})) sparkRFilesBefore <- list.files(path = sparkRDir, all.files = TRUE) diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md b/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md index 14df703270498..7a9fa3a91d143 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md @@ -155,4 +155,4 @@ server will be able to understand. This will cause the server to close the conne attacker tries to send any command to the server. The attacker can just hold the channel open for some time, which will be closed when the server times out the channel. These issues could be separately mitigated by adding a shorter timeout for the first message after authentication, and -potentially by adding host blacklists if a possible attack is detected from a particular host. +potentially by adding host reject-lists if a possible attack is detected from a particular host. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 891c5bffa52ed..a73a5e9463204 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -188,23 +188,24 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) processing.remove(path.getName) } - private val blacklist = new ConcurrentHashMap[String, Long] + private val inaccessibleList = new ConcurrentHashMap[String, Long] // Visible for testing - private[history] def isBlacklisted(path: Path): Boolean = { - blacklist.containsKey(path.getName) + private[history] def isAccessible(path: Path): Boolean = { + !inaccessibleList.containsKey(path.getName) } - private def blacklist(path: Path): Unit = { - blacklist.put(path.getName, clock.getTimeMillis()) + private def markInaccessible(path: Path): Unit = { + inaccessibleList.put(path.getName, clock.getTimeMillis()) } /** - * Removes expired entries in the blacklist, according to the provided `expireTimeInSeconds`. + * Removes expired entries in the inaccessibleList, according to the provided + * `expireTimeInSeconds`. */ - private def clearBlacklist(expireTimeInSeconds: Long): Unit = { + private def clearInaccessibleList(expireTimeInSeconds: Long): Unit = { val expiredThreshold = clock.getTimeMillis() - expireTimeInSeconds * 1000 - blacklist.asScala.retain((_, creationTime) => creationTime >= expiredThreshold) + inaccessibleList.asScala.retain((_, creationTime) => creationTime >= expiredThreshold) } private val activeUIs = new mutable.HashMap[(String, Option[String]), LoadedAppUI]() @@ -470,7 +471,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") val updated = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) - .filter { entry => !isBlacklisted(entry.getPath) } + .filter { entry => isAccessible(entry.getPath) } .filter { entry => !isProcessing(entry.getPath) } .flatMap { entry => EventLogFileReader(fs, entry) } .filter { reader => @@ -687,8 +688,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) case e: AccessControlException => // We don't have read permissions on the log file logWarning(s"Unable to read log $rootPath", e) - blacklist(rootPath) - // SPARK-28157 We should remove this blacklisted entry from the KVStore + markInaccessible(rootPath) + // SPARK-28157 We should remove this inaccessible entry from the KVStore // to handle permission-only changes with the same file sizes later. listing.delete(classOf[LogInfo], rootPath.toString) case e: Exception => @@ -956,8 +957,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - // Clean the blacklist from the expired entries. - clearBlacklist(CLEAN_INTERVAL_S) + // Clean the inaccessibleList from the expired entries. + clearInaccessibleList(CLEAN_INTERVAL_S) } private def deleteAttemptLogs( @@ -1334,7 +1335,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private def deleteLog(fs: FileSystem, log: Path): Boolean = { var deleted = false - if (isBlacklisted(log)) { + if (!isAccessible(log)) { logDebug(s"Skipping deleting $log as we don't have permissions on it.") } else { try { diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 89ef051e58549..cc1d60a097b2e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -411,7 +411,7 @@ private[spark] object RestSubmissionClient { // SPARK_HOME and SPARK_CONF_DIR are filtered out because they are usually wrong // on the remote machine (SPARK-12345) (SPARK-25934) - private val BLACKLISTED_SPARK_ENV_VARS = Set("SPARK_ENV_LOADED", "SPARK_HOME", "SPARK_CONF_DIR") + private val EXCLUDED_SPARK_ENV_VARS = Set("SPARK_ENV_LOADED", "SPARK_HOME", "SPARK_CONF_DIR") private val REPORT_DRIVER_STATUS_INTERVAL = 1000 private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" @@ -421,7 +421,7 @@ private[spark] object RestSubmissionClient { */ private[rest] def filterSystemEnvironment(env: Map[String, String]): Map[String, String] = { env.filterKeys { k => - (k.startsWith("SPARK_") && !BLACKLISTED_SPARK_ENV_VARS.contains(k)) || k.startsWith("MESOS_") + (k.startsWith("SPARK_") && !EXCLUDED_SPARK_ENV_VARS.contains(k)) || k.startsWith("MESOS_") }.toMap } diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index b382d623806e2..a5858ebf9cdcc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -151,7 +151,7 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) logInfo(s"Task was denied committing, stage: $stage.$stageAttempt, " + s"partition: $partition, attempt: $attemptNumber") case _ => - // Mark the attempt as failed to blacklist from future commit protocol + // Mark the attempt as failed to exclude from future commit protocol val taskId = TaskIdentifier(stageAttempt, attemptNumber) stageState.failures.getOrElseUpdate(partition, mutable.Set()) += taskId if (stageState.authorizedCommitters(partition) == taskId) { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 170ba2b9c6acc..ceaddb4306579 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -328,11 +328,11 @@ private[spark] object JsonProtocol { ("Accumulables" -> accumulablesToJson(taskInfo.accumulables)) } - private lazy val accumulableBlacklist = Set("internal.metrics.updatedBlockStatuses") + private lazy val accumulableExcludeList = Set("internal.metrics.updatedBlockStatuses") def accumulablesToJson(accumulables: Iterable[AccumulableInfo]): JArray = { JArray(accumulables - .filterNot(_.name.exists(accumulableBlacklist.contains)) + .filterNot(_.name.exists(accumulableExcludeList.contains)) .toList.map(accumulableInfoToJson)) } diff --git a/core/src/test/scala/org/apache/spark/ThreadAudit.scala b/core/src/test/scala/org/apache/spark/ThreadAudit.scala index 44d1f220bf6b1..1e2917621fa79 100644 --- a/core/src/test/scala/org/apache/spark/ThreadAudit.scala +++ b/core/src/test/scala/org/apache/spark/ThreadAudit.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging */ trait ThreadAudit extends Logging { - val threadWhiteList = Set( + val threadExcludeList = Set( /** * Netty related internal threads. * These are excluded because their lifecycle is handled by the netty itself @@ -108,7 +108,7 @@ trait ThreadAudit extends Logging { if (threadNamesSnapshot.nonEmpty) { val remainingThreadNames = runningThreadNames().diff(threadNamesSnapshot) - .filterNot { s => threadWhiteList.exists(s.matches(_)) } + .filterNot { s => threadExcludeList.exists(s.matches(_)) } if (remainingThreadNames.nonEmpty) { logWarning(s"\n\n===== POSSIBLE THREAD LEAK IN SUITE $shortSuiteName, " + s"thread names: ${remainingThreadNames.mkString(", ")} =====\n") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index fd2d1f56ed9b6..fd3d4bcf62f69 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -1210,17 +1210,17 @@ class SparkSubmitSuite testRemoteResources(enableHttpFs = true) } - test("force download from blacklisted schemes") { - testRemoteResources(enableHttpFs = true, blacklistSchemes = Seq("http")) + test("force download from forced schemes") { + testRemoteResources(enableHttpFs = true, forceDownloadSchemes = Seq("http")) } test("force download for all the schemes") { - testRemoteResources(enableHttpFs = true, blacklistSchemes = Seq("*")) + testRemoteResources(enableHttpFs = true, forceDownloadSchemes = Seq("*")) } private def testRemoteResources( enableHttpFs: Boolean, - blacklistSchemes: Seq[String] = Nil): Unit = { + forceDownloadSchemes: Seq[String] = Nil): Unit = { val hadoopConf = new Configuration() updateConfWithFakeS3Fs(hadoopConf) if (enableHttpFs) { @@ -1237,8 +1237,8 @@ class SparkSubmitSuite val tmpHttpJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir) val tmpHttpJarPath = s"http://${new File(tmpHttpJar.toURI).getAbsolutePath}" - val forceDownloadArgs = if (blacklistSchemes.nonEmpty) { - Seq("--conf", s"spark.yarn.dist.forceDownloadSchemes=${blacklistSchemes.mkString(",")}") + val forceDownloadArgs = if (forceDownloadSchemes.nonEmpty) { + Seq("--conf", s"spark.yarn.dist.forceDownloadSchemes=${forceDownloadSchemes.mkString(",")}") } else { Nil } @@ -1256,19 +1256,19 @@ class SparkSubmitSuite val jars = conf.get("spark.yarn.dist.jars").split(",").toSet - def isSchemeBlacklisted(scheme: String) = { - blacklistSchemes.contains("*") || blacklistSchemes.contains(scheme) + def isSchemeForcedDownload(scheme: String) = { + forceDownloadSchemes.contains("*") || forceDownloadSchemes.contains(scheme) } - if (!isSchemeBlacklisted("s3")) { + if (!isSchemeForcedDownload("s3")) { assert(jars.contains(tmpS3JarPath)) } - if (enableHttpFs && blacklistSchemes.isEmpty) { + if (enableHttpFs && forceDownloadSchemes.isEmpty) { // If Http FS is supported by yarn service, the URI of remote http resource should // still be remote. assert(jars.contains(tmpHttpJarPath)) - } else if (!enableHttpFs || isSchemeBlacklisted("http")) { + } else if (!enableHttpFs || isSchemeForcedDownload("http")) { // If Http FS is not supported by yarn service, or http scheme is configured to be force // downloading, the URI of remote http resource should be changed to a local one. val jarName = new File(tmpHttpJar.toURI).getName diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index c2f34fc3a95ed..ade03a0095c19 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -1117,7 +1117,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } - test("SPARK-24948: blacklist files we don't have read permission on") { + test("SPARK-24948: ignore files we don't have read permission on") { val clock = new ManualClock(1533132471) val provider = new FsHistoryProvider(createTestConf(), clock) val accessDenied = newLogFile("accessDenied", None, inProgress = false) @@ -1137,17 +1137,17 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { updateAndCheck(mockedProvider) { list => list.size should be(1) } - // Doing 2 times in order to check the blacklist filter too + // Doing 2 times in order to check the inaccessibleList filter too updateAndCheck(mockedProvider) { list => list.size should be(1) } val accessDeniedPath = new Path(accessDenied.getPath) - assert(mockedProvider.isBlacklisted(accessDeniedPath)) + assert(!mockedProvider.isAccessible(accessDeniedPath)) clock.advance(24 * 60 * 60 * 1000 + 1) // add a bit more than 1d isReadable = true mockedProvider.cleanLogs() updateAndCheck(mockedProvider) { list => - assert(!mockedProvider.isBlacklisted(accessDeniedPath)) + assert(mockedProvider.isAccessible(accessDeniedPath)) assert(list.exists(_.name == "accessDenied")) assert(list.exists(_.name == "accessGranted")) list.size should be(2) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index ecfdf481f4f6c..4f808f03e5dab 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -48,24 +48,28 @@ import org.apache.spark.util.CallSite private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler { - private val cssWhiteList = List("bootstrap.min.css", "vis-timeline-graph2d.min.css") + /** + * Some libraries have warn/error messages that are too noisy for the tests; exclude them from + * normal error handling to avoid logging these. + */ + private val cssExcludeList = List("bootstrap.min.css", "vis-timeline-graph2d.min.css") - private def isInWhileList(uri: String): Boolean = cssWhiteList.exists(uri.endsWith) + private def isInExcludeList(uri: String): Boolean = cssExcludeList.exists(uri.endsWith) override def warning(e: CSSParseException): Unit = { - if (!isInWhileList(e.getURI)) { + if (!isInExcludeList(e.getURI)) { super.warning(e) } } override def fatalError(e: CSSParseException): Unit = { - if (!isInWhileList(e.getURI)) { + if (!isInExcludeList(e.getURI)) { super.fatalError(e) } } override def error(e: CSSParseException): Unit = { - if (!isInWhileList(e.getURI)) { + if (!isInExcludeList(e.getURI)) { super.error(e) } } diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 2e8b8b57a5b4f..3c438e309c22d 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -32,7 +32,7 @@ class Module(object): """ def __init__(self, name, dependencies, source_file_regexes, build_profile_flags=(), environ={}, - sbt_test_goals=(), python_test_goals=(), blacklisted_python_implementations=(), + sbt_test_goals=(), python_test_goals=(), excluded_python_implementations=(), test_tags=(), should_run_r_tests=False, should_run_build_tests=False): """ Define a new module. @@ -49,7 +49,7 @@ def __init__(self, name, dependencies, source_file_regexes, build_profile_flags= module are changed. :param sbt_test_goals: A set of SBT test goals for testing this module. :param python_test_goals: A set of Python test goals for testing this module. - :param blacklisted_python_implementations: A set of Python implementations that are not + :param excluded_python_implementations: A set of Python implementations that are not supported by this module's Python components. The values in this set should match strings returned by Python's `platform.python_implementation()`. :param test_tags A set of tags that will be excluded when running unit tests if the module @@ -64,7 +64,7 @@ def __init__(self, name, dependencies, source_file_regexes, build_profile_flags= self.build_profile_flags = build_profile_flags self.environ = environ self.python_test_goals = python_test_goals - self.blacklisted_python_implementations = blacklisted_python_implementations + self.excluded_python_implementations = excluded_python_implementations self.test_tags = test_tags self.should_run_r_tests = should_run_r_tests self.should_run_build_tests = should_run_build_tests @@ -524,7 +524,7 @@ def __hash__(self): "pyspark.mllib.tests.test_streaming_algorithms", "pyspark.mllib.tests.test_util", ], - blacklisted_python_implementations=[ + excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy and it isn't available there ] ) @@ -565,7 +565,7 @@ def __hash__(self): "pyspark.ml.tests.test_tuning", "pyspark.ml.tests.test_wrapper", ], - blacklisted_python_implementations=[ + excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy and it isn't available there ] ) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 587deebcc5de4..56a455a1b8d21 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1822,7 +1822,7 @@ This is shown in the following example.
{% highlight scala %} -object WordBlacklist { +object WordExcludeList { @volatile private var instance: Broadcast[Seq[String]] = null @@ -1830,8 +1830,8 @@ object WordBlacklist { if (instance == null) { synchronized { if (instance == null) { - val wordBlacklist = Seq("a", "b", "c") - instance = sc.broadcast(wordBlacklist) + val wordExcludeList = Seq("a", "b", "c") + instance = sc.broadcast(wordExcludeList) } } } @@ -1847,7 +1847,7 @@ object DroppedWordsCounter { if (instance == null) { synchronized { if (instance == null) { - instance = sc.longAccumulator("WordsInBlacklistCounter") + instance = sc.longAccumulator("DroppedWordsCounter") } } } @@ -1856,13 +1856,13 @@ object DroppedWordsCounter { } wordCounts.foreachRDD { (rdd: RDD[(String, Int)], time: Time) => - // Get or register the blacklist Broadcast - val blacklist = WordBlacklist.getInstance(rdd.sparkContext) + // Get or register the excludeList Broadcast + val excludeList = WordExcludeList.getInstance(rdd.sparkContext) // Get or register the droppedWordsCounter Accumulator val droppedWordsCounter = DroppedWordsCounter.getInstance(rdd.sparkContext) - // Use blacklist to drop words and use droppedWordsCounter to count them + // Use excludeList to drop words and use droppedWordsCounter to count them val counts = rdd.filter { case (word, count) => - if (blacklist.value.contains(word)) { + if (excludeList.value.contains(word)) { droppedWordsCounter.add(count) false } else { @@ -1879,16 +1879,16 @@ See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_
{% highlight java %} -class JavaWordBlacklist { +class JavaWordExcludeList { private static volatile Broadcast> instance = null; public static Broadcast> getInstance(JavaSparkContext jsc) { if (instance == null) { - synchronized (JavaWordBlacklist.class) { + synchronized (JavaWordExcludeList.class) { if (instance == null) { - List wordBlacklist = Arrays.asList("a", "b", "c"); - instance = jsc.broadcast(wordBlacklist); + List wordExcludeList = Arrays.asList("a", "b", "c"); + instance = jsc.broadcast(wordExcludeList); } } } @@ -1904,7 +1904,7 @@ class JavaDroppedWordsCounter { if (instance == null) { synchronized (JavaDroppedWordsCounter.class) { if (instance == null) { - instance = jsc.sc().longAccumulator("WordsInBlacklistCounter"); + instance = jsc.sc().longAccumulator("DroppedWordsCounter"); } } } @@ -1913,13 +1913,13 @@ class JavaDroppedWordsCounter { } wordCounts.foreachRDD((rdd, time) -> { - // Get or register the blacklist Broadcast - Broadcast> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); + // Get or register the excludeList Broadcast + Broadcast> excludeList = JavaWordExcludeList.getInstance(new JavaSparkContext(rdd.context())); // Get or register the droppedWordsCounter Accumulator LongAccumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); - // Use blacklist to drop words and use droppedWordsCounter to count them + // Use excludeList to drop words and use droppedWordsCounter to count them String counts = rdd.filter(wordCount -> { - if (blacklist.value().contains(wordCount._1())) { + if (excludeList.value().contains(wordCount._1())) { droppedWordsCounter.add(wordCount._2()); return false; } else { @@ -1935,10 +1935,10 @@ See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_
{% highlight python %} -def getWordBlacklist(sparkContext): - if ("wordBlacklist" not in globals()): - globals()["wordBlacklist"] = sparkContext.broadcast(["a", "b", "c"]) - return globals()["wordBlacklist"] +def getWordExcludeList(sparkContext): + if ("wordExcludeList" not in globals()): + globals()["wordExcludeList"] = sparkContext.broadcast(["a", "b", "c"]) + return globals()["wordExcludeList"] def getDroppedWordsCounter(sparkContext): if ("droppedWordsCounter" not in globals()): @@ -1946,14 +1946,14 @@ def getDroppedWordsCounter(sparkContext): return globals()["droppedWordsCounter"] def echo(time, rdd): - # Get or register the blacklist Broadcast - blacklist = getWordBlacklist(rdd.context) + # Get or register the excludeList Broadcast + excludeList = getWordExcludeList(rdd.context) # Get or register the droppedWordsCounter Accumulator droppedWordsCounter = getDroppedWordsCounter(rdd.context) - # Use blacklist to drop words and use droppedWordsCounter to count them + # Use excludeList to drop words and use droppedWordsCounter to count them def filterFunc(wordCount): - if wordCount[0] in blacklist.value: + if wordCount[0] in excludeList.value: droppedWordsCounter.add(wordCount[1]) False else: diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java index 45a876decff8b..c01a62b078f7a 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java @@ -41,16 +41,16 @@ /** * Use this singleton to get or register a Broadcast variable. */ -class JavaWordBlacklist { +class JavaWordExcludeList { private static volatile Broadcast> instance = null; public static Broadcast> getInstance(JavaSparkContext jsc) { if (instance == null) { - synchronized (JavaWordBlacklist.class) { + synchronized (JavaWordExcludeList.class) { if (instance == null) { - List wordBlacklist = Arrays.asList("a", "b", "c"); - instance = jsc.broadcast(wordBlacklist); + List wordExcludeList = Arrays.asList("a", "b", "c"); + instance = jsc.broadcast(wordExcludeList); } } } @@ -69,7 +69,7 @@ public static LongAccumulator getInstance(JavaSparkContext jsc) { if (instance == null) { synchronized (JavaDroppedWordsCounter.class) { if (instance == null) { - instance = jsc.sc().longAccumulator("WordsInBlacklistCounter"); + instance = jsc.sc().longAccumulator("DroppedWordsCounter"); } } } @@ -133,15 +133,15 @@ private static JavaStreamingContext createContext(String ip, .reduceByKey((i1, i2) -> i1 + i2); wordCounts.foreachRDD((rdd, time) -> { - // Get or register the blacklist Broadcast - Broadcast> blacklist = - JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); + // Get or register the excludeList Broadcast + Broadcast> excludeList = + JavaWordExcludeList.getInstance(new JavaSparkContext(rdd.context())); // Get or register the droppedWordsCounter Accumulator LongAccumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); - // Use blacklist to drop words and use droppedWordsCounter to count them + // Use excludeList to drop words and use droppedWordsCounter to count them String counts = rdd.filter(wordCount -> { - if (blacklist.value().contains(wordCount._1())) { + if (excludeList.value().contains(wordCount._1())) { droppedWordsCounter.add(wordCount._2()); return false; } else { diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py index 8424556e88818..6ebe91a2f47fe 100644 --- a/examples/src/main/python/streaming/recoverable_network_wordcount.py +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -43,10 +43,10 @@ # Get or register a Broadcast variable -def getWordBlacklist(sparkContext): - if ('wordBlacklist' not in globals()): - globals()['wordBlacklist'] = sparkContext.broadcast(["a", "b", "c"]) - return globals()['wordBlacklist'] +def getWordExcludeList(sparkContext): + if ('wordExcludeList' not in globals()): + globals()['wordExcludeList'] = sparkContext.broadcast(["a", "b", "c"]) + return globals()['wordExcludeList'] # Get or register an Accumulator @@ -72,14 +72,14 @@ def createContext(host, port, outputPath): wordCounts = words.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y) def echo(time, rdd): - # Get or register the blacklist Broadcast - blacklist = getWordBlacklist(rdd.context) + # Get or register the excludeList Broadcast + excludeList = getWordExcludeList(rdd.context) # Get or register the droppedWordsCounter Accumulator droppedWordsCounter = getDroppedWordsCounter(rdd.context) - # Use blacklist to drop words and use droppedWordsCounter to count them + # Use excludeList to drop words and use droppedWordsCounter to count them def filterFunc(wordCount): - if wordCount[0] in blacklist.value: + if wordCount[0] in excludeList.value: droppedWordsCounter.add(wordCount[1]) return False else: diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 243c22e71275c..ee3bbe40fbeed 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.{IntParam, LongAccumulator} /** * Use this singleton to get or register a Broadcast variable. */ -object WordBlacklist { +object WordExcludeList { @volatile private var instance: Broadcast[Seq[String]] = null @@ -40,8 +40,8 @@ object WordBlacklist { if (instance == null) { synchronized { if (instance == null) { - val wordBlacklist = Seq("a", "b", "c") - instance = sc.broadcast(wordBlacklist) + val wordExcludeList = Seq("a", "b", "c") + instance = sc.broadcast(wordExcludeList) } } } @@ -60,7 +60,7 @@ object DroppedWordsCounter { if (instance == null) { synchronized { if (instance == null) { - instance = sc.longAccumulator("WordsInBlacklistCounter") + instance = sc.longAccumulator("DroppedWordsCounter") } } } @@ -117,13 +117,13 @@ object RecoverableNetworkWordCount { val words = lines.flatMap(_.split(" ")) val wordCounts = words.map((_, 1)).reduceByKey(_ + _) wordCounts.foreachRDD { (rdd: RDD[(String, Int)], time: Time) => - // Get or register the blacklist Broadcast - val blacklist = WordBlacklist.getInstance(rdd.sparkContext) + // Get or register the excludeList Broadcast + val excludeList = WordExcludeList.getInstance(rdd.sparkContext) // Get or register the droppedWordsCounter Accumulator val droppedWordsCounter = DroppedWordsCounter.getInstance(rdd.sparkContext) - // Use blacklist to drop words and use droppedWordsCounter to count them + // Use excludeList to drop words and use droppedWordsCounter to count them val counts = rdd.filter { case (word, count) => - if (blacklist.value.contains(word)) { + if (excludeList.value.contains(word)) { droppedWordsCounter.add(count) false } else { diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/util/DockerUtils.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/util/DockerUtils.scala index fda377e032350..5abca8df77dcd 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/util/DockerUtils.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/util/DockerUtils.scala @@ -36,7 +36,7 @@ private[spark] object DockerUtils { .orElse(findFromDockerMachine()) .orElse(Try(Seq("/bin/bash", "-c", "boot2docker ip 2>/dev/null").!!.trim).toOption) .getOrElse { - // This block of code is based on Utils.findLocalInetAddress(), but is modified to blacklist + // This block of code is based on Utils.findLocalInetAddress(), but is modified to exclude // certain interfaces. val address = InetAddress.getLocalHost // Address resolves to something like 127.0.1.1, which happens on Debian; try to find @@ -44,12 +44,12 @@ private[spark] object DockerUtils { // getNetworkInterfaces returns ifs in reverse order compared to ifconfig output order // on unix-like system. On windows, it returns in index order. // It's more proper to pick ip address following system output order. - val blackListedIFs = Seq( + val excludedIFs = Seq( "vboxnet0", // Mac "docker0" // Linux ) val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.asScala.toSeq.filter { i => - !blackListedIFs.contains(i.getName) + !excludedIFs.contains(i.getName) } val reOrderedNetworkIFs = activeNetworkIFs.reverse for (ni <- reOrderedNetworkIFs) { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5184fd85e422f..d19b514d662fa 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -444,10 +444,10 @@ object SparkBuild extends PomBuild { object SparkParallelTestGrouping { // Settings for parallelizing tests. The basic strategy here is to run the slowest suites (or // collections of suites) in their own forked JVMs, allowing us to gain parallelism within a - // SBT project. Here, we take a whitelisting approach where the default behavior is to run all + // SBT project. Here, we take an opt-in approach where the default behavior is to run all // tests sequentially in a single JVM, requiring us to manually opt-in to the extra parallelism. // - // There are a reasons why such a whitelist approach is good: + // There are a reasons why such an opt-in approach is good: // // 1. Launching one JVM per suite adds significant overhead for short-running suites. In // addition to JVM startup time and JIT warmup, it appears that initialization of Derby diff --git a/python/pylintrc b/python/pylintrc index 6a675770da69a..26d2741d3b56f 100644 --- a/python/pylintrc +++ b/python/pylintrc @@ -27,7 +27,7 @@ # Profiled execution. profile=no -# Add files or directories to the blacklist. They should be base names, not +# Add files or directories to the ignoreList. They should be base names, not # paths. ignore=pyspark.heapq3 diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 09d3a5e7cfb6f..af49c77a2d98c 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -87,8 +87,8 @@ PY2 = True PY2_WRAPPER_DESCRIPTOR_TYPE = type(object.__init__) PY2_METHOD_WRAPPER_TYPE = type(object.__eq__) - PY2_CLASS_DICT_BLACKLIST = (PY2_METHOD_WRAPPER_TYPE, - PY2_WRAPPER_DESCRIPTOR_TYPE) + PY2_CLASS_DICT_SKIP_PICKLE_METHOD_TYPE = (PY2_METHOD_WRAPPER_TYPE, + PY2_WRAPPER_DESCRIPTOR_TYPE) else: types.ClassType = type from pickle import _Pickler as Pickler @@ -327,7 +327,7 @@ def _extract_class_dict(cls): if hasattr(value, "im_func"): if value.im_func is getattr(base_value, "im_func", None): to_remove.append(name) - elif isinstance(value, PY2_CLASS_DICT_BLACKLIST): + elif isinstance(value, PY2_CLASS_DICT_SKIP_PICKLE_METHOD_TYPE): # On Python 2 we have no way to pickle those specific # methods types nor to check that they are actually # inherited. So we assume that they are always inherited diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 63b049999f255..5a352104c4eca 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -3411,9 +3411,9 @@ def udf(f=None, returnType=StringType()): evalType=PythonEvalType.SQL_BATCHED_UDF) -blacklist = ['map', 'since'] +ignored_fns = ['map', 'since'] __all__ = [k for k, v in globals().items() - if not k.startswith('_') and k[0].islower() and callable(v) and k not in blacklist] + if not k.startswith('_') and k[0].islower() and callable(v) and k not in ignored_fns] __all__ += ["PandasUDFType"] __all__.sort() diff --git a/python/pyspark/sql/pandas/typehints.py b/python/pyspark/sql/pandas/typehints.py index b0323ba1697df..e696f677cd154 100644 --- a/python/pyspark/sql/pandas/typehints.py +++ b/python/pyspark/sql/pandas/typehints.py @@ -98,8 +98,8 @@ def infer_eval_type(sig): a, parameter_check_func=lambda ua: ua == pd.Series or ua == pd.DataFrame) for a in parameters_sig) and ( - # It's tricky to whitelist which types pd.Series constructor can take. - # Simply blacklist common types used here for now (which becomes object + # It's tricky to include only types which pd.Series constructor can take. + # Simply exclude common types used here for now (which becomes object # types Spark can't recognize). return_annotation != pd.Series and return_annotation != pd.DataFrame and diff --git a/python/run-tests.py b/python/run-tests.py index 357eb8f449beb..db62f964791ac 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -268,7 +268,7 @@ def main(): [python_exec, "--version"], stderr=subprocess.STDOUT, universal_newlines=True).strip()) if should_test_modules: for module in modules_to_test: - if python_implementation not in module.blacklisted_python_implementations: + if python_implementation not in module.excluded_python_implementations: for test_goal in module.python_test_goals: heavy_tests = ['pyspark.streaming.tests', 'pyspark.mllib.tests', 'pyspark.tests', 'pyspark.sql.tests', 'pyspark.ml.tests'] diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index edcdb923ee55d..26939ef23eaab 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -539,14 +539,14 @@ private[spark] class MesosClusterScheduler( options ++= Seq("--py-files", formattedFiles) // --conf - val replicatedOptionsBlacklist = Set( + val replicatedOptionsExcludeList = Set( JARS.key, // Avoids duplicate classes in classpath SUBMIT_DEPLOY_MODE.key, // this would be set to `cluster`, but we need client "spark.master" // this contains the address of the dispatcher, not master ) val defaultConf = conf.getAllWithPrefix(config.DISPATCHER_DRIVER_DEFAULT_PREFIX).toMap val driverConf = desc.conf.getAll - .filter { case (key, _) => !replicatedOptionsBlacklist.contains(key) } + .filter { case (key, _) => !replicatedOptionsExcludeList.contains(key) } .toMap (defaultConf ++ driverConf).foreach { case (key, value) => options ++= Seq("--conf", s"${key}=${value}") } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 9cefc4011c930..9d6b776a69d85 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -107,7 +107,7 @@ object YarnSparkHadoopUtil { * Not killing the task leaves various aspects of the executor and (to some extent) the jvm in * an inconsistent state. * TODO: If the OOM is not recoverable by rescheduling it on different node, then do - * 'something' to fail job ... akin to blacklisting trackers in mapred ? + * 'something' to fail job ... akin to unhealthy trackers in mapred ? * * The handler if an OOM Exception is thrown by the JVM must be configured on Windows * differently: the 'taskkill' command should be used, whereas Unix-based systems use 'kill'. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 43dd0979eed7f..351be32ee438e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -867,7 +867,7 @@ trait CheckAnalysis extends PredicateHelper { // Simplify the predicates before validating any unsupported correlation patterns in the plan. AnalysisHelper.allowInvokingTransformsInAnalyzer { BooleanSimplification(sub).foreachUp { - // Whitelist operators allowed in a correlated subquery + // Approve operators allowed in a correlated subquery // There are 4 categories: // 1. Operators that are allowed anywhere in a correlated subquery, and, // by definition of the operators, they either do not contain diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 70a673bb42457..c145f26472355 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -168,10 +168,10 @@ private[sql] class JSONOptionsInRead( } protected override def checkedEncoding(enc: String): String = { - val isBlacklisted = JSONOptionsInRead.blacklist.contains(Charset.forName(enc)) - require(multiLine || !isBlacklisted, - s"""The ${enc} encoding must not be included in the blacklist when multiLine is disabled: - |Blacklist: ${JSONOptionsInRead.blacklist.mkString(", ")}""".stripMargin) + val isDenied = JSONOptionsInRead.denyList.contains(Charset.forName(enc)) + require(multiLine || !isDenied, + s"""The $enc encoding must not be included in the denyList when multiLine is disabled: + |denylist: ${JSONOptionsInRead.denyList.mkString(", ")}""".stripMargin) val isLineSepRequired = multiLine || Charset.forName(enc) == StandardCharsets.UTF_8 || lineSeparator.nonEmpty @@ -188,7 +188,7 @@ private[sql] object JSONOptionsInRead { // only the first lines will have the BOM which leads to impossibility for reading // the rest lines. Besides of that, the lineSep option must have the BOM in such // encodings which can never present between lines. - val blacklist = Seq( + val denyList = Seq( Charset.forName("UTF-16"), Charset.forName("UTF-32") ) 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 e481cdbd5fdf4..33da482c4eea4 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 @@ -48,7 +48,7 @@ abstract class Optimizer(catalogManager: CatalogManager) plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty) } - override protected val blacklistedOnceBatches: Set[String] = + override protected val excludedOnceBatches: Set[String] = Set( "PartitionPruning", "Extract Python UDFs", @@ -373,38 +373,38 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { /** * Remove the top-level alias from an expression when it is redundant. */ - private def removeRedundantAlias(e: Expression, blacklist: AttributeSet): Expression = e match { + private def removeRedundantAlias(e: Expression, excludeList: AttributeSet): Expression = e match { // Alias with metadata can not be stripped, or the metadata will be lost. // If the alias name is different from attribute name, we can't strip it either, or we // may accidentally change the output schema name of the root plan. case a @ Alias(attr: Attribute, name) if a.metadata == Metadata.empty && name == attr.name && - !blacklist.contains(attr) && - !blacklist.contains(a) => + !excludeList.contains(attr) && + !excludeList.contains(a) => attr case a => a } /** - * Remove redundant alias expression from a LogicalPlan and its subtree. A blacklist is used to - * prevent the removal of seemingly redundant aliases used to deduplicate the input for a (self) - * join or to prevent the removal of top-level subquery attributes. + * Remove redundant alias expression from a LogicalPlan and its subtree. A set of excludes is used + * to prevent the removal of seemingly redundant aliases used to deduplicate the input for a + * (self) join or to prevent the removal of top-level subquery attributes. */ - private def removeRedundantAliases(plan: LogicalPlan, blacklist: AttributeSet): LogicalPlan = { + private def removeRedundantAliases(plan: LogicalPlan, excluded: AttributeSet): LogicalPlan = { plan match { // We want to keep the same output attributes for subqueries. This means we cannot remove // the aliases that produce these attributes case Subquery(child, correlated) => - Subquery(removeRedundantAliases(child, blacklist ++ child.outputSet), correlated) + Subquery(removeRedundantAliases(child, excluded ++ child.outputSet), correlated) // A join has to be treated differently, because the left and the right side of the join are - // not allowed to use the same attributes. We use a blacklist to prevent us from creating a - // situation in which this happens; the rule will only remove an alias if its child + // not allowed to use the same attributes. We use an exclude list to prevent us from creating + // a situation in which this happens; the rule will only remove an alias if its child // attribute is not on the black list. case Join(left, right, joinType, condition, hint) => - val newLeft = removeRedundantAliases(left, blacklist ++ right.outputSet) - val newRight = removeRedundantAliases(right, blacklist ++ newLeft.outputSet) + val newLeft = removeRedundantAliases(left, excluded ++ right.outputSet) + val newRight = removeRedundantAliases(right, excluded ++ newLeft.outputSet) val mapping = AttributeMap( createAttributeMapping(left, newLeft) ++ createAttributeMapping(right, newRight)) @@ -417,7 +417,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { // Remove redundant aliases in the subtree(s). val currentNextAttrPairs = mutable.Buffer.empty[(Attribute, Attribute)] val newNode = plan.mapChildren { child => - val newChild = removeRedundantAliases(child, blacklist) + val newChild = removeRedundantAliases(child, excluded) currentNextAttrPairs ++= createAttributeMapping(child, newChild) newChild } @@ -430,9 +430,9 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { // Create a an expression cleaning function for nodes that can actually produce redundant // aliases, use identity otherwise. val clean: Expression => Expression = plan match { - case _: Project => removeRedundantAlias(_, blacklist) - case _: Aggregate => removeRedundantAlias(_, blacklist) - case _: Window => removeRedundantAlias(_, blacklist) + case _: Project => removeRedundantAlias(_, excluded) + case _: Aggregate => removeRedundantAlias(_, excluded) + case _: Window => removeRedundantAlias(_, excluded) case _ => identity[Expression] } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 759ce5718ed2a..773ee7708aea3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -677,7 +677,7 @@ object FoldablePropagation extends Rule[LogicalPlan] { } /** - * Whitelist of all [[UnaryNode]]s for which allow foldable propagation. + * List of all [[UnaryNode]]s which allow foldable propagation. */ private def canPropagateFoldables(u: UnaryNode): Boolean = u match { case _: Project => true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 039fd9382000a..f1a363cca752e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1028,7 +1028,7 @@ case class Deduplicate( /** * A trait to represent the commands that support subqueries. - * This is used to whitelist such commands in the subquery-related checks. + * This is used to allow such commands in the subquery-related checks. */ trait SupportsSubquery extends LogicalPlan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index bff04d317d4d2..2109e8f355c5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -80,8 +80,8 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { /** Defines a sequence of rule batches, to be overridden by the implementation. */ protected def batches: Seq[Batch] - /** Once batches that are blacklisted in the idempotence checker */ - protected val blacklistedOnceBatches: Set[String] = Set.empty + /** Once batches that are excluded in the idempotence checker */ + protected val excludedOnceBatches: Set[String] = Set.empty /** * Defines a check function that checks for structural integrity of the plan after the execution @@ -189,7 +189,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { } // Check idempotence for Once batches. if (batch.strategy == Once && - Utils.isTesting && !blacklistedOnceBatches.contains(batch.name)) { + Utils.isTesting && !excludedOnceBatches.contains(batch.name)) { checkBatchIdempotence(batch, curPlan) } continue = false diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index bb8f5f90f8508..bb7e9d04c12d9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -33,7 +33,7 @@ class FilterPushdownSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { - override protected val blacklistedOnceBatches: Set[String] = + override protected val excludedOnceBatches: Set[String] = Set("Push CNF predicate through join") val batches = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesSuite.scala index 2d86d5a97e769..e7775705edc5e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor class PullupCorrelatedPredicatesSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { - override protected val blacklistedOnceBatches = Set("PullupCorrelatedPredicates") + override protected val excludedOnceBatches = Set("PullupCorrelatedPredicates") val batches = Batch("PullupCorrelatedPredicates", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala index dfd84e344eb2a..719d72f5b9b52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala @@ -39,7 +39,7 @@ class JsonOutputWriter( case None => StandardCharsets.UTF_8 } - if (JSONOptionsInRead.blacklist.contains(encoding)) { + if (JSONOptionsInRead.denyList.contains(encoding)) { logWarning(s"The JSON file ($path) was written in the encoding ${encoding.displayName()}" + " which can be read back by Spark only if multiLine is enabled.") } diff --git a/sql/core/src/test/resources/sql-tests/inputs/blacklist.sql b/sql/core/src/test/resources/sql-tests/inputs/ignored.sql similarity index 75% rename from sql/core/src/test/resources/sql-tests/inputs/blacklist.sql rename to sql/core/src/test/resources/sql-tests/inputs/ignored.sql index d69f8147a5264..0b5331a1d0229 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/blacklist.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ignored.sql @@ -1,4 +1,4 @@ --- This is a query file that has been blacklisted. +-- This is a query file that has been ignored. -- It includes a query that should crash Spark. -- If the test case is run, the whole suite would fail. some random not working query that should crash Spark. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index f0522dfeafaac..33247455b5cdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -153,8 +153,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper .set(SQLConf.SHUFFLE_PARTITIONS, 4) /** List of test cases to ignore, in lower cases. */ - protected def blackList: Set[String] = Set( - "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. + protected def ignoreList: Set[String] = Set( + "ignored.sql" // Do NOT remove this one. It is here to test the ignore functionality. ) // Create all the test cases. @@ -222,7 +222,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper name: String, inputFile: String, resultFile: String) extends TestCase with AnsiTest protected def createScalaTestCase(testCase: TestCase): Unit = { - if (blackList.exists(t => + if (ignoreList.exists(t => testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { // Create a test case to ignore this case. ignore(testCase.name) { /* Do nothing */ } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala index aacb625d7921f..d0d484ec434ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala @@ -85,7 +85,7 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSSchema { // List up the known queries having too large code in a generated function. // A JIRA file for `modified-q3` is as follows; // [SPARK-29128] Split predicate code in OR expressions - val blackListForMethodCodeSizeCheck = Set("modified-q3") + val excludeListForMethodCodeSizeCheck = Set("modified-q3") modifiedTPCDSQueries.foreach { name => val queryString = resourceToString(s"tpcds-modifiedQueries/$name.sql", @@ -94,7 +94,7 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSSchema { test(testName) { // check the plans can be properly generated val plan = sql(queryString).queryExecution.executedPlan - checkGeneratedCode(plan, !blackListForMethodCodeSizeCheck.contains(testName)) + checkGeneratedCode(plan, !excludeListForMethodCodeSizeCheck.contains(testName)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index c7448b12626be..de01099f2db55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2488,7 +2488,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson .json(testFile("test-data/utf16LE.json")) .count() } - assert(exception.getMessage.contains("encoding must not be included in the blacklist")) + assert(exception.getMessage.contains("encoding must not be included in the denyList")) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 686dc1c9bad6b..30d911becdba7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -67,7 +67,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ } /** List of test cases to ignore, in lower cases. */ - override def blackList: Set[String] = super.blackList ++ Set( + override def ignoreList: Set[String] = super.ignoreList ++ Set( // Missing UDF "postgreSQL/boolean.sql", "postgreSQL/case.sql", @@ -208,7 +208,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ } override def createScalaTestCase(testCase: TestCase): Unit = { - if (blackList.exists(t => + if (ignoreList.exists(t => testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { // Create a test case to ignore this case. ignore(testCase.name) { /* Do nothing */ } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index d1ed55a514e7c..b7ea0630dd85f 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -85,7 +85,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { } /** A list of tests deemed out of scope currently and thus completely disregarded. */ - override def blackList: Seq[String] = Seq( + override def excludeList: Seq[String] = Seq( // These tests use hooks that are not on the classpath and thus break all subsequent execution. "hook_order", "hook_context_cs", @@ -516,7 +516,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // This test uses CREATE EXTERNAL TABLE without specifying LOCATION "alter2", - // [SPARK-16248][SQL] Whitelist the list of Hive fallback functions + // [SPARK-16248][SQL] Include the list of Hive fallback functions "udf_field", "udf_reflect2", "udf_xpath", @@ -604,7 +604,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_radians" ) - private def commonWhiteList = Seq( + private def commonIncludeList = Seq( "add_part_exist", "add_part_multiple", "add_partition_no_whitelist", @@ -1142,14 +1142,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { ) /** - * The set of tests that are believed to be working in catalyst. Tests not on whiteList or - * blacklist are implicitly marked as ignored. + * The set of tests that are believed to be working in catalyst. Tests not on includeList or + * excludeList are implicitly marked as ignored. */ - override def whiteList: Seq[String] = if (HiveUtils.isHive23) { - commonWhiteList ++ Seq( + override def includeList: Seq[String] = if (HiveUtils.isHive23) { + commonIncludeList ++ Seq( "decimal_1_1" ) } else { - commonWhiteList + commonIncludeList } } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index 2c0970c85449f..1b801ad69564c 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -763,7 +763,7 @@ class HiveWindowFunctionQueryFileSuite } } - override def blackList: Seq[String] = Seq( + override def excludeList: Seq[String] = Seq( // Partitioned table functions are not supported. "ptf*", // tests of windowing.q are in HiveWindowFunctionQueryBaseSuite @@ -791,12 +791,12 @@ class HiveWindowFunctionQueryFileSuite "windowing_adjust_rowcontainer_sz" ) - override def whiteList: Seq[String] = Seq( + override def includeList: Seq[String] = Seq( "windowing_udaf2" ) - // Only run those query tests in the realWhileList (do not try other ignored query files). + // Only run those query tests in the realIncludeList (do not try other ignored query files). override def testCases: Seq[(String, File)] = super.testCases.filter { - case (name, _) => realWhiteList.contains(name) + case (name, _) => realIncludeList.contains(name) } } diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_includelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_includelist.q new file mode 100644 index 0000000000000..17677122a1bca --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_includelist.q @@ -0,0 +1,7 @@ +SET hive.metastore.partition.name.whitelist.pattern=; +-- Test with no partition name include-list pattern + +CREATE TABLE part_noincludelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS part_noincludelist_test; + +ALTER TABLE part_noincludelist_test ADD PARTITION (ds='1,2,3,4'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q deleted file mode 100644 index f51c53c2ff627..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q +++ /dev/null @@ -1,7 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=; --- Test with no partition name whitelist pattern - -CREATE TABLE part_nowhitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); -SHOW PARTITIONS part_nowhitelist_test; - -ALTER TABLE part_nowhitelist_test ADD PARTITION (ds='1,2,3,4'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_includelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_includelist.q new file mode 100644 index 0000000000000..7e7f30dc37305 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_includelist.q @@ -0,0 +1,7 @@ +SET hive.metastore.partition.name.whitelist.pattern=[A-Za-z]*; +-- This pattern matches only letters. + +CREATE TABLE part_includelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS part_includelist_test; + +ALTER TABLE part_includelist_test ADD PARTITION (ds='Part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q deleted file mode 100644 index 009c7610ef917..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q +++ /dev/null @@ -1,9 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=[A-Za-z]*; --- This pattern matches only letters. - -CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); -SHOW PARTITIONS part_whitelist_test; - -ALTER TABLE part_whitelist_test ADD PARTITION (ds='Part'); - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_includelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_includelist.q new file mode 100644 index 0000000000000..fcef12cbaac4e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_includelist.q @@ -0,0 +1,9 @@ +SET hive.metastore.partition.name.whitelist.pattern=[A-Za-z]*; +-- This pattern matches only letters. + +CREATE TABLE part_includelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS part_includelist_test; + +ALTER TABLE part_includelist_test ADD PARTITION (ds='Part'); + +ALTER TABLE part_includelist_test PARTITION (ds='Part') rename to partition (ds='Apart'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q deleted file mode 100644 index 301362a881456..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q +++ /dev/null @@ -1,9 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=[A-Za-z]*; --- This pattern matches only letters. - -CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); -SHOW PARTITIONS part_whitelist_test; - -ALTER TABLE part_whitelist_test ADD PARTITION (ds='Part'); - -ALTER TABLE part_whitelist_test PARTITION (ds='Part') rename to partition (ds='Apart'); diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 1dd2ad3837cc3..87771eed17b1b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -781,7 +781,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te assert(math.abs(corr6 + 1.0) < 1e-12) // Test for udaf_corr in HiveCompatibilitySuite - // udaf_corr has been blacklisted due to numerical errors + // udaf_corr has been excluded due to numerical errors // We test it here: // SELECT corr(b, c) FROM covar_tab WHERE a < 1; => NULL // SELECT corr(b, c) FROM covar_tab WHERE a < 3; => NULL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index bb4ce6d3aa3f1..192fff2b98879 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -25,18 +25,18 @@ import org.apache.spark.sql.catalyst.util._ * A framework for running the query tests that are listed as a set of text files. * * TestSuites that derive from this class must provide a map of testCaseName to testCaseFiles - * that should be included. Additionally, there is support for whitelisting and blacklisting + * that should be included. Additionally, there is support for including and excluding * tests as development progresses. */ abstract class HiveQueryFileTest extends HiveComparisonTest { /** A list of tests deemed out of scope and thus completely disregarded */ - def blackList: Seq[String] = Nil + def excludeList: Seq[String] = Nil /** - * The set of tests that are believed to be working in catalyst. Tests not in whiteList - * blacklist are implicitly marked as ignored. + * The set of tests that are believed to be working in catalyst. Tests not in includeList or + * excludeList are implicitly marked as ignored. */ - def whiteList: Seq[String] = ".*" :: Nil + def includeList: Seq[String] = ".*" :: Nil def testCases: Seq[(String, File)] @@ -45,25 +45,34 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { runOnlyDirectories.nonEmpty || skipDirectories.nonEmpty - val whiteListProperty: String = "spark.hive.whitelist" - // Allow the whiteList to be overridden by a system property - val realWhiteList: Seq[String] = - Option(System.getProperty(whiteListProperty)).map(_.split(",").toSeq).getOrElse(whiteList) + val deprecatedIncludeListProperty: String = "spark.hive.whitelist" + val includeListProperty: String = "spark.hive.includelist" + if (System.getProperty(deprecatedIncludeListProperty) != null) { + logWarning(s"System property `$deprecatedIncludeListProperty` is deprecated; please update " + + s"to use new property: $includeListProperty") + } + // Allow the includeList to be overridden by a system property + val realIncludeList: Seq[String] = + Option(System.getProperty(includeListProperty)) + .orElse(Option(System.getProperty(deprecatedIncludeListProperty))) + .map(_.split(",").toSeq) + .getOrElse(includeList) // Go through all the test cases and add them to scala test. testCases.sorted.foreach { case (testCaseName, testCaseFile) => - if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { - logDebug(s"Blacklisted test skipped $testCaseName") - } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || + if (excludeList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { + logDebug(s"Excluded test skipped $testCaseName") + } else if ( + realIncludeList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) createQueryTest(testCaseName, queriesString, reset = true, tryWithoutResettingFirst = true) } else { - // Only output warnings for the built in whitelist as this clutters the output when the user - // trying to execute a single test from the commandline. - if (System.getProperty(whiteListProperty) == null && !runAll) { + // Only output warnings for the built in includeList as this clutters the output when the + // user is trying to execute a single test from the commandline. + if (System.getProperty(includeListProperty) == null && !runAll) { ignore(testCaseName) {} } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala index 97eab4f3f4f77..f8f555197daef 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala @@ -52,8 +52,8 @@ class HiveHadoopDelegationTokenManagerSuite extends SparkFunSuite { throw new ClassNotFoundException(name) } - val prefixBlacklist = Seq("java", "scala", "com.sun.", "sun.") - if (prefixBlacklist.exists(name.startsWith(_))) { + val prefixExcludeList = Seq("java", "scala", "com.sun.", "sun.") + if (prefixExcludeList.exists(name.startsWith(_))) { return currentLoader.loadClass(name) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index 6f3ee5cbaec63..2e5000159bcb7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -277,10 +277,10 @@ private[streaming] object FileBasedWriteAheadLog { } def getCallerName(): Option[String] = { - val blacklist = Seq("WriteAheadLog", "Logging", "java.lang", "scala.") + val ignoreList = Seq("WriteAheadLog", "Logging", "java.lang", "scala.") Thread.currentThread.getStackTrace() .map(_.getClassName) - .find { c => !blacklist.exists(c.contains) } + .find { c => !ignoreList.exists(c.contains) } .flatMap(_.split("\\.").lastOption) .flatMap(_.split("\\$\\$").headOption) } From b05f309bc9e51e8f7b480b5d176589773b5d59f7 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 15 Jul 2020 10:13:03 -0700 Subject: [PATCH 203/384] [SPARK-32140][ML][PYSPARK] Add training summary to FMClassificationModel ### What changes were proposed in this pull request? Add training summary for FMClassificationModel... ### Why are the changes needed? so that user can get the training process status, such as loss value of each iteration and total iteration number. ### Does this PR introduce _any_ user-facing change? Yes FMClassificationModel.summary FMClassificationModel.evaluate ### How was this patch tested? new tests Closes #28960 from huaxingao/fm_summary. Authored-by: Huaxin Gao Signed-off-by: Huaxin Gao --- .../ml/classification/FMClassifier.scala | 100 +++++++++++++++++- .../spark/ml/regression/FMRegressor.scala | 10 +- .../mllib/optimization/GradientDescent.scala | 45 +++++--- .../spark/mllib/optimization/LBFGS.scala | 11 +- .../ml/classification/FMClassifierSuite.scala | 26 +++++ python/pyspark/ml/classification.py | 48 ++++++++- .../pyspark/ml/tests/test_training_summary.py | 49 ++++++++- 7 files changed, 257 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala index 6ef42500f86f7..cc691d1c0c58c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala @@ -30,7 +30,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql._ import org.apache.spark.storage.StorageLevel /** @@ -212,14 +212,34 @@ class FMClassifier @Since("3.0.0") ( if (handlePersistence) data.persist(StorageLevel.MEMORY_AND_DISK) - val coefficients = trainImpl(data, numFeatures, LogisticLoss) + val (coefficients, objectiveHistory) = trainImpl(data, numFeatures, LogisticLoss) val (intercept, linear, factors) = splitCoefficients( coefficients, numFeatures, $(factorSize), $(fitIntercept), $(fitLinear)) if (handlePersistence) data.unpersist() - copyValues(new FMClassificationModel(uid, intercept, linear, factors)) + createModel(dataset, intercept, linear, factors, objectiveHistory) + } + + private def createModel( + dataset: Dataset[_], + intercept: Double, + linear: Vector, + factors: Matrix, + objectiveHistory: Array[Double]): FMClassificationModel = { + val model = copyValues(new FMClassificationModel(uid, intercept, linear, factors)) + val weightColName = if (!isDefined(weightCol)) "weightCol" else $(weightCol) + + val (summaryModel, probabilityColName, predictionColName) = model.findSummaryModel() + val summary = new FMClassificationTrainingSummaryImpl( + summaryModel.transform(dataset), + probabilityColName, + predictionColName, + $(labelCol), + weightColName, + objectiveHistory) + model.setSummary(Some(summary)) } @Since("3.0.0") @@ -243,7 +263,8 @@ class FMClassificationModel private[classification] ( @Since("3.0.0") val linear: Vector, @Since("3.0.0") val factors: Matrix) extends ProbabilisticClassificationModel[Vector, FMClassificationModel] - with FMClassifierParams with MLWritable { + with FMClassifierParams with MLWritable + with HasTrainingSummary[FMClassificationTrainingSummary]{ @Since("3.0.0") override val numClasses: Int = 2 @@ -251,6 +272,27 @@ class FMClassificationModel private[classification] ( @Since("3.0.0") override val numFeatures: Int = linear.size + /** + * Gets summary of model on training set. An exception is thrown + * if `hasSummary` is false. + */ + @Since("3.1.0") + override def summary: FMClassificationTrainingSummary = super.summary + + /** + * Evaluates the model on a test dataset. + * + * @param dataset Test dataset to evaluate model on. + */ + @Since("3.1.0") + def evaluate(dataset: Dataset[_]): FMClassificationSummary = { + val weightColName = if (!isDefined(weightCol)) "weightCol" else $(weightCol) + // Handle possible missing or invalid probability or prediction columns + val (summaryModel, probability, predictionColName) = findSummaryModel() + new FMClassificationSummaryImpl(summaryModel.transform(dataset), + probability, predictionColName, $(labelCol), weightColName) + } + @Since("3.0.0") override def predictRaw(features: Vector): Vector = { val rawPrediction = getRawPrediction(features, intercept, linear, factors) @@ -328,3 +370,53 @@ object FMClassificationModel extends MLReadable[FMClassificationModel] { } } } + +/** + * Abstraction for FMClassifier results for a given model. + */ +sealed trait FMClassificationSummary extends BinaryClassificationSummary + +/** + * Abstraction for FMClassifier training results. + */ +sealed trait FMClassificationTrainingSummary extends FMClassificationSummary with TrainingSummary + +/** + * FMClassifier results for a given model. + * + * @param predictions dataframe output by the model's `transform` method. + * @param scoreCol field in "predictions" which gives the probability of each instance. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + */ +private class FMClassificationSummaryImpl( + @transient override val predictions: DataFrame, + override val scoreCol: String, + override val predictionCol: String, + override val labelCol: String, + override val weightCol: String) + extends FMClassificationSummary + +/** + * FMClassifier training results. + * + * @param predictions dataframe output by the model's `transform` method. + * @param scoreCol field in "predictions" which gives the probability of each instance. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. + */ +private class FMClassificationTrainingSummaryImpl( + predictions: DataFrame, + scoreCol: String, + predictionCol: String, + labelCol: String, + weightCol: String, + override val objectiveHistory: Array[Double]) + extends FMClassificationSummaryImpl( + predictions, scoreCol, predictionCol, labelCol, weightCol) + with FMClassificationTrainingSummary diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala index b9307ebb37ac2..84c0985245a2e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala @@ -47,7 +47,7 @@ import org.apache.spark.storage.StorageLevel */ private[ml] trait FactorizationMachinesParams extends PredictorParams with HasMaxIter with HasStepSize with HasTol with HasSolver with HasSeed - with HasFitIntercept with HasRegParam { + with HasFitIntercept with HasRegParam with HasWeightCol { /** * Param for dimensionality of the factors (>= 0) @@ -134,7 +134,7 @@ private[ml] trait FactorizationMachines extends FactorizationMachinesParams { data: RDD[(Double, OldVector)], numFeatures: Int, loss: String - ): Vector = { + ): (Vector, Array[Double]) = { // initialize coefficients val initialCoefficients = initCoefficients(numFeatures) @@ -151,8 +151,8 @@ private[ml] trait FactorizationMachines extends FactorizationMachinesParams { .setRegParam($(regParam)) .setMiniBatchFraction($(miniBatchFraction)) .setConvergenceTol($(tol)) - val coefficients = optimizer.optimize(data, initialCoefficients) - coefficients.asML + val (coefficients, lossHistory) = optimizer.optimizeWithLossReturned(data, initialCoefficients) + (coefficients.asML, lossHistory) } } @@ -421,7 +421,7 @@ class FMRegressor @Since("3.0.0") ( if (handlePersistence) data.persist(StorageLevel.MEMORY_AND_DISK) - val coefficients = trainImpl(data, numFeatures, SquaredError) + val (coefficients, _) = trainImpl(data, numFeatures, SquaredError) val (intercept, linear, factors) = splitCoefficients( coefficients, numFeatures, $(factorSize), $(fitIntercept), $(fitLinear)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 1336ffd2f7d5e..796a787e77db4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -129,7 +129,20 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va * @return solution vector */ def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { - val (weights, _) = GradientDescent.runMiniBatchSGD( + val (weights, _) = optimizeWithLossReturned(data, initialWeights) + weights + } + + /** + * Runs gradient descent on the given training data. + * @param data training data + * @param initialWeights initial weights + * @return solution vector and loss value in an array + */ + def optimizeWithLossReturned( + data: RDD[(Double, Vector)], + initialWeights: Vector): (Vector, Array[Double]) = { + GradientDescent.runMiniBatchSGD( data, gradient, updater, @@ -139,7 +152,6 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va miniBatchFraction, initialWeights, convergenceTol) - weights } } @@ -195,7 +207,7 @@ object GradientDescent extends Logging { s"numIterations=$numIterations and miniBatchFraction=$miniBatchFraction") } - val stochasticLossHistory = new ArrayBuffer[Double](numIterations) + val stochasticLossHistory = new ArrayBuffer[Double](numIterations + 1) // Record previous weight and current one to calculate solution vector difference var previousWeights: Option[Vector] = None @@ -226,7 +238,7 @@ object GradientDescent extends Logging { var converged = false // indicates whether converged based on convergenceTol var i = 1 - while (!converged && i <= numIterations) { + while (!converged && (i <= numIterations + 1)) { val bcWeights = data.context.broadcast(weights) // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) @@ -249,17 +261,19 @@ object GradientDescent extends Logging { * and regVal is the regularization value computed in the previous iteration as well. */ stochasticLossHistory += lossSum / miniBatchSize + regVal - val update = updater.compute( - weights, Vectors.fromBreeze(gradientSum / miniBatchSize.toDouble), - stepSize, i, regParam) - weights = update._1 - regVal = update._2 - - previousWeights = currentWeights - currentWeights = Some(weights) - if (previousWeights != None && currentWeights != None) { - converged = isConverged(previousWeights.get, - currentWeights.get, convergenceTol) + if (i != (numIterations + 1)) { + val update = updater.compute( + weights, Vectors.fromBreeze(gradientSum / miniBatchSize.toDouble), + stepSize, i, regParam) + weights = update._1 + regVal = update._2 + + previousWeights = currentWeights + currentWeights = Some(weights) + if (previousWeights != None && currentWeights != None) { + converged = isConverged(previousWeights.get, + currentWeights.get, convergenceTol) + } } } else { logWarning(s"Iteration ($i/$numIterations). The size of sampled batch is zero") @@ -271,7 +285,6 @@ object GradientDescent extends Logging { stochasticLossHistory.takeRight(10).mkString(", "))) (weights, stochasticLossHistory.toArray) - } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 1ee9241104f87..4fc297560c088 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -136,7 +136,14 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) } override def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { - val (weights, _) = LBFGS.runLBFGS( + val (weights, _) = optimizeWithLossReturned(data, initialWeights) + weights + } + + def optimizeWithLossReturned( + data: RDD[(Double, Vector)], + initialWeights: Vector): (Vector, Array[Double]) = { + LBFGS.runLBFGS( data, gradient, updater, @@ -145,9 +152,7 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) maxNumIterations, regParam, initialWeights) - weights } - } /** diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala index d477049824b19..9a04bdc39718c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala @@ -194,6 +194,32 @@ class FMClassifierSuite extends MLTest with DefaultReadWriteTest { testPredictionModelSinglePrediction(fmModel, smallBinaryDataset) } + test("summary and training summary") { + val fm = new FMClassifier() + val model = fm.setMaxIter(5).fit(smallBinaryDataset) + + val summary = model.evaluate(smallBinaryDataset) + + assert(model.summary.accuracy === summary.accuracy) + assert(model.summary.weightedPrecision === summary.weightedPrecision) + assert(model.summary.weightedRecall === summary.weightedRecall) + assert(model.summary.pr.collect() === summary.pr.collect()) + assert(model.summary.roc.collect() === summary.roc.collect()) + assert(model.summary.areaUnderROC === summary.areaUnderROC) + } + + test("FMClassifier training summary totalIterations") { + Seq(1, 5, 10, 20, 100).foreach { maxIter => + val trainer = new FMClassifier().setMaxIter(maxIter) + val model = trainer.fit(smallBinaryDataset) + if (maxIter == 1) { + assert(model.summary.totalIterations === maxIter) + } else { + assert(model.summary.totalIterations <= maxIter) + } + } + } + test("read/write") { def checkModelData( model: FMClassificationModel, diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 7c8cbe3a9fe3e..4f2d33adbc7e7 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -52,7 +52,8 @@ 'NaiveBayes', 'NaiveBayesModel', 'MultilayerPerceptronClassifier', 'MultilayerPerceptronClassificationModel', 'OneVsRest', 'OneVsRestModel', - 'FMClassifier', 'FMClassificationModel'] + 'FMClassifier', 'FMClassificationModel', 'FMClassificationSummary', + 'FMClassificationTrainingSummary'] class _ClassifierParams(HasRawPredictionCol, _PredictorParams): @@ -3226,7 +3227,7 @@ def setRegParam(self, value): class FMClassificationModel(_JavaProbabilisticClassificationModel, _FactorizationMachinesParams, - JavaMLWritable, JavaMLReadable): + JavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by :class:`FMClassifier`. @@ -3257,6 +3258,49 @@ def factors(self): """ return self._call_java("factors") + @since("3.1.0") + def summary(self): + """ + Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + trained on the training set. An exception is thrown if `trainingSummary is None`. + """ + if self.hasSummary: + return FMClassificationTrainingSummary(super(FMClassificationModel, self).summary) + else: + raise RuntimeError("No training summary available for this %s" % + self.__class__.__name__) + + @since("3.1.0") + def evaluate(self, dataset): + """ + Evaluates the model on a test dataset. + + :param dataset: + Test dataset to evaluate model on, where dataset is an + instance of :py:class:`pyspark.sql.DataFrame` + """ + if not isinstance(dataset, DataFrame): + raise ValueError("dataset must be a DataFrame but got %s." % type(dataset)) + java_fm_summary = self._call_java("evaluate", dataset) + return FMClassificationSummary(java_fm_summary) + + +class FMClassificationSummary(_BinaryClassificationSummary): + """ + Abstraction for FMClassifier Results for a given model. + .. versionadded:: 3.1.0 + """ + pass + + +@inherit_doc +class FMClassificationTrainingSummary(FMClassificationSummary, _TrainingSummary): + """ + Abstraction for FMClassifier Training results. + .. versionadded:: 3.1.0 + """ + pass + if __name__ == "__main__": import doctest diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index 15e9ebb0f5ce8..d305be8b96cd4 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -18,8 +18,9 @@ import sys import unittest -from pyspark.ml.classification import BinaryLogisticRegressionSummary, LinearSVC, \ - LinearSVCSummary, BinaryRandomForestClassificationSummary, LogisticRegression, \ +from pyspark.ml.classification import BinaryLogisticRegressionSummary, FMClassifier, \ + FMClassificationSummary, LinearSVC, LinearSVCSummary, \ + BinaryRandomForestClassificationSummary, LogisticRegression, \ LogisticRegressionSummary, RandomForestClassificationSummary, \ RandomForestClassifier from pyspark.ml.clustering import BisectingKMeans, GaussianMixture, KMeans @@ -309,6 +310,50 @@ def test_multiclass_randomforest_classification_summary(self): self.assertFalse(isinstance(sameSummary, BinaryRandomForestClassificationSummary)) self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) + def test_fm_classification_summary(self): + df = self.spark.createDataFrame([(1.0, Vectors.dense(2.0)), + (0.0, Vectors.dense(2.0)), + (0.0, Vectors.dense(6.0)), + (1.0, Vectors.dense(3.0)) + ], + ["label", "features"]) + fm = FMClassifier(maxIter=5) + model = fm.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary() + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.scoreCol, "probability") + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.predictionCol, "prediction") + objHist = s.objectiveHistory + self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) + self.assertGreater(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertTrue(isinstance(s.roc, DataFrame)) + self.assertAlmostEqual(s.areaUnderROC, 0.625, 2) + self.assertTrue(isinstance(s.pr, DataFrame)) + self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame)) + self.assertTrue(isinstance(s.precisionByThreshold, DataFrame)) + self.assertTrue(isinstance(s.recallByThreshold, DataFrame)) + self.assertAlmostEqual(s.weightedTruePositiveRate, 0.75, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.25, 2) + self.assertAlmostEqual(s.weightedRecall, 0.75, 2) + self.assertAlmostEqual(s.weightedPrecision, 0.8333333333333333, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 0.7333333333333334, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 0.7333333333333334, 2) + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, FMClassificationSummary)) + self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) + def test_gaussian_mixture_summary(self): data = [(Vectors.dense(1.0),), (Vectors.dense(5.0),), (Vectors.dense(10.0),), (Vectors.sparse(1, [], []),)] From c28a6fa5112c9ba3839f52b737266f24fdfcf75b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 15 Jul 2020 13:26:28 -0700 Subject: [PATCH 204/384] [SPARK-29292][SQL][ML] Update rest of default modules (Hive, ML, etc) for Scala 2.13 compilation ### What changes were proposed in this pull request? Same as https://github.com/apache/spark/pull/29078 and https://github.com/apache/spark/pull/28971 . This makes the rest of the default modules (i.e. those you get without specifying `-Pyarn` etc) compile under Scala 2.13. It does not close the JIRA, as a result. this also of course does not demonstrate that tests pass yet in 2.13. Note, this does not fix the `repl` module; that's separate. ### Why are the changes needed? Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12) Closes #29111 from srowen/SPARK-29292.3. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../examples/ml/JavaTokenizerExample.java | 4 +-- .../apache/spark/examples/SparkKMeans.scala | 8 +++++- .../spark/sql/avro/SchemaConverters.scala | 4 +-- .../sql/kafka010/KafkaOffsetReader.scala | 2 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 4 +-- .../scala/org/apache/spark/ml/Estimator.scala | 2 +- .../spark/ml/clustering/GaussianMixture.scala | 28 +++++++++---------- .../spark/ml/feature/RobustScaler.scala | 4 +-- .../apache/spark/ml/feature/Word2Vec.scala | 2 +- .../org/apache/spark/ml/param/params.scala | 2 +- .../mllib/api/python/PythonMLLibAPI.scala | 8 +++--- .../mllib/clustering/BisectingKMeans.scala | 2 +- .../mllib/clustering/GaussianMixture.scala | 10 +++---- .../apache/spark/mllib/fpm/PrefixSpan.scala | 2 +- .../apache/spark/mllib/rdd/SlidingRDD.scala | 2 +- .../spark/mllib/tree/impurity/Entropy.scala | 2 +- .../spark/mllib/tree/impurity/Gini.scala | 2 +- .../spark/mllib/tree/impurity/Variance.scala | 2 +- .../spark/mllib/util/NumericParser.scala | 8 +++--- .../ml/clustering/BisectingKMeansSuite.scala | 4 +-- .../spark/ml/clustering/KMeansSuite.scala | 12 ++++---- .../evaluation/ClusteringEvaluatorSuite.scala | 2 +- .../spark/ml/feature/NormalizerSuite.scala | 12 ++++---- .../spark/ml/recommendation/ALSSuite.scala | 12 ++++---- .../spark/sql/hive/HiveExternalCatalog.scala | 8 +++--- .../spark/sql/hive/HiveInspectors.scala | 4 +-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +-- .../org/apache/spark/sql/hive/HiveUtils.scala | 4 +-- .../sql/hive/client/HiveClientImpl.scala | 24 ++++++++-------- .../spark/sql/hive/client/HiveShim.scala | 10 +++---- .../sql/hive/execution/HiveOptions.scala | 2 +- .../hive/execution/HiveTableScanExec.scala | 2 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 4 +-- .../sql/hive/HiveShowCreateTableSuite.scala | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 36 files changed, 106 insertions(+), 102 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java index a0979aa2d24e4..3b5d8e6d555eb 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java @@ -23,7 +23,7 @@ import java.util.Arrays; import java.util.List; -import scala.collection.mutable.WrappedArray; +import scala.collection.mutable.Seq; import org.apache.spark.ml.feature.RegexTokenizer; import org.apache.spark.ml.feature.Tokenizer; @@ -69,7 +69,7 @@ public static void main(String[] args) { .setPattern("\\W"); // alternatively .setPattern("\\w+").setGaps(false); spark.udf().register( - "countTokens", (WrappedArray words) -> words.size(), DataTypes.IntegerType); + "countTokens", (Seq words) -> words.size(), DataTypes.IntegerType); Dataset tokenized = tokenizer.transform(sentenceDataFrame); tokenized.select("sentence", "words") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index ec9b44ce6e3b7..cf03e0203f771 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -82,7 +82,7 @@ object SparkKMeans { while(tempDist > convergeDist) { val closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) - val pointStats = closest.reduceByKey{case ((p1, c1), (p2, c2)) => (p1 + p2, c1 + c2)} + val pointStats = closest.reduceByKey(mergeResults) val newPoints = pointStats.map {pair => (pair._1, pair._2._1 * (1.0 / pair._2._2))}.collectAsMap() @@ -102,5 +102,11 @@ object SparkKMeans { kPoints.foreach(println) spark.stop() } + + private def mergeResults( + a: (Vector[Double], Int), + b: (Vector[Double], Int)): (Vector[Double], Int) = { + (a._1 + b._1, a._2 + b._2) + } } // scalastyle:on println diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index 3947d327dfac6..75690bb7722e3 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -85,7 +85,7 @@ object SchemaConverters { StructField(f.name, schemaType.dataType, schemaType.nullable) } - SchemaType(StructType(fields), nullable = false) + SchemaType(StructType(fields.toSeq), nullable = false) case ARRAY => val schemaType = toSqlTypeHelper(avroSchema.getElementType, existingRecordNames) @@ -126,7 +126,7 @@ object SchemaConverters { StructField(s"member$i", schemaType.dataType, nullable = true) } - SchemaType(StructType(fields), nullable = false) + SchemaType(StructType(fields.toSeq), nullable = false) } case other => throw new IncompatibleSchemaException(s"Unsupported type $other") diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 216e74a85c2ae..5ab7862674956 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -336,7 +336,7 @@ private[kafka010] class KafkaOffsetReader( } }) } - incorrectOffsets + incorrectOffsets.toSeq } // Retry to fetch latest offsets when detecting incorrect offsets. We don't use diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index bdad214a91343..ee31652eaf1f4 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -1540,8 +1540,8 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { makeSureGetOffsetCalled, Execute { q => // wait to reach the last offset in every partition - q.awaitOffset( - 0, KafkaSourceOffset(partitionOffsets.mapValues(_ => 3L)), streamingTimeout.toMillis) + q.awaitOffset(0, + KafkaSourceOffset(partitionOffsets.mapValues(_ => 3L).toMap), streamingTimeout.toMillis) }, CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22), StopStream, diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index 8815eb29bc860..3a02e2be6fe04 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -76,7 +76,7 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage { * @return fitted models, matching the input parameter maps */ @Since("2.0.0") - def fit(dataset: Dataset[_], paramMaps: Array[ParamMap]): Seq[M] = { + def fit(dataset: Dataset[_], paramMaps: Seq[ParamMap]): Seq[M] = { paramMaps.map(fit(dataset, _)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 18fd220b4ca9c..90845021fc073 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -492,12 +492,7 @@ class GaussianMixture @Since("2.0.0") ( (i, (agg.means(i), agg.covs(i), agg.weights(i), ws)) } } else Iterator.empty - }.reduceByKey { case ((mean1, cov1, w1, ws1), (mean2, cov2, w2, ws2)) => - // update the weights, means and covariances for i-th distributions - BLAS.axpy(1.0, mean2, mean1) - BLAS.axpy(1.0, cov2, cov1) - (mean1, cov1, w1 + w2, ws1 + ws2) - }.mapValues { case (mean, cov, w, ws) => + }.reduceByKey(GaussianMixture.mergeWeightsMeans).mapValues { case (mean, cov, w, ws) => // Create new distributions based on the partial assignments // (often referred to as the "M" step in literature) GaussianMixture.updateWeightsAndGaussians(mean, cov, w, ws) @@ -560,12 +555,7 @@ class GaussianMixture @Since("2.0.0") ( agg.meanIter.zip(agg.covIter).zipWithIndex .map { case ((mean, cov), i) => (i, (mean, cov, agg.weights(i), ws)) } } else Iterator.empty - }.reduceByKey { case ((mean1, cov1, w1, ws1), (mean2, cov2, w2, ws2)) => - // update the weights, means and covariances for i-th distributions - BLAS.axpy(1.0, mean2, mean1) - BLAS.axpy(1.0, cov2, cov1) - (mean1, cov1, w1 + w2, ws1 + ws2) - }.mapValues { case (mean, cov, w, ws) => + }.reduceByKey(GaussianMixture.mergeWeightsMeans).mapValues { case (mean, cov, w, ws) => // Create new distributions based on the partial assignments // (often referred to as the "M" step in literature) GaussianMixture.updateWeightsAndGaussians(mean, cov, w, ws) @@ -624,8 +614,8 @@ class GaussianMixture @Since("2.0.0") ( val gaussians = Array.tabulate(numClusters) { i => val start = i * numSamples val end = start + numSamples - val sampleSlice = samples.view(start, end) - val weightSlice = sampleWeights.view(start, end) + val sampleSlice = samples.view.slice(start, end) + val weightSlice = sampleWeights.view.slice(start, end) val localWeightSum = weightSlice.sum weights(i) = localWeightSum / weightSum @@ -691,6 +681,16 @@ object GaussianMixture extends DefaultParamsReadable[GaussianMixture] { new DenseMatrix(n, n, symmetricValues) } + private def mergeWeightsMeans( + a: (DenseVector, DenseVector, Double, Double), + b: (DenseVector, DenseVector, Double, Double)): (DenseVector, DenseVector, Double, Double) = + { + // update the weights, means and covariances for i-th distributions + BLAS.axpy(1.0, b._1, a._1) + BLAS.axpy(1.0, b._2, a._2) + (a._1, a._2, a._3 + b._3, a._4 + b._4) + } + /** * Update the weight, mean and covariance of gaussian distribution. * diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala index bd9be779fedbd..72ab3dbc31016 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala @@ -201,7 +201,7 @@ object RobustScaler extends DefaultParamsReadable[RobustScaler] { } Iterator.tabulate(numFeatures)(i => (i, summaries(i).compress)) } else Iterator.empty - }.reduceByKey { case (s1, s2) => s1.merge(s2) } + }.reduceByKey { (s1, s2) => s1.merge(s2) } } else { val scale = math.max(math.ceil(math.sqrt(vectors.getNumPartitions)).toInt, 2) vectors.mapPartitionsWithIndex { case (pid, iter) => @@ -214,7 +214,7 @@ object RobustScaler extends DefaultParamsReadable[RobustScaler] { seqOp = (s, v) => s.insert(v), combOp = (s1, s2) => s1.compress.merge(s2.compress) ).map { case ((_, i), s) => (i, s) - }.reduceByKey { case (s1, s2) => s1.compress.merge(s2.compress) } + }.reduceByKey { (s1, s2) => s1.compress.merge(s2.compress) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index bbfcbfbe038ef..db2665fa2e4a3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -291,7 +291,7 @@ class Word2VecModel private[ml] ( val outputSchema = transformSchema(dataset.schema, logging = true) val vectors = wordVectors.getVectors .mapValues(vv => Vectors.dense(vv.map(_.toDouble))) - .map(identity) // mapValues doesn't return a serializable map (SI-7005) + .map(identity).toMap // mapValues doesn't return a serializable map (SI-7005) val bVectors = dataset.sparkSession.sparkContext.broadcast(vectors) val d = $(vectorSize) val emptyVec = Vectors.sparse(d, Array.emptyIntArray, Array.emptyDoubleArray) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 53ca35ccd0073..f12c1f995b7d7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -937,7 +937,7 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) /** Put param pairs with a `java.util.List` of values for Python. */ private[ml] def put(paramPairs: JList[ParamPair[_]]): this.type = { - put(paramPairs.asScala: _*) + put(paramPairs.asScala.toSeq: _*) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 259ecb3a1762f..68f6ed4281dea 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -1223,28 +1223,28 @@ private[python] class PythonMLLibAPI extends Serializable { * Python-friendly version of [[MLUtils.convertVectorColumnsToML()]]. */ def convertVectorColumnsToML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { - MLUtils.convertVectorColumnsToML(dataset, cols.asScala: _*) + MLUtils.convertVectorColumnsToML(dataset, cols.asScala.toSeq: _*) } /** * Python-friendly version of [[MLUtils.convertVectorColumnsFromML()]] */ def convertVectorColumnsFromML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { - MLUtils.convertVectorColumnsFromML(dataset, cols.asScala: _*) + MLUtils.convertVectorColumnsFromML(dataset, cols.asScala.toSeq: _*) } /** * Python-friendly version of [[MLUtils.convertMatrixColumnsToML()]]. */ def convertMatrixColumnsToML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { - MLUtils.convertMatrixColumnsToML(dataset, cols.asScala: _*) + MLUtils.convertMatrixColumnsToML(dataset, cols.asScala.toSeq: _*) } /** * Python-friendly version of [[MLUtils.convertMatrixColumnsFromML()]] */ def convertMatrixColumnsFromML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { - MLUtils.convertMatrixColumnsFromML(dataset, cols.asScala: _*) + MLUtils.convertMatrixColumnsFromML(dataset, cols.asScala.toSeq: _*) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index 7c12697be95c8..99c6e8b3e079b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -225,7 +225,7 @@ class BisectingKMeans private ( divisibleIndices.contains(parentIndex(index)) } newClusters = summarize(d, newAssignments, dMeasure) - newClusterCenters = newClusters.mapValues(_.center).map(identity) + newClusterCenters = newClusters.mapValues(_.center).map(identity).toMap } if (preIndices != null) { preIndices.unpersist() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index 4d98ba41bbb7b..d5a7882614546 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.clustering -import scala.collection.mutable.IndexedSeq - import breeze.linalg.{diag, DenseMatrix => BreezeMatrix, DenseVector => BDV, Vector => BV} import org.apache.spark.annotation.Since @@ -189,8 +187,8 @@ class GaussianMixture private ( case None => val samples = breezeData.takeSample(withReplacement = true, k * nSamples, seed) (Array.fill(k)(1.0 / k), Array.tabulate(k) { i => - val slice = samples.view(i * nSamples, (i + 1) * nSamples) - new MultivariateGaussian(vectorMean(slice), initCovariance(slice)) + val slice = samples.view.slice(i * nSamples, (i + 1) * nSamples) + new MultivariateGaussian(vectorMean(slice.toSeq), initCovariance(slice.toSeq)) }) } @@ -259,7 +257,7 @@ class GaussianMixture private ( } /** Average of dense breeze vectors */ - private def vectorMean(x: IndexedSeq[BV[Double]]): BDV[Double] = { + private def vectorMean(x: Seq[BV[Double]]): BDV[Double] = { val v = BDV.zeros[Double](x(0).length) x.foreach(xi => v += xi) v / x.length.toDouble @@ -269,7 +267,7 @@ class GaussianMixture private ( * Construct matrix where diagonal entries are element-wise * variance of input vectors (computes biased variance) */ - private def initCovariance(x: IndexedSeq[BV[Double]]): BreezeMatrix[Double] = { + private def initCovariance(x: Seq[BV[Double]]): BreezeMatrix[Double] = { val mu = vectorMean(x) val ss = BDV.zeros[Double](x(0).length) x.foreach { xi => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index ac2b576f4ac4e..de3209c34bf07 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -335,7 +335,7 @@ object PrefixSpan extends Logging { largePrefixes = newLargePrefixes } - var freqPatterns = sc.parallelize(localFreqPatterns, 1) + var freqPatterns = sc.parallelize(localFreqPatterns.toSeq, 1) val numSmallPrefixes = smallPrefixes.size logInfo(s"number of small prefixes for local processing: $numSmallPrefixes") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala index 365b2a06110f6..c669ced61d2f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala @@ -97,7 +97,7 @@ class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int } if (sizes(i) + tail.length >= offset + windowSize) { partitions += - new SlidingRDDPartition[T](partitionIndex, parentPartitions(i), tail, offset) + new SlidingRDDPartition[T](partitionIndex, parentPartitions(i), tail.toSeq, offset) partitionIndex += 1 } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 6e2732f7ae7aa..c3bda99786310 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -112,7 +112,7 @@ private[spark] class EntropyAggregator(numClasses: Int) * @param offset Start index of stats for this (node, feature, bin). */ def getCalculator(allStats: Array[Double], offset: Int): EntropyCalculator = { - new EntropyCalculator(allStats.view(offset, offset + statsSize - 1).toArray, + new EntropyCalculator(allStats.view.slice(offset, offset + statsSize - 1).toArray, allStats(offset + statsSize - 1).toLong) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 5983118c05754..70163b56408a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -107,7 +107,7 @@ private[spark] class GiniAggregator(numClasses: Int) * @param offset Start index of stats for this (node, feature, bin). */ def getCalculator(allStats: Array[Double], offset: Int): GiniCalculator = { - new GiniCalculator(allStats.view(offset, offset + statsSize - 1).toArray, + new GiniCalculator(allStats.view.slice(offset, offset + statsSize - 1).toArray, allStats(offset + statsSize - 1).toLong) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index f5b2f8d514c7e..7143fd07d7333 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -95,7 +95,7 @@ private[spark] class VarianceAggregator() * @param offset Start index of stats for this (node, feature, bin). */ def getCalculator(allStats: Array[Double], offset: Int): VarianceCalculator = { - new VarianceCalculator(allStats.view(offset, offset + statsSize - 1).toArray, + new VarianceCalculator(allStats.view.slice(offset, offset + statsSize - 1).toArray, allStats(offset + statsSize - 1).toLong) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala index 2c613348c2d92..959e54e4c7169 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala @@ -85,10 +85,10 @@ private[mllib] object NumericParser { while (parsing && tokenizer.hasMoreTokens()) { token = tokenizer.nextToken() if (token == "(") { - items.append(parseTuple(tokenizer)) + items += parseTuple(tokenizer) allowComma = true } else if (token == "[") { - items.append(parseArray(tokenizer)) + items += parseArray(tokenizer) allowComma = true } else if (token == ",") { if (allowComma) { @@ -102,14 +102,14 @@ private[mllib] object NumericParser { // ignore whitespaces between delim chars, e.g. ", [" } else { // expecting a number - items.append(parseDouble(token)) + items += parseDouble(token) allowComma = true } } if (parsing) { throw new SparkException(s"A tuple must end with ')'.") } - items + items.toSeq } private def parseDouble(s: String): Double = { diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index debd0dd65d0c8..04b20d1e58dd3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -219,7 +219,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) - val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( (Vectors.dense(1.0, 1.0), 2.0), (Vectors.dense(10.0, 10.0), 2.0), (Vectors.dense(1.0, 0.5), 2.0), (Vectors.dense(10.0, 4.4), 2.0), (Vectors.dense(-1.0, 1.0), 2.0), (Vectors.dense(-100.0, 90.0), 2.0)))) @@ -286,7 +286,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) - val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( (Vectors.dense(1.0, 1.0), 1.0), (Vectors.dense(10.0, 10.0), 2.0), (Vectors.dense(1.0, 0.5), 2.0), (Vectors.dense(10.0, 4.4), 3.0), (Vectors.dense(-1.0, 1.0), 3.0), (Vectors.dense(-100.0, 90.0), 4.0)))) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 584594436267f..61f4359d99ea9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -255,7 +255,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes } test("compare with weightCol and without weightCol") { - val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( Vectors.dense(1.0, 1.0), Vectors.dense(10.0, 10.0), Vectors.dense(10.0, 10.0), Vectors.dense(1.0, 0.5), @@ -285,7 +285,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) - val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( (Vectors.dense(1.0, 1.0), 1.0), (Vectors.dense(10.0, 10.0), 2.0), (Vectors.dense(1.0, 0.5), 1.0), @@ -322,7 +322,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes test("Two centers with weightCol") { // use the same weight for all samples. - val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( (Vectors.dense(0.0, 0.0), 2.0), (Vectors.dense(0.0, 0.1), 2.0), (Vectors.dense(0.1, 0.0), 2.0), @@ -366,7 +366,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes assert(model1.clusterCenters(1) === model1_center2) // use different weight - val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( (Vectors.dense(0.0, 0.0), 1.0), (Vectors.dense(0.0, 0.1), 2.0), (Vectors.dense(0.1, 0.0), 3.0), @@ -412,7 +412,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes test("Four centers with weightCol") { // no weight - val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( Vectors.dense(0.1, 0.1), Vectors.dense(5.0, 0.2), Vectors.dense(10.0, 0.0), @@ -444,7 +444,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) // use same weight, should have the same result as no weight - val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( (Vectors.dense(0.1, 0.1), 2.0), (Vectors.dense(5.0, 0.2), 2.0), (Vectors.dense(10.0, 0.0), 2.0), 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 d4c620adc2e3c..06f2cb2b9788b 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 @@ -189,7 +189,7 @@ class ClusteringEvaluatorSuite } test("single-element clusters with weight") { - val singleItemClusters = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val singleItemClusters = spark.createDataFrame(spark.sparkContext.parallelize(Seq( (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") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala index d97df0050d74e..1c602cd7d9a4f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -29,14 +29,14 @@ class NormalizerSuite extends MLTest with DefaultReadWriteTest { import testImplicits._ - @transient var data: Array[Vector] = _ - @transient var l1Normalized: Array[Vector] = _ - @transient var l2Normalized: Array[Vector] = _ + @transient var data: Seq[Vector] = _ + @transient var l1Normalized: Seq[Vector] = _ + @transient var l2Normalized: Seq[Vector] = _ override def beforeAll(): Unit = { super.beforeAll() - data = Array( + data = Seq( Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), Vectors.dense(0.0, 0.0, 0.0), Vectors.dense(0.6, -1.1, -3.0), @@ -44,7 +44,7 @@ class NormalizerSuite extends MLTest with DefaultReadWriteTest { Vectors.sparse(3, Seq((0, 5.7), (1, 0.72), (2, 2.7))), Vectors.sparse(3, Seq()) ) - l1Normalized = Array( + l1Normalized = Seq( Vectors.sparse(3, Seq((0, -0.465116279), (1, 0.53488372))), Vectors.dense(0.0, 0.0, 0.0), Vectors.dense(0.12765957, -0.23404255, -0.63829787), @@ -52,7 +52,7 @@ class NormalizerSuite extends MLTest with DefaultReadWriteTest { Vectors.dense(0.625, 0.07894737, 0.29605263), Vectors.sparse(3, Seq()) ) - l2Normalized = Array( + l2Normalized = Seq( Vectors.sparse(3, Seq((0, -0.65617871), (1, 0.75460552))), Vectors.dense(0.0, 0.0, 0.0), Vectors.dense(0.184549876, -0.3383414, -0.922749378), diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 9029fc96b36a8..28275eb06cf0d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -307,7 +307,7 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { } logInfo(s"Generated an explicit feedback dataset with ${training.size} ratings for training " + s"and ${test.size} for test.") - (sc.parallelize(training, 2), sc.parallelize(test, 2)) + (sc.parallelize(training.toSeq, 2), sc.parallelize(test.toSeq, 2)) } /** @@ -810,7 +810,7 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { val topItems = model.recommendForAllUsers(k) assert(topItems.count() == numUsers) assert(topItems.columns.contains("user")) - checkRecommendations(topItems, expectedUpToN, "item") + checkRecommendations(topItems, expectedUpToN.toMap, "item") } } @@ -831,7 +831,7 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { val topUsers = getALSModel.recommendForAllItems(k) assert(topUsers.count() == numItems) assert(topUsers.columns.contains("item")) - checkRecommendations(topUsers, expectedUpToN, "user") + checkRecommendations(topUsers, expectedUpToN.toMap, "user") } } @@ -853,7 +853,7 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { val topItems = model.recommendForUserSubset(userSubset, k) assert(topItems.count() == numUsersSubset) assert(topItems.columns.contains("user")) - checkRecommendations(topItems, expectedUpToN, "item") + checkRecommendations(topItems, expectedUpToN.toMap, "item") } } @@ -875,7 +875,7 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { val topUsers = model.recommendForItemSubset(itemSubset, k) assert(topUsers.count() == numItemsSubset) assert(topUsers.columns.contains("item")) - checkRecommendations(topUsers, expectedUpToN, "user") + checkRecommendations(topUsers, expectedUpToN.toMap, "user") } } @@ -1211,6 +1211,6 @@ object ALSSuite extends Logging { } logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + s"and ${test.size} for test.") - (sc.parallelize(training, 2), sc.parallelize(test, 2)) + (sc.parallelize(training.toSeq, 2), sc.parallelize(test.toSeq, 2)) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 571c25e356c08..f01a03996821a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -833,8 +833,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat updateLocationInStorageProps(table, newPath = None).copy( locationUri = tableLocation.map(CatalogUtils.stringToURI(_))) } - val storageWithoutHiveGeneratedProperties = storageWithLocation.copy( - properties = storageWithLocation.properties.filterKeys(!HIVE_GENERATED_STORAGE_PROPERTIES(_))) + val storageWithoutHiveGeneratedProperties = storageWithLocation.copy(properties = + storageWithLocation.properties.filterKeys(!HIVE_GENERATED_STORAGE_PROPERTIES(_)).toMap) val partitionProvider = table.properties.get(TABLE_PARTITION_PROVIDER) val schemaFromTableProps = getSchemaFromTableProperties(table) @@ -848,7 +848,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat partitionColumnNames = partColumnNames, bucketSpec = getBucketSpecFromTableProperties(table), tracksPartitionsInCatalog = partitionProvider == Some(TABLE_PARTITION_PROVIDER_CATALOG), - properties = table.properties.filterKeys(!HIVE_GENERATED_TABLE_PROPERTIES(_))) + properties = table.properties.filterKeys(!HIVE_GENERATED_TABLE_PROPERTIES(_)).toMap) } override def tableExists(db: String, table: String): Boolean = withClient { @@ -1125,7 +1125,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat val colStats = new mutable.HashMap[String, CatalogColumnStat] val colStatsProps = properties.filterKeys(_.startsWith(STATISTICS_COL_STATS_PREFIX)).map { case (k, v) => k.drop(STATISTICS_COL_STATS_PREFIX.length) -> v - } + }.toMap // Find all the column names by matching the KEY_VERSION properties for them. colStatsProps.keys.filter { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 16e9014340244..19aa5935a09d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -694,7 +694,7 @@ private[hive] trait HiveInspectors { } data: Any => { if (data != null) { - InternalRow.fromSeq(unwrappers.map(_(data))) + InternalRow.fromSeq(unwrappers.map(_(data)).toSeq) } else { null } @@ -872,7 +872,7 @@ private[hive] trait HiveInspectors { StructType(s.getAllStructFieldRefs.asScala.map(f => types.StructField( f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), nullable = true) - )) + ).toSeq) case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) case m: MapObjectInspector => MapType( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 2981e391c0439..a89243c331c7b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -131,12 +131,12 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log // Consider table and storage properties. For properties existing in both sides, storage // properties will supersede table properties. if (serde.contains("parquet")) { - val options = relation.tableMeta.properties.filterKeys(isParquetProperty) ++ + val options = relation.tableMeta.properties.filterKeys(isParquetProperty).toMap ++ relation.tableMeta.storage.properties + (ParquetOptions.MERGE_SCHEMA -> SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING).toString) convertToLogicalRelation(relation, options, classOf[ParquetFileFormat], "parquet") } else { - val options = relation.tableMeta.properties.filterKeys(isOrcProperty) ++ + val options = relation.tableMeta.properties.filterKeys(isOrcProperty).toMap ++ relation.tableMeta.storage.properties if (SQLConf.get.getConf(SQLConf.ORC_IMPLEMENTATION) == "native") { convertToLogicalRelation( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 04caf57efdc74..62ff2db2ecb3c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -408,7 +408,7 @@ private[spark] object HiveUtils extends Logging { logWarning(s"Hive jar path '$path' does not exist.") Nil } else { - files.filter(_.getName.toLowerCase(Locale.ROOT).endsWith(".jar")) + files.filter(_.getName.toLowerCase(Locale.ROOT).endsWith(".jar")).toSeq } case path => new File(path) :: Nil @@ -505,7 +505,7 @@ private[spark] object HiveUtils extends Logging { // partition columns are part of the schema val partCols = hiveTable.getPartCols.asScala.map(HiveClientImpl.fromHiveColumn) val dataCols = hiveTable.getCols.asScala.map(HiveClientImpl.fromHiveColumn) - table.copy(schema = StructType(dataCols ++ partCols)) + table.copy(schema = StructType((dataCols ++ partCols).toSeq)) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 6ad5e9d3c9080..3f70387a3b058 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -390,7 +390,7 @@ private[hive] class HiveClientImpl( } override def listDatabases(pattern: String): Seq[String] = withHiveState { - client.getDatabasesByPattern(pattern).asScala + client.getDatabasesByPattern(pattern).asScala.toSeq } private def getRawTableOption(dbName: String, tableName: String): Option[HiveTable] = { @@ -400,7 +400,7 @@ private[hive] class HiveClientImpl( private def getRawTablesByName(dbName: String, tableNames: Seq[String]): Seq[HiveTable] = { try { msClient.getTableObjectsByName(dbName, tableNames.asJava).asScala - .map(extraFixesForNonView).map(new HiveTable(_)) + .map(extraFixesForNonView).map(new HiveTable(_)).toSeq } catch { case ex: Exception => throw new HiveException(s"Unable to fetch tables of db $dbName", ex); @@ -434,7 +434,7 @@ private[hive] class HiveClientImpl( throw new SparkException( s"${ex.getMessage}, db: ${h.getDbName}, table: ${h.getTableName}", ex) } - val schema = StructType(cols ++ partCols) + val schema = StructType((cols ++ partCols).toSeq) val bucketSpec = if (h.getNumBuckets > 0) { val sortColumnOrders = h.getSortCols.asScala @@ -450,7 +450,7 @@ private[hive] class HiveClientImpl( } else { Seq.empty } - Option(BucketSpec(h.getNumBuckets, h.getBucketCols.asScala, sortColumnNames)) + Option(BucketSpec(h.getNumBuckets, h.getBucketCols.asScala.toSeq, sortColumnNames.toSeq)) } else { None } @@ -502,7 +502,7 @@ private[hive] class HiveClientImpl( throw new AnalysisException(s"Hive $tableTypeStr is not supported.") }, schema = schema, - partitionColumnNames = partCols.map(_.name), + partitionColumnNames = partCols.map(_.name).toSeq, // If the table is written by Spark, we will put bucketing information in table properties, // and will always overwrite the bucket spec in hive metastore by the bucketing information // in table properties. This means, if we have bucket spec in both hive metastore and @@ -539,7 +539,7 @@ private[hive] class HiveClientImpl( // that created by older versions of Spark. viewOriginalText = Option(h.getViewOriginalText), viewText = Option(h.getViewExpandedText), - unsupportedFeatures = unsupportedFeatures, + unsupportedFeatures = unsupportedFeatures.toSeq, ignoredProperties = ignoredProperties.toMap) } @@ -638,7 +638,7 @@ private[hive] class HiveClientImpl( shim.dropPartition(client, db, table, partition, !retainData, purge) } catch { case e: Exception => - val remainingParts = matchingParts.toBuffer -- droppedParts + val remainingParts = matchingParts.toBuffer --= droppedParts logError( s""" |====================== @@ -708,7 +708,7 @@ private[hive] class HiveClientImpl( assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid") client.getPartitionNames(table.database, table.identifier.table, s.asJava, -1) } - hivePartitionNames.asScala.sorted + hivePartitionNames.asScala.sorted.toSeq } override def getPartitionOption( @@ -735,7 +735,7 @@ private[hive] class HiveClientImpl( } val parts = client.getPartitions(hiveTable, partSpec.asJava).asScala.map(fromHivePartition) HiveCatalogMetrics.incrementFetchedPartitions(parts.length) - parts + parts.toSeq } override def getPartitionsByFilter( @@ -748,11 +748,11 @@ private[hive] class HiveClientImpl( } override def listTables(dbName: String): Seq[String] = withHiveState { - client.getAllTables(dbName).asScala + client.getAllTables(dbName).asScala.toSeq } override def listTables(dbName: String, pattern: String): Seq[String] = withHiveState { - client.getTablesByPattern(dbName, pattern).asScala + client.getTablesByPattern(dbName, pattern).asScala.toSeq } override def listTablesByType( @@ -766,7 +766,7 @@ private[hive] class HiveClientImpl( case _: UnsupportedOperationException => // Fallback to filter logic if getTablesByType not supported. val tableNames = client.getTablesByPattern(dbName, pattern).asScala - val tables = getTablesByName(dbName, tableNames).filter(_.tableType == tableType) + val tables = getTablesByName(dbName, tableNames.toSeq).filter(_.tableType == tableType) tables.map(_.identifier.table) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 8df43b785759e..8ff7a1abd2d6b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -363,7 +363,7 @@ private[client] class Shim_v0_12 extends Shim with Logging { override def getDriverResults(driver: Driver): Seq[String] = { val res = new JArrayList[String]() getDriverResultsMethod.invoke(driver, res) - res.asScala + res.asScala.toSeq } override def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long = { @@ -600,7 +600,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } FunctionResource(FunctionResourceType.fromString(resourceType), uri.getUri()) } - CatalogFunction(name, hf.getClassName, resources) + CatalogFunction(name, hf.getClassName, resources.toSeq) } override def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] = { @@ -623,7 +623,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { - hive.getFunctions(db, pattern).asScala + hive.getFunctions(db, pattern).asScala.toSeq } /** @@ -843,7 +843,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { case s: String => s case a: Array[Object] => a(0).asInstanceOf[String] } - } + }.toSeq } override def getDatabaseOwnerName(db: Database): String = { @@ -1252,7 +1252,7 @@ private[client] class Shim_v2_3 extends Shim_v2_1 { pattern: String, tableType: TableType): Seq[String] = { getTablesByTypeMethod.invoke(hive, dbName, pattern, tableType) - .asInstanceOf[JList[String]].asScala + .asInstanceOf[JList[String]].asScala.toSeq } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala index 802ddafdbee4d..7b51618772edc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala @@ -87,7 +87,7 @@ class HiveOptions(@transient private val parameters: CaseInsensitiveMap[String]) def serdeProperties: Map[String, String] = parameters.filterKeys { k => !lowerCasedOptionNames.contains(k.toLowerCase(Locale.ROOT)) - }.map { case (k, v) => delimiterOptions.getOrElse(k, k) -> v } + }.map { case (k, v) => delimiterOptions.getOrElse(k, k) -> v }.toMap } object HiveOptions { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index 4dccacef337e9..41820b0135f4a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -156,7 +156,7 @@ case class HiveTableScanExec( // Only partitioned values are needed here, since the predicate has already been bound to // partition key attribute references. - val row = InternalRow.fromSeq(castedValues) + val row = InternalRow.fromSeq(castedValues.toSeq) shouldKeep.eval(row).asInstanceOf[Boolean] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 05d608a2016a5..8ad5cb70d248b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -224,7 +224,7 @@ private[hive] case class HiveGenericUDTF( override lazy val elementSchema = StructType(outputInspector.getAllStructFieldRefs.asScala.map { field => StructField(field.getFieldName, inspectorToDataType(field.getFieldObjectInspector), nullable = true) - }) + }.toSeq) @transient private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray @@ -257,7 +257,7 @@ private[hive] case class HiveGenericUDTF( def collectRows(): Seq[InternalRow] = { val toCollect = collected collected = new ArrayBuffer[InternalRow] - toCollect + toCollect.toSeq } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala index cfcf70c0e79f0..446923ad23201 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala @@ -279,7 +279,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet table.copy( createTime = 0L, lastAccessTime = 0L, - properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)).toMap, stats = None, ignoredProperties = Map.empty, storage = table.storage.copy(properties = Map.empty), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index c1eab63ec073f..be6d023302293 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -911,7 +911,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto */ private def getStatsProperties(tableName: String): Map[String, String] = { val hTable = hiveClient.getTable(spark.sessionState.catalog.getCurrentDatabase, tableName) - hTable.properties.filterKeys(_.startsWith(STATISTICS_PREFIX)) + hTable.properties.filterKeys(_.startsWith(STATISTICS_PREFIX)).toMap } test("change stats after insert command for hive table") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index f95251a669715..fbd1fc1ea98df 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -128,7 +128,7 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA createTime = 0L, lastAccessTime = 0L, owner = "", - properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)).toMap, // View texts are checked separately viewText = None ) From db47c6e340a63100d7c0e85abf237adc4e2174cc Mon Sep 17 00:00:00 2001 From: Warren Zhu Date: Thu, 16 Jul 2020 11:31:24 +0800 Subject: [PATCH 205/384] [SPARK-32125][UI] Support get taskList by status in Web UI and SHS Rest API ### What changes were proposed in this pull request? Support fetching taskList by status as below: ``` /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskList?status=failed ``` ### Why are the changes needed? When there're large number of tasks in one stage, current api is hard to get taskList by status ### Does this PR introduce _any_ user-facing change? Yes. Updated monitoring doc. ### How was this patch tested? Added tests in `HistoryServerSuite` Closes #28942 from warrenzhu25/SPARK-32125. Authored-by: Warren Zhu Signed-off-by: Gengliang Wang --- .../spark/status/api/v1/TaskStatus.java | 32 + .../apache/spark/status/AppStatusStore.scala | 16 +- .../spark/status/api/v1/StagesResource.scala | 5 +- ..._status___offset___length_expectation.json | 99 ++ ...rtBy_short_names__runtime_expectation.json | 981 ++++++++++++++++++ ...stage_task_list_w__status_expectation.json | 531 ++++++++++ .../deploy/history/HistoryServerSuite.scala | 6 + docs/monitoring.md | 3 +- 8 files changed, 1666 insertions(+), 7 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/status/api/v1/TaskStatus.java create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status___offset___length_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status___sortBy_short_names__runtime_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status_expectation.json diff --git a/core/src/main/java/org/apache/spark/status/api/v1/TaskStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/TaskStatus.java new file mode 100644 index 0000000000000..dec9c31321839 --- /dev/null +++ b/core/src/main/java/org/apache/spark/status/api/v1/TaskStatus.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1; + +import org.apache.spark.util.EnumUtil; + +public enum TaskStatus { + RUNNING, + KILLED, + FAILED, + SUCCESS, + UNKNOWN; + + public static TaskStatus fromString(String str) { + return EnumUtil.parseIgnoreCase(TaskStatus.class, str); + } +} diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 106d272948b9f..0a8d188dc1553 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -386,7 +386,8 @@ private[spark] class AppStatusStore( stageAttemptId: Int, offset: Int, length: Int, - sortBy: v1.TaskSorting): Seq[v1.TaskData] = { + sortBy: v1.TaskSorting, + statuses: JList[v1.TaskStatus]): Seq[v1.TaskData] = { val (indexName, ascending) = sortBy match { case v1.TaskSorting.ID => (None, true) @@ -395,7 +396,7 @@ private[spark] class AppStatusStore( case v1.TaskSorting.DECREASING_RUNTIME => (Some(TaskIndexNames.EXEC_RUN_TIME), false) } - taskList(stageId, stageAttemptId, offset, length, indexName, ascending) + taskList(stageId, stageAttemptId, offset, length, indexName, ascending, statuses) } def taskList( @@ -404,7 +405,8 @@ private[spark] class AppStatusStore( offset: Int, length: Int, sortBy: Option[String], - ascending: Boolean): Seq[v1.TaskData] = { + ascending: Boolean, + statuses: JList[v1.TaskStatus] = List().asJava): Seq[v1.TaskData] = { val stageKey = Array(stageId, stageAttemptId) val base = store.view(classOf[TaskDataWrapper]) val indexed = sortBy match { @@ -417,7 +419,13 @@ private[spark] class AppStatusStore( } val ordered = if (ascending) indexed else indexed.reverse() - val taskDataWrapperIter = ordered.skip(offset).max(length).asScala + val taskDataWrapperIter = if (statuses != null && !statuses.isEmpty) { + val statusesStr = statuses.asScala.map(_.toString).toSet + ordered.asScala.filter(s => statusesStr.contains(s.status)).slice(offset, offset + length) + } else { + ordered.skip(offset).max(length).asScala + } + constructTaskDataList(taskDataWrapperIter) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 44ee322a22a10..05a7e96882d77 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -96,8 +96,9 @@ private[v1] class StagesResource extends BaseAppResource { @PathParam("stageAttemptId") stageAttemptId: Int, @DefaultValue("0") @QueryParam("offset") offset: Int, @DefaultValue("20") @QueryParam("length") length: Int, - @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = { - withUI(_.store.taskList(stageId, stageAttemptId, offset, length, sortBy)) + @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting, + @QueryParam("status") statuses: JList[TaskStatus]): Seq[TaskData] = { + withUI(_.store.taskList(stageId, stageAttemptId, offset, length, sortBy, statuses)) } // This api needs to stay formatted exactly as it is below, since, it is being used by the diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status___offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status___offset___length_expectation.json new file mode 100644 index 0000000000000..28509e33c5dcc --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status___offset___length_expectation.json @@ -0,0 +1,99 @@ +[ { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 421, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 350, + "executorCpuTime" : 0, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3934399, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 40, + "gettingResultTime" : 0 +}, { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.503GMT", + "duration" : 419, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 32, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 348, + "executorCpuTime" : 0, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89885, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 37, + "gettingResultTime" : 0 +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status___sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status___sortBy_short_names__runtime_expectation.json new file mode 100644 index 0000000000000..01eef1b565bf6 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status___sortBy_short_names__runtime_expectation.json @@ -0,0 +1,981 @@ +[ { + "taskId" : 40, + "index" : 40, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.197GMT", + "duration" : 24, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 14, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94792, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 +}, { + "taskId" : 41, + "index" : 41, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.200GMT", + "duration" : 24, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 16, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 90765, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 +}, { + "taskId" : 43, + "index" : 43, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.204GMT", + "duration" : 39, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 16, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 171516, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 21, + "gettingResultTime" : 0 +}, { + "taskId" : 57, + "index" : 57, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.257GMT", + "duration" : 21, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 16, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 96849, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 2, + "gettingResultTime" : 0 +}, { + "taskId" : 58, + "index" : 58, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.263GMT", + "duration" : 23, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 16, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 97521, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 +}, { + "taskId" : 68, + "index" : 68, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.306GMT", + "duration" : 22, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 16, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 101750, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 +}, { + "taskId" : 86, + "index" : 86, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.374GMT", + "duration" : 28, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 16, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95848, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 8, + "gettingResultTime" : 0 +}, { + "taskId" : 32, + "index" : 32, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.148GMT", + "duration" : 33, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89603, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 +}, { + "taskId" : 39, + "index" : 39, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.180GMT", + "duration" : 32, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98748, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 +}, { + "taskId" : 42, + "index" : 42, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.203GMT", + "duration" : 42, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 103713, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 15, + "gettingResultTime" : 0 +}, { + "taskId" : 51, + "index" : 51, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.242GMT", + "duration" : 21, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 96013, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 2, + "gettingResultTime" : 0 +}, { + "taskId" : 59, + "index" : 59, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.265GMT", + "duration" : 23, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100753, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 +}, { + "taskId" : 63, + "index" : 63, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.276GMT", + "duration" : 40, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 20, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102779, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 +}, { + "taskId" : 87, + "index" : 87, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.374GMT", + "duration" : 36, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102159, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 7, + "gettingResultTime" : 0 +}, { + "taskId" : 90, + "index" : 90, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.385GMT", + "duration" : 23, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98472, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 +}, { + "taskId" : 99, + "index" : 99, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.426GMT", + "duration" : 22, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 17, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70565, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 133964, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 +}, { + "taskId" : 44, + "index" : 44, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.205GMT", + "duration" : 37, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 18, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98293, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 +}, { + "taskId" : 47, + "index" : 47, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.212GMT", + "duration" : 33, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 18, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 103015, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 +}, { + "taskId" : 50, + "index" : 50, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.240GMT", + "duration" : 26, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 18, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 90836, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 +}, { + "taskId" : 52, + "index" : 52, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.243GMT", + "duration" : 28, + "executorId" : "driver", + "host" : "localhost", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 18, + "executorCpuTime" : 0, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89664, + "recordsWritten" : 10 + } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status_expectation.json new file mode 100644 index 0000000000000..9896aceb275de --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__status_expectation.json @@ -0,0 +1,531 @@ +[ { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2016-11-15T23:20:44.052GMT", + "duration" : 675, + "executorId" : "0", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 494, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 30, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" + }, + "schedulerDelay" : 181, + "gettingResultTime" : 0 +}, { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2016-11-15T23:20:44.053GMT", + "duration" : 725, + "executorId" : "2", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 456, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 32, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + }, + "schedulerDelay" : 269, + "gettingResultTime" : 0 +}, { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2016-11-15T23:20:44.055GMT", + "duration" : 665, + "executorId" : "0", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 495, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 30, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" + }, + "schedulerDelay" : 170, + "gettingResultTime" : 0 +}, { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2016-11-15T23:20:44.056GMT", + "duration" : 685, + "executorId" : "2", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 448, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 32, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + }, + "schedulerDelay" : 237, + "gettingResultTime" : 0 +}, { + "taskId" : 9, + "index" : 9, + "attempt" : 0, + "launchTime" : "2016-11-15T23:20:44.057GMT", + "duration" : 732, + "executorId" : "0", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 503, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 30, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" + }, + "schedulerDelay" : 229, + "gettingResultTime" : 0 +}, { + "taskId" : 11, + "index" : 11, + "attempt" : 0, + "launchTime" : "2016-11-15T23:20:44.058GMT", + "duration" : 678, + "executorId" : "2", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 451, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 32, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + }, + "schedulerDelay" : 227, + "gettingResultTime" : 0 +}, { + "taskId" : 13, + "index" : 13, + "attempt" : 0, + "launchTime" : "2016-11-15T23:20:44.060GMT", + "duration" : 669, + "executorId" : "0", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 494, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 30, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" + }, + "schedulerDelay" : 175, + "gettingResultTime" : 0 +}, { + "taskId" : 15, + "index" : 15, + "attempt" : 0, + "launchTime" : "2016-11-15T23:20:44.065GMT", + "duration" : 672, + "executorId" : "2", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 446, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 32, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + }, + "schedulerDelay" : 226, + "gettingResultTime" : 0 +}, { + "taskId" : 19, + "index" : 11, + "attempt" : 1, + "launchTime" : "2016-11-15T23:20:44.736GMT", + "duration" : 13, + "executorId" : "2", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 2, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + }, + "schedulerDelay" : 11, + "gettingResultTime" : 0 +}, { + "taskId" : 20, + "index" : 15, + "attempt" : 1, + "launchTime" : "2016-11-15T23:20:44.737GMT", + "duration" : 19, + "executorId" : "2", + "host" : "172.22.0.111", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 10, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + }, + "schedulerDelay" : 9, + "gettingResultTime" : 0 +} ] diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index c856cc1897660..39b339caea385 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -154,6 +154,12 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "applications/local-1430917381534/stages/0/0/taskList?sortBy=-runtime", "stage task list w/ sortBy short names: runtime" -> "applications/local-1430917381534/stages/0/0/taskList?sortBy=runtime", + "stage task list w/ status" -> + "applications/app-20161115172038-0000/stages/0/0/taskList?status=failed", + "stage task list w/ status & offset & length" -> + "applications/local-1430917381534/stages/0/0/taskList?status=success&offset=1&length=2", + "stage task list w/ status & sortBy short names: runtime" -> + "applications/local-1430917381534/stages/0/0/taskList?status=success&sortBy=runtime", "stage list with accumulable json" -> "applications/local-1426533911241/1/stages", "stage with accumulable json" -> "applications/local-1426533911241/1/stages/0/0", diff --git a/docs/monitoring.md b/docs/monitoring.md index 81c386aa90dd8..2ab7b30a1dca9 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -499,7 +499,8 @@ can be identified by their `[attempt-id]`. In the API listed below, when running A list of all tasks for the given stage attempt.
?offset=[offset]&length=[len] list tasks in the given range.
?sortBy=[runtime|-runtime] sort the tasks. -
Example: ?offset=10&length=50&sortBy=runtime +
?status=[running|success|killed|failed|unknown] list only tasks in the state. +
Example: ?offset=10&length=50&sortBy=runtime&status=running From bdeb626c5a6f16101917140b1d30296e7d35b2ce Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 16 Jul 2020 13:01:53 +0000 Subject: [PATCH 206/384] [SPARK-32272][SQL] Add SQL standard command SET TIME ZONE ### What changes were proposed in this pull request? This PR adds the SQL standard command - `SET TIME ZONE` to the current default time zone displacement for the current SQL-session, which is the same as the existing `set spark.sql.session.timeZone=xxx'. All in all, this PR adds syntax as following, ``` SET TIME ZONE LOCAL; SET TIME ZONE 'valid time zone'; -- zone offset or region SET TIME ZONE INTERVAL XXXX; -- xxx must in [-18, + 18] hours, * this range is bigger than ansi [-14, + 14] ``` ### Why are the changes needed? ANSI compliance and supply pure SQL users a way to retrieve all supported TimeZones ### Does this PR introduce _any_ user-facing change? yes, add new syntax. ### How was this patch tested? add unit tests. and locally verified reference doc ![image](https://user-images.githubusercontent.com/8326978/87510244-c8dc3680-c6a5-11ea-954c-b098be84afee.png) Closes #29064 from yaooqinn/SPARK-32272. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- docs/_data/menu-sql.yaml | 2 + docs/sql-ref-ansi-compliance.md | 2 + ...l-ref-syntax-aux-conf-mgmt-set-timezone.md | 67 +++++++++ docs/sql-ref-syntax-aux-conf-mgmt.md | 1 + .../spark/sql/catalyst/parser/SqlBase.g4 | 8 ++ .../sql/catalyst/parser/AstBuilder.scala | 11 +- .../apache/spark/sql/internal/SQLConf.scala | 6 +- .../spark/sql/execution/SparkSqlParser.scala | 39 ++++- .../resources/sql-tests/inputs/timezone.sql | 15 ++ .../sql-tests/results/timezone.sql.out | 135 ++++++++++++++++++ .../spark/sql/internal/SQLConfSuite.scala | 28 ++++ 11 files changed, 308 insertions(+), 6 deletions(-) create mode 100644 docs/sql-ref-syntax-aux-conf-mgmt-set-timezone.md create mode 100644 sql/core/src/test/resources/sql-tests/inputs/timezone.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/timezone.sql.out diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index eea657e684495..2d26326a00b0e 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -249,6 +249,8 @@ url: sql-ref-syntax-aux-conf-mgmt-set.html - text: RESET url: sql-ref-syntax-aux-conf-mgmt-reset.html + - text: SET TIME ZONE + url: sql-ref-syntax-aux-conf-mgmt-set-timezone.html - text: RESOURCE MANAGEMENT url: sql-ref-syntax-aux-resource-mgmt.html subitems: diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 6488ad9cd34c9..f083f880cc073 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -355,6 +355,7 @@ Below is a list of all the keywords in Spark SQL. |TEMPORARY|non-reserved|non-reserved|non-reserved| |TERMINATED|non-reserved|non-reserved|non-reserved| |THEN|reserved|non-reserved|reserved| +|TIME|reserved|non-reserved|reserved| |TO|reserved|non-reserved|reserved| |TOUCH|non-reserved|non-reserved|non-reserved| |TRAILING|reserved|non-reserved|reserved| @@ -385,3 +386,4 @@ Below is a list of all the keywords in Spark SQL. |WINDOW|non-reserved|non-reserved|reserved| |WITH|reserved|non-reserved|reserved| |YEAR|reserved|non-reserved|reserved| +|ZONE|non-reserved|non-reserved|non-reserved| diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-set-timezone.md b/docs/sql-ref-syntax-aux-conf-mgmt-set-timezone.md new file mode 100644 index 0000000000000..47dd2be77ae90 --- /dev/null +++ b/docs/sql-ref-syntax-aux-conf-mgmt-set-timezone.md @@ -0,0 +1,67 @@ +--- +layout: global +title: SET TIME ZONE +displayTitle: SET TIME ZONE +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +### Description + +The SET TIME ZONE command sets the time zone of the current session. + +### Syntax + +```sql +SET TIME ZONE LOCAL +SET TIME ZONE 'timezone_value' +SET TIME ZONE INTERVAL interval_literal +``` + +### Parameters + +* **LOCAL** + + Set the time zone to the one specified in the java `user.timezone` property, or to the environment variable `TZ` if `user.timezone` is undefined, or to the system time zone if both of them are undefined. + +* **timezone_value** + + The ID of session local timezone in the format of either region-based zone IDs or zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in the format '`(+|-)HH`', '`(+|-)HH:mm`' or '`(+|-)HH:mm:ss`', e.g '-08', '+01:00' or '-13:33:33'. Also, 'UTC' and 'Z' are supported as aliases of '+00:00'. Other short names are not recommended to use because they can be ambiguous. + +* **interval_literal** + + The [interval literal](sql-ref-literals.html#interval-literal) represents the difference between the session time zone to the 'UTC'. It must be in the range of [-18, 18] hours and max to second precision, e.g. `INTERVAL 2 HOURS 30 MINITUES` or `INTERVAL '15:40:32' HOUR TO SECOND`. + +### Examples + +```sql +-- Set time zone to the system default. +SET TIME ZONE LOCAL; + +-- Set time zone to the region-based zone ID. +SET TIME ZONE 'America/Los_Angeles'; + +-- Set time zone to the Zone offset. +SET TIME ZONE '+08:00'; + +-- Set time zone with intervals. +SET TIME ZONE INTERVAL 1 HOUR 30 MINUTES; +SET TIME ZONE INTERVAL '08:30:00' HOUR TO SECOND; +``` + +### Related Statements + +* [SET](sql-ref-syntax-aux-conf-mgmt-set.html) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt.md b/docs/sql-ref-syntax-aux-conf-mgmt.md index 1900fb7f1cb9a..3312bcb503500 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt.md @@ -21,3 +21,4 @@ license: | * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) * [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) + * [SET TIME ZONE](sql-ref-syntax-aux-conf-mgmt-set-timezone.html) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index d29fa1319daa5..bc7e982830584 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -240,6 +240,9 @@ statement | MSCK REPAIR TABLE multipartIdentifier #repairTable | op=(ADD | LIST) identifier (STRING | .*?) #manageResource | SET ROLE .*? #failNativeCommand + | SET TIME ZONE interval #setTimeZone + | SET TIME ZONE timezone=(STRING | LOCAL) #setTimeZone + | SET TIME ZONE .*? #setTimeZone | SET .*? #setConfiguration | RESET #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand @@ -1190,6 +1193,7 @@ ansiNonReserved | VIEW | VIEWS | WINDOW + | ZONE //--ANSI-NON-RESERVED-END ; @@ -1431,6 +1435,7 @@ nonReserved | TEMPORARY | TERMINATED | THEN + | TIME | TO | TOUCH | TRAILING @@ -1459,6 +1464,7 @@ nonReserved | WINDOW | WITH | YEAR + | ZONE ; // NOTE: If you add a new token in the list below, you should update the list of keywords @@ -1691,6 +1697,7 @@ TBLPROPERTIES: 'TBLPROPERTIES'; TEMPORARY: 'TEMPORARY' | 'TEMP'; TERMINATED: 'TERMINATED'; THEN: 'THEN'; +TIME: 'TIME'; TO: 'TO'; TOUCH: 'TOUCH'; TRAILING: 'TRAILING'; @@ -1721,6 +1728,7 @@ WHERE: 'WHERE'; WINDOW: 'WINDOW'; WITH: 'WITH'; YEAR: 'YEAR'; +ZONE: 'ZONE'; //--SPARK-KEYWORD-LIST-END //============================ // End of the keywords list 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 29621e11e534c..740892dfdb1c6 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 @@ -2090,6 +2090,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * - from-to unit, for instance: interval '1-2' year to month. */ override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) { + Literal(parseIntervalLiteral(ctx), CalendarIntervalType) + } + + /** + * Create a [[CalendarInterval]] object + */ + protected def parseIntervalLiteral(ctx: IntervalContext): CalendarInterval = withOrigin(ctx) { if (ctx.errorCapturingMultiUnitsInterval != null) { val innerCtx = ctx.errorCapturingMultiUnitsInterval if (innerCtx.unitToUnitInterval != null) { @@ -2097,7 +2104,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Can only have a single from-to unit in the interval literal syntax", innerCtx.unitToUnitInterval) } - Literal(visitMultiUnitsInterval(innerCtx.multiUnitsInterval), CalendarIntervalType) + visitMultiUnitsInterval(innerCtx.multiUnitsInterval) } else if (ctx.errorCapturingUnitToUnitInterval != null) { val innerCtx = ctx.errorCapturingUnitToUnitInterval if (innerCtx.error1 != null || innerCtx.error2 != null) { @@ -2106,7 +2113,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Can only have a single from-to unit in the interval literal syntax", errorCtx) } - Literal(visitUnitToUnitInterval(innerCtx.body), CalendarIntervalType) + visitUnitToUnitInterval(innerCtx.body) } else { throw new ParseException("at least one time unit should be given for interval literal", ctx) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9be0497e46603..77d49aff33c53 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1723,9 +1723,9 @@ object SQLConf { val SESSION_LOCAL_TIMEZONE = buildConf("spark.sql.session.timeZone") .doc("The ID of session local timezone in the format of either region-based zone IDs or " + "zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. " + - "Zone offsets must be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. " + - "Also 'UTC' and 'Z' are supported as aliases of '+00:00'. Other short names are not " + - "recommended to use because they can be ambiguous.") + "Zone offsets must be in the format '(+|-)HH', '(+|-)HH:mm' or '(+|-)HH:mm:ss', e.g '-08', " + + "'+01:00' or '-13:33:33'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. Other " + + "short names are not recommended to use because they can be ambiguous.") .version("2.2.0") .stringConf .checkValue(isValidTimezone, s"Cannot resolve the given timezone with" + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 363282ea95997..44d5285e56074 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution -import java.util.Locale +import java.time.ZoneOffset +import java.util.{Locale, TimeZone} import javax.ws.rs.core.UriBuilder import scala.collection.JavaConverters._ @@ -32,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} @@ -90,6 +92,41 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ResetCommand } + /** + * Create a [[SetCommand]] logical plan to set [[SQLConf.SESSION_LOCAL_TIMEZONE]] + * Example SQL : + * {{{ + * SET TIME ZONE LOCAL; + * SET TIME ZONE 'Asia/Shanghai'; + * SET TIME ZONE INTERVAL 10 HOURS; + * }}} + */ + override def visitSetTimeZone(ctx: SetTimeZoneContext): LogicalPlan = withOrigin(ctx) { + val key = SQLConf.SESSION_LOCAL_TIMEZONE.key + if (ctx.interval != null) { + val interval = parseIntervalLiteral(ctx.interval) + if (interval.months != 0 || interval.days != 0 || + math.abs(interval.microseconds) > 18 * DateTimeConstants.MICROS_PER_HOUR || + interval.microseconds % DateTimeConstants.MICROS_PER_SECOND != 0) { + throw new ParseException("The interval value must be in the range of [-18, +18] hours" + + " with second precision", + ctx.interval()) + } else { + val seconds = (interval.microseconds / DateTimeConstants.MICROS_PER_SECOND).toInt + SetCommand(Some(key -> Some(ZoneOffset.ofTotalSeconds(seconds).toString))) + } + } else if (ctx.timezone != null) { + ctx.timezone.getType match { + case SqlBaseParser.LOCAL => + SetCommand(Some(key -> Some(TimeZone.getDefault.getID))) + case _ => + SetCommand(Some(key -> Some(string(ctx.STRING)))) + } + } else { + throw new ParseException("Invalid time zone displacement value", ctx) + } + } + /** * Create a [[RefreshResource]] logical plan. */ diff --git a/sql/core/src/test/resources/sql-tests/inputs/timezone.sql b/sql/core/src/test/resources/sql-tests/inputs/timezone.sql new file mode 100644 index 0000000000000..105f2816fb549 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/timezone.sql @@ -0,0 +1,15 @@ +-- valid time zones +SET TIME ZONE 'Asia/Hong_Kong'; +SET TIME ZONE 'GMT+1'; +SET TIME ZONE INTERVAL 10 HOURS; +SET TIME ZONE INTERVAL '15:40:32' HOUR TO SECOND; +SET TIME ZONE LOCAL; + +-- invalid time zone +SET TIME ZONE; +SET TIME ZONE 'invalid/zone'; +SET TIME ZONE INTERVAL 3 DAYS; +SET TIME ZONE INTERVAL 24 HOURS; +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND; +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1'; +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND; diff --git a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out new file mode 100644 index 0000000000000..d816043130737 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out @@ -0,0 +1,135 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query +SET TIME ZONE 'Asia/Hong_Kong' +-- !query schema +struct +-- !query output +spark.sql.session.timeZone Asia/Hong_Kong + + +-- !query +SET TIME ZONE 'GMT+1' +-- !query schema +struct +-- !query output +spark.sql.session.timeZone GMT+1 + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS +-- !query schema +struct +-- !query output +spark.sql.session.timeZone +10:00 + + +-- !query +SET TIME ZONE INTERVAL '15:40:32' HOUR TO SECOND +-- !query schema +struct +-- !query output +spark.sql.session.timeZone +15:40:32 + + +-- !query +SET TIME ZONE LOCAL +-- !query schema +struct +-- !query output +spark.sql.session.timeZone America/Los_Angeles + + +-- !query +SET TIME ZONE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Invalid time zone displacement value(line 1, pos 0) + +== SQL == +SET TIME ZONE +^^^ + + +-- !query +SET TIME ZONE 'invalid/zone' +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Cannot resolve the given timezone with ZoneId.of(_, ZoneId.SHORT_IDS) + + +-- !query +SET TIME ZONE INTERVAL 3 DAYS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) + +== SQL == +SET TIME ZONE INTERVAL 3 DAYS +--------------^^^ + + +-- !query +SET TIME ZONE INTERVAL 24 HOURS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) + +== SQL == +SET TIME ZONE INTERVAL 24 HOURS +--------------^^^ + + +-- !query +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) + +== SQL == +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND +--------------^^^ + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Invalid time zone displacement value(line 1, pos 0) + +== SQL == +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1' +^^^ + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) + +== SQL == +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND +--------------^^^ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index f1cd37f91d78a..36cb5d94a027d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.sql.internal +import java.util.TimeZone + import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.log4j.Level import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.MIT import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext} @@ -383,4 +386,29 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } assert(e.getMessage === "Cannot resolve the given timezone with ZoneId.of(_, ZoneId.SHORT_IDS)") } + + test("set time zone") { + TimeZone.getAvailableIDs().foreach { zid => + sql(s"set time zone '$zid'") + assert(spark.conf.get(SQLConf.SESSION_LOCAL_TIMEZONE) === zid) + } + sql("set time zone local") + assert(spark.conf.get(SQLConf.SESSION_LOCAL_TIMEZONE) === TimeZone.getDefault.getID) + + val e1 = intercept[IllegalArgumentException](sql("set time zone 'invalid'")) + assert(e1.getMessage === "Cannot resolve the given timezone with" + + " ZoneId.of(_, ZoneId.SHORT_IDS)") + + (-18 to 18).map(v => (v, s"interval '$v' hours")).foreach { case (i, interval) => + sql(s"set time zone $interval") + val zone = spark.conf.get(SQLConf.SESSION_LOCAL_TIMEZONE) + if (i == 0) { + assert(zone === "Z") + } else { + assert(zone === String.format("%+03d:00", new Integer(i))) + } + } + val e2 = intercept[ParseException](sql("set time zone interval 19 hours")) + assert(e2.getMessage contains "The interval value must be in the range of [-18, +18] hours") + } } From 6be8b935a4f7ce0dea2d7aaaf747c2e8e1a9f47a Mon Sep 17 00:00:00 2001 From: SaurabhChawla Date: Thu, 16 Jul 2020 13:11:47 +0000 Subject: [PATCH 207/384] [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables ### What changes were proposed in this pull request? Spark sql commands are failing on selecting the orc tables Steps to reproduce Example 1 - Prerequisite - This is the location(/Users/test/tpcds_scale5data/date_dim) for orc data which is generated by the hive. ``` val table = """CREATE TABLE `date_dim` ( `d_date_sk` INT, `d_date_id` STRING, `d_date` TIMESTAMP, `d_month_seq` INT, `d_week_seq` INT, `d_quarter_seq` INT, `d_year` INT, `d_dow` INT, `d_moy` INT, `d_dom` INT, `d_qoy` INT, `d_fy_year` INT, `d_fy_quarter_seq` INT, `d_fy_week_seq` INT, `d_day_name` STRING, `d_quarter_name` STRING, `d_holiday` STRING, `d_weekend` STRING, `d_following_holiday` STRING, `d_first_dom` INT, `d_last_dom` INT, `d_same_day_ly` INT, `d_same_day_lq` INT, `d_current_day` STRING, `d_current_week` STRING, `d_current_month` STRING, `d_current_quarter` STRING, `d_current_year` STRING) USING orc LOCATION '/Users/test/tpcds_scale5data/date_dim'""" spark.sql(table).collect val u = """select date_dim.d_date_id from date_dim limit 5""" spark.sql(u).collect ``` Example 2 ``` val table = """CREATE TABLE `test_orc_data` ( `_col1` INT, `_col2` STRING, `_col3` INT) USING orc""" spark.sql(table).collect spark.sql("insert into test_orc_data values(13, '155', 2020)").collect val df = """select _col2 from test_orc_data limit 5""" spark.sql(df).collect ``` Its Failing with below error ``` org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 2, 192.168.0.103, executor driver): java.lang.ArrayIndexOutOfBoundsException: 1 at org.apache.spark.sql.execution.datasources.orc.OrcColumnarBatchReader.initBatch(OrcColumnarBatchReader.java:156) at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat.$anonfun$buildReaderWithPartitionValues$7(OrcFileFormat.scala:258) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:141) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:203) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:116) at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.hasNext(DataSourceScanExec.scala:620) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown Source) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729) at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:343) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:895) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:895) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:372) at org.apache.spark.rdd.RDD.iterator(RDD.scala:336) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:133) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:445) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1489) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:448) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748)` ``` The reason behind this initBatch is not getting the schema that is needed to find out the column value in OrcFileFormat.scala ``` batchReader.initBatch( TypeDescription.fromString(resultSchemaString) ``` ### Why are the changes needed? Spark sql queries for orc tables are failing ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test is added for this .Also Tested through spark shell and spark submit the failing queries Closes #29045 from SaurabhChawla100/SPARK-32234. Lead-authored-by: SaurabhChawla Co-authored-by: SaurabhChawla Signed-off-by: Wenchen Fan --- .../datasources/orc/OrcFileFormat.scala | 10 ++--- .../execution/datasources/orc/OrcUtils.scala | 41 +++++++++++++++---- .../v2/orc/OrcPartitionReaderFactory.scala | 21 +++++----- .../sql/hive/orc/HiveOrcQuerySuite.scala | 28 +++++++++++++ 4 files changed, 78 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index fd791ce7c5e19..4dff1ec7ebfb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -164,8 +164,6 @@ class OrcFileFormat val enableVectorizedReader = supportBatch(sparkSession, resultSchema) val capacity = sqlConf.orcVectorizedReaderBatchSize - val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) - OrcConf.MAPRED_INPUT_SCHEMA.setString(hadoopConf, resultSchemaString) OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(hadoopConf, sqlConf.caseSensitiveAnalysis) val broadcastedConf = @@ -179,16 +177,18 @@ class OrcFileFormat val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val requestedColIdsOrEmptyFile = + val resultedColPruneInfo = Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, requiredSchema, reader, conf) } - if (requestedColIdsOrEmptyFile.isEmpty) { + if (resultedColPruneInfo.isEmpty) { Iterator.empty } else { - val requestedColIds = requestedColIdsOrEmptyFile.get + val (requestedColIds, canPruneCols) = resultedColPruneInfo.get + val resultSchemaString = OrcUtils.orcResultSchemaString(canPruneCols, + dataSchema, resultSchema, partitionSchema, conf) assert(requestedColIds.length == requiredSchema.length, "[BUG] requested column IDs do not match required schema") val taskConf = new Configuration(conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index d274bcd0edd2c..e10253989788b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.orc.{OrcFile, Reader, TypeDescription, Writer} +import org.apache.orc.{OrcConf, OrcFile, Reader, TypeDescription, Writer} import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} import org.apache.spark.deploy.SparkHadoopUtil @@ -116,15 +116,17 @@ object OrcUtils extends Logging { } /** - * Returns the requested column ids from the given ORC file. Column id can be -1, which means the - * requested column doesn't exist in the ORC file. Returns None if the given ORC file is empty. + * @return Returns the combination of requested column ids from the given ORC file and + * boolean flag to find if the pruneCols is allowed or not. Requested Column id can be + * -1, which means the requested column doesn't exist in the ORC file. Returns None + * if the given ORC file is empty. */ def requestedColumnIds( isCaseSensitive: Boolean, dataSchema: StructType, requiredSchema: StructType, reader: Reader, - conf: Configuration): Option[Array[Int]] = { + conf: Configuration): Option[(Array[Int], Boolean)] = { val orcFieldNames = reader.getSchema.getFieldNames.asScala if (orcFieldNames.isEmpty) { // SPARK-8501: Some old empty ORC files always have an empty schema stored in their footer. @@ -136,6 +138,10 @@ object OrcUtils extends Logging { assert(orcFieldNames.length <= dataSchema.length, "The given data schema " + s"${dataSchema.catalogString} has less fields than the actual ORC physical schema, " + "no idea which columns were dropped, fail to read.") + // for ORC file written by Hive, no field names + // in the physical schema, there is a need to send the + // entire dataSchema instead of required schema. + // So pruneCols is not done in this case Some(requiredSchema.fieldNames.map { name => val index = dataSchema.fieldIndex(name) if (index < orcFieldNames.length) { @@ -143,7 +149,7 @@ object OrcUtils extends Logging { } else { -1 } - }) + }, false) } else { if (isCaseSensitive) { Some(requiredSchema.fieldNames.zipWithIndex.map { case (name, idx) => @@ -152,7 +158,7 @@ object OrcUtils extends Logging { } else { -1 } - }) + }, true) } else { // Do case-insensitive resolution only if in case-insensitive mode val caseInsensitiveOrcFieldMap = orcFieldNames.groupBy(_.toLowerCase(Locale.ROOT)) @@ -170,7 +176,7 @@ object OrcUtils extends Logging { idx } }.getOrElse(-1) - }) + }, true) } } } @@ -199,4 +205,25 @@ object OrcUtils extends Logging { s"map<${orcTypeDescriptionString(m.keyType)},${orcTypeDescriptionString(m.valueType)}>" case _ => dt.catalogString } + + /** + * @return Returns the result schema string based on the canPruneCols flag. + * resultSchemaString will be created using resultsSchema in case of + * canPruneCols is true and for canPruneCols as false value + * resultSchemaString will be created using the actual dataSchema. + */ + def orcResultSchemaString( + canPruneCols: Boolean, + dataSchema: StructType, + resultSchema: StructType, + partitionSchema: StructType, + conf: Configuration): String = { + val resultSchemaString = if (canPruneCols) { + OrcUtils.orcTypeDescriptionString(resultSchema) + } else { + OrcUtils.orcTypeDescriptionString(StructType(dataSchema.fields ++ partitionSchema.fields)) + } + OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString) + resultSchemaString + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 03d58fdcb7f67..7f25f7bd135f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -66,24 +66,24 @@ case class OrcPartitionReaderFactory( override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { val conf = broadcastedConf.value.value - val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) - OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString) OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, isCaseSensitive) val filePath = new Path(new URI(file.filePath)) val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val requestedColIdsOrEmptyFile = + val resultedColPruneInfo = Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, readDataSchema, reader, conf) } - if (requestedColIdsOrEmptyFile.isEmpty) { + if (resultedColPruneInfo.isEmpty) { new EmptyPartitionReader[InternalRow] } else { - val requestedColIds = requestedColIdsOrEmptyFile.get + val (requestedColIds, canPruneCols) = resultedColPruneInfo.get + val resultSchemaString = OrcUtils.orcResultSchemaString(canPruneCols, + dataSchema, resultSchema, partitionSchema, conf) assert(requestedColIds.length == readDataSchema.length, "[BUG] requested column IDs do not match required schema") @@ -112,24 +112,25 @@ case class OrcPartitionReaderFactory( override def buildColumnarReader(file: PartitionedFile): PartitionReader[ColumnarBatch] = { val conf = broadcastedConf.value.value - val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) - OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString) OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, isCaseSensitive) val filePath = new Path(new URI(file.filePath)) val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val requestedColIdsOrEmptyFile = + val resultedColPruneInfo = Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, readDataSchema, reader, conf) } - if (requestedColIdsOrEmptyFile.isEmpty) { + if (resultedColPruneInfo.isEmpty) { new EmptyPartitionReader } else { - val requestedColIds = requestedColIdsOrEmptyFile.get ++ Array.fill(partitionSchema.length)(-1) + val (requestedDataColIds, canPruneCols) = resultedColPruneInfo.get + val resultSchemaString = OrcUtils.orcResultSchemaString(canPruneCols, + dataSchema, resultSchema, partitionSchema, conf) + val requestedColIds = requestedDataColIds ++ Array.fill(partitionSchema.length)(-1) assert(requestedColIds.length == resultSchema.length, "[BUG] requested column IDs do not match required schema") val taskConf = new Configuration(conf) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala index 990d9425fb7fc..12ee5bea7c2f9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala @@ -288,4 +288,32 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { } } } + + test("SPARK-32234 read ORC table with column names all starting with '_col'") { + Seq("native", "hive").foreach { orcImpl => + Seq("false", "true").foreach { vectorized => + withSQLConf( + SQLConf.ORC_IMPLEMENTATION.key -> orcImpl, + SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized) { + withTable("test_hive_orc_impl") { + spark.sql( + s""" + | CREATE TABLE test_hive_orc_impl + | (_col1 INT, _col2 STRING, _col3 INT) + | STORED AS ORC + """.stripMargin) + spark.sql( + s""" + | INSERT INTO + | test_hive_orc_impl + | VALUES(9, '12', 2020) + """.stripMargin) + + val df = spark.sql("SELECT _col2 FROM test_hive_orc_impl") + checkAnswer(df, Row("12")) + } + } + } + } + } } From c1f160e0972858d727d58ceab5dce0f6b48425dd Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 17 Jul 2020 00:01:13 +0900 Subject: [PATCH 208/384] [SPARK-30648][SQL] Support filters pushdown in JSON datasource ### What changes were proposed in this pull request? In the PR, I propose to support pushed down filters in JSON datasource. The reason of pushing a filter up to `JacksonParser` is to apply the filter as soon as all its attributes become available i.e. converted from JSON field values to desired values according to the schema. This allows to skip parsing of the rest of JSON record and conversions of other values if the filter returns `false`. This can improve performance when pushed filters are highly selective and conversion of JSON string fields to desired values are comparably expensive ( for example, the conversion to `TIMESTAMP` values). The main idea behind of `JsonFilters` is to group pushdown filters by their references, convert the grouped filters to expressions, and then compile to predicates. The predicates are indexed by schema field positions. Each predicate has a state with reference counter to non-set row fields. As soon as the counter reaches `0`, it can be applied to the row because all its dependencies has been set. Before processing new row, predicate's reference counter is reset to total number of predicate references (dependencies in a row). The common code shared between `CSVFilters` and `JsonFilters` is moved to the `StructFilters` class and its companion object. ### Why are the changes needed? The changes improve performance on synthetic benchmarks up to **27 times** on JDK 8 and **25** times on JDK 11: ``` OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1044-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 25230 25255 22 0.0 252299.6 1.0X pushdown disabled 25248 25282 33 0.0 252475.6 1.0X w/ filters 905 911 8 0.1 9047.9 27.9X ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? - Added new test suites `JsonFiltersSuite` and `JacksonParserSuite`. - By new end-to-end and case sensitivity tests in `JsonSuite`. - By `CSVFiltersSuite`, `UnivocityParserSuite` and `CSVSuite`. - Re-running `CSVBenchmark` and `JsonBenchmark` using Amazon EC2: | Item | Description | | ---- | ----| | Region | us-west-2 (Oregon) | | Instance | r3.xlarge (spot instance) | | AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) | | Java | OpenJDK8/11 installed by`sudo add-apt-repository ppa:openjdk-r/ppa` & `sudo apt install openjdk-11-jdk`| and `./dev/run-benchmarks`: ```python #!/usr/bin/env python3 import os from sparktestsupport.shellutils import run_cmd benchmarks = [ ['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 #27366 from MaxGekk/json-filters-pushdown. Lead-authored-by: Maxim Gekk Co-authored-by: Max Gekk Signed-off-by: HyukjinKwon --- .../spark/sql/catalyst/StructFilters.scala | 166 ++++++++++++++++++ .../spark/sql/catalyst/csv/CSVFilters.scala | 124 ++----------- .../sql/catalyst/json/JacksonParser.scala | 27 ++- .../spark/sql/catalyst/json/JsonFilters.scala | 156 ++++++++++++++++ .../apache/spark/sql/internal/SQLConf.scala | 8 + .../sql/catalyst/StructFiltersSuite.scala | 136 ++++++++++++++ .../sql/catalyst/csv/CSVFiltersSuite.scala | 116 +----------- .../catalyst/json/JacksonParserSuite.scala | 57 ++++++ .../sql/catalyst/json/JsonFiltersSuite.scala | 28 +++ .../benchmarks/CSVBenchmark-jdk11-results.txt | 64 +++---- sql/core/benchmarks/CSVBenchmark-results.txt | 64 +++---- .../JsonBenchmark-jdk11-results.txt | 94 +++++----- sql/core/benchmarks/JsonBenchmark-results.txt | 94 +++++----- .../datasources/json/JsonFileFormat.scala | 6 +- .../datasources/v2/csv/CSVScanBuilder.scala | 4 +- .../v2/json/JsonPartitionReaderFactory.scala | 11 +- .../datasources/v2/json/JsonScan.scala | 12 +- .../datasources/v2/json/JsonScanBuilder.scala | 26 ++- .../execution/datasources/csv/CSVSuite.scala | 30 ++++ .../datasources/json/JsonBenchmark.scala | 45 ++++- .../datasources/json/JsonSuite.scala | 151 +++++++++++++++- 21 files changed, 1029 insertions(+), 390 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/StructFiltersSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonFiltersSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala new file mode 100644 index 0000000000000..fed1b323f5773 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import scala.util.Try + +import org.apache.spark.sql.catalyst.StructFilters._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.{BooleanType, StructType} + +/** + * The class provides API for applying pushed down filters to partially or + * fully set internal rows that have the struct schema. + * + * `StructFilters` assumes that: + * - `reset()` is called before any `skipRow()` calls for new row. + * + * @param pushedFilters The pushed down source filters. The filters should refer to + * the fields of the provided schema. + * @param schema The required schema of records from datasource files. + */ +abstract class StructFilters(pushedFilters: Seq[sources.Filter], schema: StructType) { + + protected val filters = StructFilters.pushedFilters(pushedFilters.toArray, schema) + + /** + * Applies pushed down source filters to the given row assuming that + * value at `index` has been already set. + * + * @param row The row with fully or partially set values. + * @param index The index of already set value. + * @return `true` if currently processed row can be skipped otherwise false. + */ + def skipRow(row: InternalRow, index: Int): Boolean + + /** + * Resets states of pushed down filters. The method must be called before + * precessing any new row otherwise `skipRow()` may return wrong result. + */ + def reset(): Unit + + /** + * Compiles source filters to a predicate. + */ + def toPredicate(filters: Seq[sources.Filter]): BasePredicate = { + val reducedExpr = filters + .sortBy(_.references.length) + .flatMap(filterToExpression(_, toRef)) + .reduce(And) + Predicate.create(reducedExpr) + } + + // Finds a filter attribute in the schema and converts it to a `BoundReference` + private def toRef(attr: String): Option[BoundReference] = { + // The names have been normalized and case sensitivity is not a concern here. + schema.getFieldIndex(attr).map { index => + val field = schema(index) + BoundReference(index, field.dataType, field.nullable) + } + } +} + +object StructFilters { + private def checkFilterRefs(filter: sources.Filter, fieldNames: Set[String]): Boolean = { + // The names have been normalized and case sensitivity is not a concern here. + filter.references.forall(fieldNames.contains) + } + + /** + * Returns the filters currently supported by the datasource. + * @param filters The filters pushed down to the datasource. + * @param schema data schema of datasource files. + * @return a sub-set of `filters` that can be handled by the datasource. + */ + def pushedFilters(filters: Array[sources.Filter], schema: StructType): Array[sources.Filter] = { + val fieldNames = schema.fieldNames.toSet + filters.filter(checkFilterRefs(_, fieldNames)) + } + + private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = { + a.zip(b).headOption + } + + private def toLiteral(value: Any): Option[Literal] = { + Try(Literal(value)).toOption + } + + /** + * Converts a filter to an expression and binds it to row positions. + * + * @param filter The filter to convert. + * @param toRef The function converts a filter attribute to a bound reference. + * @return some expression with resolved attributes or `None` if the conversion + * of the given filter to an expression is impossible. + */ + def filterToExpression( + filter: sources.Filter, + toRef: String => Option[BoundReference]): Option[Expression] = { + def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { + zip(toRef(name), toLiteral(value)) + } + def translate(filter: sources.Filter): Option[Expression] = filter match { + case sources.And(left, right) => + zip(translate(left), translate(right)).map(And.tupled) + case sources.Or(left, right) => + zip(translate(left), translate(right)).map(Or.tupled) + case sources.Not(child) => + translate(child).map(Not) + case sources.EqualTo(attribute, value) => + zipAttributeAndValue(attribute, value).map(EqualTo.tupled) + case sources.EqualNullSafe(attribute, value) => + zipAttributeAndValue(attribute, value).map(EqualNullSafe.tupled) + case sources.IsNull(attribute) => + toRef(attribute).map(IsNull) + case sources.IsNotNull(attribute) => + toRef(attribute).map(IsNotNull) + case sources.In(attribute, values) => + val literals = values.toSeq.flatMap(toLiteral) + if (literals.length == values.length) { + toRef(attribute).map(In(_, literals)) + } else { + None + } + case sources.GreaterThan(attribute, value) => + zipAttributeAndValue(attribute, value).map(GreaterThan.tupled) + case sources.GreaterThanOrEqual(attribute, value) => + zipAttributeAndValue(attribute, value).map(GreaterThanOrEqual.tupled) + case sources.LessThan(attribute, value) => + zipAttributeAndValue(attribute, value).map(LessThan.tupled) + case sources.LessThanOrEqual(attribute, value) => + zipAttributeAndValue(attribute, value).map(LessThanOrEqual.tupled) + case sources.StringContains(attribute, value) => + zipAttributeAndValue(attribute, value).map(Contains.tupled) + case sources.StringStartsWith(attribute, value) => + zipAttributeAndValue(attribute, value).map(StartsWith.tupled) + case sources.StringEndsWith(attribute, value) => + zipAttributeAndValue(attribute, value).map(EndsWith.tupled) + case sources.AlwaysTrue() => + Some(Literal(true, BooleanType)) + case sources.AlwaysFalse() => + Some(Literal(false, BooleanType)) + } + translate(filter) + } +} + +class NoopFilters extends StructFilters(Seq.empty, new StructType()) { + override def skipRow(row: InternalRow, index: Int): Boolean = false + override def reset(): Unit = {} +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index b50a76a496556..d2cb2c4d8134a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -17,13 +17,11 @@ package org.apache.spark.sql.catalyst.csv -import scala.util.Try - -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, StructFilters} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources -import org.apache.spark.sql.types.{BooleanType, StructType} +import org.apache.spark.sql.types.StructType /** * An instance of the class compiles filters to predicates and allows to @@ -33,7 +31,8 @@ import org.apache.spark.sql.types.{BooleanType, StructType} * @param filters The filters pushed down to CSV datasource. * @param requiredSchema The schema with only fields requested by the upper layer. */ -class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) { +class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) + extends StructFilters(filters, requiredSchema) { /** * Converted filters to predicates and grouped by maximum field index * in the read schema. For example, if an filter refers to 2 attributes @@ -54,30 +53,27 @@ class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) { for (filter <- filters) { val refs = filter.references val index = if (refs.isEmpty) { - // For example, AlwaysTrue and AlwaysFalse doesn't have any references + // For example, `AlwaysTrue` and `AlwaysFalse` doesn't have any references // Filters w/o refs always return the same result. Taking into account - // that predicates are combined via And, we can apply such filters only + // that predicates are combined via `And`, we can apply such filters only // once at the position 0. 0 } else { // readSchema must contain attributes of all filters. - // Accordingly, fieldIndex() returns a valid index always. + // Accordingly, `fieldIndex()` returns a valid index always. refs.map(requiredSchema.fieldIndex).max } groupedFilters(index) :+= filter } if (len > 0 && !groupedFilters(0).isEmpty) { - // We assume that filters w/o refs like AlwaysTrue and AlwaysFalse + // We assume that filters w/o refs like `AlwaysTrue` and `AlwaysFalse` // can be evaluated faster that others. We put them in front of others. val (literals, others) = groupedFilters(0).partition(_.references.isEmpty) groupedFilters(0) = literals ++ others } for (i <- 0 until len) { if (!groupedFilters(i).isEmpty) { - val reducedExpr = groupedFilters(i) - .flatMap(CSVFilters.filterToExpression(_, toRef)) - .reduce(And) - groupedPredicates(i) = Predicate.create(reducedExpr) + groupedPredicates(i) = toPredicate(groupedFilters(i)) } } } @@ -85,107 +81,21 @@ class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) { } /** - * Applies all filters that refer to row fields at the positions from 0 to index. + * Applies all filters that refer to row fields at the positions from 0 to `index`. * @param row The internal row to check. * @param index Maximum field index. The function assumes that all fields - * from 0 to index position are set. - * @return false iff row fields at the position from 0 to index pass filters + * from 0 to `index` position are set. + * @return false` iff row fields at the position from 0 to `index` pass filters * or there are no applicable filters - * otherwise false if at least one of the filters returns false. + * otherwise `false` if at least one of the filters returns `false`. */ def skipRow(row: InternalRow, index: Int): Boolean = { val predicate = predicates(index) predicate != null && !predicate.eval(row) } - // Finds a filter attribute in the read schema and converts it to a `BoundReference` - private def toRef(attr: String): Option[BoundReference] = { - requiredSchema.getFieldIndex(attr).map { index => - val field = requiredSchema(index) - BoundReference(requiredSchema.fieldIndex(attr), field.dataType, field.nullable) - } - } -} - -object CSVFilters { - private def checkFilterRefs(filter: sources.Filter, schema: StructType): Boolean = { - val fieldNames = schema.fields.map(_.name).toSet - filter.references.forall(fieldNames.contains(_)) - } - - /** - * Returns the filters currently supported by CSV datasource. - * @param filters The filters pushed down to CSV datasource. - * @param schema data schema of CSV files. - * @return a sub-set of `filters` that can be handled by CSV datasource. - */ - def pushedFilters(filters: Array[sources.Filter], schema: StructType): Array[sources.Filter] = { - filters.filter(checkFilterRefs(_, schema)) - } - - private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = { - a.zip(b).headOption - } - - private def toLiteral(value: Any): Option[Literal] = { - Try(Literal(value)).toOption - } - - /** - * Converts a filter to an expression and binds it to row positions. - * - * @param filter The filter to convert. - * @param toRef The function converts a filter attribute to a bound reference. - * @return some expression with resolved attributes or None if the conversion - * of the given filter to an expression is impossible. - */ - def filterToExpression( - filter: sources.Filter, - toRef: String => Option[BoundReference]): Option[Expression] = { - def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { - zip(toRef(name), toLiteral(value)) - } - def translate(filter: sources.Filter): Option[Expression] = filter match { - case sources.And(left, right) => - zip(translate(left), translate(right)).map(And.tupled) - case sources.Or(left, right) => - zip(translate(left), translate(right)).map(Or.tupled) - case sources.Not(child) => - translate(child).map(Not) - case sources.EqualTo(attribute, value) => - zipAttributeAndValue(attribute, value).map(EqualTo.tupled) - case sources.EqualNullSafe(attribute, value) => - zipAttributeAndValue(attribute, value).map(EqualNullSafe.tupled) - case sources.IsNull(attribute) => - toRef(attribute).map(IsNull) - case sources.IsNotNull(attribute) => - toRef(attribute).map(IsNotNull) - case sources.In(attribute, values) => - val literals = values.toSeq.flatMap(toLiteral) - if (literals.length == values.length) { - toRef(attribute).map(In(_, literals)) - } else { - None - } - case sources.GreaterThan(attribute, value) => - zipAttributeAndValue(attribute, value).map(GreaterThan.tupled) - case sources.GreaterThanOrEqual(attribute, value) => - zipAttributeAndValue(attribute, value).map(GreaterThanOrEqual.tupled) - case sources.LessThan(attribute, value) => - zipAttributeAndValue(attribute, value).map(LessThan.tupled) - case sources.LessThanOrEqual(attribute, value) => - zipAttributeAndValue(attribute, value).map(LessThanOrEqual.tupled) - case sources.StringContains(attribute, value) => - zipAttributeAndValue(attribute, value).map(Contains.tupled) - case sources.StringStartsWith(attribute, value) => - zipAttributeAndValue(attribute, value).map(StartsWith.tupled) - case sources.StringEndsWith(attribute, value) => - zipAttributeAndValue(attribute, value).map(EndsWith.tupled) - case sources.AlwaysTrue() => - Some(Literal(true, BooleanType)) - case sources.AlwaysFalse() => - Some(Literal(false, BooleanType)) - } - translate(filter) - } + // CSV filters are applied sequentially, and no need to track which filter references + // point out to already set row values. The `reset()` method is trivial because + // the filters don't have any states. + def reset(): Unit = {} } 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 7f69180e87e7e..9ddbacbbc3e6e 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 @@ -27,11 +27,12 @@ import com.fasterxml.jackson.core._ import org.apache.spark.SparkUpgradeException import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, StructFilters} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.Utils @@ -42,7 +43,8 @@ import org.apache.spark.util.Utils class JacksonParser( schema: DataType, val options: JSONOptions, - allowArrayAsStructs: Boolean) extends Logging { + allowArrayAsStructs: Boolean, + filters: Seq[Filter] = Seq.empty) extends Logging { import JacksonUtils._ import com.fasterxml.jackson.core.JsonToken._ @@ -85,8 +87,9 @@ class JacksonParser( private def makeStructRootConverter(st: StructType): JsonParser => Iterable[InternalRow] = { val elementConverter = makeConverter(st) val fieldConverters = st.map(_.dataType).map(makeConverter).toArray + val jsonFilters = new JsonFilters(filters, st) (parser: JsonParser) => parseJsonToken[Iterable[InternalRow]](parser, st) { - case START_OBJECT => Some(convertObject(parser, st, fieldConverters)) + case START_OBJECT => convertObject(parser, st, fieldConverters, jsonFilters) // SPARK-3308: support reading top level JSON arrays and take every element // in such an array as a row // @@ -146,7 +149,7 @@ class JacksonParser( // val st = at.elementType.asInstanceOf[StructType] val fieldConverters = st.map(_.dataType).map(makeConverter).toArray - Some(InternalRow(new GenericArrayData(Seq(convertObject(parser, st, fieldConverters))))) + Some(InternalRow(new GenericArrayData(convertObject(parser, st, fieldConverters).toArray))) } } @@ -291,7 +294,7 @@ class JacksonParser( case st: StructType => val fieldConverters = st.map(_.dataType).map(makeConverter).toArray (parser: JsonParser) => parseJsonToken[InternalRow](parser, dataType) { - case START_OBJECT => convertObject(parser, st, fieldConverters) + case START_OBJECT => convertObject(parser, st, fieldConverters).get } case at: ArrayType => @@ -375,15 +378,19 @@ class JacksonParser( private def convertObject( parser: JsonParser, schema: StructType, - fieldConverters: Array[ValueConverter]): InternalRow = { + fieldConverters: Array[ValueConverter], + structFilters: StructFilters = new NoopFilters()): Option[InternalRow] = { val row = new GenericInternalRow(schema.length) var badRecordException: Option[Throwable] = None + var skipRow = false - while (nextUntil(parser, JsonToken.END_OBJECT)) { + structFilters.reset() + while (!skipRow && nextUntil(parser, JsonToken.END_OBJECT)) { schema.getFieldIndex(parser.getCurrentName) match { case Some(index) => try { row.update(index, fieldConverters(index).apply(parser)) + skipRow = structFilters.skipRow(row, index) } catch { case e: SparkUpgradeException => throw e case NonFatal(e) => @@ -395,8 +402,10 @@ class JacksonParser( } } - if (badRecordException.isEmpty) { - row + if (skipRow) { + None + } else if (badRecordException.isEmpty) { + Some(row) } else { throw PartialResultException(row, badRecordException.get) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala new file mode 100644 index 0000000000000..09022bfc15403 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.json + +import org.apache.spark.sql.catalyst.{InternalRow, StructFilters} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType + +/** + * The class provides API for applying pushed down source filters to rows with + * a struct schema parsed from JSON records. The class should be used in this way: + * 1. Before processing of the next row, `JacksonParser` (parser for short) resets the internal + * state of `JsonFilters` by calling the `reset()` method. + * 2. The parser reads JSON fields one-by-one in streaming fashion. It converts an incoming + * field value to the desired type from the schema. After that, it sets the value to an instance + * of `InternalRow` at the position according to the schema. Order of parsed JSON fields can + * be different from the order in the schema. + * 3. Per every JSON field of the top-level JSON object, the parser calls `skipRow` by passing + * an `InternalRow` in which some of fields can be already set, and the position of the JSON + * field according to the schema. + * 3.1 `skipRow` finds a group of predicates that refers to this JSON field. + * 3.2 Per each predicate from the group, `skipRow` decrements its reference counter. + * 3.2.1 If predicate reference counter becomes 0, it means that all predicate attributes have + * been already set in the internal row, and the predicate can be applied to it. `skipRow` + * invokes the predicate for the row. + * 3.3 `skipRow` applies predicates until one of them returns `false`. In that case, the method + * returns `true` to the parser. + * 3.4 If all predicates with zero reference counter return `true`, the final result of + * the method is `false` which tells the parser to not skip the row. + * 4. If the parser gets `true` from `JsonFilters.skipRow`, it must not call the method anymore + * for this internal row, and should go the step 1. + * + * Besides of `StructFilters` assumptions, `JsonFilters` assumes that: + * - `skipRow()` can be called for any valid index of the struct fields, + * and in any order. + * - After `skipRow()` returns `true`, the internal state of `JsonFilters` can be inconsistent, + * so, `skipRow()` must not be called for the current row anymore without `reset()`. + * + * @param pushedFilters The pushed down source filters. The filters should refer to + * the fields of the provided schema. + * @param schema The required schema of records from datasource files. + */ +class JsonFilters(pushedFilters: Seq[sources.Filter], schema: StructType) + extends StructFilters(pushedFilters, schema) { + + /** + * Stateful JSON predicate that keeps track of its dependent references in the + * current row via `refCount`. + * + * @param predicate The predicate compiled from pushed down source filters. + * @param totalRefs The total amount of all filters references which the predicate + * compiled from. + */ + case class JsonPredicate(predicate: BasePredicate, totalRefs: Int) { + // The current number of predicate references in the row that have been not set yet. + // When `refCount` reaches zero, the predicate has all dependencies are set, and can + // be applied to the row. + var refCount: Int = totalRefs + + def reset(): Unit = { + refCount = totalRefs + } + } + + // Predicates compiled from the pushed down filters. The predicates are grouped by their + // attributes. The i-th group contains predicates that refer to the i-th field of the given + // schema. A predicates can be placed to many groups if it has many attributes. For example: + // schema: i INTEGER, s STRING + // filters: IsNotNull("i"), AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc")) + // predicates: + // 0: Array(IsNotNull("i"), AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + // 1: Array(AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + private val predicates: Array[Array[JsonPredicate]] = { + val groupedPredicates = Array.fill(schema.length)(Array.empty[JsonPredicate]) + if (SQLConf.get.jsonFilterPushDown) { + val groupedByRefSet: Map[Set[String], JsonPredicate] = filters + // Group filters that have the same set of references. For example: + // IsNotNull("i") -> Set("i"), AlwaysTrue -> Set(), + // Or(EqualTo("i", 0), StringStartsWith("s", "abc")) -> Set("i", "s") + // By grouping filters we could avoid tracking their state of references in the + // current row separately. + .groupBy(_.references.toSet) + // Combine all filters from the same group by `And` because all filters should + // return `true` to do not skip a row. The result is compiled to a predicate. + .map { case (refSet, refsFilters) => + (refSet, JsonPredicate(toPredicate(refsFilters), refSet.size)) + } + // Apply predicates w/o references like `AlwaysTrue` and `AlwaysFalse` to all fields. + // We cannot set such predicates to a particular position because skipRow() can + // be invoked for any index due to unpredictable order of JSON fields in JSON records. + val withLiterals: Map[Set[String], JsonPredicate] = groupedByRefSet.map { + case (refSet, pred) if refSet.isEmpty => + (schema.fields.map(_.name).toSet, pred.copy(totalRefs = 1)) + case others => others + } + // Build a map where key is only one field and value is seq of predicates refer to the field + // "i" -> Seq(AlwaysTrue, IsNotNull("i"), Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + // "s" -> Seq(AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + val groupedByFields: Map[String, Seq[(String, JsonPredicate)]] = withLiterals.toSeq + .flatMap { case (refSet, pred) => refSet.map((_, pred)) } + .groupBy(_._1) + // Build the final array by converting keys of `groupedByFields` to their + // indexes in the provided schema. + groupedByFields.foreach { case (fieldName, fieldPredicates) => + val fieldIndex = schema.fieldIndex(fieldName) + groupedPredicates(fieldIndex) = fieldPredicates.map(_._2).toArray + } + } + groupedPredicates + } + + /** + * Applies predicates (compiled filters) associated with the row field value + * at the position `index` only if other predicates dependencies are already + * set in the given row. + * + * Note: If the function returns `true`, `refCount` of some predicates can be not decremented. + * + * @param row The row with fully or partially set values. + * @param index The index of already set value. + * @return `true` if at least one of applicable predicates (all dependent row values are set) + * return `false`. It returns `false` if all predicates return `true`. + */ + def skipRow(row: InternalRow, index: Int): Boolean = { + var skip = false + for (pred <- predicates(index) if !skip) { + pred.refCount -= 1 + assert(pred.refCount >= 0, + s"Predicate reference counter cannot be negative but got ${pred.refCount}.") + skip = pred.refCount == 0 && !pred.predicate.eval(row) + } + skip + } + + /** + * Reset states of all predicates by re-initializing reference counters. + */ + override def reset(): Unit = predicates.foreach(_.foreach(_.reset)) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 77d49aff33c53..f0d0a601ff196 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2543,6 +2543,12 @@ object SQLConf { .booleanConf .createWithDefault(true) + val JSON_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.json.filterPushdown.enabled") + .doc("When true, enable filter pushdown to JSON datasource.") + .version("3.1.0") + .booleanConf + .createWithDefault(true) + val ADD_PARTITION_BATCH_SIZE = buildConf("spark.sql.addPartitionInBatch.size") .internal() @@ -3255,6 +3261,8 @@ class SQLConf extends Serializable with Logging { def csvFilterPushDown: Boolean = getConf(CSV_FILTER_PUSHDOWN_ENABLED) + def jsonFilterPushDown: Boolean = getConf(JSON_FILTER_PUSHDOWN_ENABLED) + def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID) def legacyAllowCastNumericToTimestamp: Boolean = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/StructFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/StructFiltersSuite.scala new file mode 100644 index 0000000000000..3893b2b07c519 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/StructFiltersSuite.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.sources +import org.apache.spark.sql.sources.{AlwaysFalse, AlwaysTrue, Filter} +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.unsafe.types.UTF8String + +abstract class StructFiltersSuite extends SparkFunSuite { + + protected def createFilters(filters: Seq[sources.Filter], schema: StructType): StructFilters + + test("filter to expression conversion") { + val ref = BoundReference(0, IntegerType, true) + def check(f: Filter, expr: Expression): Unit = { + assert(StructFilters.filterToExpression(f, _ => Some(ref)).get === expr) + } + + check(sources.AlwaysTrue, Literal(true)) + check(sources.AlwaysFalse, Literal(false)) + check(sources.IsNull("a"), IsNull(ref)) + check(sources.Not(sources.IsNull("a")), Not(IsNull(ref))) + check(sources.IsNotNull("a"), IsNotNull(ref)) + check(sources.EqualTo("a", "b"), EqualTo(ref, Literal("b"))) + check(sources.EqualNullSafe("a", "b"), EqualNullSafe(ref, Literal("b"))) + check(sources.StringStartsWith("a", "b"), StartsWith(ref, Literal("b"))) + check(sources.StringEndsWith("a", "b"), EndsWith(ref, Literal("b"))) + check(sources.StringContains("a", "b"), Contains(ref, Literal("b"))) + check(sources.LessThanOrEqual("a", 1), LessThanOrEqual(ref, Literal(1))) + check(sources.LessThan("a", 1), LessThan(ref, Literal(1))) + check(sources.GreaterThanOrEqual("a", 1), GreaterThanOrEqual(ref, Literal(1))) + check(sources.GreaterThan("a", 1), GreaterThan(ref, Literal(1))) + check(sources.And(sources.AlwaysTrue, sources.AlwaysTrue), And(Literal(true), Literal(true))) + check(sources.Or(sources.AlwaysTrue, sources.AlwaysTrue), Or(Literal(true), Literal(true))) + check(sources.In("a", Array(1)), In(ref, Seq(Literal(1)))) + } + + private def getSchema(str: String): StructType = str match { + case "" => new StructType() + case _ => StructType.fromDDL(str) + } + + test("skipping rows") { + def check( + requiredSchema: String = "i INTEGER, d DOUBLE", + filters: Seq[Filter], + row: InternalRow, + pos: Int, + skip: Boolean): Unit = { + val structFilters = createFilters(filters, getSchema(requiredSchema)) + structFilters.reset() + assert(structFilters.skipRow(row, pos) === skip) + } + + check(filters = Seq(), row = InternalRow(3.14), pos = 0, skip = false) + check(filters = Seq(AlwaysTrue), row = InternalRow(1), pos = 0, skip = false) + check(filters = Seq(AlwaysFalse), row = InternalRow(1), pos = 0, skip = true) + check( + filters = Seq(sources.EqualTo("i", 1), sources.LessThan("d", 10), sources.AlwaysFalse), + row = InternalRow(1, 3.14), + pos = 0, + skip = true) + check( + filters = Seq(sources.EqualTo("i", 10)), + row = InternalRow(10, 3.14), + pos = 0, + skip = false) + check( + filters = Seq(sources.IsNotNull("d"), sources.GreaterThanOrEqual("d", 2.96)), + row = InternalRow(3.14), + pos = 0, + skip = false) + check( + filters = Seq(sources.In("i", Array(10, 20)), sources.LessThanOrEqual("d", 2.96)), + row = InternalRow(10, 3.14), + pos = 1, + skip = true) + val filters1 = Seq( + sources.Or( + sources.AlwaysTrue, + sources.And( + sources.Not(sources.IsNull("i")), + sources.Not( + sources.And( + sources.StringEndsWith("s", "ab"), + sources.StringEndsWith("s", "cd") + ) + ) + ) + ), + sources.GreaterThan("d", 0), + sources.LessThan("i", 500) + ) + val filters2 = Seq( + sources.And( + sources.StringContains("s", "abc"), + sources.And( + sources.Not(sources.IsNull("i")), + sources.And( + sources.StringEndsWith("s", "ab"), + sources.StringEndsWith("s", "bc") + ) + ) + ), + sources.GreaterThan("d", 100), + sources.LessThan("i", 0) + ) + Seq(filters1 -> false, filters2 -> true).foreach { case (filters, skip) => + val schema = "i INTEGER, d DOUBLE, s STRING" + val row = InternalRow(10, 3.14, UTF8String.fromString("abc")) + val structFilters = createFilters(filters, getSchema(schema)) + structFilters.reset() + for { p <- 0 until 3 if !skip } { + assert(structFilters.skipRow(row, p) === skip, s"p = $p filters = $filters skip = $skip") + } + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala index 499bbaf452aee..21bef20d7d4d9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -17,118 +17,12 @@ package org.apache.spark.sql.catalyst.csv -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.{StructFilters, StructFiltersSuite} import org.apache.spark.sql.sources -import org.apache.spark.sql.sources.{AlwaysFalse, AlwaysTrue, Filter} -import org.apache.spark.sql.types.{IntegerType, StructType} -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.sql.types.StructType -class CSVFiltersSuite extends SparkFunSuite { - test("filter to expression conversion") { - val ref = BoundReference(0, IntegerType, true) - def check(f: Filter, expr: Expression): Unit = { - assert(CSVFilters.filterToExpression(f, _ => Some(ref)).get === expr) - } - - check(sources.AlwaysTrue, Literal(true)) - check(sources.AlwaysFalse, Literal(false)) - check(sources.IsNull("a"), IsNull(ref)) - check(sources.Not(sources.IsNull("a")), Not(IsNull(ref))) - check(sources.IsNotNull("a"), IsNotNull(ref)) - check(sources.EqualTo("a", "b"), EqualTo(ref, Literal("b"))) - check(sources.EqualNullSafe("a", "b"), EqualNullSafe(ref, Literal("b"))) - check(sources.StringStartsWith("a", "b"), StartsWith(ref, Literal("b"))) - check(sources.StringEndsWith("a", "b"), EndsWith(ref, Literal("b"))) - check(sources.StringContains("a", "b"), Contains(ref, Literal("b"))) - check(sources.LessThanOrEqual("a", 1), LessThanOrEqual(ref, Literal(1))) - check(sources.LessThan("a", 1), LessThan(ref, Literal(1))) - check(sources.GreaterThanOrEqual("a", 1), GreaterThanOrEqual(ref, Literal(1))) - check(sources.GreaterThan("a", 1), GreaterThan(ref, Literal(1))) - check(sources.And(sources.AlwaysTrue, sources.AlwaysTrue), And(Literal(true), Literal(true))) - check(sources.Or(sources.AlwaysTrue, sources.AlwaysTrue), Or(Literal(true), Literal(true))) - check(sources.In("a", Array(1)), In(ref, Seq(Literal(1)))) - } - - private def getSchema(str: String): StructType = str match { - case "" => new StructType() - case _ => StructType.fromDDL(str) - } - - test("skipping rows") { - def check( - requiredSchema: String = "i INTEGER, d DOUBLE", - filters: Seq[Filter], - row: InternalRow, - pos: Int, - skip: Boolean): Unit = { - val csvFilters = new CSVFilters(filters, getSchema(requiredSchema)) - assert(csvFilters.skipRow(row, pos) === skip) - } - - check(filters = Seq(), row = InternalRow(3.14), pos = 0, skip = false) - check(filters = Seq(AlwaysTrue), row = InternalRow(1), pos = 0, skip = false) - check(filters = Seq(AlwaysFalse), row = InternalRow(1), pos = 0, skip = true) - check( - filters = Seq(sources.EqualTo("i", 1), sources.LessThan("d", 10), sources.AlwaysFalse), - row = InternalRow(1, 3.14), - pos = 0, - skip = true) - check( - filters = Seq(sources.EqualTo("i", 10)), - row = InternalRow(10, 3.14), - pos = 0, - skip = false) - check( - filters = Seq(sources.IsNotNull("d"), sources.GreaterThanOrEqual("d", 2.96)), - row = InternalRow(3.14), - pos = 0, - skip = false) - check( - filters = Seq(sources.In("i", Array(10, 20)), sources.LessThanOrEqual("d", 2.96)), - row = InternalRow(10, 3.14), - pos = 1, - skip = true) - val filters1 = Seq( - sources.Or( - sources.AlwaysTrue, - sources.And( - sources.Not(sources.IsNull("i")), - sources.Not( - sources.And( - sources.StringEndsWith("s", "ab"), - sources.StringEndsWith("s", "cd") - ) - ) - ) - ), - sources.GreaterThan("d", 0), - sources.LessThan("i", 500) - ) - val filters2 = Seq( - sources.And( - sources.StringContains("s", "abc"), - sources.And( - sources.Not(sources.IsNull("i")), - sources.And( - sources.StringEndsWith("s", "ab"), - sources.StringEndsWith("s", "bc") - ) - ) - ), - sources.GreaterThan("d", 100), - sources.LessThan("i", 0) - ) - Seq(filters1 -> false, filters2 -> true).foreach { case (filters, skip) => - for (p <- 0 until 3) { - check( - requiredSchema = "i INTEGER, d DOUBLE, s STRING", - filters = filters, - row = InternalRow(10, 3.14, UTF8String.fromString("abc")), - pos = p, - skip = skip) - } - } +class CSVFiltersSuite extends StructFiltersSuite { + override def createFilters(filters: Seq[sources.Filter], schema: StructType): StructFilters = { + new CSVFilters(filters, schema) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala new file mode 100644 index 0000000000000..587e22e787b87 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.json + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.{EqualTo, Filter, StringStartsWith} +import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.types.UTF8String + +class JacksonParserSuite extends SparkFunSuite { + test("skipping rows using pushdown filters") { + def check( + input: String = """{"i":1, "s": "a"}""", + schema: StructType = StructType.fromDDL("i INTEGER"), + filters: Seq[Filter], + expected: Seq[InternalRow]): Unit = { + val options = new JSONOptions(Map.empty[String, String], "GMT", "") + val parser = new JacksonParser(schema, options, false, filters) + val createParser = CreateJacksonParser.string _ + val actual = parser.parse(input, createParser, UTF8String.fromString) + assert(actual === expected) + } + + check(filters = Seq(), expected = Seq(InternalRow(1))) + check(filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow(1))) + check(filters = Seq(EqualTo("i", 2)), expected = Seq.empty) + check( + schema = StructType.fromDDL("s STRING"), + filters = Seq(StringStartsWith("s", "b")), + expected = Seq.empty) + check( + schema = StructType.fromDDL("i INTEGER, s STRING"), + filters = Seq(StringStartsWith("s", "a")), + expected = Seq(InternalRow(1, UTF8String.fromString("a")))) + check( + input = """{"i":1,"s": "a", "d": 3.14}""", + schema = StructType.fromDDL("i INTEGER, d DOUBLE"), + filters = Seq(EqualTo("d", 3.14)), + expected = Seq(InternalRow(1, 3.14))) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonFiltersSuite.scala new file mode 100644 index 0000000000000..82f5e71d67964 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonFiltersSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.json + +import org.apache.spark.sql.catalyst.{StructFilters, StructFiltersSuite} +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType + +class JsonFiltersSuite extends StructFiltersSuite { + override def createFilters(filters: Seq[sources.Filter], schema: StructType): StructFilters = { + new JsonFilters(filters, schema) + } +} diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index 0e82b632793d2..03c51ddad1c62 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -6,62 +6,62 @@ 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 Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 46568 46683 198 0.0 931358.6 1.0X +One quoted string 53332 53484 194 0.0 1066633.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 Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +Select 1000 columns 127472 128337 1185 0.0 127472.4 1.0X +Select 100 columns 43731 43856 130 0.0 43730.7 2.9X +Select one column 37347 37401 47 0.0 37347.4 3.4X +count() 8014 8028 25 0.1 8013.8 15.9X +Select 100 columns, one bad input field 95603 95726 106 0.0 95603.0 1.3X +Select 100 columns, corrupt record field 111851 111969 171 0.0 111851.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 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() 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 +Select 10 columns + count() 20364 20481 110 0.5 2036.4 1.0X +Select 1 column + count() 14706 14803 153 0.7 1470.6 1.4X +count() 3855 3880 32 2.6 385.5 5.3X 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 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 +Create a dataset of timestamps 2191 2205 14 4.6 219.1 1.0X +to_csv(timestamp) 13209 13253 43 0.8 1320.9 0.2X +write timestamps to files 12300 12380 71 0.8 1230.0 0.2X +Create a dataset of dates 2254 2269 14 4.4 225.4 1.0X +to_csv(date) 7980 8006 22 1.3 798.0 0.3X +write dates to files 7076 7100 26 1.4 707.6 0.3X 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 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 +read timestamp text from files 2405 2408 5 4.2 240.5 1.0X +read timestamps from files 54502 54624 207 0.2 5450.2 0.0X +infer timestamps from files 112896 113040 135 0.1 11289.6 0.0X +read date text from files 2127 2141 23 4.7 212.7 1.1X +read date from files 30229 30257 29 0.3 3022.9 0.1X +infer date from files 28156 28621 409 0.4 2815.6 0.1X +timestamp strings 3096 3097 1 3.2 309.6 0.8X +parse timestamps from Dataset[String] 63096 63751 571 0.2 6309.6 0.0X +infer timestamps from Dataset[String] 120916 121262 556 0.1 12091.6 0.0X +date strings 3445 3457 13 2.9 344.5 0.7X +parse dates from Dataset[String] 37481 37585 91 0.3 3748.1 0.1X +from_csv(timestamp) 57933 57996 69 0.2 5793.3 0.0X +from_csv(date) 35312 35469 164 0.3 3531.2 0.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 Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +w/o filters 24751 24829 67 0.0 247510.6 1.0X +pushdown disabled 24856 24889 29 0.0 248558.7 1.0X +w/ filters 1881 1892 11 0.1 18814.4 13.2X diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index a3af46c037bf9..a0d8c0c6fd492 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -6,62 +6,62 @@ 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 Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 45457 45731 344 0.0 909136.8 1.0X +One quoted string 47588 47831 244 0.0 951755.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 Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +Select 1000 columns 129509 130323 1388 0.0 129509.4 1.0X +Select 100 columns 42474 42572 108 0.0 42473.6 3.0X +Select one column 35479 35586 93 0.0 35479.1 3.7X +count() 11021 11071 47 0.1 11021.3 11.8X +Select 100 columns, one bad input field 94652 94795 134 0.0 94652.0 1.4X +Select 100 columns, corrupt record field 115336 115542 350 0.0 115336.0 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 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() 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 +Select 10 columns + count() 19959 20022 76 0.5 1995.9 1.0X +Select 1 column + count() 13920 13968 54 0.7 1392.0 1.4X +count() 3928 3938 11 2.5 392.8 5.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 Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +Create a dataset of timestamps 1940 1977 56 5.2 194.0 1.0X +to_csv(timestamp) 15398 15669 458 0.6 1539.8 0.1X +write timestamps to files 12438 12454 19 0.8 1243.8 0.2X +Create a dataset of dates 2157 2171 18 4.6 215.7 0.9X +to_csv(date) 11764 11839 95 0.9 1176.4 0.2X +write dates to files 8893 8907 12 1.1 889.3 0.2X 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 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 +read timestamp text from files 2219 2230 11 4.5 221.9 1.0X +read timestamps from files 51519 51725 192 0.2 5151.9 0.0X +infer timestamps from files 104744 104885 124 0.1 10474.4 0.0X +read date text from files 1940 1943 4 5.2 194.0 1.1X +read date from files 27099 27118 33 0.4 2709.9 0.1X +infer date from files 27662 27703 61 0.4 2766.2 0.1X +timestamp strings 4225 4242 15 2.4 422.5 0.5X +parse timestamps from Dataset[String] 56090 56479 376 0.2 5609.0 0.0X +infer timestamps from Dataset[String] 115629 116245 1049 0.1 11562.9 0.0X +date strings 4337 4344 10 2.3 433.7 0.5X +parse dates from Dataset[String] 32373 32476 120 0.3 3237.3 0.1X +from_csv(timestamp) 54952 55157 300 0.2 5495.2 0.0X +from_csv(date) 30924 30985 66 0.3 3092.4 0.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 Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -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 +w/o filters 25630 25636 8 0.0 256301.4 1.0X +pushdown disabled 25673 25681 9 0.0 256734.0 1.0X +w/ filters 1873 1886 15 0.1 18733.1 13.7X diff --git a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt index 2d506f03d9f7e..6f68d60ce619a 100644 --- a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt @@ -7,106 +7,114 @@ 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 JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 73307 73400 141 1.4 733.1 1.0X -UTF-8 is set 143834 143925 152 0.7 1438.3 0.5X +No encoding 70753 71127 471 1.4 707.5 1.0X +UTF-8 is set 128105 129183 1165 0.8 1281.1 0.6X Preparing data for benchmarking ... 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 50894 51065 292 2.0 508.9 1.0X -UTF-8 is set 98462 99455 1173 1.0 984.6 0.5X +No encoding 59588 59643 73 1.7 595.9 1.0X +UTF-8 is set 97081 97122 62 1.0 970.8 0.6X Preparing data for benchmarking ... 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 64011 64969 1001 0.2 6401.1 1.0X -UTF-8 is set 102757 102984 311 0.1 10275.7 0.6X +No encoding 58835 59259 659 0.2 5883.5 1.0X +UTF-8 is set 103117 103218 88 0.1 10311.7 0.6X Preparing data for benchmarking ... 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 132559 133561 1010 0.0 265117.3 1.0X -UTF-8 is set 151458 152129 611 0.0 302915.4 0.9X +No encoding 142993 143485 436 0.0 285985.3 1.0X +UTF-8 is set 165446 165496 60 0.0 330892.4 0.9X Preparing data for benchmarking ... 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 21148 21202 87 0.5 2114.8 1.0X -Select 1 column 24701 24724 21 0.4 2470.1 0.9X +Select 10 columns 21557 21593 61 0.5 2155.7 1.0X +Select 1 column 24197 24236 35 0.4 2419.7 0.9X Preparing data for benchmarking ... 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 6945 6998 59 1.4 694.5 1.0X -Short column with UTF-8 11510 11569 51 0.9 1151.0 0.6X -Wide column without encoding 95004 95795 790 0.1 9500.4 0.1X -Wide column with UTF-8 149223 149409 276 0.1 14922.3 0.0X +Short column without encoding 9795 9820 29 1.0 979.5 1.0X +Short column with UTF-8 16442 16536 146 0.6 1644.2 0.6X +Wide column without encoding 99134 99475 300 0.1 9913.4 0.1X +Wide column with UTF-8 155913 156369 692 0.1 15591.3 0.1X Preparing data for benchmarking ... 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 649 652 3 15.4 64.9 1.0X -from_json 22284 22393 99 0.4 2228.4 0.0X -json_tuple 32310 32824 484 0.3 3231.0 0.0X -get_json_object 22111 22751 568 0.5 2211.1 0.0X +Text read 671 679 7 14.9 67.1 1.0X +from_json 25356 25432 79 0.4 2535.6 0.0X +json_tuple 29464 29927 672 0.3 2946.4 0.0X +get_json_object 21841 21877 32 0.5 2184.1 0.0X Preparing data for benchmarking ... 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 2894 2903 8 17.3 57.9 1.0X -schema inferring 26724 26785 62 1.9 534.5 0.1X -parsing 37502 37632 131 1.3 750.0 0.1X +Text read 3109 3116 12 16.1 62.2 1.0X +schema inferring 28751 28765 15 1.7 575.0 0.1X +parsing 34923 35030 151 1.4 698.5 0.1X Preparing data for benchmarking ... 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 10994 11010 16 4.5 219.9 1.0X -Schema inferring 45654 45677 37 1.1 913.1 0.2X -Parsing without charset 34476 34559 73 1.5 689.5 0.3X -Parsing with UTF-8 56987 57002 13 0.9 1139.7 0.2X +Text read 10787 10818 32 4.6 215.7 1.0X +Schema inferring 49577 49775 184 1.0 991.5 0.2X +Parsing without charset 35343 35433 87 1.4 706.9 0.3X +Parsing with UTF-8 60253 60290 35 0.8 1205.1 0.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 Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2150 2188 35 4.7 215.0 1.0X -to_json(timestamp) 17874 18080 294 0.6 1787.4 0.1X -write timestamps to files 12518 12538 34 0.8 1251.8 0.2X -Create a dataset of dates 2298 2310 18 4.4 229.8 0.9X -to_json(date) 11673 11703 27 0.9 1167.3 0.2X -write dates to files 7121 7135 12 1.4 712.1 0.3X +Create a dataset of timestamps 2200 2209 8 4.5 220.0 1.0X +to_json(timestamp) 18410 18602 264 0.5 1841.0 0.1X +write timestamps to files 11841 12032 305 0.8 1184.1 0.2X +Create a dataset of dates 2353 2363 9 4.3 235.3 0.9X +to_json(date) 12135 12182 72 0.8 1213.5 0.2X +write dates to files 6776 6801 33 1.5 677.6 0.3X 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 2616 2641 34 3.8 261.6 1.0X -read timestamps from files 37481 37517 58 0.3 3748.1 0.1X -infer timestamps from files 84774 84964 201 0.1 8477.4 0.0X -read date text from files 2362 2365 3 4.2 236.2 1.1X -read date from files 16583 16612 29 0.6 1658.3 0.2X -timestamp strings 3927 3963 40 2.5 392.7 0.7X -parse timestamps from Dataset[String] 52827 53004 243 0.2 5282.7 0.0X -infer timestamps from Dataset[String] 101108 101644 769 0.1 10110.8 0.0X -date strings 4886 4906 26 2.0 488.6 0.5X -parse dates from Dataset[String] 27623 27694 62 0.4 2762.3 0.1X -from_json(timestamp) 71764 71887 124 0.1 7176.4 0.0X -from_json(date) 46200 46314 99 0.2 4620.0 0.1X +read timestamp text from files 2563 2580 20 3.9 256.3 1.0X +read timestamps from files 41261 41360 97 0.2 4126.1 0.1X +infer timestamps from files 92292 92517 243 0.1 9229.2 0.0X +read date text from files 2332 2340 11 4.3 233.2 1.1X +read date from files 18753 18768 13 0.5 1875.3 0.1X +timestamp strings 3108 3123 13 3.2 310.8 0.8X +parse timestamps from Dataset[String] 51078 51448 323 0.2 5107.8 0.1X +infer timestamps from Dataset[String] 101373 101429 65 0.1 10137.3 0.0X +date strings 4126 4138 15 2.4 412.6 0.6X +parse dates from Dataset[String] 29365 29398 30 0.3 2936.5 0.1X +from_json(timestamp) 67033 67098 63 0.1 6703.3 0.0X +from_json(date) 44495 44581 125 0.2 4449.5 0.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 +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 30167 30223 48 0.0 301674.9 1.0X +pushdown disabled 30291 30311 30 0.0 302914.8 1.0X +w/ filters 901 915 14 0.1 9012.4 33.5X diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt index c22118f91b3fc..38ad9d0077f9a 100644 --- a/sql/core/benchmarks/JsonBenchmark-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -7,106 +7,114 @@ 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 JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 63839 64000 263 1.6 638.4 1.0X -UTF-8 is set 124633 124945 429 0.8 1246.3 0.5X +No encoding 78058 78116 76 1.3 780.6 1.0X +UTF-8 is set 125709 126521 1367 0.8 1257.1 0.6X Preparing data for benchmarking ... 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 51720 51901 157 1.9 517.2 1.0X -UTF-8 is set 91161 91190 25 1.1 911.6 0.6X +No encoding 60424 60567 188 1.7 604.2 1.0X +UTF-8 is set 92714 92864 140 1.1 927.1 0.7X Preparing data for benchmarking ... 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 58486 59038 714 0.2 5848.6 1.0X -UTF-8 is set 103045 103350 358 0.1 10304.5 0.6X +No encoding 65047 65761 662 0.2 6504.7 1.0X +UTF-8 is set 101823 101918 113 0.1 10182.3 0.6X Preparing data for benchmarking ... 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 134909 135024 105 0.0 269818.6 1.0X -UTF-8 is set 154418 154593 155 0.0 308836.7 0.9X +No encoding 145471 146067 601 0.0 290941.4 1.0X +UTF-8 is set 158504 159237 635 0.0 317008.4 0.9X Preparing data for benchmarking ... 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 19538 19620 70 0.5 1953.8 1.0X -Select 1 column 26142 26159 15 0.4 2614.2 0.7X +Select 10 columns 21386 21451 112 0.5 2138.6 1.0X +Select 1 column 27172 27214 58 0.4 2717.2 0.8X Preparing data for benchmarking ... 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 8103 8162 53 1.2 810.3 1.0X -Short column with UTF-8 13104 13150 58 0.8 1310.4 0.6X -Wide column without encoding 135280 135593 375 0.1 13528.0 0.1X -Wide column with UTF-8 175189 175483 278 0.1 17518.9 0.0X +Short column without encoding 9283 9363 69 1.1 928.3 1.0X +Short column with UTF-8 15330 15369 61 0.7 1533.0 0.6X +Wide column without encoding 138885 139153 239 0.1 13888.5 0.1X +Wide column with UTF-8 177201 177650 501 0.1 17720.1 0.1X Preparing data for benchmarking ... 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 1225 1234 8 8.2 122.5 1.0X -from_json 22482 22552 95 0.4 2248.2 0.1X -json_tuple 30203 30338 146 0.3 3020.3 0.0X -get_json_object 22219 22245 26 0.5 2221.9 0.1X +Text read 1224 1243 17 8.2 122.4 1.0X +from_json 25191 25327 214 0.4 2519.1 0.0X +json_tuple 30333 30380 42 0.3 3033.3 0.0X +get_json_object 21611 21739 112 0.5 2161.1 0.1X Preparing data for benchmarking ... 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 5897 5904 10 8.5 117.9 1.0X -schema inferring 30282 30340 50 1.7 605.6 0.2X -parsing 33304 33577 289 1.5 666.1 0.2X +Text read 5923 5941 32 8.4 118.5 1.0X +schema inferring 34089 34238 135 1.5 681.8 0.2X +parsing 44699 45952 1108 1.1 894.0 0.1X Preparing data for benchmarking ... 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 9710 9757 80 5.1 194.2 1.0X -Schema inferring 35929 35939 9 1.4 718.6 0.3X -Parsing without charset 39175 39227 87 1.3 783.5 0.2X -Parsing with UTF-8 59188 59294 109 0.8 1183.8 0.2X +Text read 9727 9776 50 5.1 194.5 1.0X +Schema inferring 52529 52643 98 1.0 1050.6 0.2X +Parsing without charset 44563 44692 132 1.1 891.3 0.2X +Parsing with UTF-8 55558 55755 218 0.9 1111.2 0.2X 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 1967 1977 9 5.1 196.7 1.0X -to_json(timestamp) 17086 17304 371 0.6 1708.6 0.1X -write timestamps to files 12691 12716 28 0.8 1269.1 0.2X -Create a dataset of dates 2192 2217 39 4.6 219.2 0.9X -to_json(date) 10541 10656 137 0.9 1054.1 0.2X -write dates to files 7259 7311 46 1.4 725.9 0.3X +Create a dataset of timestamps 1945 1964 22 5.1 194.5 1.0X +to_json(timestamp) 17990 18135 249 0.6 1799.0 0.1X +write timestamps to files 13198 13234 45 0.8 1319.8 0.1X +Create a dataset of dates 2202 2213 11 4.5 220.2 0.9X +to_json(date) 11219 11240 29 0.9 1121.9 0.2X +write dates to files 6932 6966 32 1.4 693.2 0.3X 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 2318 2326 13 4.3 231.8 1.0X -read timestamps from files 43345 43627 258 0.2 4334.5 0.1X -infer timestamps from files 89570 89621 59 0.1 8957.0 0.0X -read date text from files 2099 2107 9 4.8 209.9 1.1X -read date from files 18000 18065 98 0.6 1800.0 0.1X -timestamp strings 3937 3956 32 2.5 393.7 0.6X -parse timestamps from Dataset[String] 56001 56429 539 0.2 5600.1 0.0X -infer timestamps from Dataset[String] 109410 109963 559 0.1 10941.0 0.0X -date strings 4530 4540 9 2.2 453.0 0.5X -parse dates from Dataset[String] 29723 29767 72 0.3 2972.3 0.1X -from_json(timestamp) 74106 74619 728 0.1 7410.6 0.0X -from_json(date) 46599 46632 32 0.2 4659.9 0.0X +read timestamp text from files 2354 2368 12 4.2 235.4 1.0X +read timestamps from files 43681 43771 100 0.2 4368.1 0.1X +infer timestamps from files 90608 90771 161 0.1 9060.8 0.0X +read date text from files 2121 2129 9 4.7 212.1 1.1X +read date from files 19069 19103 32 0.5 1906.9 0.1X +timestamp strings 3943 3967 24 2.5 394.3 0.6X +parse timestamps from Dataset[String] 55239 55324 74 0.2 5523.9 0.0X +infer timestamps from Dataset[String] 106155 106258 99 0.1 10615.5 0.0X +date strings 4567 4572 5 2.2 456.7 0.5X +parse dates from Dataset[String] 31258 31461 321 0.3 3125.8 0.1X +from_json(timestamp) 76499 77031 504 0.1 7649.9 0.0X +from_json(date) 44188 44199 9 0.2 4418.8 0.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 +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 30314 30334 28 0.0 303139.1 1.0X +pushdown disabled 30394 30429 54 0.0 303944.7 1.0X +w/ filters 906 913 8 0.1 9059.1 33.5X diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 95a63c3d1e2d9..e0fa4584185e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -125,7 +125,11 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { } (file: PartitionedFile) => { - val parser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) + val parser = new JacksonParser( + actualSchema, + parsedOptions, + allowArrayAsStructs = true, + filters) JsonDataSource(parsedOptions).readFile( broadcastedHadoopConf.value.value, file, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala index 81a234e254000..f7a79bf31948e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.csv.CSVFilters +import org.apache.spark.sql.catalyst.StructFilters import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder @@ -49,7 +49,7 @@ case class CSVScanBuilder( override def pushFilters(filters: Array[Filter]): Array[Filter] = { if (sparkSession.sessionState.conf.csvFilterPushDown) { - _pushedFilters = CSVFilters.pushedFilters(filters, dataSchema) + _pushedFilters = StructFilters.pushedFilters(filters, dataSchema) } filters } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala index 698423948f916..9737803b597a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.json.JsonDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -36,6 +37,7 @@ import org.apache.spark.util.SerializableConfiguration * @param readDataSchema Required schema of JSON files. * @param partitionSchema Schema of partitions. * @param parsedOptions Options for parsing JSON files. + * @param filters The filters pushed down to JSON datasource. */ case class JsonPartitionReaderFactory( sqlConf: SQLConf, @@ -43,12 +45,17 @@ case class JsonPartitionReaderFactory( dataSchema: StructType, readDataSchema: StructType, partitionSchema: StructType, - parsedOptions: JSONOptionsInRead) extends FilePartitionReaderFactory { + parsedOptions: JSONOptionsInRead, + filters: Seq[Filter]) extends FilePartitionReaderFactory { override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { val actualSchema = StructType(readDataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) - val parser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) + val parser = new JacksonParser( + actualSchema, + parsedOptions, + allowArrayAsStructs = true, + filters) val iter = JsonDataSource(parsedOptions).readFile( broadcastedConf.value.value, partitionedFile, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala index 75231625676ff..7ad106627a083 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.json.JsonDataSource import org.apache.spark.sql.execution.datasources.v2.{FileScan, TextBasedFileScan} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -39,6 +40,7 @@ case class JsonScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, + pushedFilters: Array[Filter], partitionFilters: Seq[Expression] = Seq.empty, dataFilters: Seq[Expression] = Seq.empty) extends TextBasedFileScan(sparkSession, options) { @@ -86,7 +88,7 @@ case class JsonScan( // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. JsonPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, parsedOptions) + dataSchema, readDataSchema, readPartitionSchema, parsedOptions, pushedFilters) } override def withFilters( @@ -94,10 +96,14 @@ case class JsonScan( this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) override def equals(obj: Any): Boolean = obj match { - case j: JsonScan => super.equals(j) && dataSchema == j.dataSchema && options == j.options - + case j: JsonScan => super.equals(j) && dataSchema == j.dataSchema && options == j.options && + equivalentFilters(pushedFilters, j.pushedFilters) case _ => false } override def hashCode(): Int = super.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala index be53b1b1676f1..cf1204566ddbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution.datasources.v2.json import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.catalyst.StructFilters +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -29,8 +31,26 @@ class JsonScanBuilder ( schema: StructType, dataSchema: StructType, options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) with SupportsPushDownFilters { override def build(): Scan = { - JsonScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + JsonScan( + sparkSession, + fileIndex, + dataSchema, + readDataSchema(), + readPartitionSchema(), + options, + pushedFilters()) } + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + if (sparkSession.sessionState.conf.jsonFilterPushDown) { + _pushedFilters = StructFilters.pushedFilters(filters, dataSchema) + } + filters + } + + override def pushedFilters(): Array[Filter] = _pushedFilters } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index fcb7bdc25f08f..db6f45247d130 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2353,6 +2353,36 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa assert(df.schema.last == StructField("col_mixed_types", StringType, true)) } } + + test("case sensitivity of filters references") { + Seq(true, false).foreach { filterPushdown => + withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString) { + withTempPath { path => + Seq( + """aaa,BBB""", + """0,1""", + """2,3""").toDF().repartition(1).write.text(path.getCanonicalPath) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val readback = spark.read.schema("aaa integer, BBB integer") + .option("header", true) + .csv(path.getCanonicalPath) + checkAnswer(readback, Seq(Row(2, 3), Row(0, 1))) + checkAnswer(readback.filter($"AAA" === 2 && $"bbb" === 3), Seq(Row(2, 3))) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val readback = spark.read.schema("aaa integer, BBB integer") + .option("header", true) + .csv(path.getCanonicalPath) + checkAnswer(readback, Seq(Row(2, 3), Row(0, 1))) + val errorMsg = intercept[AnalysisException] { + readback.filter($"AAA" === 2 && $"bbb" === 3).collect() + }.getMessage + assert(errorMsg.contains("cannot resolve '`AAA`'")) + } + } + } + } + } } class CSVv1Suite extends CSVSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala index 0dbd6b5754afb..9ff35c0946cc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala @@ -20,9 +20,10 @@ import java.io.File import java.time.{Instant, LocalDate} import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.{Column, Dataset, Row} import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -495,6 +496,45 @@ object JsonBenchmark extends SqlBasedBenchmark { } } + private def filtersPushdownBenchmark(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("Filters pushdown", rowsNum, output = output) + val colsNum = 100 + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", TimestampType)) + val schema = StructType(StructField("key", IntegerType) +: fields) + def columns(): Seq[Column] = { + val ts = Seq.tabulate(colsNum) { i => + lit(Instant.ofEpochSecond(i * 12345678)).as(s"col$i") + } + ($"id" % 1000).as("key") +: ts + } + withTempPath { path => + spark.range(rowsNum).select(columns(): _*).write.json(path.getAbsolutePath) + def readback = { + spark.read.schema(schema).json(path.getAbsolutePath) + } + + benchmark.addCase("w/o filters", numIters) { _ => + readback.noop() + } + + def withFilter(configEnabled: Boolean): Unit = { + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> configEnabled.toString()) { + readback.filter($"key" === 0).noop() + } + } + + benchmark.addCase("pushdown disabled", numIters) { _ => + withFilter(configEnabled = false) + } + + benchmark.addCase("w/ filters", numIters) { _ => + withFilter(configEnabled = true) + } + + benchmark.run() + } + } + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val numIters = 3 runBenchmark("Benchmark for performance of JSON parsing") { @@ -508,6 +548,9 @@ object JsonBenchmark extends SqlBasedBenchmark { jsonInDS(50 * 1000 * 1000, numIters) jsonInFile(50 * 1000 * 1000, numIters) datetimeBenchmark(rowsNum = 10 * 1000 * 1000, numIters) + // Benchmark pushdown filters that refer to top-level columns. + // TODO (SPARK-32325): Add benchmarks for filters with nested column attributes. + filtersPushdownBenchmark(rowsNum = 100 * 1000, numIters) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index de01099f2db55..2a98cba663a1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -35,12 +35,14 @@ import org.apache.spark.sql.{functions => F, _} import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.ExternalRDD -import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, NoopCache} +import org.apache.spark.sql.execution.datasources.v2.json.JsonScanBuilder import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.sql.types.StructType.fromDDL import org.apache.spark.sql.types.TestUDT.{MyDenseVector, MyDenseVectorUDT} +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils class TestFileFilter extends PathFilter { @@ -2688,6 +2690,122 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson checkAnswer(json, Row(null)) } } + + test("filters push down") { + withTempPath { path => + val t = "2019-12-17 00:01:02" + Seq( + """{"c0": "abc", "c1": {"c2": 1, "c3": "2019-11-14 20:35:30"}}""", + s"""{"c0": "def", "c1": {"c2": 2, "c3": "$t"}}""", + s"""{"c0": "defa", "c1": {"c2": 3, "c3": "$t"}}""", + s"""{"c0": "define", "c1": {"c2": 2, "c3": "$t"}}""").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + Seq(true, false).foreach { filterPushdown => + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString) { + Seq("PERMISSIVE", "DROPMALFORMED", "FAILFAST").foreach { mode => + val readback = spark.read + .option("mode", mode) + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .schema("c0 string, c1 struct") + .json(path.getAbsolutePath) + .where($"c1.c2" === 2 && $"c0".startsWith("def")) + .select($"c1.c3") + assert(readback.count() === 2) + checkAnswer(readback, Seq(Row(Timestamp.valueOf(t)), Row(Timestamp.valueOf(t)))) + } + } + } + } + } + + test("apply filters to malformed rows") { + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { path => + Seq( + "{}", + """{"invalid": 0}""", + """{"i":}""", + """{"i": 0}""", + """{"i": 1, "t": "2020-01-28 01:00:00"}""", + """{"t": "2020-01-28 02:00:00"}""", + """{"i": "abc", "t": "2020-01-28 03:00:00"}""", + """{"i": 2, "t": "2020-01-28 04:00:00", "d": 3.14}""").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + val schema = "i INTEGER, t TIMESTAMP" + val readback = spark.read + .schema(schema) + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .json(path.getAbsolutePath) + // readback: + // +----+-------------------+ + // |i |t | + // +----+-------------------+ + // |null|null | + // |null|null | + // |null|null | + // |0 |null | + // |1 |2020-01-28 01:00:00| + // |null|2020-01-28 02:00:00| + // |null|2020-01-28 03:00:00| + // |2 |2020-01-28 04:00:00| + // +----+-------------------+ + checkAnswer( + readback.where($"i".isNull && $"t".isNotNull), + Seq( + Row(null, Timestamp.valueOf("2020-01-28 02:00:00")), + Row(null, Timestamp.valueOf("2020-01-28 03:00:00")))) + checkAnswer( + readback.where($"i" >= 0 && $"t" > "2020-01-28 00:00:00"), + Seq( + Row(1, Timestamp.valueOf("2020-01-28 01:00:00")), + Row(2, Timestamp.valueOf("2020-01-28 04:00:00")))) + checkAnswer( + readback.where($"t".isNull).select($"i"), + Seq(Row(null), Row(null), Row(null), Row(0))) + } + } + } + + test("case sensitivity of filters references") { + Seq(true, false).foreach { filterPushdown => + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString) { + withTempPath { path => + Seq( + """{"aaa": 0, "BBB": 1}""", + """{"AAA": 2, "bbb": 3}""").toDF().write.text(path.getCanonicalPath) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val readback = spark.read.schema("aaa integer, BBB integer") + .json(path.getCanonicalPath) + checkAnswer(readback, Seq(Row(null, null), Row(0, 1))) + checkAnswer(readback.filter($"AAA" === 0 && $"bbb" === 1), Seq(Row(0, 1))) + checkAnswer(readback.filter($"AAA" === 2 && $"bbb" === 3), Seq()) + // Schema inferring + val errorMsg = intercept[AnalysisException] { + spark.read.json(path.getCanonicalPath).collect() + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the data schema")) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val readback = spark.read.schema("aaa integer, BBB integer") + .json(path.getCanonicalPath) + checkAnswer(readback, Seq(Row(null, null), Row(0, 1))) + val errorMsg = intercept[AnalysisException] { + readback.filter($"AAA" === 0 && $"bbb" === 1).collect() + }.getMessage + assert(errorMsg.contains("cannot resolve '`AAA`'")) + // Schema inferring + val readback2 = spark.read.json(path.getCanonicalPath) + checkAnswer( + readback2.filter($"AAA" === 2).select($"AAA", $"bbb"), + Seq(Row(2, 3))) + checkAnswer(readback2.filter($"aaa" === 2).select($"AAA", $"bbb"), Seq()) + } + } + } + } + } } class JsonV1Suite extends JsonSuite { @@ -2702,6 +2820,37 @@ class JsonV2Suite extends JsonSuite { super .sparkConf .set(SQLConf.USE_V1_SOURCE_LIST, "") + + test("get pushed filters") { + val attr = "col" + def getBuilder(path: String): JsonScanBuilder = { + val fileIndex = new InMemoryFileIndex( + spark, + Seq(new org.apache.hadoop.fs.Path(path, "file.json")), + Map.empty, + None, + NoopCache) + val schema = new StructType().add(attr, IntegerType) + val options = CaseInsensitiveStringMap.empty() + new JsonScanBuilder(spark, fileIndex, schema, schema, options) + } + val filters: Array[sources.Filter] = Array(sources.IsNotNull(attr)) + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushFilters(filters) === filters) + assert(scanBuilder.pushedFilters() === filters) + } + } + + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "false") { + withTempPath { file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushFilters(filters) === filters) + assert(scanBuilder.pushedFilters() === Array.empty[sources.Filter]) + } + } + } } class JsonLegacyTimeParserSuite extends JsonSuite { From d5c672af589d55318d311593a9add24e02c215f5 Mon Sep 17 00:00:00 2001 From: dzlab Date: Thu, 16 Jul 2020 09:25:52 -0700 Subject: [PATCH 209/384] [SPARK-32315][ML] Provide an explanation error message when calling require ### What changes were proposed in this pull request? Small improvement in the error message shown to user https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala#L537-L538 ### Why are the changes needed? The current behavior is an exception is thrown but without any specific details on the cause ``` Caused by: java.lang.IllegalArgumentException: requirement failedCaused by: java.lang.IllegalArgumentException: requirement failed at scala.Predef$.require(Predef.scala:212) at org.apache.spark.mllib.util.MLUtils$.fastSquaredDistance(MLUtils.scala:508) at org.apache.spark.mllib.clustering.EuclideanDistanceMeasure$.fastSquaredDistance(DistanceMeasure.scala:232) at org.apache.spark.mllib.clustering.EuclideanDistanceMeasure.isCenterConverged(DistanceMeasure.scala:190) at org.apache.spark.mllib.clustering.KMeans$$anonfun$runAlgorithm$4.apply(KMeans.scala:336) at org.apache.spark.mllib.clustering.KMeans$$anonfun$runAlgorithm$4.apply(KMeans.scala:334) at scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245) at scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245) at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:130) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:130) at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:236) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40) at scala.collection.mutable.HashMap.foreach(HashMap.scala:130) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732) at scala.collection.MapLike$MappedValues.foreach(MapLike.scala:245) at org.apache.spark.mllib.clustering.KMeans.runAlgorithm(KMeans.scala:334) at org.apache.spark.mllib.clustering.KMeans.run(KMeans.scala:251) at org.apache.spark.mllib.clustering.KMeans.run(KMeans.scala:233) ``` ### Does this PR introduce _any_ user-facing change? Yes, this PR adds an explanation message to be shown to user when requirement check is not meant ### How was this patch tested? manually Closes #29115 from dzlab/patch/SPARK-32315. Authored-by: dzlab Signed-off-by: Huaxin Gao --- .../main/scala/org/apache/spark/mllib/util/MLUtils.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index d177364d012f4..0ac8cd014c6e4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -534,8 +534,10 @@ object MLUtils extends Logging { norm2: Double, precision: Double = 1e-6): Double = { val n = v1.size - require(v2.size == n) - require(norm1 >= 0.0 && norm2 >= 0.0) + require(v2.size == n, + s"Both vectors should have same length, found v1 is $n while v2 is ${v2.size}") + require(norm1 >= 0.0 && norm2 >= 0.0, + s"Both norms should be greater or equal to 0.0, found norm1=$norm1, norm2=$norm2") var sqDist = 0.0 /* * The relative error is From 383f5e9cbed636ae91b84252a2c249120c698588 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 16 Jul 2020 11:12:29 -0700 Subject: [PATCH 210/384] [SPARK-32310][ML][PYSPARK] ML params default value parity in classification, regression, clustering and fpm ### What changes were proposed in this pull request? set params default values in trait ...Params in both Scala and Python. I will do this in two PRs. I will change classification, regression, clustering and fpm in this PR. Will change the rest in another PR. ### Why are the changes needed? Make ML has the same default param values between estimator and its corresponding transformer, and also between Scala and Python. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests Closes #29112 from huaxingao/set_default. Authored-by: Huaxin Gao Signed-off-by: Huaxin Gao --- .../ml/classification/FMClassifier.scala | 10 ---- .../spark/ml/classification/LinearSVC.scala | 12 +--- .../classification/LogisticRegression.scala | 14 ++--- .../spark/ml/classification/NaiveBayes.scala | 4 +- .../spark/ml/clustering/BisectingKMeans.scala | 7 +-- .../spark/ml/clustering/GaussianMixture.scala | 8 +-- .../apache/spark/ml/clustering/KMeans.scala | 11 +--- .../org/apache/spark/ml/clustering/LDA.scala | 11 ++-- .../clustering/PowerIterationClustering.scala | 7 +-- .../BinaryClassificationEvaluator.scala | 4 +- .../MulticlassClassificationEvaluator.scala | 8 +-- .../MultilabelClassificationEvaluator.scala | 6 +- .../ml/evaluation/RankingEvaluator.scala | 6 +- .../ml/evaluation/RegressionEvaluator.scala | 4 +- .../org/apache/spark/ml/fpm/FPGrowth.scala | 5 +- .../ml/regression/AFTSurvivalRegression.scala | 11 ++-- .../ml/regression/LinearRegression.scala | 15 ++--- python/pyspark/ml/classification.py | 58 ++++++++++++------- python/pyspark/ml/clustering.py | 33 ++++++++--- python/pyspark/ml/fpm.py | 7 ++- python/pyspark/ml/regression.py | 57 +++++++++++------- 21 files changed, 141 insertions(+), 157 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala index cc691d1c0c58c..88e6e5c090a21 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala @@ -85,7 +85,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setFactorSize(value: Int): this.type = set(factorSize, value) - setDefault(factorSize -> 8) /** * Set whether to fit intercept term. @@ -95,7 +94,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Set whether to fit linear term. @@ -105,7 +103,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setFitLinear(value: Boolean): this.type = set(fitLinear, value) - setDefault(fitLinear -> true) /** * Set the L2 regularization parameter. @@ -115,7 +112,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set the mini-batch fraction parameter. @@ -125,7 +121,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setMiniBatchFraction(value: Double): this.type = set(miniBatchFraction, value) - setDefault(miniBatchFraction -> 1.0) /** * Set the standard deviation of initial coefficients. @@ -135,7 +130,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setInitStd(value: Double): this.type = set(initStd, value) - setDefault(initStd -> 0.01) /** * Set the maximum number of iterations. @@ -145,7 +139,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the initial step size for the first step (like learning rate). @@ -155,7 +148,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setStepSize(value: Double): this.type = set(stepSize, value) - setDefault(stepSize -> 1.0) /** * Set the convergence tolerance of iterations. @@ -165,7 +157,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Set the solver algorithm used for optimization. @@ -176,7 +167,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> AdamW) /** * Set the random seed for weight initialization. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 4adc527c89b36..77272c65eb231 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -55,6 +55,9 @@ private[classification] trait LinearSVCParams extends ClassifierParams with HasR */ final override val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction applied to rawPrediction") + + setDefault(regParam -> 0.0, maxIter -> 100, fitIntercept -> true, tol -> 1E-6, + standardization -> true, threshold -> 0.0, aggregationDepth -> 2, blockSize -> 1) } /** @@ -82,7 +85,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set the maximum number of iterations. @@ -92,7 +94,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Whether to fit an intercept term. @@ -102,7 +103,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Set the convergence tolerance of iterations. @@ -113,7 +113,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Whether to standardize the training features before fitting the model. @@ -123,7 +122,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setStandardization(value: Boolean): this.type = set(standardization, value) - setDefault(standardization -> true) /** * Set the value of param [[weightCol]]. @@ -142,7 +140,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setThreshold(value: Double): this.type = set(threshold, value) - setDefault(threshold -> 0.0) /** * Suggested depth for treeAggregate (greater than or equal to 2). @@ -154,7 +151,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) /** * Set block size for stacking input data in matrices. @@ -173,7 +169,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) @Since("2.2.0") override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) @@ -381,7 +376,6 @@ class LinearSVCModel private[classification] ( @Since("2.2.0") def setThreshold(value: Double): this.type = set(threshold, value) - setDefault(threshold, 0.0) private val margin: Vector => Double = (features) => { BLAS.dot(features, coefficients) + intercept diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 47b3e2de7695c..0e3708bfe9a68 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -243,6 +243,10 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas @Since("2.2.0") def getUpperBoundsOnIntercepts: Vector = $(upperBoundsOnIntercepts) + setDefault(regParam -> 0.0, elasticNetParam -> 0.0, maxIter -> 100, tol -> 1E-6, + fitIntercept -> true, family -> "auto", standardization -> true, threshold -> 0.5, + aggregationDepth -> 2, blockSize -> 1) + protected def usingBoundConstrainedOptimization: Boolean = { isSet(lowerBoundsOnCoefficients) || isSet(upperBoundsOnCoefficients) || isSet(lowerBoundsOnIntercepts) || isSet(upperBoundsOnIntercepts) @@ -290,7 +294,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.2.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set the ElasticNet mixing parameter. @@ -306,7 +309,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.4.0") def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) - setDefault(elasticNetParam -> 0.0) /** * Set the maximum number of iterations. @@ -316,7 +318,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.2.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the convergence tolerance of iterations. @@ -327,7 +328,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.4.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Whether to fit an intercept term. @@ -337,7 +337,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.4.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Sets the value of param [[family]]. @@ -347,7 +346,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("2.1.0") def setFamily(value: String): this.type = set(family, value) - setDefault(family -> "auto") /** * Whether to standardize the training features before fitting the model. @@ -361,11 +359,9 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.5.0") def setStandardization(value: Boolean): this.type = set(standardization, value) - setDefault(standardization -> true) @Since("1.5.0") override def setThreshold(value: Double): this.type = super.setThreshold(value) - setDefault(threshold -> 0.5) @Since("1.5.0") override def getThreshold: Double = super.getThreshold @@ -396,7 +392,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("2.1.0") def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) /** * Set the lower bounds on coefficients if fitting under bound constrained optimization. @@ -447,7 +442,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) private def assertBoundConstrainedOptimizationParamsValid( numCoefficientSets: Int, diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index e65295dbdaf55..6b1537bcc5069 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -64,6 +64,8 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW /** @group getParam */ final def getModelType: String = $(modelType) + + setDefault(smoothing -> 1.0, modelType -> NaiveBayes.Multinomial) } // scalastyle:off line.size.limit @@ -107,7 +109,6 @@ class NaiveBayes @Since("1.5.0") ( */ @Since("1.5.0") def setSmoothing(value: Double): this.type = set(smoothing, value) - setDefault(smoothing -> 1.0) /** * Set the model type using a string (case-sensitive). @@ -117,7 +118,6 @@ class NaiveBayes @Since("1.5.0") ( */ @Since("1.5.0") def setModelType(value: String): this.type = set(modelType, value) - setDefault(modelType -> Multinomial) /** * Sets the value of param [[weightCol]]. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index b09f11dcfe156..5a60bed2652f7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -72,6 +72,8 @@ private[clustering] trait BisectingKMeansParams extends Params with HasMaxIter @Since("2.0.0") def getMinDivisibleClusterSize: Double = $(minDivisibleClusterSize) + setDefault(k -> 4, maxIter -> 20, minDivisibleClusterSize -> 1.0) + /** * Validates and transforms the input schema. * @param schema input schema @@ -226,11 +228,6 @@ class BisectingKMeans @Since("2.0.0") ( @Since("2.0.0") override val uid: String) extends Estimator[BisectingKMeansModel] with BisectingKMeansParams with DefaultParamsWritable { - setDefault( - k -> 4, - maxIter -> 20, - minDivisibleClusterSize -> 1.0) - @Since("2.0.0") override def copy(extra: ParamMap): BisectingKMeans = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 90845021fc073..7df87d2c3336a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -59,6 +59,8 @@ private[clustering] trait GaussianMixtureParams extends Params with HasMaxIter w @Since("2.0.0") def getK: Int = $(k) + setDefault(k -> 2, maxIter -> 100, tol -> 0.01, blockSize -> 1) + /** * Validates and transforms the input schema. * @@ -328,11 +330,6 @@ class GaussianMixture @Since("2.0.0") ( @Since("2.0.0") override val uid: String) extends Estimator[GaussianMixtureModel] with GaussianMixtureParams with DefaultParamsWritable { - setDefault( - k -> 2, - maxIter -> 100, - tol -> 0.01) - @Since("2.0.0") override def copy(extra: ParamMap): GaussianMixture = defaultCopy(extra) @@ -392,7 +389,6 @@ class GaussianMixture @Since("2.0.0") ( */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) /** * Number of samples per cluster to use when initializing Gaussians. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 806015b633c23..5c06973e618bd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -87,6 +87,9 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe @Since("1.5.0") def getInitSteps: Int = $(initSteps) + setDefault(k -> 2, maxIter -> 20, initMode -> MLlibKMeans.K_MEANS_PARALLEL, initSteps -> 2, + tol -> 1e-4, distanceMeasure -> DistanceMeasure.EUCLIDEAN) + /** * Validates and transforms the input schema. * @param schema input schema @@ -271,14 +274,6 @@ class KMeans @Since("1.5.0") ( @Since("1.5.0") override val uid: String) extends Estimator[KMeansModel] with KMeansParams with DefaultParamsWritable { - setDefault( - k -> 2, - maxIter -> 20, - initMode -> MLlibKMeans.K_MEANS_PARALLEL, - initSteps -> 2, - tol -> 1e-4, - distanceMeasure -> DistanceMeasure.EUCLIDEAN) - @Since("1.5.0") override def copy(extra: ParamMap): KMeans = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 4e86b3b247ace..c1b76fb40b2f6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -199,8 +199,6 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM " with estimates of the topic mixture distribution for each document (often called \"theta\"" + " in the literature). Returns a vector of zeros for an empty document.") - setDefault(topicDistributionCol -> "topicDistribution") - /** @group getParam */ @Since("1.6.0") def getTopicDistributionCol: String = $(topicDistributionCol) @@ -315,6 +313,11 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM @Since("2.0.0") def getKeepLastCheckpoint: Boolean = $(keepLastCheckpoint) + setDefault(maxIter -> 20, k -> 10, optimizer -> "online", checkpointInterval -> 10, + learningOffset -> 1024, learningDecay -> 0.51, subsamplingRate -> 0.05, + optimizeDocConcentration -> true, keepLastCheckpoint -> true, + topicDistributionCol -> "topicDistribution") + /** * Validates and transforms the input schema. * @@ -863,10 +866,6 @@ class LDA @Since("1.6.0") ( @Since("1.6.0") def this() = this(Identifiable.randomUID("lda")) - setDefault(maxIter -> 20, k -> 10, optimizer -> "online", checkpointInterval -> 10, - learningOffset -> 1024, learningDecay -> 0.51, subsamplingRate -> 0.05, - optimizeDocConcentration -> true, keepLastCheckpoint -> true) - /** * The features for LDA should be a `Vector` representing the word counts in a document. * The vector should be of length vocabSize, with counts for each term (word). diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala index 812a426a062c1..1466b32bef530 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala @@ -91,7 +91,7 @@ private[clustering] trait PowerIterationClusteringParams extends Params with Has @Since("2.4.0") def getDstCol: String = $(dstCol) - setDefault(srcCol -> "src", dstCol -> "dst") + setDefault(srcCol -> "src", dstCol -> "dst", k -> 2, maxIter -> 20, initMode -> "random") } /** @@ -111,11 +111,6 @@ class PowerIterationClustering private[clustering] ( @Since("2.4.0") override val uid: String) extends PowerIterationClusteringParams with DefaultParamsWritable { - setDefault( - k -> 2, - maxIter -> 20, - initMode -> "random") - @Since("2.4.0") def this() = this(Identifiable.randomUID("PowerIterationClustering")) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 52be22f714981..93b66f3ab7007 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -81,8 +81,6 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("3.0.0") def setNumBins(value: Int): this.type = set(numBins, value) - setDefault(numBins -> 1000) - /** @group setParam */ @Since("1.5.0") def setRawPredictionCol(value: String): this.type = set(rawPredictionCol, value) @@ -95,7 +93,7 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("3.0.0") def setWeightCol(value: String): this.type = set(weightCol, value) - setDefault(metricName -> "areaUnderROC") + setDefault(metricName -> "areaUnderROC", numBins -> 1000) @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 3d77792c4fc88..beeefde8c5fac 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -64,8 +64,6 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("1.5.0") def setMetricName(value: String): this.type = set(metricName, value) - setDefault(metricName -> "f1") - /** @group setParam */ @Since("1.5.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -105,8 +103,6 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("3.0.0") def setMetricLabel(value: Double): this.type = set(metricLabel, value) - setDefault(metricLabel -> 0.0) - /** * The beta value, which controls precision vs recall weighting, * used in `"weightedFMeasure"`, `"fMeasureByLabel"`. @@ -128,8 +124,6 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("3.0.0") def setBeta(value: Double): this.type = set(beta, value) - setDefault(beta -> 1.0) - /** * param for eps. log-loss is undefined for p=0 or p=1, so probabilities are clipped to * max(eps, min(1 - eps, p)). Must be in range (0, 0.5). The default value is 1e-15. @@ -150,7 +144,7 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("3.0.0") def setEps(value: Double): this.type = set(eps, value) - setDefault(eps -> 1e-15) + setDefault(metricName -> "f1", eps -> 1e-15, metricLabel -> 0.0, beta -> 1.0) @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala index 1a82ac7a9472f..8ed26502407a8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala @@ -65,8 +65,6 @@ class MultilabelClassificationEvaluator @Since("3.0.0") (@Since("3.0.0") overrid @Since("3.0.0") def setMetricName(value: String): this.type = set(metricName, value) - setDefault(metricName -> "f1Measure") - /** * param for the class whose metric will be computed in `"precisionByLabel"`, `"recallByLabel"`, * `"f1MeasureByLabel"`. @@ -86,8 +84,6 @@ class MultilabelClassificationEvaluator @Since("3.0.0") (@Since("3.0.0") overrid /** @group setParam */ def setMetricLabel(value: Double): this.type = set(metricLabel, value) - setDefault(metricLabel -> 0.0) - /** @group setParam */ @Since("3.0.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -96,6 +92,8 @@ class MultilabelClassificationEvaluator @Since("3.0.0") (@Since("3.0.0") overrid @Since("3.0.0") def setLabelCol(value: String): this.type = set(labelCol, value) + setDefault(metricLabel -> 0.0, metricName -> "f1Measure") + @Since("3.0.0") override def evaluate(dataset: Dataset[_]): Double = { val metrics = getMetrics(dataset) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala index 82dda4109771d..01fb0599160b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala @@ -61,8 +61,6 @@ class RankingEvaluator @Since("3.0.0") (@Since("3.0.0") override val uid: String @Since("3.0.0") def setMetricName(value: String): this.type = set(metricName, value) - setDefault(metricName -> "meanAveragePrecision") - /** * param for ranking position value used in `"meanAveragePrecisionAtK"`, `"precisionAtK"`, * `"ndcgAtK"`, `"recallAtK"`. Must be > 0. The default value is 10. @@ -83,8 +81,6 @@ class RankingEvaluator @Since("3.0.0") (@Since("3.0.0") override val uid: String @Since("3.0.0") def setK(value: Int): this.type = set(k, value) - setDefault(k -> 10) - /** @group setParam */ @Since("3.0.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -93,6 +89,8 @@ class RankingEvaluator @Since("3.0.0") (@Since("3.0.0") override val uid: String @Since("3.0.0") def setLabelCol(value: String): this.type = set(labelCol, value) + setDefault(k -> 10, metricName -> "meanAveragePrecision") + @Since("3.0.0") override def evaluate(dataset: Dataset[_]): Double = { val metrics = getMetrics(dataset) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala index f0b7c345c3285..902869cc681b8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala @@ -80,8 +80,6 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("3.0.0") def setThroughOrigin(value: Boolean): this.type = set(throughOrigin, value) - setDefault(throughOrigin -> false) - /** @group setParam */ @Since("1.4.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -94,7 +92,7 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("3.0.0") def setWeightCol(value: String): this.type = set(weightCol, value) - setDefault(metricName -> "rmse") + setDefault(metricName -> "rmse", throughOrigin -> false) @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala index f1a68edaed950..7aab4ef62c4d9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -50,7 +50,6 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { */ @Since("2.2.0") val itemsCol: Param[String] = new Param[String](this, "itemsCol", "items column name") - setDefault(itemsCol -> "items") /** @group getParam */ @Since("2.2.0") @@ -66,7 +65,6 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { val minSupport: DoubleParam = new DoubleParam(this, "minSupport", "the minimal support level of a frequent pattern", ParamValidators.inRange(0.0, 1.0)) - setDefault(minSupport -> 0.3) /** @group getParam */ @Since("2.2.0") @@ -95,12 +93,13 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { val minConfidence: DoubleParam = new DoubleParam(this, "minConfidence", "minimal confidence for generating Association Rule", ParamValidators.inRange(0.0, 1.0)) - setDefault(minConfidence -> 0.8) /** @group getParam */ @Since("2.2.0") def getMinConfidence: Double = $(minConfidence) + setDefault(minSupport -> 0.3, itemsCol -> "items", minConfidence -> 0.8) + /** * Validates and transforms the input schema. * @param schema input schema diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 2c30e44b93467..f301c349a2dc7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -62,7 +62,6 @@ private[regression] trait AFTSurvivalRegressionParams extends PredictorParams /** @group getParam */ @Since("1.6.0") def getCensorCol: String = $(censorCol) - setDefault(censorCol -> "censor") /** * Param for quantile probabilities array. @@ -78,7 +77,6 @@ private[regression] trait AFTSurvivalRegressionParams extends PredictorParams /** @group getParam */ @Since("1.6.0") def getQuantileProbabilities: Array[Double] = $(quantileProbabilities) - setDefault(quantileProbabilities -> Array(0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99)) /** * Param for quantiles column name. @@ -92,6 +90,10 @@ private[regression] trait AFTSurvivalRegressionParams extends PredictorParams @Since("1.6.0") def getQuantilesCol: String = $(quantilesCol) + setDefault(censorCol -> "censor", + quantileProbabilities -> Array(0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99), + fitIntercept -> true, maxIter -> 100, tol -> 1E-6, aggregationDepth -> 2, blockSize -> 1) + /** Checks whether the input has quantiles column name. */ private[regression] def hasQuantilesCol: Boolean = { isDefined(quantilesCol) && $(quantilesCol).nonEmpty @@ -153,7 +155,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("1.6.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Set the maximum number of iterations. @@ -162,7 +163,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("1.6.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the convergence tolerance of iterations. @@ -172,7 +172,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("1.6.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Suggested depth for treeAggregate (greater than or equal to 2). @@ -183,7 +182,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("2.1.0") def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) /** * Set block size for stacking input data in matrices. @@ -202,7 +200,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) /** * Extract [[featuresCol]], [[labelCol]] and [[censorCol]] from input dataset, diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index de559142a9261..235a7f9b6ebd5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -105,6 +105,10 @@ private[regression] trait LinearRegressionParams extends PredictorParams @Since("2.3.0") def getEpsilon: Double = $(epsilon) + setDefault(regParam -> 0.0, fitIntercept -> true, standardization -> true, + elasticNetParam -> 0.0, maxIter -> 100, tol -> 1E-6, solver -> Auto, + aggregationDepth -> 2, loss -> SquaredError, epsilon -> 1.35, blockSize -> 1) + override protected def validateAndTransformSchema( schema: StructType, fitting: Boolean, @@ -191,7 +195,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.3.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set if we should fit the intercept. @@ -201,7 +204,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.5.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Whether to standardize the training features before fitting the model. @@ -217,7 +219,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.5.0") def setStandardization(value: Boolean): this.type = set(standardization, value) - setDefault(standardization -> true) /** * Set the ElasticNet mixing parameter. @@ -233,7 +234,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.4.0") def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) - setDefault(elasticNetParam -> 0.0) /** * Set the maximum number of iterations. @@ -243,7 +243,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.3.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the convergence tolerance of iterations. @@ -254,7 +253,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.4.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Whether to over-/under-sample training instances according to the given weights in weightCol. @@ -283,7 +281,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.6.0") def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> Auto) /** * Suggested depth for treeAggregate (greater than or equal to 2). @@ -295,7 +292,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("2.1.0") def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) /** * Sets the value of param [[loss]]. @@ -305,7 +301,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("2.3.0") def setLoss(value: String): this.type = set(loss, value) - setDefault(loss -> SquaredError) /** * Sets the value of param [[epsilon]]. @@ -315,7 +310,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("2.3.0") def setEpsilon(value: Double): this.type = set(epsilon, value) - setDefault(epsilon -> 1.35) /** * Set block size for stacking input data in matrices. @@ -334,7 +328,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) override protected def train(dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => instr.logPipelineStage(this) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4f2d33adbc7e7..e192e8c252d50 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -513,6 +513,12 @@ class _LinearSVCParams(_ClassifierParams, HasRegParam, HasMaxIter, HasFitInterce " all predictions 0.0 and -Inf will make all predictions 1.0.", typeConverter=TypeConverters.toFloat) + def __init__(self): + super(_LinearSVCParams, self).__init__() + self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, + standardization=True, threshold=0.0, aggregationDepth=2, + blockSize=1) + @inherit_doc class LinearSVC(_JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable): @@ -599,9 +605,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(LinearSVC, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LinearSVC", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, - standardization=True, threshold=0.0, aggregationDepth=2, - blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -817,6 +820,11 @@ class _LogisticRegressionParams(_ProbabilisticClassifierParams, HasRegParam, "classes for multinomial regression.", typeConverter=TypeConverters.toVector) + def __init__(self): + super(_LogisticRegressionParams, self).__init__() + self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto", + blockSize=1) + @since("1.4.0") def setThreshold(self, value): """ @@ -1038,8 +1046,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(LogisticRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LogisticRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto", - blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) self._checkThresholdConsistency() @@ -1306,7 +1312,12 @@ class _DecisionTreeClassifierParams(_DecisionTreeParams, _TreeClassifierParams): """ Params for :py:class:`DecisionTreeClassifier` and :py:class:`DecisionTreeClassificationModel`. """ - pass + + def __init__(self): + super(_DecisionTreeClassifierParams, self).__init__() + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + impurity="gini", leafCol="", minWeightFractionPerNode=0.0) @inherit_doc @@ -1405,9 +1416,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(DecisionTreeClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.DecisionTreeClassifier", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="gini", leafCol="", minWeightFractionPerNode=0.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1541,7 +1549,14 @@ class _RandomForestClassifierParams(_RandomForestParams, _TreeClassifierParams): """ Params for :py:class:`RandomForestClassifier` and :py:class:`RandomForestClassificationModel`. """ - pass + + def __init__(self): + super(_RandomForestClassifierParams, self).__init__() + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + impurity="gini", numTrees=20, featureSubsetStrategy="auto", + subsamplingRate=1.0, leafCol="", minWeightFractionPerNode=0.0, + bootstrap=True) @inherit_doc @@ -1635,11 +1650,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(RandomForestClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.RandomForestClassifier", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="gini", numTrees=20, featureSubsetStrategy="auto", - subsamplingRate=1.0, leafCol="", minWeightFractionPerNode=0.0, - bootstrap=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1883,6 +1893,14 @@ class _GBTClassifierParams(_GBTParams, _HasVarianceImpurity): "Supported options: " + ", ".join(supportedLossTypes), typeConverter=TypeConverters.toString) + def __init__(self): + super(_GBTClassifierParams, self).__init__() + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + lossType="logistic", maxIter=20, stepSize=0.1, subsamplingRate=1.0, + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, + leafCol="", minWeightFractionPerNode=0.0) + @since("1.4.0") def getLossType(self): """ @@ -2010,11 +2028,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(GBTClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.GBTClassifier", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - lossType="logistic", maxIter=20, stepSize=0.1, subsamplingRate=1.0, - impurity="variance", featureSubsetStrategy="all", validationTol=0.01, - leafCol="", minWeightFractionPerNode=0.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2212,6 +2225,10 @@ class _NaiveBayesParams(_PredictorParams, HasWeightCol): "and gaussian.", typeConverter=TypeConverters.toString) + def __init__(self): + super(_NaiveBayesParams, self).__init__() + self._setDefault(smoothing=1.0, modelType="multinomial") + @since("1.5.0") def getSmoothing(self): """ @@ -2330,7 +2347,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(NaiveBayes, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.NaiveBayes", self.uid) - self._setDefault(smoothing=1.0, modelType="multinomial") kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 54a184bc081ee..6ca413d696368 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -109,6 +109,10 @@ class _GaussianMixtureParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionC k = Param(Params._dummy(), "k", "Number of independent Gaussians in the mixture model. " + "Must be > 1.", typeConverter=TypeConverters.toInt) + def __init__(self): + super(_GaussianMixtureParams, self).__init__() + self._setDefault(k=2, tol=0.01, maxIter=100, aggregationDepth=2, blockSize=1) + @since("2.0.0") def getK(self): """ @@ -339,7 +343,6 @@ def __init__(self, featuresCol="features", predictionCol="prediction", k=2, super(GaussianMixture, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.GaussianMixture", self.uid) - self._setDefault(k=2, tol=0.01, maxIter=100, aggregationDepth=2, blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -500,6 +503,11 @@ class _KMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionCol, HasTo initSteps = Param(Params._dummy(), "initSteps", "The number of steps for k-means|| " + "initialization mode. Must be > 0.", typeConverter=TypeConverters.toInt) + def __init__(self): + super(_KMeansParams, self).__init__() + self._setDefault(k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, + distanceMeasure="euclidean") + @since("1.5.0") def getK(self): """ @@ -644,8 +652,6 @@ def __init__(self, featuresCol="features", predictionCol="prediction", k=2, """ super(KMeans, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.KMeans", self.uid) - self._setDefault(k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, - distanceMeasure="euclidean") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -754,6 +760,10 @@ class _BisectingKMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionC "proportion of points (if < 1.0) of a divisible cluster.", typeConverter=TypeConverters.toFloat) + def __init__(self): + super(_BisectingKMeansParams, self).__init__() + self._setDefault(maxIter=20, k=4, minDivisibleClusterSize=1.0) + @since("2.0.0") def getK(self): """ @@ -920,7 +930,6 @@ def __init__(self, featuresCol="features", predictionCol="prediction", maxIter=2 super(BisectingKMeans, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.BisectingKMeans", self.uid) - self._setDefault(maxIter=20, k=4, minDivisibleClusterSize=1.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1063,6 +1072,13 @@ class _LDAParams(HasMaxIter, HasFeaturesCol, HasSeed, HasCheckpointInterval): " partition is lost, so set this bit with care.", TypeConverters.toBoolean) + def __init__(self): + super(_LDAParams, self).__init__() + self._setDefault(maxIter=20, checkpointInterval=10, + k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51, + subsamplingRate=0.05, optimizeDocConcentration=True, + topicDistributionCol="topicDistribution", keepLastCheckpoint=True) + @since("2.0.0") def getK(self): """ @@ -1392,10 +1408,6 @@ def __init__(self, featuresCol="features", maxIter=20, seed=None, checkpointInte """ super(LDA, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid) - self._setDefault(maxIter=20, checkpointInterval=10, - k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51, - subsamplingRate=0.05, optimizeDocConcentration=True, - topicDistributionCol="topicDistribution", keepLastCheckpoint=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1588,6 +1600,10 @@ class _PowerIterationClusteringParams(HasMaxIter, HasWeightCol): "Name of the input column for destination vertex IDs.", typeConverter=TypeConverters.toString) + def __init__(self): + super(_PowerIterationClusteringParams, self).__init__() + self._setDefault(k=2, maxIter=20, initMode="random", srcCol="src", dstCol="dst") + @since("2.4.0") def getK(self): """ @@ -1675,7 +1691,6 @@ def __init__(self, k=2, maxIter=20, initMode="random", srcCol="src", dstCol="dst super(PowerIterationClustering, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.clustering.PowerIterationClustering", self.uid) - self._setDefault(k=2, maxIter=20, initMode="random", srcCol="src", dstCol="dst") kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index b91788a82c19a..a1a8a4e3e3ac4 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -55,6 +55,11 @@ class _FPGrowthParams(HasPredictionCol): "but will affect the association rules generation.", typeConverter=TypeConverters.toFloat) + def __init__(self): + super(_FPGrowthParams, self).__init__() + self._setDefault(minSupport=0.3, minConfidence=0.8, + itemsCol="items", predictionCol="prediction") + def getItemsCol(self): """ Gets the value of itemsCol or its default value. @@ -204,8 +209,6 @@ def __init__(self, minSupport=0.3, minConfidence=0.8, itemsCol="items", """ super(FPGrowth, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.fpm.FPGrowth", self.uid) - self._setDefault(minSupport=0.3, minConfidence=0.8, - itemsCol="items", predictionCol="prediction") kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index e82a35c8e78f1..6fe6486c5a04a 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -104,6 +104,11 @@ class _LinearRegressionParams(_PredictorParams, HasRegParam, HasElasticNetParam, "robustness. Must be > 1.0. Only valid when loss is huber", typeConverter=TypeConverters.toFloat) + def __init__(self): + super(_LinearRegressionParams, self).__init__() + self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35, + blockSize=1) + @since("2.3.0") def getEpsilon(self): """ @@ -206,8 +211,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(LinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.LinearRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35, - blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -619,6 +622,10 @@ class _IsotonicRegressionParams(HasFeaturesCol, HasLabelCol, HasPredictionCol, H "The index of the feature if featuresCol is a vector column, no effect otherwise.", typeConverter=TypeConverters.toInt) + def __init__(self): + super(_IsotonicRegressionParams, self).__init__() + self._setDefault(isotonic=True, featureIndex=0) + def getIsotonic(self): """ Gets the value of isotonic or its default value. @@ -681,7 +688,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(IsotonicRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.IsotonicRegression", self.uid) - self._setDefault(isotonic=True, featureIndex=0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -808,7 +814,11 @@ class _DecisionTreeRegressorParams(_DecisionTreeParams, _TreeRegressorParams, Ha .. versionadded:: 3.0.0 """ - pass + def __init__(self): + super(_DecisionTreeRegressorParams, self).__init__() + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + impurity="variance", leafCol="", minWeightFractionPerNode=0.0) @inherit_doc @@ -895,9 +905,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(DecisionTreeRegressor, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.DecisionTreeRegressor", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="variance", leafCol="", minWeightFractionPerNode=0.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1052,7 +1059,14 @@ class _RandomForestRegressorParams(_RandomForestParams, _TreeRegressorParams): .. versionadded:: 3.0.0 """ - pass + + def __init__(self): + super(_RandomForestRegressorParams, self).__init__() + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + impurity="variance", subsamplingRate=1.0, numTrees=20, + featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, + bootstrap=True) @inherit_doc @@ -1135,11 +1149,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(RandomForestRegressor, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.RandomForestRegressor", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="variance", subsamplingRate=1.0, numTrees=20, - featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, - bootstrap=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1310,6 +1319,14 @@ class _GBTRegressorParams(_GBTParams, _TreeRegressorParams): "Supported options: " + ", ".join(supportedLossTypes), typeConverter=TypeConverters.toString) + def __init__(self): + super(_GBTRegressorParams, self).__init__() + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, + checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, + leafCol="", minWeightFractionPerNode=0.0) + @since("1.4.0") def getLossType(self): """ @@ -1407,11 +1424,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred """ super(GBTRegressor, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.regression.GBTRegressor", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, - checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, - impurity="variance", featureSubsetStrategy="all", validationTol=0.01, - leafCol="", minWeightFractionPerNode=0.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1630,6 +1642,12 @@ class _AFTSurvivalRegressionParams(_PredictorParams, HasMaxIter, HasTol, HasFitI "corresponding quantileProbabilities if it is set.", typeConverter=TypeConverters.toString) + def __init__(self): + super(_AFTSurvivalRegressionParams, self).__init__() + self._setDefault(censorCol="censor", + quantileProbabilities=[0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99], + maxIter=100, tol=1E-6, blockSize=1) + @since("1.6.0") def getCensorCol(self): """ @@ -1722,9 +1740,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(AFTSurvivalRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.AFTSurvivalRegression", self.uid) - self._setDefault(censorCol="censor", - quantileProbabilities=[0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99], - maxIter=100, tol=1E-6, blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) From fb519251237892ad474592d19bf4f193e2a9e2b6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 16 Jul 2020 11:21:14 -0700 Subject: [PATCH 211/384] [SPARK-32335][K8S][TESTS] Remove Python2 test from K8s IT ### What changes were proposed in this pull request? This PR aims to remove Python 2 test case from K8s IT. ### Why are the changes needed? Since Apache Spark 3.1.0 dropped Python 2.7, 3.4 and 3.5 support officially via SPARK-32138, K8s IT fails. ``` KubernetesSuite: - Run SparkPi with no resources - Run SparkPi with a very long application name. - Use SparkLauncher.NO_RESOURCE - Run SparkPi with a master URL without a scheme. - Run SparkPi with an argument. - Run SparkPi with custom labels, annotations, and environment variables. - All pods have the same service account by default - Run extraJVMOptions check on driver - Run SparkRemoteFileTest using a remote data file - Run SparkPi with env and mount secrets. - Run PySpark on simple pi.py example - Run PySpark with Python2 to test a pyfiles example *** FAILED *** The code passed to eventually never returned normally. Attempted 113 times over 2.0014854648999996 minutes. Last failure message: false was not true. (KubernetesSuite.scala:370) - Run PySpark with Python3 to test a pyfiles example - Run PySpark with memory customization - Run in client mode. - Start pod creation from template - PVs with local storage - Launcher client dependencies - Test basic decommissioning - Run SparkR on simple dataframe.R example Run completed in 11 minutes, 15 seconds. Total number of tests run: 20 Suites: completed 2, aborted 0 Tests: succeeded 19, failed 1, canceled 0, ignored 0, pending 0 *** 1 TEST FAILED *** ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass Jenkins K8s IT. Closes #29136 from dongjoon-hyun/SPARK-32335. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../integrationtest/PythonTestsSuite.scala | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala index 3a6e5e7a6255d..b16ccb429074f 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala @@ -35,25 +35,6 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => isJVM = false) } - test("Run PySpark with Python2 to test a pyfiles example", k8sTestTag) { - sparkAppConf - .set("spark.kubernetes.container.image", pyImage) - .set("spark.kubernetes.pyspark.pythonVersion", "2") - runSparkApplicationAndVerifyCompletion( - appResource = PYSPARK_FILES, - mainClass = "", - expectedLogOnCompletion = Seq( - "Python runtime version check is: True", - "Python environment version check is: True", - "Python runtime version check for executor is: True"), - appArgs = Array("python"), - driverPodChecker = doBasicDriverPyPodCheck, - executorPodChecker = doBasicExecutorPyPodCheck, - appLocator = appLocator, - isJVM = false, - pyFiles = Some(PYSPARK_CONTAINER_TESTS)) - } - test("Run PySpark with Python3 to test a pyfiles example", k8sTestTag) { sparkAppConf .set("spark.kubernetes.container.image", pyImage) From 9747e8fc9d1c67a4f27f7e4ff94b5f0aca394919 Mon Sep 17 00:00:00 2001 From: Frank Yin Date: Fri, 17 Jul 2020 11:14:25 +0900 Subject: [PATCH 212/384] [SPARK-31831][SQL][TESTS][FOLLOWUP] Put mocks for HiveSessionImplSuite in hive version related subdirectories ### What changes were proposed in this pull request? This patch fixes the build issue on Hive 1.2 profile brought by #29069, via putting mocks for HiveSessionImplSuite in hive version related subdirectories, so that maven build will pick up the proper source code according to the profile. ### Why are the changes needed? #29069 fixed the flakiness of HiveSessionImplSuite, but given the patch relied on the default profile (Hive 2.3) it broke the build with Hive 1.2 profile. This patch addresses both Hive versions. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually confirmed the test suite via below command: > Hive 1.2 ``` build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.HiveSessionImplSuite test -Phive-1.2 -Phadoop-2.7 -Phive-thriftserver ``` > Hive 2.3 ``` build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.HiveSessionImplSuite test -Phive-2.3 -Phadoop-3.2 -Phive-thriftserver ``` Closes #29129 from frankyin-factual/hive-tests. Authored-by: Frank Yin Signed-off-by: Jungtaek Lim (HeartSaVioR) --- sql/hive-thriftserver/pom.xml | 12 +++++ .../thriftserver/HiveSessionImplSuite.scala | 29 +---------- .../GetCatalogsOperationMock.scala | 50 +++++++++++++++++++ .../GetCatalogsOperationMock.scala | 50 +++++++++++++++++++ 4 files changed, 113 insertions(+), 28 deletions(-) create mode 100644 sql/hive-thriftserver/v1.2/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala create mode 100644 sql/hive-thriftserver/v2.3/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5bf20b209aff7..0d9d7515d85ed 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -152,6 +152,18 @@ + + add-test-source + generate-sources + + add-test-source + + + + v${hive.version.short}/src/test/scala + + + diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala index 42d86e98a7273..47db7e34a5a2c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.hive.thriftserver import java.lang.reflect.InvocationTargetException -import java.nio.ByteBuffer -import java.util.UUID import scala.collection.JavaConverters._ import scala.collection.mutable @@ -27,7 +25,6 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hive.service.cli.OperationHandle import org.apache.hive.service.cli.operation.{GetCatalogsOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.{HiveSession, HiveSessionImpl, SessionManager} -import org.apache.hive.service.rpc.thrift.{THandleIdentifier, TOperationHandle, TOperationType} import org.apache.spark.SparkFunSuite @@ -65,31 +62,6 @@ class HiveSessionImplSuite extends SparkFunSuite { } } -class GetCatalogsOperationMock(parentSession: HiveSession) - extends GetCatalogsOperation(parentSession) { - - override def runInternal(): Unit = {} - - override def getHandle: OperationHandle = { - val uuid: UUID = UUID.randomUUID() - val tHandleIdentifier: THandleIdentifier = new THandleIdentifier() - tHandleIdentifier.setGuid(getByteBufferFromUUID(uuid)) - tHandleIdentifier.setSecret(getByteBufferFromUUID(uuid)) - val tOperationHandle: TOperationHandle = new TOperationHandle() - tOperationHandle.setOperationId(tHandleIdentifier) - tOperationHandle.setOperationType(TOperationType.GET_TYPE_INFO) - tOperationHandle.setHasResultSetIsSet(false) - new OperationHandle(tOperationHandle) - } - - private def getByteBufferFromUUID(uuid: UUID): Array[Byte] = { - val bb: ByteBuffer = ByteBuffer.wrap(new Array[Byte](16)) - bb.putLong(uuid.getMostSignificantBits) - bb.putLong(uuid.getLeastSignificantBits) - bb.array - } -} - class OperationManagerMock extends OperationManager { private val calledHandles: mutable.Set[OperationHandle] = new mutable.HashSet[OperationHandle]() @@ -114,3 +86,4 @@ class OperationManagerMock extends OperationManager { def getCalledHandles: mutable.Set[OperationHandle] = calledHandles } + diff --git a/sql/hive-thriftserver/v1.2/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala b/sql/hive-thriftserver/v1.2/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala new file mode 100644 index 0000000000000..1e684fa65b284 --- /dev/null +++ b/sql/hive-thriftserver/v1.2/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.thriftserver + +import java.nio.ByteBuffer +import java.util.UUID + +import org.apache.hive.service.cli.OperationHandle +import org.apache.hive.service.cli.operation.GetCatalogsOperation +import org.apache.hive.service.cli.session.HiveSession +import org.apache.hive.service.cli.thrift.{THandleIdentifier, TOperationHandle, TOperationType} + +class GetCatalogsOperationMock(parentSession: HiveSession) + extends GetCatalogsOperation(parentSession) { + + override def runInternal(): Unit = {} + + override def getHandle: OperationHandle = { + val uuid: UUID = UUID.randomUUID() + val tHandleIdentifier: THandleIdentifier = new THandleIdentifier() + tHandleIdentifier.setGuid(getByteBufferFromUUID(uuid)) + tHandleIdentifier.setSecret(getByteBufferFromUUID(uuid)) + val tOperationHandle: TOperationHandle = new TOperationHandle() + tOperationHandle.setOperationId(tHandleIdentifier) + tOperationHandle.setOperationType(TOperationType.GET_TYPE_INFO) + tOperationHandle.setHasResultSetIsSet(false) + new OperationHandle(tOperationHandle) + } + + private def getByteBufferFromUUID(uuid: UUID): Array[Byte] = { + val bb: ByteBuffer = ByteBuffer.wrap(new Array[Byte](16)) + bb.putLong(uuid.getMostSignificantBits) + bb.putLong(uuid.getLeastSignificantBits) + bb.array + } +} diff --git a/sql/hive-thriftserver/v2.3/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala b/sql/hive-thriftserver/v2.3/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala new file mode 100644 index 0000000000000..1bc9aaf672c3b --- /dev/null +++ b/sql/hive-thriftserver/v2.3/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.thriftserver + +import java.nio.ByteBuffer +import java.util.UUID + +import org.apache.hive.service.cli.OperationHandle +import org.apache.hive.service.cli.operation.GetCatalogsOperation +import org.apache.hive.service.cli.session.HiveSession +import org.apache.hive.service.rpc.thrift.{THandleIdentifier, TOperationHandle, TOperationType} + +class GetCatalogsOperationMock(parentSession: HiveSession) + extends GetCatalogsOperation(parentSession) { + + override def runInternal(): Unit = {} + + override def getHandle: OperationHandle = { + val uuid: UUID = UUID.randomUUID() + val tHandleIdentifier: THandleIdentifier = new THandleIdentifier() + tHandleIdentifier.setGuid(getByteBufferFromUUID(uuid)) + tHandleIdentifier.setSecret(getByteBufferFromUUID(uuid)) + val tOperationHandle: TOperationHandle = new TOperationHandle() + tOperationHandle.setOperationId(tHandleIdentifier) + tOperationHandle.setOperationType(TOperationType.GET_TYPE_INFO) + tOperationHandle.setHasResultSetIsSet(false) + new OperationHandle(tOperationHandle) + } + + private def getByteBufferFromUUID(uuid: UUID): Array[Byte] = { + val bb: ByteBuffer = ByteBuffer.wrap(new Array[Byte](16)) + bb.putLong(uuid.getMostSignificantBits) + bb.putLong(uuid.getLeastSignificantBits) + bb.array + } +} From ea9e8f365a6670972b64da9ff2342568268067e3 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 17 Jul 2020 11:49:18 +0900 Subject: [PATCH 213/384] [SPARK-32094][PYTHON] Update cloudpickle to v1.5.0 ### What changes were proposed in this pull request? This PR aims to upgrade PySpark's embedded cloudpickle to the latest cloudpickle v1.5.0 (See https://github.com/cloudpipe/cloudpickle/blob/v1.5.0/cloudpickle/cloudpickle.py) ### Why are the changes needed? There are many bug fixes. For example, the bug described in the JIRA: dill unpickling fails because they define `types.ClassType`, which is undefined in dill. This results in the following error: ``` Traceback (most recent call last): File "/usr/local/lib/python3.6/site-packages/apache_beam/internal/pickler.py", line 279, in loads return dill.loads(s) File "/usr/local/lib/python3.6/site-packages/dill/_dill.py", line 317, in loads return load(file, ignore) File "/usr/local/lib/python3.6/site-packages/dill/_dill.py", line 305, in load obj = pik.load() File "/usr/local/lib/python3.6/site-packages/dill/_dill.py", line 577, in _load_type return _reverse_typemap[name] KeyError: 'ClassType' ``` See also https://github.com/cloudpipe/cloudpickle/issues/82. This was fixed for cloudpickle 1.3.0+ (https://github.com/cloudpipe/cloudpickle/pull/337), but PySpark's cloudpickle.py doesn't have this change yet. More notably, now it supports C pickle implementation with Python 3.8 which hugely improve performance. This is already adopted in another project such as Ray. ### Does this PR introduce _any_ user-facing change? Yes, as described above, the bug fixes. Internally, users also could leverage the fast cloudpickle backed by C pickle. ### How was this patch tested? Jenkins will test it out. Closes #29114 from HyukjinKwon/SPARK-32094. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- LICENSE | 2 +- dev/.rat-excludes | 4 +- dev/tox.ini | 2 +- python/pyspark/cloudpickle.py | 1362 ----------------- python/pyspark/cloudpickle/__init__.py | 7 + python/pyspark/cloudpickle/cloudpickle.py | 830 ++++++++++ .../pyspark/cloudpickle/cloudpickle_fast.py | 747 +++++++++ python/pyspark/cloudpickle/compat.py | 13 + python/setup.py | 1 + 9 files changed, 1602 insertions(+), 1366 deletions(-) delete mode 100644 python/pyspark/cloudpickle.py create mode 100644 python/pyspark/cloudpickle/__init__.py create mode 100644 python/pyspark/cloudpickle/cloudpickle.py create mode 100644 python/pyspark/cloudpickle/cloudpickle_fast.py create mode 100644 python/pyspark/cloudpickle/compat.py diff --git a/LICENSE b/LICENSE index 6b169b1447f14..af2cdd275d28d 100644 --- a/LICENSE +++ b/LICENSE @@ -229,7 +229,7 @@ BSD 3-Clause ------------ python/lib/py4j-*-src.zip -python/pyspark/cloudpickle.py +python/pyspark/cloudpickle/*.py python/pyspark/join.py core/src/main/resources/org/apache/spark/ui/static/d3.min.js diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 326e561529073..db6a4ce644edd 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -47,8 +47,8 @@ jsonFormatter.min.js .*json .*data .*log -pyspark-coverage-site/ -cloudpickle.py +pyspark-coverage-site/* +cloudpickle/* heapq3.py join.py SparkExprTyper.scala diff --git a/dev/tox.ini b/dev/tox.ini index 3ee13c09b5de7..ba5df084daad7 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -16,4 +16,4 @@ [pycodestyle] ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 -exclude=cloudpickle.py,heapq3.py,shared.py,python/docs/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* +exclude=python/pyspark/cloudpickle/*.py,heapq3.py,shared.py,python/docs/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py deleted file mode 100644 index af49c77a2d98c..0000000000000 --- a/python/pyspark/cloudpickle.py +++ /dev/null @@ -1,1362 +0,0 @@ -""" -This class is defined to override standard pickle functionality - -The goals of it follow: --Serialize lambdas and nested functions to compiled byte code --Deal with main module correctly --Deal with other non-serializable objects - -It does not include an unpickler, as standard python unpickling suffices. - -This module was extracted from the `cloud` package, developed by `PiCloud, Inc. -`_. - -Copyright (c) 2012, Regents of the University of California. -Copyright (c) 2009 `PiCloud, Inc. `_. -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - * Neither the name of the University of California, Berkeley nor the - names of its contributors may be used to endorse or promote - products derived from this software without specific prior written - permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED -TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -""" -from __future__ import print_function - -import dis -from functools import partial -import io -import itertools -import logging -import opcode -import operator -import pickle -import struct -import sys -import traceback -import types -import weakref -import uuid -import threading - - -try: - from enum import Enum -except ImportError: - Enum = None - -# cloudpickle is meant for inter process communication: we expect all -# communicating processes to run the same Python version hence we favor -# communication speed over compatibility: -DEFAULT_PROTOCOL = pickle.HIGHEST_PROTOCOL - -# Track the provenance of reconstructed dynamic classes to make it possible to -# recontruct instances from the matching singleton class definition when -# appropriate and preserve the usual "isinstance" semantics of Python objects. -_DYNAMIC_CLASS_TRACKER_BY_CLASS = weakref.WeakKeyDictionary() -_DYNAMIC_CLASS_TRACKER_BY_ID = weakref.WeakValueDictionary() -_DYNAMIC_CLASS_TRACKER_LOCK = threading.Lock() - -if sys.version_info[0] < 3: # pragma: no branch - from pickle import Pickler - try: - from cStringIO import StringIO - except ImportError: - from StringIO import StringIO - string_types = (basestring,) # noqa - PY3 = False - PY2 = True - PY2_WRAPPER_DESCRIPTOR_TYPE = type(object.__init__) - PY2_METHOD_WRAPPER_TYPE = type(object.__eq__) - PY2_CLASS_DICT_SKIP_PICKLE_METHOD_TYPE = (PY2_METHOD_WRAPPER_TYPE, - PY2_WRAPPER_DESCRIPTOR_TYPE) -else: - types.ClassType = type - from pickle import _Pickler as Pickler - from io import BytesIO as StringIO - string_types = (str,) - PY3 = True - PY2 = False - - -def _ensure_tracking(class_def): - with _DYNAMIC_CLASS_TRACKER_LOCK: - class_tracker_id = _DYNAMIC_CLASS_TRACKER_BY_CLASS.get(class_def) - if class_tracker_id is None: - class_tracker_id = uuid.uuid4().hex - _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id - _DYNAMIC_CLASS_TRACKER_BY_ID[class_tracker_id] = class_def - return class_tracker_id - - -def _lookup_class_or_track(class_tracker_id, class_def): - if class_tracker_id is not None: - with _DYNAMIC_CLASS_TRACKER_LOCK: - class_def = _DYNAMIC_CLASS_TRACKER_BY_ID.setdefault( - class_tracker_id, class_def) - _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id - return class_def - - -def _make_cell_set_template_code(): - """Get the Python compiler to emit LOAD_FAST(arg); STORE_DEREF - - Notes - ----- - In Python 3, we could use an easier function: - - .. code-block:: python - - def f(): - cell = None - - def _stub(value): - nonlocal cell - cell = value - - return _stub - - _cell_set_template_code = f().__code__ - - This function is _only_ a LOAD_FAST(arg); STORE_DEREF, but that is - invalid syntax on Python 2. If we use this function we also don't need - to do the weird freevars/cellvars swap below - """ - def inner(value): - lambda: cell # make ``cell`` a closure so that we get a STORE_DEREF - cell = value - - co = inner.__code__ - - # NOTE: we are marking the cell variable as a free variable intentionally - # so that we simulate an inner function instead of the outer function. This - # is what gives us the ``nonlocal`` behavior in a Python 2 compatible way. - if PY2: # pragma: no branch - return types.CodeType( - co.co_argcount, - co.co_nlocals, - co.co_stacksize, - co.co_flags, - co.co_code, - co.co_consts, - co.co_names, - co.co_varnames, - co.co_filename, - co.co_name, - co.co_firstlineno, - co.co_lnotab, - co.co_cellvars, # this is the trickery - (), - ) - else: - if hasattr(types.CodeType, "co_posonlyargcount"): # pragma: no branch - return types.CodeType( - co.co_argcount, - co.co_posonlyargcount, # Python3.8 with PEP570 - co.co_kwonlyargcount, - co.co_nlocals, - co.co_stacksize, - co.co_flags, - co.co_code, - co.co_consts, - co.co_names, - co.co_varnames, - co.co_filename, - co.co_name, - co.co_firstlineno, - co.co_lnotab, - co.co_cellvars, # this is the trickery - (), - ) - else: - return types.CodeType( - co.co_argcount, - co.co_kwonlyargcount, - co.co_nlocals, - co.co_stacksize, - co.co_flags, - co.co_code, - co.co_consts, - co.co_names, - co.co_varnames, - co.co_filename, - co.co_name, - co.co_firstlineno, - co.co_lnotab, - co.co_cellvars, # this is the trickery - (), - ) - -_cell_set_template_code = _make_cell_set_template_code() - - -def cell_set(cell, value): - """Set the value of a closure cell. - """ - return types.FunctionType( - _cell_set_template_code, - {}, - '_cell_set_inner', - (), - (cell,), - )(value) - - -# relevant opcodes -STORE_GLOBAL = opcode.opmap['STORE_GLOBAL'] -DELETE_GLOBAL = opcode.opmap['DELETE_GLOBAL'] -LOAD_GLOBAL = opcode.opmap['LOAD_GLOBAL'] -GLOBAL_OPS = (STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL) -HAVE_ARGUMENT = dis.HAVE_ARGUMENT -EXTENDED_ARG = dis.EXTENDED_ARG - - -def islambda(func): - return getattr(func, '__name__') == '' - - -_BUILTIN_TYPE_NAMES = {} -for k, v in types.__dict__.items(): - if type(v) is type: - _BUILTIN_TYPE_NAMES[v] = k - - -def _builtin_type(name): - return getattr(types, name) - - -def _make__new__factory(type_): - def _factory(): - return type_.__new__ - return _factory - - -# NOTE: These need to be module globals so that they're pickleable as globals. -_get_dict_new = _make__new__factory(dict) -_get_frozenset_new = _make__new__factory(frozenset) -_get_list_new = _make__new__factory(list) -_get_set_new = _make__new__factory(set) -_get_tuple_new = _make__new__factory(tuple) -_get_object_new = _make__new__factory(object) - -# Pre-defined set of builtin_function_or_method instances that can be -# serialized. -_BUILTIN_TYPE_CONSTRUCTORS = { - dict.__new__: _get_dict_new, - frozenset.__new__: _get_frozenset_new, - set.__new__: _get_set_new, - list.__new__: _get_list_new, - tuple.__new__: _get_tuple_new, - object.__new__: _get_object_new, -} - - -if sys.version_info < (3, 4): # pragma: no branch - def _walk_global_ops(code): - """ - Yield (opcode, argument number) tuples for all - global-referencing instructions in *code*. - """ - code = getattr(code, 'co_code', b'') - if PY2: # pragma: no branch - code = map(ord, code) - - n = len(code) - i = 0 - extended_arg = 0 - while i < n: - op = code[i] - i += 1 - if op >= HAVE_ARGUMENT: - oparg = code[i] + code[i + 1] * 256 + extended_arg - extended_arg = 0 - i += 2 - if op == EXTENDED_ARG: - extended_arg = oparg * 65536 - if op in GLOBAL_OPS: - yield op, oparg - -else: - def _walk_global_ops(code): - """ - Yield (opcode, argument number) tuples for all - global-referencing instructions in *code*. - """ - for instr in dis.get_instructions(code): - op = instr.opcode - if op in GLOBAL_OPS: - yield op, instr.arg - - -def _extract_class_dict(cls): - """Retrieve a copy of the dict of a class without the inherited methods""" - clsdict = dict(cls.__dict__) # copy dict proxy to a dict - if len(cls.__bases__) == 1: - inherited_dict = cls.__bases__[0].__dict__ - else: - inherited_dict = {} - for base in reversed(cls.__bases__): - inherited_dict.update(base.__dict__) - to_remove = [] - for name, value in clsdict.items(): - try: - base_value = inherited_dict[name] - if value is base_value: - to_remove.append(name) - elif PY2: - # backward compat for Python 2 - if hasattr(value, "im_func"): - if value.im_func is getattr(base_value, "im_func", None): - to_remove.append(name) - elif isinstance(value, PY2_CLASS_DICT_SKIP_PICKLE_METHOD_TYPE): - # On Python 2 we have no way to pickle those specific - # methods types nor to check that they are actually - # inherited. So we assume that they are always inherited - # from builtin types. - to_remove.append(name) - except KeyError: - pass - for name in to_remove: - clsdict.pop(name) - return clsdict - - -class CloudPickler(Pickler): - - dispatch = Pickler.dispatch.copy() - - def __init__(self, file, protocol=None): - if protocol is None: - protocol = DEFAULT_PROTOCOL - Pickler.__init__(self, file, protocol=protocol) - # map ids to dictionary. used to ensure that functions can share global env - self.globals_ref = {} - - def dump(self, obj): - self.inject_addons() - try: - return Pickler.dump(self, obj) - except RuntimeError as e: - if 'recursion' in e.args[0]: - msg = """Could not pickle object as excessively deep recursion required.""" - raise pickle.PicklingError(msg) - else: - raise - - def save_memoryview(self, obj): - self.save(obj.tobytes()) - - dispatch[memoryview] = save_memoryview - - if PY2: # pragma: no branch - def save_buffer(self, obj): - self.save(str(obj)) - - dispatch[buffer] = save_buffer # noqa: F821 'buffer' was removed in Python 3 - - def save_module(self, obj): - """ - Save a module as an import - """ - if _is_dynamic(obj): - self.save_reduce(dynamic_subimport, (obj.__name__, vars(obj)), - obj=obj) - else: - self.save_reduce(subimport, (obj.__name__,), obj=obj) - - dispatch[types.ModuleType] = save_module - - def save_codeobject(self, obj): - """ - Save a code object - """ - if PY3: # pragma: no branch - if hasattr(obj, "co_posonlyargcount"): # pragma: no branch - args = ( - obj.co_argcount, obj.co_posonlyargcount, - obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, - obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, - obj.co_varnames, obj.co_filename, obj.co_name, - obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, - obj.co_cellvars - ) - else: - args = ( - obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, - obj.co_stacksize, obj.co_flags, obj.co_code, obj.co_consts, - obj.co_names, obj.co_varnames, obj.co_filename, - obj.co_name, obj.co_firstlineno, obj.co_lnotab, - obj.co_freevars, obj.co_cellvars - ) - else: - args = ( - obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, - obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, - obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars - ) - self.save_reduce(types.CodeType, args, obj=obj) - - dispatch[types.CodeType] = save_codeobject - - def save_function(self, obj, name=None): - """ Registered with the dispatch to handle all function types. - - Determines what kind of function obj is (e.g. lambda, defined at - interactive prompt, etc) and handles the pickling appropriately. - """ - try: - should_special_case = obj in _BUILTIN_TYPE_CONSTRUCTORS - except TypeError: - # Methods of builtin types aren't hashable in python 2. - should_special_case = False - - if should_special_case: - # We keep a special-cased cache of built-in type constructors at - # global scope, because these functions are structured very - # differently in different python versions and implementations (for - # example, they're instances of types.BuiltinFunctionType in - # CPython, but they're ordinary types.FunctionType instances in - # PyPy). - # - # If the function we've received is in that cache, we just - # serialize it as a lookup into the cache. - return self.save_reduce(_BUILTIN_TYPE_CONSTRUCTORS[obj], (), obj=obj) - - write = self.write - - if name is None: - name = obj.__name__ - try: - # whichmodule() could fail, see - # https://bitbucket.org/gutworth/six/issues/63/importing-six-breaks-pickling - modname = pickle.whichmodule(obj, name) - except Exception: - modname = None - # print('which gives %s %s %s' % (modname, obj, name)) - try: - themodule = sys.modules[modname] - except KeyError: - # eval'd items such as namedtuple give invalid items for their function __module__ - modname = '__main__' - - if modname == '__main__': - themodule = None - - try: - lookedup_by_name = getattr(themodule, name, None) - except Exception: - lookedup_by_name = None - - if themodule: - if lookedup_by_name is obj: - return self.save_global(obj, name) - - # a builtin_function_or_method which comes in as an attribute of some - # object (e.g., itertools.chain.from_iterable) will end - # up with modname "__main__" and so end up here. But these functions - # have no __code__ attribute in CPython, so the handling for - # user-defined functions below will fail. - # So we pickle them here using save_reduce; have to do it differently - # for different python versions. - if not hasattr(obj, '__code__'): - if PY3: # pragma: no branch - rv = obj.__reduce_ex__(self.proto) - else: - if hasattr(obj, '__self__'): - rv = (getattr, (obj.__self__, name)) - else: - raise pickle.PicklingError("Can't pickle %r" % obj) - return self.save_reduce(obj=obj, *rv) - - # if func is lambda, def'ed at prompt, is in main, or is nested, then - # we'll pickle the actual function object rather than simply saving a - # reference (as is done in default pickler), via save_function_tuple. - if (islambda(obj) - or getattr(obj.__code__, 'co_filename', None) == '' - or themodule is None): - self.save_function_tuple(obj) - return - else: - # func is nested - if lookedup_by_name is None or lookedup_by_name is not obj: - self.save_function_tuple(obj) - return - - if obj.__dict__: - # essentially save_reduce, but workaround needed to avoid recursion - self.save(_restore_attr) - write(pickle.MARK + pickle.GLOBAL + modname + '\n' + name + '\n') - self.memoize(obj) - self.save(obj.__dict__) - write(pickle.TUPLE + pickle.REDUCE) - else: - write(pickle.GLOBAL + modname + '\n' + name + '\n') - self.memoize(obj) - - dispatch[types.FunctionType] = save_function - - def _save_subimports(self, code, top_level_dependencies): - """ - Save submodules used by a function but not listed in its globals. - - In the example below: - - ``` - import concurrent.futures - import cloudpickle - - - def func(): - x = concurrent.futures.ThreadPoolExecutor - - - if __name__ == '__main__': - cloudpickle.dumps(func) - ``` - - the globals extracted by cloudpickle in the function's state include - the concurrent module, but not its submodule (here, - concurrent.futures), which is the module used by func. - - To ensure that calling the depickled function does not raise an - AttributeError, this function looks for any currently loaded submodule - that the function uses and whose parent is present in the function - globals, and saves it before saving the function. - """ - - # check if any known dependency is an imported package - for x in top_level_dependencies: - if isinstance(x, types.ModuleType) and hasattr(x, '__package__') and x.__package__: - # check if the package has any currently loaded sub-imports - prefix = x.__name__ + '.' - # A concurrent thread could mutate sys.modules, - # make sure we iterate over a copy to avoid exceptions - for name in list(sys.modules): - # Older versions of pytest will add a "None" module to sys.modules. - if name is not None and name.startswith(prefix): - # check whether the function can address the sub-module - tokens = set(name[len(prefix):].split('.')) - if not tokens - set(code.co_names): - # ensure unpickler executes this import - self.save(sys.modules[name]) - # then discards the reference to it - self.write(pickle.POP) - - def _save_dynamic_enum(self, obj, clsdict): - """Special handling for dynamic Enum subclasses - - Use a dedicated Enum constructor (inspired by EnumMeta.__call__) as the - EnumMeta metaclass has complex initialization that makes the Enum - subclasses hold references to their own instances. - """ - members = dict((e.name, e.value) for e in obj) - - # Python 2.7 with enum34 can have no qualname: - qualname = getattr(obj, "__qualname__", None) - - self.save_reduce(_make_skeleton_enum, - (obj.__bases__, obj.__name__, qualname, members, - obj.__module__, _ensure_tracking(obj), None), - obj=obj) - - # Cleanup the clsdict that will be passed to _rehydrate_skeleton_class: - # Those attributes are already handled by the metaclass. - for attrname in ["_generate_next_value_", "_member_names_", - "_member_map_", "_member_type_", - "_value2member_map_"]: - clsdict.pop(attrname, None) - for member in members: - clsdict.pop(member) - - def save_dynamic_class(self, obj): - """Save a class that can't be stored as module global. - - This method is used to serialize classes that are defined inside - functions, or that otherwise can't be serialized as attribute lookups - from global modules. - """ - clsdict = _extract_class_dict(obj) - clsdict.pop('__weakref__', None) - - # For ABCMeta in python3.7+, remove _abc_impl as it is not picklable. - # This is a fix which breaks the cache but this only makes the first - # calls to issubclass slower. - if "_abc_impl" in clsdict: - import abc - (registry, _, _, _) = abc._get_dump(obj) - clsdict["_abc_impl"] = [subclass_weakref() - for subclass_weakref in registry] - - # On PyPy, __doc__ is a readonly attribute, so we need to include it in - # the initial skeleton class. This is safe because we know that the - # doc can't participate in a cycle with the original class. - type_kwargs = {'__doc__': clsdict.pop('__doc__', None)} - - if hasattr(obj, "__slots__"): - type_kwargs['__slots__'] = obj.__slots__ - # pickle string length optimization: member descriptors of obj are - # created automatically from obj's __slots__ attribute, no need to - # save them in obj's state - if isinstance(obj.__slots__, string_types): - clsdict.pop(obj.__slots__) - else: - for k in obj.__slots__: - clsdict.pop(k, None) - - # If type overrides __dict__ as a property, include it in the type - # kwargs. In Python 2, we can't set this attribute after construction. - __dict__ = clsdict.pop('__dict__', None) - if isinstance(__dict__, property): - type_kwargs['__dict__'] = __dict__ - - save = self.save - write = self.write - - # We write pickle instructions explicitly here to handle the - # possibility that the type object participates in a cycle with its own - # __dict__. We first write an empty "skeleton" version of the class and - # memoize it before writing the class' __dict__ itself. We then write - # instructions to "rehydrate" the skeleton class by restoring the - # attributes from the __dict__. - # - # A type can appear in a cycle with its __dict__ if an instance of the - # type appears in the type's __dict__ (which happens for the stdlib - # Enum class), or if the type defines methods that close over the name - # of the type, (which is common for Python 2-style super() calls). - - # Push the rehydration function. - save(_rehydrate_skeleton_class) - - # Mark the start of the args tuple for the rehydration function. - write(pickle.MARK) - - # Create and memoize an skeleton class with obj's name and bases. - if Enum is not None and issubclass(obj, Enum): - # Special handling of Enum subclasses - self._save_dynamic_enum(obj, clsdict) - else: - # "Regular" class definition: - tp = type(obj) - self.save_reduce(_make_skeleton_class, - (tp, obj.__name__, obj.__bases__, type_kwargs, - _ensure_tracking(obj), None), - obj=obj) - - # Now save the rest of obj's __dict__. Any references to obj - # encountered while saving will point to the skeleton class. - save(clsdict) - - # Write a tuple of (skeleton_class, clsdict). - write(pickle.TUPLE) - - # Call _rehydrate_skeleton_class(skeleton_class, clsdict) - write(pickle.REDUCE) - - def save_function_tuple(self, func): - """ Pickles an actual func object. - - A func comprises: code, globals, defaults, closure, and dict. We - extract and save these, injecting reducing functions at certain points - to recreate the func object. Keep in mind that some of these pieces - can contain a ref to the func itself. Thus, a naive save on these - pieces could trigger an infinite loop of save's. To get around that, - we first create a skeleton func object using just the code (this is - safe, since this won't contain a ref to the func), and memoize it as - soon as it's created. The other stuff can then be filled in later. - """ - if is_tornado_coroutine(func): - self.save_reduce(_rebuild_tornado_coroutine, (func.__wrapped__,), - obj=func) - return - - save = self.save - write = self.write - - code, f_globals, defaults, closure_values, dct, base_globals = self.extract_func_data(func) - - save(_fill_function) # skeleton function updater - write(pickle.MARK) # beginning of tuple that _fill_function expects - - self._save_subimports( - code, - itertools.chain(f_globals.values(), closure_values or ()), - ) - - # create a skeleton function object and memoize it - save(_make_skel_func) - save(( - code, - len(closure_values) if closure_values is not None else -1, - base_globals, - )) - write(pickle.REDUCE) - self.memoize(func) - - # save the rest of the func data needed by _fill_function - state = { - 'globals': f_globals, - 'defaults': defaults, - 'dict': dct, - 'closure_values': closure_values, - 'module': func.__module__, - 'name': func.__name__, - 'doc': func.__doc__, - } - if hasattr(func, '__annotations__') and sys.version_info >= (3, 7): - state['annotations'] = func.__annotations__ - if hasattr(func, '__qualname__'): - state['qualname'] = func.__qualname__ - if hasattr(func, '__kwdefaults__'): - state['kwdefaults'] = func.__kwdefaults__ - save(state) - write(pickle.TUPLE) - write(pickle.REDUCE) # applies _fill_function on the tuple - - _extract_code_globals_cache = ( - weakref.WeakKeyDictionary() - if not hasattr(sys, "pypy_version_info") - else {}) - - @classmethod - def extract_code_globals(cls, co): - """ - Find all globals names read or written to by codeblock co - """ - out_names = cls._extract_code_globals_cache.get(co) - if out_names is None: - try: - names = co.co_names - except AttributeError: - # PyPy "builtin-code" object - out_names = set() - else: - out_names = {names[oparg] for _, oparg in _walk_global_ops(co)} - - # see if nested function have any global refs - if co.co_consts: - for const in co.co_consts: - if type(const) is types.CodeType: - out_names |= cls.extract_code_globals(const) - - cls._extract_code_globals_cache[co] = out_names - - return out_names - - def extract_func_data(self, func): - """ - Turn the function into a tuple of data necessary to recreate it: - code, globals, defaults, closure_values, dict - """ - code = func.__code__ - - # extract all global ref's - func_global_refs = self.extract_code_globals(code) - - # process all variables referenced by global environment - f_globals = {} - for var in func_global_refs: - if var in func.__globals__: - f_globals[var] = func.__globals__[var] - - # defaults requires no processing - defaults = func.__defaults__ - - # process closure - closure = ( - list(map(_get_cell_contents, func.__closure__)) - if func.__closure__ is not None - else None - ) - - # save the dict - dct = func.__dict__ - - # base_globals represents the future global namespace of func at - # unpickling time. Looking it up and storing it in globals_ref allow - # functions sharing the same globals at pickling time to also - # share them once unpickled, at one condition: since globals_ref is - # an attribute of a Cloudpickler instance, and that a new CloudPickler is - # created each time pickle.dump or pickle.dumps is called, functions - # also need to be saved within the same invokation of - # cloudpickle.dump/cloudpickle.dumps (for example: cloudpickle.dumps([f1, f2])). There - # is no such limitation when using Cloudpickler.dump, as long as the - # multiple invokations are bound to the same Cloudpickler. - base_globals = self.globals_ref.setdefault(id(func.__globals__), {}) - - if base_globals == {}: - # Add module attributes used to resolve relative imports - # instructions inside func. - for k in ["__package__", "__name__", "__path__", "__file__"]: - # Some built-in functions/methods such as object.__new__ have - # their __globals__ set to None in PyPy - if func.__globals__ is not None and k in func.__globals__: - base_globals[k] = func.__globals__[k] - - return (code, f_globals, defaults, closure, dct, base_globals) - - def save_builtin_function(self, obj): - if obj.__module__ == "__builtin__": - return self.save_global(obj) - return self.save_function(obj) - - dispatch[types.BuiltinFunctionType] = save_builtin_function - - def save_global(self, obj, name=None, pack=struct.pack): - """ - Save a "global". - - The name of this method is somewhat misleading: all types get - dispatched here. - """ - if obj is type(None): - return self.save_reduce(type, (None,), obj=obj) - elif obj is type(Ellipsis): - return self.save_reduce(type, (Ellipsis,), obj=obj) - elif obj is type(NotImplemented): - return self.save_reduce(type, (NotImplemented,), obj=obj) - - if obj.__module__ == "__main__": - return self.save_dynamic_class(obj) - - try: - return Pickler.save_global(self, obj, name=name) - except Exception: - if obj.__module__ == "__builtin__" or obj.__module__ == "builtins": - if obj in _BUILTIN_TYPE_NAMES: - return self.save_reduce( - _builtin_type, (_BUILTIN_TYPE_NAMES[obj],), obj=obj) - - typ = type(obj) - if typ is not obj and isinstance(obj, (type, types.ClassType)): - return self.save_dynamic_class(obj) - - raise - - dispatch[type] = save_global - dispatch[types.ClassType] = save_global - - def save_instancemethod(self, obj): - # Memoization rarely is ever useful due to python bounding - if obj.__self__ is None: - self.save_reduce(getattr, (obj.im_class, obj.__name__)) - else: - if PY3: # pragma: no branch - self.save_reduce(types.MethodType, (obj.__func__, obj.__self__), obj=obj) - else: - self.save_reduce(types.MethodType, (obj.__func__, obj.__self__, obj.__self__.__class__), - obj=obj) - - dispatch[types.MethodType] = save_instancemethod - - def save_inst(self, obj): - """Inner logic to save instance. Based off pickle.save_inst""" - cls = obj.__class__ - - # Try the dispatch table (pickle module doesn't do it) - f = self.dispatch.get(cls) - if f: - f(self, obj) # Call unbound method with explicit self - return - - memo = self.memo - write = self.write - save = self.save - - if hasattr(obj, '__getinitargs__'): - args = obj.__getinitargs__() - len(args) # XXX Assert it's a sequence - pickle._keep_alive(args, memo) - else: - args = () - - write(pickle.MARK) - - if self.bin: - save(cls) - for arg in args: - save(arg) - write(pickle.OBJ) - else: - for arg in args: - save(arg) - write(pickle.INST + cls.__module__ + '\n' + cls.__name__ + '\n') - - self.memoize(obj) - - try: - getstate = obj.__getstate__ - except AttributeError: - stuff = obj.__dict__ - else: - stuff = getstate() - pickle._keep_alive(stuff, memo) - save(stuff) - write(pickle.BUILD) - - if PY2: # pragma: no branch - dispatch[types.InstanceType] = save_inst - - def save_property(self, obj): - # properties not correctly saved in python - self.save_reduce(property, (obj.fget, obj.fset, obj.fdel, obj.__doc__), obj=obj) - - dispatch[property] = save_property - - def save_classmethod(self, obj): - orig_func = obj.__func__ - self.save_reduce(type(obj), (orig_func,), obj=obj) - - dispatch[classmethod] = save_classmethod - dispatch[staticmethod] = save_classmethod - - def save_itemgetter(self, obj): - """itemgetter serializer (needed for namedtuple support)""" - class Dummy: - def __getitem__(self, item): - return item - items = obj(Dummy()) - if not isinstance(items, tuple): - items = (items,) - return self.save_reduce(operator.itemgetter, items) - - if type(operator.itemgetter) is type: - dispatch[operator.itemgetter] = save_itemgetter - - def save_attrgetter(self, obj): - """attrgetter serializer""" - class Dummy(object): - def __init__(self, attrs, index=None): - self.attrs = attrs - self.index = index - def __getattribute__(self, item): - attrs = object.__getattribute__(self, "attrs") - index = object.__getattribute__(self, "index") - if index is None: - index = len(attrs) - attrs.append(item) - else: - attrs[index] = ".".join([attrs[index], item]) - return type(self)(attrs, index) - attrs = [] - obj(Dummy(attrs)) - return self.save_reduce(operator.attrgetter, tuple(attrs)) - - if type(operator.attrgetter) is type: - dispatch[operator.attrgetter] = save_attrgetter - - def save_file(self, obj): - """Save a file""" - try: - import StringIO as pystringIO # we can't use cStringIO as it lacks the name attribute - except ImportError: - import io as pystringIO - - if not hasattr(obj, 'name') or not hasattr(obj, 'mode'): - raise pickle.PicklingError("Cannot pickle files that do not map to an actual file") - if obj is sys.stdout: - return self.save_reduce(getattr, (sys, 'stdout'), obj=obj) - if obj is sys.stderr: - return self.save_reduce(getattr, (sys, 'stderr'), obj=obj) - if obj is sys.stdin: - raise pickle.PicklingError("Cannot pickle standard input") - if obj.closed: - raise pickle.PicklingError("Cannot pickle closed files") - if hasattr(obj, 'isatty') and obj.isatty(): - raise pickle.PicklingError("Cannot pickle files that map to tty objects") - if 'r' not in obj.mode and '+' not in obj.mode: - raise pickle.PicklingError("Cannot pickle files that are not opened for reading: %s" % obj.mode) - - name = obj.name - - retval = pystringIO.StringIO() - - try: - # Read the whole file - curloc = obj.tell() - obj.seek(0) - contents = obj.read() - obj.seek(curloc) - except IOError: - raise pickle.PicklingError("Cannot pickle file %s as it cannot be read" % name) - retval.write(contents) - retval.seek(curloc) - - retval.name = name - self.save(retval) - self.memoize(obj) - - def save_ellipsis(self, obj): - self.save_reduce(_gen_ellipsis, ()) - - def save_not_implemented(self, obj): - self.save_reduce(_gen_not_implemented, ()) - - try: # Python 2 - dispatch[file] = save_file - except NameError: # Python 3 # pragma: no branch - dispatch[io.TextIOWrapper] = save_file - - dispatch[type(Ellipsis)] = save_ellipsis - dispatch[type(NotImplemented)] = save_not_implemented - - def save_weakset(self, obj): - self.save_reduce(weakref.WeakSet, (list(obj),)) - - dispatch[weakref.WeakSet] = save_weakset - - def save_logger(self, obj): - self.save_reduce(logging.getLogger, (obj.name,), obj=obj) - - dispatch[logging.Logger] = save_logger - - def save_root_logger(self, obj): - self.save_reduce(logging.getLogger, (), obj=obj) - - dispatch[logging.RootLogger] = save_root_logger - - if hasattr(types, "MappingProxyType"): # pragma: no branch - def save_mappingproxy(self, obj): - self.save_reduce(types.MappingProxyType, (dict(obj),), obj=obj) - - dispatch[types.MappingProxyType] = save_mappingproxy - - """Special functions for Add-on libraries""" - def inject_addons(self): - """Plug in system. Register additional pickling functions if modules already loaded""" - pass - - -# Tornado support - -def is_tornado_coroutine(func): - """ - Return whether *func* is a Tornado coroutine function. - Running coroutines are not supported. - """ - if 'tornado.gen' not in sys.modules: - return False - gen = sys.modules['tornado.gen'] - if not hasattr(gen, "is_coroutine_function"): - # Tornado version is too old - return False - return gen.is_coroutine_function(func) - - -def _rebuild_tornado_coroutine(func): - from tornado import gen - return gen.coroutine(func) - - -# Shorthands for legacy support - -def dump(obj, file, protocol=None): - """Serialize obj as bytes streamed into file - - protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to - pickle.HIGHEST_PROTOCOL. This setting favors maximum communication speed - between processes running the same Python version. - - Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure - compatibility with older versions of Python. - """ - CloudPickler(file, protocol=protocol).dump(obj) - - -def dumps(obj, protocol=None): - """Serialize obj as a string of bytes allocated in memory - - protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to - pickle.HIGHEST_PROTOCOL. This setting favors maximum communication speed - between processes running the same Python version. - - Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure - compatibility with older versions of Python. - """ - file = StringIO() - try: - cp = CloudPickler(file, protocol=protocol) - cp.dump(obj) - return file.getvalue() - finally: - file.close() - - -# including pickles unloading functions in this namespace -load = pickle.load -loads = pickle.loads - - -# hack for __import__ not working as desired -def subimport(name): - __import__(name) - return sys.modules[name] - - -def dynamic_subimport(name, vars): - mod = types.ModuleType(name) - mod.__dict__.update(vars) - return mod - - -# restores function attributes -def _restore_attr(obj, attr): - for key, val in attr.items(): - setattr(obj, key, val) - return obj - - -def _gen_ellipsis(): - return Ellipsis - - -def _gen_not_implemented(): - return NotImplemented - - -def _get_cell_contents(cell): - try: - return cell.cell_contents - except ValueError: - # sentinel used by ``_fill_function`` which will leave the cell empty - return _empty_cell_value - - -def instance(cls): - """Create a new instance of a class. - - Parameters - ---------- - cls : type - The class to create an instance of. - - Returns - ------- - instance : cls - A new instance of ``cls``. - """ - return cls() - - -@instance -class _empty_cell_value(object): - """sentinel for empty closures - """ - @classmethod - def __reduce__(cls): - return cls.__name__ - - -def _fill_function(*args): - """Fills in the rest of function data into the skeleton function object - - The skeleton itself is create by _make_skel_func(). - """ - if len(args) == 2: - func = args[0] - state = args[1] - elif len(args) == 5: - # Backwards compat for cloudpickle v0.4.0, after which the `module` - # argument was introduced - func = args[0] - keys = ['globals', 'defaults', 'dict', 'closure_values'] - state = dict(zip(keys, args[1:])) - elif len(args) == 6: - # Backwards compat for cloudpickle v0.4.1, after which the function - # state was passed as a dict to the _fill_function it-self. - func = args[0] - keys = ['globals', 'defaults', 'dict', 'module', 'closure_values'] - state = dict(zip(keys, args[1:])) - else: - raise ValueError('Unexpected _fill_value arguments: %r' % (args,)) - - # - At pickling time, any dynamic global variable used by func is - # serialized by value (in state['globals']). - # - At unpickling time, func's __globals__ attribute is initialized by - # first retrieving an empty isolated namespace that will be shared - # with other functions pickled from the same original module - # by the same CloudPickler instance and then updated with the - # content of state['globals'] to populate the shared isolated - # namespace with all the global variables that are specifically - # referenced for this function. - func.__globals__.update(state['globals']) - - func.__defaults__ = state['defaults'] - func.__dict__ = state['dict'] - if 'annotations' in state: - func.__annotations__ = state['annotations'] - if 'doc' in state: - func.__doc__ = state['doc'] - if 'name' in state: - func.__name__ = state['name'] - if 'module' in state: - func.__module__ = state['module'] - if 'qualname' in state: - func.__qualname__ = state['qualname'] - if 'kwdefaults' in state: - func.__kwdefaults__ = state['kwdefaults'] - - cells = func.__closure__ - if cells is not None: - for cell, value in zip(cells, state['closure_values']): - if value is not _empty_cell_value: - cell_set(cell, value) - - return func - - -def _make_empty_cell(): - if False: - # trick the compiler into creating an empty cell in our lambda - cell = None - raise AssertionError('this route should not be executed') - - return (lambda: cell).__closure__[0] - - -def _make_skel_func(code, cell_count, base_globals=None): - """ Creates a skeleton function object that contains just the provided - code and the correct number of cells in func_closure. All other - func attributes (e.g. func_globals) are empty. - """ - # This is backward-compatibility code: for cloudpickle versions between - # 0.5.4 and 0.7, base_globals could be a string or None. base_globals - # should now always be a dictionary. - if base_globals is None or isinstance(base_globals, str): - base_globals = {} - - base_globals['__builtins__'] = __builtins__ - - closure = ( - tuple(_make_empty_cell() for _ in range(cell_count)) - if cell_count >= 0 else - None - ) - return types.FunctionType(code, base_globals, None, None, closure) - - -def _make_skeleton_class(type_constructor, name, bases, type_kwargs, - class_tracker_id, extra): - """Build dynamic class with an empty __dict__ to be filled once memoized - - If class_tracker_id is not None, try to lookup an existing class definition - matching that id. If none is found, track a newly reconstructed class - definition under that id so that other instances stemming from the same - class id will also reuse this class definition. - - The "extra" variable is meant to be a dict (or None) that can be used for - forward compatibility shall the need arise. - """ - skeleton_class = type_constructor(name, bases, type_kwargs) - return _lookup_class_or_track(class_tracker_id, skeleton_class) - - -def _rehydrate_skeleton_class(skeleton_class, class_dict): - """Put attributes from `class_dict` back on `skeleton_class`. - - See CloudPickler.save_dynamic_class for more info. - """ - registry = None - for attrname, attr in class_dict.items(): - if attrname == "_abc_impl": - registry = attr - else: - setattr(skeleton_class, attrname, attr) - if registry is not None: - for subclass in registry: - skeleton_class.register(subclass) - - return skeleton_class - - -def _make_skeleton_enum(bases, name, qualname, members, module, - class_tracker_id, extra): - """Build dynamic enum with an empty __dict__ to be filled once memoized - - The creation of the enum class is inspired by the code of - EnumMeta._create_. - - If class_tracker_id is not None, try to lookup an existing enum definition - matching that id. If none is found, track a newly reconstructed enum - definition under that id so that other instances stemming from the same - class id will also reuse this enum definition. - - The "extra" variable is meant to be a dict (or None) that can be used for - forward compatibility shall the need arise. - """ - # enums always inherit from their base Enum class at the last position in - # the list of base classes: - enum_base = bases[-1] - metacls = enum_base.__class__ - classdict = metacls.__prepare__(name, bases) - - for member_name, member_value in members.items(): - classdict[member_name] = member_value - enum_class = metacls.__new__(metacls, name, bases, classdict) - enum_class.__module__ = module - - # Python 2.7 compat - if qualname is not None: - enum_class.__qualname__ = qualname - - return _lookup_class_or_track(class_tracker_id, enum_class) - - -def _is_dynamic(module): - """ - Return True if the module is special module that cannot be imported by its - name. - """ - # Quick check: module that have __file__ attribute are not dynamic modules. - if hasattr(module, '__file__'): - return False - - if hasattr(module, '__spec__'): - return module.__spec__ is None - else: - # Backward compat for Python 2 - import imp - try: - path = None - for part in module.__name__.split('.'): - if path is not None: - path = [path] - f, path, description = imp.find_module(part, path) - if f is not None: - f.close() - except ImportError: - return True - return False - - -""" Use copy_reg to extend global pickle definitions """ - -if sys.version_info < (3, 4): # pragma: no branch - method_descriptor = type(str.upper) - - def _reduce_method_descriptor(obj): - return (getattr, (obj.__objclass__, obj.__name__)) - - try: - import copy_reg as copyreg - except ImportError: - import copyreg - copyreg.pickle(method_descriptor, _reduce_method_descriptor) \ No newline at end of file diff --git a/python/pyspark/cloudpickle/__init__.py b/python/pyspark/cloudpickle/__init__.py new file mode 100644 index 0000000000000..4e85b637800dc --- /dev/null +++ b/python/pyspark/cloudpickle/__init__.py @@ -0,0 +1,7 @@ +from __future__ import absolute_import + + +from pyspark.cloudpickle.cloudpickle import * # noqa +from pyspark.cloudpickle.cloudpickle_fast import CloudPickler, dumps, dump # noqa + +__version__ = '1.5.0' diff --git a/python/pyspark/cloudpickle/cloudpickle.py b/python/pyspark/cloudpickle/cloudpickle.py new file mode 100644 index 0000000000000..8e683e7a6988b --- /dev/null +++ b/python/pyspark/cloudpickle/cloudpickle.py @@ -0,0 +1,830 @@ +""" +This class is defined to override standard pickle functionality + +The goals of it follow: +-Serialize lambdas and nested functions to compiled byte code +-Deal with main module correctly +-Deal with other non-serializable objects + +It does not include an unpickler, as standard python unpickling suffices. + +This module was extracted from the `cloud` package, developed by `PiCloud, Inc. +`_. + +Copyright (c) 2012, Regents of the University of California. +Copyright (c) 2009 `PiCloud, Inc. `_. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the University of California, Berkeley nor the + names of its contributors may be used to endorse or promote + products derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED +TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +""" +from __future__ import print_function + +import builtins +import dis +import opcode +import platform +import sys +import types +import weakref +import uuid +import threading +import typing +import warnings + +from .compat import pickle +from typing import Generic, Union, Tuple, Callable +from pickle import _getattribute +from importlib._bootstrap import _find_spec + +try: # pragma: no branch + import typing_extensions as _typing_extensions + from typing_extensions import Literal, Final +except ImportError: + _typing_extensions = Literal = Final = None + +if sys.version_info >= (3, 5, 3): + from typing import ClassVar +else: # pragma: no cover + ClassVar = None + +if sys.version_info >= (3, 8): + from types import CellType +else: + def f(): + a = 1 + + def g(): + return a + return g + CellType = type(f().__closure__[0]) + + +# cloudpickle is meant for inter process communication: we expect all +# communicating processes to run the same Python version hence we favor +# communication speed over compatibility: +DEFAULT_PROTOCOL = pickle.HIGHEST_PROTOCOL + +# Track the provenance of reconstructed dynamic classes to make it possible to +# recontruct instances from the matching singleton class definition when +# appropriate and preserve the usual "isinstance" semantics of Python objects. +_DYNAMIC_CLASS_TRACKER_BY_CLASS = weakref.WeakKeyDictionary() +_DYNAMIC_CLASS_TRACKER_BY_ID = weakref.WeakValueDictionary() +_DYNAMIC_CLASS_TRACKER_LOCK = threading.Lock() + +PYPY = platform.python_implementation() == "PyPy" + +builtin_code_type = None +if PYPY: + # builtin-code objects only exist in pypy + builtin_code_type = type(float.__new__.__code__) + +_extract_code_globals_cache = weakref.WeakKeyDictionary() + + +def _get_or_create_tracker_id(class_def): + with _DYNAMIC_CLASS_TRACKER_LOCK: + class_tracker_id = _DYNAMIC_CLASS_TRACKER_BY_CLASS.get(class_def) + if class_tracker_id is None: + class_tracker_id = uuid.uuid4().hex + _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id + _DYNAMIC_CLASS_TRACKER_BY_ID[class_tracker_id] = class_def + return class_tracker_id + + +def _lookup_class_or_track(class_tracker_id, class_def): + if class_tracker_id is not None: + with _DYNAMIC_CLASS_TRACKER_LOCK: + class_def = _DYNAMIC_CLASS_TRACKER_BY_ID.setdefault( + class_tracker_id, class_def) + _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id + return class_def + + +def _whichmodule(obj, name): + """Find the module an object belongs to. + + This function differs from ``pickle.whichmodule`` in two ways: + - it does not mangle the cases where obj's module is __main__ and obj was + not found in any module. + - Errors arising during module introspection are ignored, as those errors + are considered unwanted side effects. + """ + if sys.version_info[:2] < (3, 7) and isinstance(obj, typing.TypeVar): # pragma: no branch # noqa + # Workaround bug in old Python versions: prior to Python 3.7, + # T.__module__ would always be set to "typing" even when the TypeVar T + # would be defined in a different module. + # + # For such older Python versions, we ignore the __module__ attribute of + # TypeVar instances and instead exhaustively lookup those instances in + # all currently imported modules. + module_name = None + else: + module_name = getattr(obj, '__module__', None) + + if module_name is not None: + return module_name + # Protect the iteration by using a copy of sys.modules against dynamic + # modules that trigger imports of other modules upon calls to getattr or + # other threads importing at the same time. + for module_name, module in sys.modules.copy().items(): + # Some modules such as coverage can inject non-module objects inside + # sys.modules + if ( + module_name == '__main__' or + module is None or + not isinstance(module, types.ModuleType) + ): + continue + try: + if _getattribute(module, name)[0] is obj: + return module_name + except Exception: + pass + return None + + +def _is_importable(obj, name=None): + """Dispatcher utility to test the importability of various constructs.""" + if isinstance(obj, types.FunctionType): + return _lookup_module_and_qualname(obj, name=name) is not None + elif issubclass(type(obj), type): + return _lookup_module_and_qualname(obj, name=name) is not None + elif isinstance(obj, types.ModuleType): + # We assume that sys.modules is primarily used as a cache mechanism for + # the Python import machinery. Checking if a module has been added in + # is sys.modules therefore a cheap and simple heuristic to tell us whether + # we can assume that a given module could be imported by name in + # another Python process. + return obj.__name__ in sys.modules + else: + raise TypeError( + "cannot check importability of {} instances".format( + type(obj).__name__) + ) + + +def _lookup_module_and_qualname(obj, name=None): + if name is None: + name = getattr(obj, '__qualname__', None) + if name is None: # pragma: no cover + # This used to be needed for Python 2.7 support but is probably not + # needed anymore. However we keep the __name__ introspection in case + # users of cloudpickle rely on this old behavior for unknown reasons. + name = getattr(obj, '__name__', None) + + module_name = _whichmodule(obj, name) + + if module_name is None: + # In this case, obj.__module__ is None AND obj was not found in any + # imported module. obj is thus treated as dynamic. + return None + + if module_name == "__main__": + return None + + # Note: if module_name is in sys.modules, the corresponding module is + # assumed importable at unpickling time. See #357 + module = sys.modules.get(module_name, None) + if module is None: + # The main reason why obj's module would not be imported is that this + # module has been dynamically created, using for example + # types.ModuleType. The other possibility is that module was removed + # from sys.modules after obj was created/imported. But this case is not + # supported, as the standard pickle does not support it either. + return None + + try: + obj2, parent = _getattribute(module, name) + except AttributeError: + # obj was not found inside the module it points to + return None + if obj2 is not obj: + return None + return module, name + + +def _extract_code_globals(co): + """ + Find all globals names read or written to by codeblock co + """ + out_names = _extract_code_globals_cache.get(co) + if out_names is None: + names = co.co_names + out_names = {names[oparg] for _, oparg in _walk_global_ops(co)} + + # Declaring a function inside another one using the "def ..." + # syntax generates a constant code object corresonding to the one + # of the nested function's As the nested function may itself need + # global variables, we need to introspect its code, extract its + # globals, (look for code object in it's co_consts attribute..) and + # add the result to code_globals + if co.co_consts: + for const in co.co_consts: + if isinstance(const, types.CodeType): + out_names |= _extract_code_globals(const) + + _extract_code_globals_cache[co] = out_names + + return out_names + + +def _find_imported_submodules(code, top_level_dependencies): + """ + Find currently imported submodules used by a function. + + Submodules used by a function need to be detected and referenced for the + function to work correctly at depickling time. Because submodules can be + referenced as attribute of their parent package (``package.submodule``), we + need a special introspection technique that does not rely on GLOBAL-related + opcodes to find references of them in a code object. + + Example: + ``` + import concurrent.futures + import cloudpickle + def func(): + x = concurrent.futures.ThreadPoolExecutor + if __name__ == '__main__': + cloudpickle.dumps(func) + ``` + The globals extracted by cloudpickle in the function's state include the + concurrent package, but not its submodule (here, concurrent.futures), which + is the module used by func. Find_imported_submodules will detect the usage + of concurrent.futures. Saving this module alongside with func will ensure + that calling func once depickled does not fail due to concurrent.futures + not being imported + """ + + subimports = [] + # check if any known dependency is an imported package + for x in top_level_dependencies: + if (isinstance(x, types.ModuleType) and + hasattr(x, '__package__') and x.__package__): + # check if the package has any currently loaded sub-imports + prefix = x.__name__ + '.' + # A concurrent thread could mutate sys.modules, + # make sure we iterate over a copy to avoid exceptions + for name in list(sys.modules): + # Older versions of pytest will add a "None" module to + # sys.modules. + if name is not None and name.startswith(prefix): + # check whether the function can address the sub-module + tokens = set(name[len(prefix):].split('.')) + if not tokens - set(code.co_names): + subimports.append(sys.modules[name]) + return subimports + + +def cell_set(cell, value): + """Set the value of a closure cell. + + The point of this function is to set the cell_contents attribute of a cell + after its creation. This operation is necessary in case the cell contains a + reference to the function the cell belongs to, as when calling the + function's constructor + ``f = types.FunctionType(code, globals, name, argdefs, closure)``, + closure will not be able to contain the yet-to-be-created f. + + In Python3.7, cell_contents is writeable, so setting the contents of a cell + can be done simply using + >>> cell.cell_contents = value + + In earlier Python3 versions, the cell_contents attribute of a cell is read + only, but this limitation can be worked around by leveraging the Python 3 + ``nonlocal`` keyword. + + In Python2 however, this attribute is read only, and there is no + ``nonlocal`` keyword. For this reason, we need to come up with more + complicated hacks to set this attribute. + + The chosen approach is to create a function with a STORE_DEREF opcode, + which sets the content of a closure variable. Typically: + + >>> def inner(value): + ... lambda: cell # the lambda makes cell a closure + ... cell = value # cell is a closure, so this triggers a STORE_DEREF + + (Note that in Python2, A STORE_DEREF can never be triggered from an inner + function. The function g for example here + >>> def f(var): + ... def g(): + ... var += 1 + ... return g + + will not modify the closure variable ``var```inplace, but instead try to + load a local variable var and increment it. As g does not assign the local + variable ``var`` any initial value, calling f(1)() will fail at runtime.) + + Our objective is to set the value of a given cell ``cell``. So we need to + somewhat reference our ``cell`` object into the ``inner`` function so that + this object (and not the smoke cell of the lambda function) gets affected + by the STORE_DEREF operation. + + In inner, ``cell`` is referenced as a cell variable (an enclosing variable + that is referenced by the inner function). If we create a new function + cell_set with the exact same code as ``inner``, but with ``cell`` marked as + a free variable instead, the STORE_DEREF will be applied on its closure - + ``cell``, which we can specify explicitly during construction! The new + cell_set variable thus actually sets the contents of a specified cell! + + Note: we do not make use of the ``nonlocal`` keyword to set the contents of + a cell in early python3 versions to limit possible syntax errors in case + test and checker libraries decide to parse the whole file. + """ + + if sys.version_info[:2] >= (3, 7): # pragma: no branch + cell.cell_contents = value + else: + _cell_set = types.FunctionType( + _cell_set_template_code, {}, '_cell_set', (), (cell,),) + _cell_set(value) + + +def _make_cell_set_template_code(): + def _cell_set_factory(value): + lambda: cell + cell = value + + co = _cell_set_factory.__code__ + + _cell_set_template_code = types.CodeType( + co.co_argcount, + co.co_kwonlyargcount, # Python 3 only argument + co.co_nlocals, + co.co_stacksize, + co.co_flags, + co.co_code, + co.co_consts, + co.co_names, + co.co_varnames, + co.co_filename, + co.co_name, + co.co_firstlineno, + co.co_lnotab, + co.co_cellvars, # co_freevars is initialized with co_cellvars + (), # co_cellvars is made empty + ) + return _cell_set_template_code + + +if sys.version_info[:2] < (3, 7): + _cell_set_template_code = _make_cell_set_template_code() + +# relevant opcodes +STORE_GLOBAL = opcode.opmap['STORE_GLOBAL'] +DELETE_GLOBAL = opcode.opmap['DELETE_GLOBAL'] +LOAD_GLOBAL = opcode.opmap['LOAD_GLOBAL'] +GLOBAL_OPS = (STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL) +HAVE_ARGUMENT = dis.HAVE_ARGUMENT +EXTENDED_ARG = dis.EXTENDED_ARG + + +_BUILTIN_TYPE_NAMES = {} +for k, v in types.__dict__.items(): + if type(v) is type: + _BUILTIN_TYPE_NAMES[v] = k + + +def _builtin_type(name): + if name == "ClassType": # pragma: no cover + # Backward compat to load pickle files generated with cloudpickle + # < 1.3 even if loading pickle files from older versions is not + # officially supported. + return type + return getattr(types, name) + + +def _walk_global_ops(code): + """ + Yield (opcode, argument number) tuples for all + global-referencing instructions in *code*. + """ + for instr in dis.get_instructions(code): + op = instr.opcode + if op in GLOBAL_OPS: + yield op, instr.arg + + +def _extract_class_dict(cls): + """Retrieve a copy of the dict of a class without the inherited methods""" + clsdict = dict(cls.__dict__) # copy dict proxy to a dict + if len(cls.__bases__) == 1: + inherited_dict = cls.__bases__[0].__dict__ + else: + inherited_dict = {} + for base in reversed(cls.__bases__): + inherited_dict.update(base.__dict__) + to_remove = [] + for name, value in clsdict.items(): + try: + base_value = inherited_dict[name] + if value is base_value: + to_remove.append(name) + except KeyError: + pass + for name in to_remove: + clsdict.pop(name) + return clsdict + + +if sys.version_info[:2] < (3, 7): # pragma: no branch + def _is_parametrized_type_hint(obj): + # This is very cheap but might generate false positives. + # general typing Constructs + is_typing = getattr(obj, '__origin__', None) is not None + + # typing_extensions.Literal + is_litteral = getattr(obj, '__values__', None) is not None + + # typing_extensions.Final + is_final = getattr(obj, '__type__', None) is not None + + # typing.Union/Tuple for old Python 3.5 + is_union = getattr(obj, '__union_params__', None) is not None + is_tuple = getattr(obj, '__tuple_params__', None) is not None + is_callable = ( + getattr(obj, '__result__', None) is not None and + getattr(obj, '__args__', None) is not None + ) + return any((is_typing, is_litteral, is_final, is_union, is_tuple, + is_callable)) + + def _create_parametrized_type_hint(origin, args): + return origin[args] +else: + _is_parametrized_type_hint = None + _create_parametrized_type_hint = None + + +def parametrized_type_hint_getinitargs(obj): + # The distorted type check sematic for typing construct becomes: + # ``type(obj) is type(TypeHint)``, which means "obj is a + # parametrized TypeHint" + if type(obj) is type(Literal): # pragma: no branch + initargs = (Literal, obj.__values__) + elif type(obj) is type(Final): # pragma: no branch + initargs = (Final, obj.__type__) + elif type(obj) is type(ClassVar): + initargs = (ClassVar, obj.__type__) + elif type(obj) is type(Generic): + parameters = obj.__parameters__ + if len(obj.__parameters__) > 0: + # in early Python 3.5, __parameters__ was sometimes + # preferred to __args__ + initargs = (obj.__origin__, parameters) + + else: + initargs = (obj.__origin__, obj.__args__) + elif type(obj) is type(Union): + if sys.version_info < (3, 5, 3): # pragma: no cover + initargs = (Union, obj.__union_params__) + else: + initargs = (Union, obj.__args__) + elif type(obj) is type(Tuple): + if sys.version_info < (3, 5, 3): # pragma: no cover + initargs = (Tuple, obj.__tuple_params__) + else: + initargs = (Tuple, obj.__args__) + elif type(obj) is type(Callable): + if sys.version_info < (3, 5, 3): # pragma: no cover + args = obj.__args__ + result = obj.__result__ + if args != Ellipsis: + if isinstance(args, tuple): + args = list(args) + else: + args = [args] + else: + (*args, result) = obj.__args__ + if len(args) == 1 and args[0] is Ellipsis: + args = Ellipsis + else: + args = list(args) + initargs = (Callable, (args, result)) + else: # pragma: no cover + raise pickle.PicklingError( + "Cloudpickle Error: Unknown type {}".format(type(obj)) + ) + return initargs + + +# Tornado support + +def is_tornado_coroutine(func): + """ + Return whether *func* is a Tornado coroutine function. + Running coroutines are not supported. + """ + if 'tornado.gen' not in sys.modules: + return False + gen = sys.modules['tornado.gen'] + if not hasattr(gen, "is_coroutine_function"): + # Tornado version is too old + return False + return gen.is_coroutine_function(func) + + +def _rebuild_tornado_coroutine(func): + from tornado import gen + return gen.coroutine(func) + + +# including pickles unloading functions in this namespace +load = pickle.load +loads = pickle.loads + + +# hack for __import__ not working as desired +def subimport(name): + __import__(name) + return sys.modules[name] + + +def dynamic_subimport(name, vars): + mod = types.ModuleType(name) + mod.__dict__.update(vars) + mod.__dict__['__builtins__'] = builtins.__dict__ + return mod + + +def _gen_ellipsis(): + return Ellipsis + + +def _gen_not_implemented(): + return NotImplemented + + +def _get_cell_contents(cell): + try: + return cell.cell_contents + except ValueError: + # sentinel used by ``_fill_function`` which will leave the cell empty + return _empty_cell_value + + +def instance(cls): + """Create a new instance of a class. + + Parameters + ---------- + cls : type + The class to create an instance of. + + Returns + ------- + instance : cls + A new instance of ``cls``. + """ + return cls() + + +@instance +class _empty_cell_value(object): + """sentinel for empty closures + """ + @classmethod + def __reduce__(cls): + return cls.__name__ + + +def _fill_function(*args): + """Fills in the rest of function data into the skeleton function object + + The skeleton itself is create by _make_skel_func(). + """ + if len(args) == 2: + func = args[0] + state = args[1] + elif len(args) == 5: + # Backwards compat for cloudpickle v0.4.0, after which the `module` + # argument was introduced + func = args[0] + keys = ['globals', 'defaults', 'dict', 'closure_values'] + state = dict(zip(keys, args[1:])) + elif len(args) == 6: + # Backwards compat for cloudpickle v0.4.1, after which the function + # state was passed as a dict to the _fill_function it-self. + func = args[0] + keys = ['globals', 'defaults', 'dict', 'module', 'closure_values'] + state = dict(zip(keys, args[1:])) + else: + raise ValueError('Unexpected _fill_value arguments: %r' % (args,)) + + # - At pickling time, any dynamic global variable used by func is + # serialized by value (in state['globals']). + # - At unpickling time, func's __globals__ attribute is initialized by + # first retrieving an empty isolated namespace that will be shared + # with other functions pickled from the same original module + # by the same CloudPickler instance and then updated with the + # content of state['globals'] to populate the shared isolated + # namespace with all the global variables that are specifically + # referenced for this function. + func.__globals__.update(state['globals']) + + func.__defaults__ = state['defaults'] + func.__dict__ = state['dict'] + if 'annotations' in state: + func.__annotations__ = state['annotations'] + if 'doc' in state: + func.__doc__ = state['doc'] + if 'name' in state: + func.__name__ = state['name'] + if 'module' in state: + func.__module__ = state['module'] + if 'qualname' in state: + func.__qualname__ = state['qualname'] + if 'kwdefaults' in state: + func.__kwdefaults__ = state['kwdefaults'] + # _cloudpickle_subimports is a set of submodules that must be loaded for + # the pickled function to work correctly at unpickling time. Now that these + # submodules are depickled (hence imported), they can be removed from the + # object's state (the object state only served as a reference holder to + # these submodules) + if '_cloudpickle_submodules' in state: + state.pop('_cloudpickle_submodules') + + cells = func.__closure__ + if cells is not None: + for cell, value in zip(cells, state['closure_values']): + if value is not _empty_cell_value: + cell_set(cell, value) + + return func + + +def _make_empty_cell(): + if False: + # trick the compiler into creating an empty cell in our lambda + cell = None + raise AssertionError('this route should not be executed') + + return (lambda: cell).__closure__[0] + + +def _make_cell(value=_empty_cell_value): + cell = _make_empty_cell() + if value is not _empty_cell_value: + cell_set(cell, value) + return cell + + +def _make_skel_func(code, cell_count, base_globals=None): + """ Creates a skeleton function object that contains just the provided + code and the correct number of cells in func_closure. All other + func attributes (e.g. func_globals) are empty. + """ + # This function is deprecated and should be removed in cloudpickle 1.7 + warnings.warn( + "A pickle file created using an old (<=1.4.1) version of cloudpicke " + "is currently being loaded. This is not supported by cloudpickle and " + "will break in cloudpickle 1.7", category=UserWarning + ) + # This is backward-compatibility code: for cloudpickle versions between + # 0.5.4 and 0.7, base_globals could be a string or None. base_globals + # should now always be a dictionary. + if base_globals is None or isinstance(base_globals, str): + base_globals = {} + + base_globals['__builtins__'] = __builtins__ + + closure = ( + tuple(_make_empty_cell() for _ in range(cell_count)) + if cell_count >= 0 else + None + ) + return types.FunctionType(code, base_globals, None, None, closure) + + +def _make_skeleton_class(type_constructor, name, bases, type_kwargs, + class_tracker_id, extra): + """Build dynamic class with an empty __dict__ to be filled once memoized + + If class_tracker_id is not None, try to lookup an existing class definition + matching that id. If none is found, track a newly reconstructed class + definition under that id so that other instances stemming from the same + class id will also reuse this class definition. + + The "extra" variable is meant to be a dict (or None) that can be used for + forward compatibility shall the need arise. + """ + skeleton_class = types.new_class( + name, bases, {'metaclass': type_constructor}, + lambda ns: ns.update(type_kwargs) + ) + return _lookup_class_or_track(class_tracker_id, skeleton_class) + + +def _rehydrate_skeleton_class(skeleton_class, class_dict): + """Put attributes from `class_dict` back on `skeleton_class`. + + See CloudPickler.save_dynamic_class for more info. + """ + registry = None + for attrname, attr in class_dict.items(): + if attrname == "_abc_impl": + registry = attr + else: + setattr(skeleton_class, attrname, attr) + if registry is not None: + for subclass in registry: + skeleton_class.register(subclass) + + return skeleton_class + + +def _make_skeleton_enum(bases, name, qualname, members, module, + class_tracker_id, extra): + """Build dynamic enum with an empty __dict__ to be filled once memoized + + The creation of the enum class is inspired by the code of + EnumMeta._create_. + + If class_tracker_id is not None, try to lookup an existing enum definition + matching that id. If none is found, track a newly reconstructed enum + definition under that id so that other instances stemming from the same + class id will also reuse this enum definition. + + The "extra" variable is meant to be a dict (or None) that can be used for + forward compatibility shall the need arise. + """ + # enums always inherit from their base Enum class at the last position in + # the list of base classes: + enum_base = bases[-1] + metacls = enum_base.__class__ + classdict = metacls.__prepare__(name, bases) + + for member_name, member_value in members.items(): + classdict[member_name] = member_value + enum_class = metacls.__new__(metacls, name, bases, classdict) + enum_class.__module__ = module + enum_class.__qualname__ = qualname + + return _lookup_class_or_track(class_tracker_id, enum_class) + + +def _make_typevar(name, bound, constraints, covariant, contravariant, + class_tracker_id): + tv = typing.TypeVar( + name, *constraints, bound=bound, + covariant=covariant, contravariant=contravariant + ) + if class_tracker_id is not None: + return _lookup_class_or_track(class_tracker_id, tv) + else: # pragma: nocover + # Only for Python 3.5.3 compat. + return tv + + +def _decompose_typevar(obj): + try: + class_tracker_id = _get_or_create_tracker_id(obj) + except TypeError: # pragma: nocover + # TypeVar instances are not weakref-able in Python 3.5.3 + class_tracker_id = None + return ( + obj.__name__, obj.__bound__, obj.__constraints__, + obj.__covariant__, obj.__contravariant__, + class_tracker_id, + ) + + +def _typevar_reduce(obj): + # TypeVar instances have no __qualname__ hence we pass the name explicitly. + module_and_name = _lookup_module_and_qualname(obj, name=obj.__name__) + if module_and_name is None: + return (_make_typevar, _decompose_typevar(obj)) + return (getattr, module_and_name) + + +def _get_bases(typ): + if hasattr(typ, '__orig_bases__'): + # For generic types (see PEP 560) + bases_attr = '__orig_bases__' + else: + # For regular class objects + bases_attr = '__bases__' + return getattr(typ, bases_attr) diff --git a/python/pyspark/cloudpickle/cloudpickle_fast.py b/python/pyspark/cloudpickle/cloudpickle_fast.py new file mode 100644 index 0000000000000..e8e46b88fdc91 --- /dev/null +++ b/python/pyspark/cloudpickle/cloudpickle_fast.py @@ -0,0 +1,747 @@ +""" +New, fast version of the CloudPickler. + +This new CloudPickler class can now extend the fast C Pickler instead of the +previous Python implementation of the Pickler class. Because this functionality +is only available for Python versions 3.8+, a lot of backward-compatibility +code is also removed. + +Note that the C Pickler sublassing API is CPython-specific. Therefore, some +guards present in cloudpickle.py that were written to handle PyPy specificities +are not present in cloudpickle_fast.py +""" +import abc +import copyreg +import io +import itertools +import logging +import sys +import struct +import types +import weakref +import typing + +from enum import Enum +from collections import ChainMap + +from .compat import pickle, Pickler +from .cloudpickle import ( + _extract_code_globals, _BUILTIN_TYPE_NAMES, DEFAULT_PROTOCOL, + _find_imported_submodules, _get_cell_contents, _is_importable, + _builtin_type, _get_or_create_tracker_id, _make_skeleton_class, + _make_skeleton_enum, _extract_class_dict, dynamic_subimport, subimport, + _typevar_reduce, _get_bases, _make_cell, _make_empty_cell, CellType, + _is_parametrized_type_hint, PYPY, cell_set, + parametrized_type_hint_getinitargs, _create_parametrized_type_hint, + builtin_code_type + +) + + +if pickle.HIGHEST_PROTOCOL >= 5 and not PYPY: + # Shorthands similar to pickle.dump/pickle.dumps + + def dump(obj, file, protocol=None, buffer_callback=None): + """Serialize obj as bytes streamed into file + + protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to + pickle.HIGHEST_PROTOCOL. This setting favors maximum communication + speed between processes running the same Python version. + + Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure + compatibility with older versions of Python. + """ + CloudPickler( + file, protocol=protocol, buffer_callback=buffer_callback + ).dump(obj) + + def dumps(obj, protocol=None, buffer_callback=None): + """Serialize obj as a string of bytes allocated in memory + + protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to + pickle.HIGHEST_PROTOCOL. This setting favors maximum communication + speed between processes running the same Python version. + + Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure + compatibility with older versions of Python. + """ + with io.BytesIO() as file: + cp = CloudPickler( + file, protocol=protocol, buffer_callback=buffer_callback + ) + cp.dump(obj) + return file.getvalue() + +else: + # Shorthands similar to pickle.dump/pickle.dumps + def dump(obj, file, protocol=None): + """Serialize obj as bytes streamed into file + + protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to + pickle.HIGHEST_PROTOCOL. This setting favors maximum communication + speed between processes running the same Python version. + + Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure + compatibility with older versions of Python. + """ + CloudPickler(file, protocol=protocol).dump(obj) + + def dumps(obj, protocol=None): + """Serialize obj as a string of bytes allocated in memory + + protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to + pickle.HIGHEST_PROTOCOL. This setting favors maximum communication + speed between processes running the same Python version. + + Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure + compatibility with older versions of Python. + """ + with io.BytesIO() as file: + cp = CloudPickler(file, protocol=protocol) + cp.dump(obj) + return file.getvalue() + + +load, loads = pickle.load, pickle.loads + + +# COLLECTION OF OBJECTS __getnewargs__-LIKE METHODS +# ------------------------------------------------- + +def _class_getnewargs(obj): + type_kwargs = {} + if "__slots__" in obj.__dict__: + type_kwargs["__slots__"] = obj.__slots__ + + __dict__ = obj.__dict__.get('__dict__', None) + if isinstance(__dict__, property): + type_kwargs['__dict__'] = __dict__ + + return (type(obj), obj.__name__, _get_bases(obj), type_kwargs, + _get_or_create_tracker_id(obj), None) + + +def _enum_getnewargs(obj): + members = dict((e.name, e.value) for e in obj) + return (obj.__bases__, obj.__name__, obj.__qualname__, members, + obj.__module__, _get_or_create_tracker_id(obj), None) + + +# COLLECTION OF OBJECTS RECONSTRUCTORS +# ------------------------------------ +def _file_reconstructor(retval): + return retval + + +# COLLECTION OF OBJECTS STATE GETTERS +# ----------------------------------- +def _function_getstate(func): + # - Put func's dynamic attributes (stored in func.__dict__) in state. These + # attributes will be restored at unpickling time using + # f.__dict__.update(state) + # - Put func's members into slotstate. Such attributes will be restored at + # unpickling time by iterating over slotstate and calling setattr(func, + # slotname, slotvalue) + slotstate = { + "__name__": func.__name__, + "__qualname__": func.__qualname__, + "__annotations__": func.__annotations__, + "__kwdefaults__": func.__kwdefaults__, + "__defaults__": func.__defaults__, + "__module__": func.__module__, + "__doc__": func.__doc__, + "__closure__": func.__closure__, + } + + f_globals_ref = _extract_code_globals(func.__code__) + f_globals = {k: func.__globals__[k] for k in f_globals_ref if k in + func.__globals__} + + closure_values = ( + list(map(_get_cell_contents, func.__closure__)) + if func.__closure__ is not None else () + ) + + # Extract currently-imported submodules used by func. Storing these modules + # in a smoke _cloudpickle_subimports attribute of the object's state will + # trigger the side effect of importing these modules at unpickling time + # (which is necessary for func to work correctly once depickled) + slotstate["_cloudpickle_submodules"] = _find_imported_submodules( + func.__code__, itertools.chain(f_globals.values(), closure_values)) + slotstate["__globals__"] = f_globals + + state = func.__dict__ + return state, slotstate + + +def _class_getstate(obj): + clsdict = _extract_class_dict(obj) + clsdict.pop('__weakref__', None) + + if issubclass(type(obj), abc.ABCMeta): + # If obj is an instance of an ABCMeta subclass, dont pickle the + # cache/negative caches populated during isinstance/issubclass + # checks, but pickle the list of registered subclasses of obj. + clsdict.pop('_abc_cache', None) + clsdict.pop('_abc_negative_cache', None) + clsdict.pop('_abc_negative_cache_version', None) + registry = clsdict.pop('_abc_registry', None) + if registry is None: + # in Python3.7+, the abc caches and registered subclasses of a + # class are bundled into the single _abc_impl attribute + clsdict.pop('_abc_impl', None) + (registry, _, _, _) = abc._get_dump(obj) + + clsdict["_abc_impl"] = [subclass_weakref() + for subclass_weakref in registry] + else: + # In the above if clause, registry is a set of weakrefs -- in + # this case, registry is a WeakSet + clsdict["_abc_impl"] = [type_ for type_ in registry] + + if "__slots__" in clsdict: + # pickle string length optimization: member descriptors of obj are + # created automatically from obj's __slots__ attribute, no need to + # save them in obj's state + if isinstance(obj.__slots__, str): + clsdict.pop(obj.__slots__) + else: + for k in obj.__slots__: + clsdict.pop(k, None) + + clsdict.pop('__dict__', None) # unpicklable property object + + return (clsdict, {}) + + +def _enum_getstate(obj): + clsdict, slotstate = _class_getstate(obj) + + members = dict((e.name, e.value) for e in obj) + # Cleanup the clsdict that will be passed to _rehydrate_skeleton_class: + # Those attributes are already handled by the metaclass. + for attrname in ["_generate_next_value_", "_member_names_", + "_member_map_", "_member_type_", + "_value2member_map_"]: + clsdict.pop(attrname, None) + for member in members: + clsdict.pop(member) + # Special handling of Enum subclasses + return clsdict, slotstate + + +# COLLECTIONS OF OBJECTS REDUCERS +# ------------------------------- +# A reducer is a function taking a single argument (obj), and that returns a +# tuple with all the necessary data to re-construct obj. Apart from a few +# exceptions (list, dict, bytes, int, etc.), a reducer is necessary to +# correctly pickle an object. +# While many built-in objects (Exceptions objects, instances of the "object" +# class, etc), are shipped with their own built-in reducer (invoked using +# obj.__reduce__), some do not. The following methods were created to "fill +# these holes". + +def _code_reduce(obj): + """codeobject reducer""" + if hasattr(obj, "co_posonlyargcount"): # pragma: no branch + args = ( + obj.co_argcount, obj.co_posonlyargcount, + obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, + obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, + obj.co_varnames, obj.co_filename, obj.co_name, + obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, + obj.co_cellvars + ) + else: + args = ( + obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, + obj.co_stacksize, obj.co_flags, obj.co_code, obj.co_consts, + obj.co_names, obj.co_varnames, obj.co_filename, + obj.co_name, obj.co_firstlineno, obj.co_lnotab, + obj.co_freevars, obj.co_cellvars + ) + return types.CodeType, args + + +def _cell_reduce(obj): + """Cell (containing values of a function's free variables) reducer""" + try: + obj.cell_contents + except ValueError: # cell is empty + return _make_empty_cell, () + else: + return _make_cell, (obj.cell_contents, ) + + +def _classmethod_reduce(obj): + orig_func = obj.__func__ + return type(obj), (orig_func,) + + +def _file_reduce(obj): + """Save a file""" + import io + + if not hasattr(obj, "name") or not hasattr(obj, "mode"): + raise pickle.PicklingError( + "Cannot pickle files that do not map to an actual file" + ) + if obj is sys.stdout: + return getattr, (sys, "stdout") + if obj is sys.stderr: + return getattr, (sys, "stderr") + if obj is sys.stdin: + raise pickle.PicklingError("Cannot pickle standard input") + if obj.closed: + raise pickle.PicklingError("Cannot pickle closed files") + if hasattr(obj, "isatty") and obj.isatty(): + raise pickle.PicklingError( + "Cannot pickle files that map to tty objects" + ) + if "r" not in obj.mode and "+" not in obj.mode: + raise pickle.PicklingError( + "Cannot pickle files that are not opened for reading: %s" + % obj.mode + ) + + name = obj.name + + retval = io.StringIO() + + try: + # Read the whole file + curloc = obj.tell() + obj.seek(0) + contents = obj.read() + obj.seek(curloc) + except IOError as e: + raise pickle.PicklingError( + "Cannot pickle file %s as it cannot be read" % name + ) from e + retval.write(contents) + retval.seek(curloc) + + retval.name = name + return _file_reconstructor, (retval,) + + +def _getset_descriptor_reduce(obj): + return getattr, (obj.__objclass__, obj.__name__) + + +def _mappingproxy_reduce(obj): + return types.MappingProxyType, (dict(obj),) + + +def _memoryview_reduce(obj): + return bytes, (obj.tobytes(),) + + +def _module_reduce(obj): + if _is_importable(obj): + return subimport, (obj.__name__,) + else: + obj.__dict__.pop('__builtins__', None) + return dynamic_subimport, (obj.__name__, vars(obj)) + + +def _method_reduce(obj): + return (types.MethodType, (obj.__func__, obj.__self__)) + + +def _logger_reduce(obj): + return logging.getLogger, (obj.name,) + + +def _root_logger_reduce(obj): + return logging.getLogger, () + + +def _property_reduce(obj): + return property, (obj.fget, obj.fset, obj.fdel, obj.__doc__) + + +def _weakset_reduce(obj): + return weakref.WeakSet, (list(obj),) + + +def _dynamic_class_reduce(obj): + """ + Save a class that can't be stored as module global. + + This method is used to serialize classes that are defined inside + functions, or that otherwise can't be serialized as attribute lookups + from global modules. + """ + if Enum is not None and issubclass(obj, Enum): + return ( + _make_skeleton_enum, _enum_getnewargs(obj), _enum_getstate(obj), + None, None, _class_setstate + ) + else: + return ( + _make_skeleton_class, _class_getnewargs(obj), _class_getstate(obj), + None, None, _class_setstate + ) + + +def _class_reduce(obj): + """Select the reducer depending on the dynamic nature of the class obj""" + if obj is type(None): # noqa + return type, (None,) + elif obj is type(Ellipsis): + return type, (Ellipsis,) + elif obj is type(NotImplemented): + return type, (NotImplemented,) + elif obj in _BUILTIN_TYPE_NAMES: + return _builtin_type, (_BUILTIN_TYPE_NAMES[obj],) + elif not _is_importable(obj): + return _dynamic_class_reduce(obj) + return NotImplemented + + +# COLLECTIONS OF OBJECTS STATE SETTERS +# ------------------------------------ +# state setters are called at unpickling time, once the object is created and +# it has to be updated to how it was at unpickling time. + + +def _function_setstate(obj, state): + """Update the state of a dynaamic function. + + As __closure__ and __globals__ are readonly attributes of a function, we + cannot rely on the native setstate routine of pickle.load_build, that calls + setattr on items of the slotstate. Instead, we have to modify them inplace. + """ + state, slotstate = state + obj.__dict__.update(state) + + obj_globals = slotstate.pop("__globals__") + obj_closure = slotstate.pop("__closure__") + # _cloudpickle_subimports is a set of submodules that must be loaded for + # the pickled function to work correctly at unpickling time. Now that these + # submodules are depickled (hence imported), they can be removed from the + # object's state (the object state only served as a reference holder to + # these submodules) + slotstate.pop("_cloudpickle_submodules") + + obj.__globals__.update(obj_globals) + obj.__globals__["__builtins__"] = __builtins__ + + if obj_closure is not None: + for i, cell in enumerate(obj_closure): + try: + value = cell.cell_contents + except ValueError: # cell is empty + continue + cell_set(obj.__closure__[i], value) + + for k, v in slotstate.items(): + setattr(obj, k, v) + + +def _class_setstate(obj, state): + state, slotstate = state + registry = None + for attrname, attr in state.items(): + if attrname == "_abc_impl": + registry = attr + else: + setattr(obj, attrname, attr) + if registry is not None: + for subclass in registry: + obj.register(subclass) + + return obj + + +class CloudPickler(Pickler): + # set of reducers defined and used by cloudpickle (private) + _dispatch_table = {} + _dispatch_table[classmethod] = _classmethod_reduce + _dispatch_table[io.TextIOWrapper] = _file_reduce + _dispatch_table[logging.Logger] = _logger_reduce + _dispatch_table[logging.RootLogger] = _root_logger_reduce + _dispatch_table[memoryview] = _memoryview_reduce + _dispatch_table[property] = _property_reduce + _dispatch_table[staticmethod] = _classmethod_reduce + _dispatch_table[CellType] = _cell_reduce + _dispatch_table[types.CodeType] = _code_reduce + _dispatch_table[types.GetSetDescriptorType] = _getset_descriptor_reduce + _dispatch_table[types.ModuleType] = _module_reduce + _dispatch_table[types.MethodType] = _method_reduce + _dispatch_table[types.MappingProxyType] = _mappingproxy_reduce + _dispatch_table[weakref.WeakSet] = _weakset_reduce + _dispatch_table[typing.TypeVar] = _typevar_reduce + + dispatch_table = ChainMap(_dispatch_table, copyreg.dispatch_table) + + # function reducers are defined as instance methods of CloudPickler + # objects, as they rely on a CloudPickler attribute (globals_ref) + def _dynamic_function_reduce(self, func): + """Reduce a function that is not pickleable via attribute lookup.""" + newargs = self._function_getnewargs(func) + state = _function_getstate(func) + return (types.FunctionType, newargs, state, None, None, + _function_setstate) + + def _function_reduce(self, obj): + """Reducer for function objects. + + If obj is a top-level attribute of a file-backed module, this + reducer returns NotImplemented, making the CloudPickler fallback to + traditional _pickle.Pickler routines to save obj. Otherwise, it reduces + obj using a custom cloudpickle reducer designed specifically to handle + dynamic functions. + + As opposed to cloudpickle.py, There no special handling for builtin + pypy functions because cloudpickle_fast is CPython-specific. + """ + if _is_importable(obj): + return NotImplemented + else: + return self._dynamic_function_reduce(obj) + + def _function_getnewargs(self, func): + code = func.__code__ + + # base_globals represents the future global namespace of func at + # unpickling time. Looking it up and storing it in + # CloudpiPickler.globals_ref allow functions sharing the same globals + # at pickling time to also share them once unpickled, at one condition: + # since globals_ref is an attribute of a CloudPickler instance, and + # that a new CloudPickler is created each time pickle.dump or + # pickle.dumps is called, functions also need to be saved within the + # same invocation of cloudpickle.dump/cloudpickle.dumps (for example: + # cloudpickle.dumps([f1, f2])). There is no such limitation when using + # CloudPickler.dump, as long as the multiple invocations are bound to + # the same CloudPickler. + base_globals = self.globals_ref.setdefault(id(func.__globals__), {}) + + if base_globals == {}: + # Add module attributes used to resolve relative imports + # instructions inside func. + for k in ["__package__", "__name__", "__path__", "__file__"]: + if k in func.__globals__: + base_globals[k] = func.__globals__[k] + + # Do not bind the free variables before the function is created to + # avoid infinite recursion. + if func.__closure__ is None: + closure = None + else: + closure = tuple( + _make_empty_cell() for _ in range(len(code.co_freevars))) + + return code, base_globals, None, None, closure + + def dump(self, obj): + try: + return Pickler.dump(self, obj) + except RuntimeError as e: + if "recursion" in e.args[0]: + msg = ( + "Could not pickle object as excessively deep recursion " + "required." + ) + raise pickle.PicklingError(msg) from e + else: + raise + + if pickle.HIGHEST_PROTOCOL >= 5: + # `CloudPickler.dispatch` is only left for backward compatibility - note + # that when using protocol 5, `CloudPickler.dispatch` is not an + # extension of `Pickler.dispatch` dictionary, because CloudPickler + # subclasses the C-implemented Pickler, which does not expose a + # `dispatch` attribute. Earlier versions of the protocol 5 CloudPickler + # used `CloudPickler.dispatch` as a class-level attribute storing all + # reducers implemented by cloudpickle, but the attribute name was not a + # great choice given the meaning of `Cloudpickler.dispatch` when + # `CloudPickler` extends the pure-python pickler. + dispatch = dispatch_table + + # Implementation of the reducer_override callback, in order to + # efficiently serialize dynamic functions and classes by subclassing + # the C-implemented Pickler. + # TODO: decorrelate reducer_override (which is tied to CPython's + # implementation - would it make sense to backport it to pypy? - and + # pickle's protocol 5 which is implementation agnostic. Currently, the + # availability of both notions coincide on CPython's pickle and the + # pickle5 backport, but it may not be the case anymore when pypy + # implements protocol 5 + def __init__(self, file, protocol=None, buffer_callback=None): + if protocol is None: + protocol = DEFAULT_PROTOCOL + Pickler.__init__( + self, file, protocol=protocol, buffer_callback=buffer_callback + ) + # map functions __globals__ attribute ids, to ensure that functions + # sharing the same global namespace at pickling time also share + # their global namespace at unpickling time. + self.globals_ref = {} + self.proto = int(protocol) + + def reducer_override(self, obj): + """Type-agnostic reducing callback for function and classes. + + For performance reasons, subclasses of the C _pickle.Pickler class + cannot register custom reducers for functions and classes in the + dispatch_table. Reducer for such types must instead implemented in + the special reducer_override method. + + Note that method will be called for any object except a few + builtin-types (int, lists, dicts etc.), which differs from reducers + in the Pickler's dispatch_table, each of them being invoked for + objects of a specific type only. + + This property comes in handy for classes: although most classes are + instances of the ``type`` metaclass, some of them can be instances + of other custom metaclasses (such as enum.EnumMeta for example). In + particular, the metaclass will likely not be known in advance, and + thus cannot be special-cased using an entry in the dispatch_table. + reducer_override, among other things, allows us to register a + reducer that will be called for any class, independently of its + type. + + + Notes: + + * reducer_override has the priority over dispatch_table-registered + reducers. + * reducer_override can be used to fix other limitations of + cloudpickle for other types that suffered from type-specific + reducers, such as Exceptions. See + https://github.com/cloudpipe/cloudpickle/issues/248 + """ + if sys.version_info[:2] < (3, 7) and _is_parametrized_type_hint(obj): # noqa # pragma: no branch + return ( + _create_parametrized_type_hint, + parametrized_type_hint_getinitargs(obj) + ) + t = type(obj) + try: + is_anyclass = issubclass(t, type) + except TypeError: # t is not a class (old Boost; see SF #502085) + is_anyclass = False + + if is_anyclass: + return _class_reduce(obj) + elif isinstance(obj, types.FunctionType): + return self._function_reduce(obj) + else: + # fallback to save_global, including the Pickler's + # distpatch_table + return NotImplemented + + else: + # When reducer_override is not available, hack the pure-Python + # Pickler's types.FunctionType and type savers. Note: the type saver + # must override Pickler.save_global, because pickle.py contains a + # hard-coded call to save_global when pickling meta-classes. + dispatch = Pickler.dispatch.copy() + + def __init__(self, file, protocol=None): + if protocol is None: + protocol = DEFAULT_PROTOCOL + Pickler.__init__(self, file, protocol=protocol) + # map functions __globals__ attribute ids, to ensure that functions + # sharing the same global namespace at pickling time also share + # their global namespace at unpickling time. + self.globals_ref = {} + assert hasattr(self, 'proto') + + def _save_reduce_pickle5(self, func, args, state=None, listitems=None, + dictitems=None, state_setter=None, obj=None): + save = self.save + write = self.write + self.save_reduce( + func, args, state=None, listitems=listitems, + dictitems=dictitems, obj=obj + ) + # backport of the Python 3.8 state_setter pickle operations + save(state_setter) + save(obj) # simple BINGET opcode as obj is already memoized. + save(state) + write(pickle.TUPLE2) + # Trigger a state_setter(obj, state) function call. + write(pickle.REDUCE) + # The purpose of state_setter is to carry-out an + # inplace modification of obj. We do not care about what the + # method might return, so its output is eventually removed from + # the stack. + write(pickle.POP) + + def save_global(self, obj, name=None, pack=struct.pack): + """ + Save a "global". + + The name of this method is somewhat misleading: all types get + dispatched here. + """ + if obj is type(None): # noqa + return self.save_reduce(type, (None,), obj=obj) + elif obj is type(Ellipsis): + return self.save_reduce(type, (Ellipsis,), obj=obj) + elif obj is type(NotImplemented): + return self.save_reduce(type, (NotImplemented,), obj=obj) + elif obj in _BUILTIN_TYPE_NAMES: + return self.save_reduce( + _builtin_type, (_BUILTIN_TYPE_NAMES[obj],), obj=obj) + + if sys.version_info[:2] < (3, 7) and _is_parametrized_type_hint(obj): # noqa # pragma: no branch + # Parametrized typing constructs in Python < 3.7 are not + # compatible with type checks and ``isinstance`` semantics. For + # this reason, it is easier to detect them using a + # duck-typing-based check (``_is_parametrized_type_hint``) than + # to populate the Pickler's dispatch with type-specific savers. + self.save_reduce( + _create_parametrized_type_hint, + parametrized_type_hint_getinitargs(obj), + obj=obj + ) + elif name is not None: + Pickler.save_global(self, obj, name=name) + elif not _is_importable(obj, name=name): + self._save_reduce_pickle5(*_dynamic_class_reduce(obj), obj=obj) + else: + Pickler.save_global(self, obj, name=name) + dispatch[type] = save_global + + def save_function(self, obj, name=None): + """ Registered with the dispatch to handle all function types. + + Determines what kind of function obj is (e.g. lambda, defined at + interactive prompt, etc) and handles the pickling appropriately. + """ + if _is_importable(obj, name=name): + return Pickler.save_global(self, obj, name=name) + elif PYPY and isinstance(obj.__code__, builtin_code_type): + return self.save_pypy_builtin_func(obj) + else: + return self._save_reduce_pickle5( + *self._dynamic_function_reduce(obj), obj=obj + ) + + def save_pypy_builtin_func(self, obj): + """Save pypy equivalent of builtin functions. + PyPy does not have the concept of builtin-functions. Instead, + builtin-functions are simple function instances, but with a + builtin-code attribute. + Most of the time, builtin functions should be pickled by attribute. + But PyPy has flaky support for __qualname__, so some builtin + functions such as float.__new__ will be classified as dynamic. For + this reason only, we created this special routine. Because + builtin-functions are not expected to have closure or globals, + there is no additional hack (compared the one already implemented + in pickle) to protect ourselves from reference cycles. A simple + (reconstructor, newargs, obj.__dict__) tuple is save_reduced. Note + also that PyPy improved their support for __qualname__ in v3.6, so + this routing should be removed when cloudpickle supports only PyPy + 3.6 and later. + """ + rv = (types.FunctionType, (obj.__code__, {}, obj.__name__, + obj.__defaults__, obj.__closure__), + obj.__dict__) + self.save_reduce(*rv, obj=obj) + + dispatch[types.FunctionType] = save_function diff --git a/python/pyspark/cloudpickle/compat.py b/python/pyspark/cloudpickle/compat.py new file mode 100644 index 0000000000000..afa285f62903d --- /dev/null +++ b/python/pyspark/cloudpickle/compat.py @@ -0,0 +1,13 @@ +import sys + + +if sys.version_info < (3, 8): + try: + import pickle5 as pickle # noqa: F401 + from pickle5 import Pickler # noqa: F401 + except ImportError: + import pickle # noqa: F401 + from pickle import _Pickler as Pickler # noqa: F401 +else: + import pickle # noqa: F401 + from _pickle import Pickler # noqa: F401 diff --git a/python/setup.py b/python/setup.py index c456a32fea87c..294a34c7a1916 100755 --- a/python/setup.py +++ b/python/setup.py @@ -161,6 +161,7 @@ def _supports_symlinks(): author_email='dev@spark.apache.org', url='https://github.com/apache/spark/tree/master/python', packages=['pyspark', + 'pyspark.cloudpickle', 'pyspark.mllib', 'pyspark.mllib.linalg', 'pyspark.mllib.stat', From efa70b87555840932c5ac22ce3a63e2c461a0081 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 17 Jul 2020 04:50:26 +0000 Subject: [PATCH 214/384] [SPARK-32145][SQL][FOLLOWUP] Fix type in the error log of SparkOperation ### What changes were proposed in this pull request? Fix typo error in the error log of SparkOperation trait, reported by https://github.com/apache/spark/pull/28963#discussion_r454954542 ### Why are the changes needed? fix error in thrift server driver log ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Passing GitHub actions Closes #29140 from yaooqinn/SPARK-32145-F. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/hive/thriftserver/SparkOperation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index 446669d08e76b..18d2950a02745 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -96,7 +96,7 @@ private[hive] trait SparkOperation extends Operation with Logging { protected def onError(): PartialFunction[Throwable, Unit] = { case e: Throwable => - logError(s"Error executing get catalogs operation with $statementId", e) + logError(s"Error operating $getType with $statementId", e) super.setState(OperationState.ERROR) HiveThriftServer2.eventManager.onStatementError( statementId, e.getMessage, Utils.exceptionString(e)) From ffdbbae1d465fe2c710d020de62ca1a6b0b924d9 Mon Sep 17 00:00:00 2001 From: Devesh Agrawal Date: Fri, 17 Jul 2020 06:04:34 +0000 Subject: [PATCH 215/384] [SPARK-32215] Expose a (protected) /workers/kill endpoint on the MasterWebUI ### What changes were proposed in this pull request? This PR allows an external agent to inform the Master that certain hosts are being decommissioned. ### Why are the changes needed? The current decommissioning is triggered by the Worker getting getting a SIGPWR (out of band possibly by some cleanup hook), which then informs the Master about it. This approach may not be feasible in some environments that cannot trigger a clean up hook on the Worker. In addition, when a large number of worker nodes are being decommissioned then the master will get a flood of messages. So we add a new post endpoint `/workers/kill` on the MasterWebUI that allows an external agent to inform the master about all the nodes being decommissioned in bulk. The list of nodes is specified by providing a list of hostnames. All workers on those hosts will be decommissioned. This API is merely a new entry point into the existing decommissioning logic. It does not change how the decommissioning request is handled in its core. ### Does this PR introduce _any_ user-facing change? Yes, a new endpoint `/workers/kill` is added to the MasterWebUI. By default only requests originating from an IP address local to the MasterWebUI are allowed. ### How was this patch tested? Added unit tests Closes #29015 from agrawaldevesh/master_decom_endpoint. Authored-by: Devesh Agrawal Signed-off-by: Wenchen Fan --- .../apache/spark/deploy/DeployMessage.scala | 7 +++ .../apache/spark/deploy/master/Master.scala | 37 +++++++++++ .../spark/deploy/master/ui/MasterWebUI.scala | 48 ++++++++++++++- .../org/apache/spark/internal/config/UI.scala | 12 ++++ .../spark/deploy/master/MasterSuite.scala | 61 ++++++++++++++++++- .../deploy/master/ui/MasterWebUISuite.scala | 30 ++++++++- 6 files changed, 190 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 18305ad3746a6..c8c6e5a192a24 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -108,6 +108,13 @@ private[deploy] object DeployMessages { case class Heartbeat(workerId: String, worker: RpcEndpointRef) extends DeployMessage + /** + * Used by the MasterWebUI to request the master to decommission all workers that are active on + * any of the given hostnames. + * @param hostnames: A list of hostnames without the ports. Like "localhost", "foo.bar.com" etc + */ + case class DecommissionWorkersOnHosts(hostnames: Seq[String]) + // Master to Worker sealed trait RegisterWorkerResponse diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index d2e65db970380..0070df1d66dee 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -22,7 +22,9 @@ import java.util.{Date, Locale} import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable import scala.util.Random +import scala.util.control.NonFatal import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} @@ -525,6 +527,13 @@ private[deploy] class Master( case KillExecutors(appId, executorIds) => val formattedExecutorIds = formatExecutorIds(executorIds) context.reply(handleKillExecutors(appId, formattedExecutorIds)) + + case DecommissionWorkersOnHosts(hostnames) => + if (state != RecoveryState.STANDBY) { + context.reply(decommissionWorkersOnHosts(hostnames)) + } else { + context.reply(0) + } } override def onDisconnected(address: RpcAddress): Unit = { @@ -863,6 +872,34 @@ private[deploy] class Master( true } + /** + * Decommission all workers that are active on any of the given hostnames. The decommissioning is + * asynchronously done by enqueueing WorkerDecommission messages to self. No checks are done about + * the prior state of the worker. So an already decommissioned worker will match as well. + * + * @param hostnames: A list of hostnames without the ports. Like "localhost", "foo.bar.com" etc + * + * Returns the number of workers that matched the hostnames. + */ + private def decommissionWorkersOnHosts(hostnames: Seq[String]): Integer = { + val hostnamesSet = hostnames.map(_.toLowerCase(Locale.ROOT)).toSet + val workersToRemove = addressToWorker + .filterKeys(addr => hostnamesSet.contains(addr.host.toLowerCase(Locale.ROOT))) + .values + + val workersToRemoveHostPorts = workersToRemove.map(_.hostPort) + logInfo(s"Decommissioning the workers with host:ports ${workersToRemoveHostPorts}") + + // The workers are removed async to avoid blocking the receive loop for the entire batch + workersToRemove.foreach(wi => { + logInfo(s"Sending the worker decommission to ${wi.id} and ${wi.endpoint}") + self.send(WorkerDecommission(wi.id, wi.endpoint)) + }) + + // Return the count of workers actually removed + workersToRemove.size + } + private def decommissionWorker(worker: WorkerInfo): Unit = { if (worker.state != WorkerState.DECOMMISSIONED) { logInfo("Decommissioning worker %s on %s:%d".format(worker.id, worker.host, worker.port)) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 86554ec4ec1c9..035f9d379471c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,9 +17,14 @@ package org.apache.spark.deploy.master.ui -import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import java.net.{InetAddress, NetworkInterface, SocketException} +import java.util.Locale +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} + +import org.apache.spark.deploy.DeployMessages.{DecommissionWorkersOnHosts, MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.MASTER_UI_DECOMMISSION_ALLOW_MODE import org.apache.spark.internal.config.UI.UI_KILL_ENABLED import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -36,6 +41,7 @@ class MasterWebUI( val masterEndpointRef = master.self val killEnabled = master.conf.get(UI_KILL_ENABLED) + val decommissionAllowMode = master.conf.get(MASTER_UI_DECOMMISSION_ALLOW_MODE) initialize() @@ -49,6 +55,27 @@ class MasterWebUI( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST"))) + attachHandler(createServletHandler("/workers/kill", new HttpServlet { + override def doPost(req: HttpServletRequest, resp: HttpServletResponse): Unit = { + val hostnames: Seq[String] = Option(req.getParameterValues("host")) + .getOrElse(Array[String]()).toSeq + if (!isDecommissioningRequestAllowed(req)) { + resp.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } else { + val removedWorkers = masterEndpointRef.askSync[Integer]( + DecommissionWorkersOnHosts(hostnames)) + logInfo(s"Decommissioning of hosts $hostnames decommissioned $removedWorkers workers") + if (removedWorkers > 0) { + resp.setStatus(HttpServletResponse.SC_OK) + } else if (removedWorkers == 0) { + resp.sendError(HttpServletResponse.SC_NOT_FOUND) + } else { + // We shouldn't even see this case. + resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + } + } + } + }, "")) } def addProxy(): Unit = { @@ -64,6 +91,25 @@ class MasterWebUI( maybeWorkerUiAddress.orElse(maybeAppUiAddress) } + private def isLocal(address: InetAddress): Boolean = { + if (address.isAnyLocalAddress || address.isLoopbackAddress) { + return true + } + try { + NetworkInterface.getByInetAddress(address) != null + } catch { + case _: SocketException => false + } + } + + private def isDecommissioningRequestAllowed(req: HttpServletRequest): Boolean = { + decommissionAllowMode match { + case "ALLOW" => true + case "LOCAL" => isLocal(InetAddress.getByName(req.getRemoteAddr)) + case _ => false + } + } + } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/internal/config/UI.scala b/core/src/main/scala/org/apache/spark/internal/config/UI.scala index 231eecf086bbe..fcbe2b9775841 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/UI.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/UI.scala @@ -17,6 +17,7 @@ package org.apache.spark.internal.config +import java.util.Locale import java.util.concurrent.TimeUnit import org.apache.spark.network.util.ByteUnit @@ -191,4 +192,15 @@ private[spark] object UI { .version("3.0.0") .stringConf .createOptional + + val MASTER_UI_DECOMMISSION_ALLOW_MODE = ConfigBuilder("spark.master.ui.decommission.allow.mode") + .doc("Specifies the behavior of the Master Web UI's /workers/kill endpoint. Possible choices" + + " are: `LOCAL` means allow this endpoint from IP's that are local to the machine running" + + " the Master, `DENY` means to completely disable this endpoint, `ALLOW` means to allow" + + " calling this endpoint from any IP.") + .internal() + .version("3.1.0") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .createWithDefault("LOCAL") } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 91128af82b022..d98a6b29be9e8 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.duration._ import scala.io.Source import scala.reflect.ClassTag @@ -726,6 +726,65 @@ class MasterSuite extends SparkFunSuite } } + def testWorkerDecommissioning( + numWorkers: Int, + numWorkersExpectedToDecom: Int, + hostnames: Seq[String]): Unit = { + val conf = new SparkConf() + val master = makeAliveMaster(conf) + val workerRegs = (1 to numWorkers).map{idx => + val worker = new MockWorker(master.self, conf) + worker.rpcEnv.setupEndpoint("worker", worker) + val workerReg = RegisterWorker( + worker.id, + "localhost", + worker.self.address.port, + worker.self, + 10, + 1024, + "http://localhost:8080", + RpcAddress("localhost", 10000)) + master.self.send(workerReg) + workerReg + } + + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.workers.length === numWorkers) + assert(masterState.workers.forall(_.state == WorkerState.ALIVE)) + assert(masterState.workers.map(_.id).toSet == workerRegs.map(_.id).toSet) + } + + val decomWorkersCount = master.self.askSync[Integer](DecommissionWorkersOnHosts(hostnames)) + assert(decomWorkersCount === numWorkersExpectedToDecom) + + // Decommissioning is actually async ... wait for the workers to actually be decommissioned by + // polling the master's state. + eventually(timeout(30.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.workers.length === numWorkers) + val workersActuallyDecomed = masterState.workers.count(_.state == WorkerState.DECOMMISSIONED) + assert(workersActuallyDecomed === numWorkersExpectedToDecom) + } + + // Decommissioning a worker again should return the same answer since we want this call to be + // idempotent. + val decomWorkersCountAgain = master.self.askSync[Integer](DecommissionWorkersOnHosts(hostnames)) + assert(decomWorkersCountAgain === numWorkersExpectedToDecom) + } + + test("All workers on a host should be decommissioned") { + testWorkerDecommissioning(2, 2, Seq("LoCalHost", "localHOST")) + } + + test("No workers should be decommissioned with invalid host") { + testWorkerDecommissioning(2, 0, Seq("NoSuchHost1", "NoSuchHost2")) + } + + test("Only worker on host should be decommissioned") { + testWorkerDecommissioning(1, 1, Seq("lOcalHost", "NoSuchHost")) + } + test("SPARK-19900: there should be a corresponding driver for the app after relaunching driver") { val conf = new SparkConf().set(WORKER_TIMEOUT, 1L) val master = makeAliveMaster(conf) diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala index e2d7facdd77e0..35de457ec48ce 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -21,6 +21,7 @@ import java.io.DataOutputStream import java.net.{HttpURLConnection, URL} import java.nio.charset.StandardCharsets import java.util.Date +import javax.servlet.http.HttpServletResponse import scala.collection.mutable.HashMap @@ -28,15 +29,16 @@ import org.mockito.Mockito.{mock, times, verify, when} import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, RequestKillDriver} +import org.apache.spark.deploy.DeployMessages.{DecommissionWorkersOnHosts, KillDriverResponse, RequestKillDriver} import org.apache.spark.deploy.DeployTestUtils._ import org.apache.spark.deploy.master._ +import org.apache.spark.internal.config.UI import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { - val conf = new SparkConf + val conf = new SparkConf() val securityMgr = new SecurityManager(conf) val rpcEnv = mock(classOf[RpcEnv]) val master = mock(classOf[Master]) @@ -88,10 +90,32 @@ class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { verify(masterEndpointRef, times(1)).ask[KillDriverResponse](RequestKillDriver(activeDriverId)) } - private def convPostDataToString(data: Map[String, String]): String = { + private def testKillWorkers(hostnames: Seq[String]): Unit = { + val url = s"http://localhost:${masterWebUI.boundPort}/workers/kill/" + val body = convPostDataToString(hostnames.map(("host", _))) + val conn = sendHttpRequest(url, "POST", body) + // The master is mocked here, so cannot assert on the response code + conn.getResponseCode + // Verify that master was asked to kill driver with the correct id + verify(masterEndpointRef).askSync[Integer](DecommissionWorkersOnHosts(hostnames)) + } + + test("Kill one host") { + testKillWorkers(Seq("localhost")) + } + + test("Kill multiple hosts") { + testKillWorkers(Seq("noSuchHost", "LocalHost")) + } + + private def convPostDataToString(data: Seq[(String, String)]): String = { (for ((name, value) <- data) yield s"$name=$value").mkString("&") } + private def convPostDataToString(data: Map[String, String]): String = { + convPostDataToString(data.toSeq) + } + /** * Send an HTTP request to the given URL using the method and the body specified. * Return the connection object. From 34baed8139bf2d1b0a332d123c44434f626aaa18 Mon Sep 17 00:00:00 2001 From: Yaroslav Tkachenko Date: Fri, 17 Jul 2020 13:40:54 +0000 Subject: [PATCH 216/384] [SPARK-30616][SQL] Introduce TTL config option for SQL Metadata Cache ### What changes were proposed in this pull request? New `spark.sql.metadataCacheTTLSeconds` option that adds time-to-live cache behaviour to the existing caches in `FileStatusCache` and `SessionCatalog`. ### Why are the changes needed? Currently Spark [caches file listing for tables](https://spark.apache.org/docs/2.4.4/sql-data-sources-parquet.html#metadata-refreshing) and requires issuing `REFRESH TABLE` any time the file listing has changed outside of Spark. Unfortunately, simply submitting `REFRESH TABLE` commands could be very cumbersome. Assuming frequently added files, hundreds of tables and dozens of users querying the data (and expecting up-to-date results), manually refreshing metadata for each table is not a solution. This is a pretty common use-case for streaming ingestion of data, which can be done outside of Spark (with tools like Kafka Connect, etc.). A similar feature exists in Presto: `hive.file-status-cache-expire-time` can be found [here](https://prestosql.io/docs/current/connector/hive.html#hive-configuration-properties). ### Does this PR introduce _any_ user-facing change? Yes, it's controlled with the new `spark.sql.metadataCacheTTLSeconds` option. When it's set to `-1` (by default), the behaviour of caches doesn't change, so it stays _backwards-compatible_. Otherwise, you can specify a value in seconds, for example `spark.sql.metadataCacheTTLSeconds: 60` means 1-minute cache TTL. ### How was this patch tested? Added new tests in: - FileIndexSuite - SessionCatalogSuite Closes #28852 from sap1ens/SPARK-30616-metadata-cache-ttl. Authored-by: Yaroslav Tkachenko Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/SessionCatalog.scala | 12 ++++- .../apache/spark/sql/internal/SQLConf.scala | 2 + .../spark/sql/internal/StaticSQLConf.scala | 13 ++++++ .../catalog/SessionCatalogSuite.scala | 45 +++++++++++++++++-- .../datasources/FileStatusCache.scala | 17 +++++-- .../datasources/FileIndexSuite.scala | 28 +++++++++++- 6 files changed, 107 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index b79857cdccd22..2b3f05f61b483 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI import java.util.Locale import java.util.concurrent.Callable +import java.util.concurrent.TimeUnit import javax.annotation.concurrent.GuardedBy import scala.collection.mutable @@ -135,7 +136,16 @@ class SessionCatalog( private val tableRelationCache: Cache[QualifiedTableName, LogicalPlan] = { val cacheSize = conf.tableRelationCacheSize - CacheBuilder.newBuilder().maximumSize(cacheSize).build[QualifiedTableName, LogicalPlan]() + val cacheTTL = conf.metadataCacheTTL + + var builder = CacheBuilder.newBuilder() + .maximumSize(cacheSize) + + if (cacheTTL > 0) { + builder = builder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS) + } + + builder.build[QualifiedTableName, LogicalPlan]() } /** This method provides a way to get a cached plan. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f0d0a601ff196..e95ef3d77d549 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3268,6 +3268,8 @@ class SQLConf extends Serializable with Logging { def legacyAllowCastNumericToTimestamp: Boolean = getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP) + def metadataCacheTTL: Long = getConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 9618ff6062635..9bc910b5e1cdf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.internal import java.util.Locale +import java.util.concurrent.TimeUnit import org.apache.spark.util.Utils @@ -226,4 +227,16 @@ object StaticSQLConf { .version("3.0.0") .intConf .createWithDefault(100) + + val METADATA_CACHE_TTL_SECONDS = buildStaticConf("spark.sql.metadataCacheTTLSeconds") + .doc("Time-to-live (TTL) value for the metadata caches: partition file metadata cache and " + + "session catalog cache. This configuration only has an effect when this value having " + + "a positive value (> 0). It also requires setting " + + s"${StaticSQLConf.CATALOG_IMPLEMENTATION} to `hive`, setting " + + s"${SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE} > 0 and setting " + + s"${SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS} to `true` " + + "to be applied to the partition file metadata cache.") + .version("3.1.0") + .timeConf(TimeUnit.SECONDS) + .createWithDefault(-1) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 4d88a8d7ee546..ad40cc010361c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -17,15 +17,19 @@ package org.apache.spark.sql.catalyst.catalog +import scala.concurrent.duration._ + +import org.scalatest.concurrent.Eventually + import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.{Command, Range, SubqueryAlias, View} import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types._ class InMemorySessionCatalogSuite extends SessionCatalogSuite { @@ -45,7 +49,7 @@ class InMemorySessionCatalogSuite extends SessionCatalogSuite { * signatures but do not extend a common parent. This is largely by design but * unfortunately leads to very similar test code in two places. */ -abstract class SessionCatalogSuite extends AnalysisTest { +abstract class SessionCatalogSuite extends AnalysisTest with Eventually { protected val utils: CatalogTestUtils protected val isHiveExternalCatalog = false @@ -70,6 +74,16 @@ abstract class SessionCatalogSuite extends AnalysisTest { catalog.reset() } } + + private def withConfAndEmptyCatalog(conf: SQLConf)(f: SessionCatalog => Unit): Unit = { + val catalog = new SessionCatalog(newEmptyCatalog(), new SimpleFunctionRegistry(), conf) + catalog.createDatabase(newDb("default"), ignoreIfExists = true) + try { + f(catalog) + } finally { + catalog.reset() + } + } // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -1641,4 +1655,27 @@ abstract class SessionCatalogSuite extends AnalysisTest { assert(cause.cause.get.getMessage.contains("Actual error")) } } + + test("expire table relation cache if TTL is configured") { + case class TestCommand() extends Command + + val conf = new SQLConf() + conf.setConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS, 1L) + + withConfAndEmptyCatalog(conf) { catalog => + val table = QualifiedTableName(catalog.getCurrentDatabase, "test") + + // First, make sure the test table is not cached. + assert(catalog.getCachedTable(table) === null) + + catalog.cacheTable(table, TestCommand()) + assert(catalog.getCachedTable(table) !== null) + + // Wait until the cache expiration. + eventually(timeout(3.seconds)) { + // And the cache is gone. + assert(catalog.getCachedTable(table) === null) + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala index aea27bd4c4d7f..b5d800f02862e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ @@ -44,7 +45,9 @@ object FileStatusCache { session.sqlContext.conf.filesourcePartitionFileCacheSize > 0) { if (sharedCache == null) { sharedCache = new SharedInMemoryCache( - session.sqlContext.conf.filesourcePartitionFileCacheSize) + session.sqlContext.conf.filesourcePartitionFileCacheSize, + session.sqlContext.conf.metadataCacheTTL + ) } sharedCache.createForNewClient() } else { @@ -89,7 +92,7 @@ abstract class FileStatusCache { * * @param maxSizeInBytes max allowable cache size before entries start getting evicted */ -private class SharedInMemoryCache(maxSizeInBytes: Long) extends Logging { +private class SharedInMemoryCache(maxSizeInBytes: Long, cacheTTL: Long) extends Logging { // Opaque object that uniquely identifies a shared cache user private type ClientId = Object @@ -129,11 +132,17 @@ private class SharedInMemoryCache(maxSizeInBytes: Long) extends Logging { } } } - CacheBuilder.newBuilder() + + var builder = CacheBuilder.newBuilder() .weigher(weigher) .removalListener(removalListener) .maximumWeight(maxSizeInBytes / weightScale) - .build[(ClientId, Path), Array[FileStatus]]() + + if (cacheTTL > 0) { + builder = builder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS) + } + + builder.build[(ClientId, Path), Array[FileStatus]]() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index ea15f1891b006..040996276063b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -21,6 +21,7 @@ import java.io.{File, FileNotFoundException} import java.net.URI import scala.collection.mutable +import scala.concurrent.duration._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path, RawLocalFileSystem, RemoteIterator} @@ -33,7 +34,7 @@ import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.functions.col -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.KnownSizeEstimation @@ -488,6 +489,31 @@ class FileIndexSuite extends SharedSparkSession { val fileIndex = new TestInMemoryFileIndex(spark, path) assert(fileIndex.leafFileStatuses.toSeq == statuses) } + + test("expire FileStatusCache if TTL is configured") { + val previousValue = SQLConf.get.getConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS) + try { + // using 'SQLConf.get.setConf' instead of 'withSQLConf' to set a static config at runtime + SQLConf.get.setConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS, 1L) + + val path = new Path("/dummy_tmp", "abc") + val files = (1 to 3).map(_ => new FileStatus()) + + FileStatusCache.resetForTesting() + val fileStatusCache = FileStatusCache.getOrCreate(spark) + fileStatusCache.putLeafFiles(path, files.toArray) + + // Exactly 3 files are cached. + assert(fileStatusCache.getLeafFiles(path).get.length === 3) + // Wait until the cache expiration. + eventually(timeout(3.seconds)) { + // And the cache is gone. + assert(fileStatusCache.getLeafFiles(path).isEmpty === true) + } + } finally { + SQLConf.get.setConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS, previousValue) + } + } } object DeletionRaceFileSystem { From 5daf244d0f89668007f7168809b8b83decb979eb Mon Sep 17 00:00:00 2001 From: williamhyun <62487364+williamhyun@users.noreply.github.com> Date: Fri, 17 Jul 2020 11:59:19 -0500 Subject: [PATCH 217/384] [SPARK-32329][TESTS] Rename HADOOP2_MODULE_PROFILES to HADOOP_MODULE_PROFILES ### What changes were proposed in this pull request? This PR aims to rename `HADOOP2_MODULE_PROFILES` to `HADOOP_MODULE_PROFILES` because Hadoop 3 is now the default. ### Why are the changes needed? Hadoop 3 is now the default. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GitHub Action dependency test. Closes #29128 from williamhyun/williamhyun-patch-3. Authored-by: williamhyun <62487364+williamhyun@users.noreply.github.com> Signed-off-by: Sean Owen --- dev/test-dependencies.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index b3e68bed1d1e7..129b073d75254 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -29,7 +29,7 @@ export LC_ALL=C # TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution. # NOTE: These should match those in the release publishing script -HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive" +HADOOP_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive" MVN="build/mvn" HADOOP_HIVE_PROFILES=( hadoop-2.7-hive-1.2 @@ -79,14 +79,14 @@ for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do HIVE_PROFILE=hive-1.2 fi echo "Performing Maven install for $HADOOP_HIVE_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q + $MVN $HADOOP_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q echo "Performing Maven validate for $HADOOP_HIVE_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE validate -q + $MVN $HADOOP_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE validate -q echo "Generating dependency manifest for $HADOOP_HIVE_PROFILE" mkdir -p dev/pr-deps - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE dependency:build-classpath -pl assembly -am \ + $MVN $HADOOP_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE dependency:build-classpath -pl assembly -am \ | grep "Dependencies classpath:" -A 1 \ | tail -n 1 | tr ":" "\n" | awk -F '/' '{ # For each dependency classpath, we fetch the last three parts split by "/": artifact id, version, and jar name. From 3a60b4194980e95c461021b38912e4f08e6761be Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 17 Jul 2020 12:00:49 -0500 Subject: [PATCH 218/384] [SPARK-32298][ML] tree models prediction optimization ### What changes were proposed in this pull request? use while-loop instead of the recursive way ### Why are the changes needed? 3% ~ 10% faster ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? existing testsuites Closes #29095 from zhengruifeng/tree_pred_opt. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../scala/org/apache/spark/ml/tree/Node.scala | 46 +++++++++++-------- 1 file changed, 28 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index cc21e9cbbe8de..dfa4961d9ffb8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -174,22 +174,32 @@ class InternalNode private[ml] ( } override private[ml] def predictImpl(features: Vector): LeafNode = { - if (split.shouldGoLeft(features)) { - leftChild.predictImpl(features) - } else { - rightChild.predictImpl(features) + var node: Node = this + while (node.isInstanceOf[InternalNode]) { + val n = node.asInstanceOf[InternalNode] + if (n.split.shouldGoLeft(features)) { + node = n.leftChild + } else { + node = n.rightChild + } } + node.asInstanceOf[LeafNode] } override private[ml] def predictBinned( binned: Array[Int], splits: Array[Array[Split]]): LeafNode = { - val i = split.featureIndex - if (split.shouldGoLeft(binned(i), splits(i))) { - leftChild.predictBinned(binned, splits) - } else { - rightChild.predictBinned(binned, splits) + var node: Node = this + while (node.isInstanceOf[InternalNode]) { + val n = node.asInstanceOf[InternalNode] + val i = n.split.featureIndex + if (n.split.shouldGoLeft(binned(i), splits(i))) { + node = n.leftChild + } else { + node = n.rightChild + } } + node.asInstanceOf[LeafNode] } override private[tree] def numDescendants: Int = { @@ -326,27 +336,27 @@ private[tree] class LearningNode( * [[org.apache.spark.ml.tree.impl.RandomForest.findBestSplits()]]. */ def predictImpl(binnedFeatures: Array[Int], splits: Array[Array[Split]]): Int = { - if (this.isLeaf || this.split.isEmpty) { - this.id - } else { - val split = this.split.get + var node = this + while (!node.isLeaf && node.split.nonEmpty) { + val split = node.split.get val featureIndex = split.featureIndex val splitLeft = split.shouldGoLeft(binnedFeatures(featureIndex), splits(featureIndex)) - if (this.leftChild.isEmpty) { + if (node.leftChild.isEmpty) { // Not yet split. Return next layer of nodes to train if (splitLeft) { - LearningNode.leftChildIndex(this.id) + return LearningNode.leftChildIndex(node.id) } else { - LearningNode.rightChildIndex(this.id) + return LearningNode.rightChildIndex(node.id) } } else { if (splitLeft) { - this.leftChild.get.predictImpl(binnedFeatures, splits) + node = node.leftChild.get } else { - this.rightChild.get.predictImpl(binnedFeatures, splits) + node = node.rightChild.get } } } + node.id } } From 7dc1d8917dd01b5d5808460a5eb6e846795ab4bd Mon Sep 17 00:00:00 2001 From: William Hyun Date: Fri, 17 Jul 2020 12:05:45 -0700 Subject: [PATCH 219/384] [SPARK-32353][TEST] Update docker/spark-test and clean up unused stuff ### What changes were proposed in this pull request? This PR aims to update the docker/spark-test and clean up unused stuff. ### Why are the changes needed? Since Spark 3.0.0, Java 11 is supported. We had better use the latest Java and OS. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually do the following as described in https://github.com/apache/spark/blob/master/external/docker/spark-test/README.md . ``` docker run -v $SPARK_HOME:/opt/spark spark-test-master docker run -v $SPARK_HOME:/opt/spark spark-test-worker spark://:7077 ``` Closes #29150 from williamhyun/docker. Authored-by: William Hyun Signed-off-by: Dongjoon Hyun --- external/docker/spark-test/base/Dockerfile | 15 +++------------ external/docker/spark-test/master/default_cmd | 3 --- external/docker/spark-test/worker/default_cmd | 3 --- 3 files changed, 3 insertions(+), 18 deletions(-) diff --git a/external/docker/spark-test/base/Dockerfile b/external/docker/spark-test/base/Dockerfile index 5bec5d3f16548..d4a30c4681cba 100644 --- a/external/docker/spark-test/base/Dockerfile +++ b/external/docker/spark-test/base/Dockerfile @@ -15,23 +15,14 @@ # limitations under the License. # -FROM ubuntu:xenial +FROM ubuntu:20.04 # Upgrade package index -# install a few other useful packages plus Open Jdk 8 +# install a few other useful packages plus Open Java 11 # Remove unneeded /var/lib/apt/lists/* after install to reduce the # docker image size (by ~30MB) RUN apt-get update && \ - apt-get install -y less openjdk-8-jre-headless iproute2 vim-tiny sudo openssh-server && \ + apt-get install -y less openjdk-11-jre-headless iproute2 vim-tiny sudo openssh-server && \ rm -rf /var/lib/apt/lists/* -ENV SCALA_VERSION 2.12.10 -ENV CDH_VERSION cdh4 -ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark -ENV PATH $SPARK_HOME:$SCALA_HOME/bin:$PATH - -# Install Scala -ADD https://www.scala-lang.org/files/archive/scala-$SCALA_VERSION.tgz / -RUN (cd / && gunzip < scala-$SCALA_VERSION.tgz)|(cd /opt && tar -xvf -) -RUN rm /scala-$SCALA_VERSION.tgz diff --git a/external/docker/spark-test/master/default_cmd b/external/docker/spark-test/master/default_cmd index 5a7da3446f6d2..96a36cd0bb682 100755 --- a/external/docker/spark-test/master/default_cmd +++ b/external/docker/spark-test/master/default_cmd @@ -22,7 +22,4 @@ echo "CONTAINER_IP=$IP" export SPARK_LOCAL_IP=$IP export SPARK_PUBLIC_DNS=$IP -# Avoid the default Docker behavior of mapping our IP address to an unreachable host name -umount /etc/hosts - /opt/spark/bin/spark-class org.apache.spark.deploy.master.Master -i $IP diff --git a/external/docker/spark-test/worker/default_cmd b/external/docker/spark-test/worker/default_cmd index 31b06cb0eb047..2401f5565aa0b 100755 --- a/external/docker/spark-test/worker/default_cmd +++ b/external/docker/spark-test/worker/default_cmd @@ -22,7 +22,4 @@ echo "CONTAINER_IP=$IP" export SPARK_LOCAL_IP=$IP export SPARK_PUBLIC_DNS=$IP -# Avoid the default Docker behavior of mapping our IP address to an unreachable host name -umount /etc/hosts - /opt/spark/bin/spark-class org.apache.spark.deploy.worker.Worker $1 From 0678afe393b1e4f65b70470483fe0cdb1fe139dc Mon Sep 17 00:00:00 2001 From: Prakhar Jain Date: Fri, 17 Jul 2020 16:11:02 -0700 Subject: [PATCH 220/384] [SPARK-21040][CORE] Speculate tasks which are running on decommission executors ### What changes were proposed in this pull request? This PR adds functionality to consider the running tasks on decommission executors based on some config. In spark-on-cloud , we sometimes already know that an executor won't be alive for more than fix amount of time. Ex- In AWS Spot nodes, once we get the notification, we know that a node will be gone in 120 seconds. So if the running tasks on the decommissioning executors may run beyond currentTime+120 seconds, then they are candidate for speculation. ### Why are the changes needed? Currently when an executor is decommission, we stop scheduling new tasks on those executors but the already running tasks keeps on running on them. Based on the cloud, we might know beforehand that an executor won't be alive for more than a preconfigured time. Different cloud providers gives different timeouts before they take away the nodes. For Ex- In case of AWS spot nodes, an executor won't be alive for more than 120 seconds. We can utilize this information in cloud environments and take better decisions about speculating the already running tasks on decommission executors. ### Does this PR introduce _any_ user-facing change? Yes. This PR adds a new config "spark.executor.decommission.killInterval" which they can explicitly set based on the cloud environment where they are running. ### How was this patch tested? Added UT. Closes #28619 from prakharjain09/SPARK-21040-speculate-decommission-exec-tasks. Authored-by: Prakhar Jain Signed-off-by: Holden Karau --- .../spark/internal/config/package.scala | 11 ++ .../spark/scheduler/TaskSetManager.scala | 28 ++++- .../spark/scheduler/TaskSetManagerSuite.scala | 106 ++++++++++++++++++ 3 files changed, 141 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index ca75a19af7bf6..f0b292b71eef7 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1843,6 +1843,17 @@ package object config { .timeConf(TimeUnit.MILLISECONDS) .createOptional + private[spark] val EXECUTOR_DECOMMISSION_KILL_INTERVAL = + ConfigBuilder("spark.executor.decommission.killInterval") + .doc("Duration after which a decommissioned executor will be killed forcefully." + + "This config is useful for cloud environments where we know in advance when " + + "an executor is going to go down after decommissioning signal i.e. around 2 mins " + + "in aws spot nodes, 1/2 hrs in spot block nodes etc. This config is currently " + + "used to decide what tasks running on decommission executors to speculate.") + .version("3.1.0") + .timeConf(TimeUnit.SECONDS) + .createOptional + private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir") .doc("Staging directory used while submitting applications.") .version("2.0.0") diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a302f680a272e..4b31ff0c790da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.{ConcurrentLinkedQueue, TimeUnit} import scala.collection.immutable.Map import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -102,6 +102,8 @@ private[spark] class TaskSetManager( } numTasks <= slots } + val executorDecommissionKillInterval = conf.get(EXECUTOR_DECOMMISSION_KILL_INTERVAL).map( + TimeUnit.SECONDS.toMillis) // For each task, tracks whether a copy of the task has succeeded. A task will also be // marked as "succeeded" if it failed with a fetch failure, in which case it should not @@ -165,6 +167,7 @@ private[spark] class TaskSetManager( // Task index, start and finish time for each task attempt (indexed by task ID) private[scheduler] val taskInfos = new HashMap[Long, TaskInfo] + private[scheduler] val tidToExecutorKillTimeMapping = new HashMap[Long, Long] // Use a MedianHeap to record durations of successful tasks so we know when to launch // speculative tasks. This is only used when speculation is enabled, to avoid the overhead @@ -933,6 +936,7 @@ private[spark] class TaskSetManager( /** If the given task ID is in the set of running tasks, removes it. */ def removeRunningTask(tid: Long): Unit = { + tidToExecutorKillTimeMapping.remove(tid) if (runningTasksSet.remove(tid) && parent != null) { parent.decreaseRunningTasks(1) } @@ -1042,7 +1046,19 @@ private[spark] class TaskSetManager( // bound based on that. logDebug("Task length threshold for speculation: " + threshold) for (tid <- runningTasksSet) { - foundTasks |= checkAndSubmitSpeculatableTask(tid, time, threshold) + var speculated = checkAndSubmitSpeculatableTask(tid, time, threshold) + if (!speculated && tidToExecutorKillTimeMapping.contains(tid)) { + // Check whether this task will finish before the exectorKillTime assuming + // it will take medianDuration overall. If this task cannot finish within + // executorKillInterval, then this task is a candidate for speculation + val taskEndTimeBasedOnMedianDuration = taskInfos(tid).launchTime + medianDuration + val canExceedDeadline = tidToExecutorKillTimeMapping(tid) < + taskEndTimeBasedOnMedianDuration + if (canExceedDeadline) { + speculated = checkAndSubmitSpeculatableTask(tid, time, 0) + } + } + foundTasks |= speculated } } else if (speculationTaskDurationThresOpt.isDefined && speculationTasksLessEqToSlots) { val time = clock.getTimeMillis() @@ -1100,8 +1116,12 @@ private[spark] class TaskSetManager( def executorDecommission(execId: String): Unit = { recomputeLocality() - // Future consideration: if an executor is decommissioned it may make sense to add the current - // tasks to the spec exec queue. + executorDecommissionKillInterval.foreach { interval => + val executorKillTime = clock.getTimeMillis() + interval + runningTasksSet.filter(taskInfos(_).executorId == execId).foreach { tid => + tidToExecutorKillTimeMapping(tid) = executorKillTime + } + } } def recomputeLocality(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 95c8197abbf0b..ae51b55aa1373 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1911,6 +1911,112 @@ class TaskSetManagerSuite testSpeculationDurationThreshold(true, 2, 1) } + test("SPARK-21040: Check speculative tasks are launched when an executor is decommissioned" + + " and the tasks running on it cannot finish within EXECUTOR_DECOMMISSION_KILL_INTERVAL") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3")) + val taskSet = FakeTask.createTaskSet(4) + sc.conf.set(config.SPECULATION_ENABLED, true) + sc.conf.set(config.SPECULATION_MULTIPLIER, 1.5) + sc.conf.set(config.SPECULATION_QUANTILE, 0.5) + sc.conf.set(config.EXECUTOR_DECOMMISSION_KILL_INTERVAL.key, "5s") + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => + task.metrics.internalAccums + } + + // Start TASK 0,1 on exec1, TASK 2 on exec2 + (0 until 2).foreach { _ => + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF)._1 + assert(taskOption.isDefined) + assert(taskOption.get.executorId === "exec1") + } + val taskOption2 = manager.resourceOffer("exec2", "host2", NO_PREF)._1 + assert(taskOption2.isDefined) + assert(taskOption2.get.executorId === "exec2") + + clock.advance(6*1000) // time = 6s + // Start TASK 3 on exec2 after some delay + val taskOption3 = manager.resourceOffer("exec2", "host2", NO_PREF)._1 + assert(taskOption3.isDefined) + assert(taskOption3.get.executorId === "exec2") + + assert(sched.startedTasks.toSet === Set(0, 1, 2, 3)) + + clock.advance(4*1000) // time = 10s + // Complete the first 2 tasks and leave the other 2 tasks in running + for (id <- Set(0, 1)) { + manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) + assert(sched.endedTasks(id) === Success) + } + + // checkSpeculatableTasks checks that the task runtime is greater than the threshold for + // speculating. Since we use a SPECULATION_MULTIPLIER of 1.5, So tasks need to be running for + // > 15s for speculation + assert(!manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.toSet === Set()) + + // decommission exec-2. All tasks running on exec-2 (i.e. TASK 2,3) will be added to + // executorDecommissionSpeculationTriggerTimeoutOpt + // (TASK 2 -> 15, TASK 3 -> 15) + manager.executorDecommission("exec2") + assert(manager.tidToExecutorKillTimeMapping.keySet === Set(2, 3)) + assert(manager.tidToExecutorKillTimeMapping(2) === 15*1000) + assert(manager.tidToExecutorKillTimeMapping(3) === 15*1000) + + assert(manager.checkSpeculatableTasks(0)) + // TASK 2 started at t=0s, so it can still finish before t=15s (Median task runtime = 10s) + // TASK 3 started at t=6s, so it might not finish before t=15s. So TASK 3 should be part + // of speculativeTasks + assert(sched.speculativeTasks.toSet === Set(3)) + assert(manager.copiesRunning(3) === 1) + + // Offer resource to start the speculative attempt for the running task + val taskOption3New = manager.resourceOffer("exec3", "host3", NO_PREF)._1 + // Offer more resources. Nothing should get scheduled now. + assert(manager.resourceOffer("exec3", "host3", NO_PREF)._1.isEmpty) + assert(taskOption3New.isDefined) + + // Assert info about the newly launched speculative task + val speculativeTask3 = taskOption3New.get + assert(speculativeTask3.index === 3) + assert(speculativeTask3.taskId === 4) + assert(speculativeTask3.executorId === "exec3") + assert(speculativeTask3.attemptNumber === 1) + + clock.advance(1*1000) // time = 11s + // Running checkSpeculatableTasks again should return false + assert(!manager.checkSpeculatableTasks(0)) + assert(manager.copiesRunning(2) === 1) + assert(manager.copiesRunning(3) === 2) + + clock.advance(5*1000) // time = 16s + // At t=16s, TASK 2 has been running for 16s. It is more than the + // SPECULATION_MULTIPLIER * medianRuntime = 1.5 * 10 = 15s. So now TASK 2 will + // be selected for speculation. Here we are verifying that regular speculation configs + // should still take effect even when a EXECUTOR_DECOMMISSION_KILL_INTERVAL is provided and + // corresponding executor is decommissioned + assert(manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.toSet === Set(2, 3)) + assert(manager.copiesRunning(2) === 1) + assert(manager.copiesRunning(3) === 2) + val taskOption2New = manager.resourceOffer("exec3", "host3", NO_PREF)._1 + assert(taskOption2New.isDefined) + val speculativeTask2 = taskOption2New.get + // Ensure that TASK 2 is re-launched on exec3, host3 + assert(speculativeTask2.index === 2) + assert(speculativeTask2.taskId === 5) + assert(speculativeTask2.executorId === "exec3") + assert(speculativeTask2.attemptNumber === 1) + + assert(manager.copiesRunning(2) === 2) + assert(manager.copiesRunning(3) === 2) + + // Offering additional resources should not lead to any speculative tasks being respawned + assert(manager.resourceOffer("exec1", "host1", ANY)._1.isEmpty) + } + test("SPARK-29976 Regular speculation configs should still take effect even when a " + "threshold is provided") { val (manager, clock) = testSpeculationDurationSetup( From f9f9309bec4bac33a182046ccedabef4234e8f77 Mon Sep 17 00:00:00 2001 From: Sudharshann D Date: Sat, 18 Jul 2020 13:04:58 -0500 Subject: [PATCH 221/384] [SPARK-31579][SQL] replaced floorDiv to Div ### What changes were proposed in this pull request? Replaced floorDiv to just / in `localRebaseGregorianToJulianDays()` in `spark/sql/catalyst/util/RebaseDateTime.scala` ### Why are the changes needed? Easier to understand the logic/code and a little more efficiency. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? Proof of concept [here](https://github.com/apache/spark/pull/28573/files). The operation `utcCal.getTimeInMillis / MILLIS_PER_DAY` results in an interger value already. Closes #29008 from Sudhar287/SPARK-31579. Authored-by: Sudharshann D Signed-off-by: Sean Owen --- .../org/apache/spark/sql/catalyst/util/RebaseDateTime.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index 24fe9fa42c19f..1a78422e57a4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -186,7 +186,8 @@ object RebaseDateTime { .setTimeZone(TimeZoneUTC) .setDate(localDate.getYear, localDate.getMonthValue - 1, localDate.getDayOfMonth) .build() - Math.toIntExact(Math.floorDiv(utcCal.getTimeInMillis, MILLIS_PER_DAY)) + assert(utcCal.getTimeInMillis % MILLIS_PER_DAY == 0) + Math.toIntExact(utcCal.getTimeInMillis / MILLIS_PER_DAY) } /** From ee624821a903a263c844ed849d6833df8e9ad43e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 18 Jul 2020 15:08:00 -0700 Subject: [PATCH 222/384] [SPARK-29292][YARN][K8S][MESOS] Fix Scala 2.13 compilation for remaining modules ### What changes were proposed in this pull request? See again the related PRs like https://github.com/apache/spark/pull/28971 This completes fixing compilation for 2.13 for all but `repl`, which is a separate task. ### Why are the changes needed? Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12) Closes #29147 from srowen/SPARK-29292.4. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../spark/streaming/kinesis/KinesisTestUtils.scala | 6 +++--- .../streaming/kinesis/KPLBasedKinesisTestUtils.scala | 2 +- .../streaming/kinesis/KinesisBackedBlockRDDSuite.scala | 4 ++-- .../k8s/ExecutorPodsPollingSnapshotSource.scala | 2 +- .../cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala | 2 +- .../k8s/DeterministicExecutorPodsSnapshotsStore.scala | 2 +- .../deploy/k8s/integrationtest/ProcessUtils.scala | 2 +- .../cluster/mesos/MesosClusterScheduler.scala | 8 ++++---- .../mesos/MesosCoarseGrainedSchedulerBackend.scala | 4 ++-- .../scheduler/cluster/mesos/MesosSchedulerUtils.scala | 2 +- .../cluster/mesos/MesosClusterSchedulerSuite.scala | 2 +- .../mesos/MesosFineGrainedSchedulerBackendSuite.scala | 6 ++++-- .../scala/org/apache/spark/deploy/yarn/Client.scala | 4 ++-- .../deploy/yarn/ClientDistributedCacheManager.scala | 10 +++++----- .../org/apache/spark/deploy/yarn/YarnAllocator.scala | 8 ++++---- .../executor/YarnCoarseGrainedExecutorBackend.scala | 4 ++-- .../spark/sql/hive/thriftserver/SparkOperation.scala | 2 +- 17 files changed, 36 insertions(+), 34 deletions(-) diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index 2ac83c8ee6b38..4dacc03c72719 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -97,7 +97,7 @@ private[kinesis] class KinesisTestUtils(streamShardCount: Int = 2) extends Loggi } def getShards(): Seq[Shard] = { - kinesisClient.describeStream(_streamName).getStreamDescription.getShards.asScala + kinesisClient.describeStream(_streamName).getStreamDescription.getShards.asScala.toSeq } def splitShard(shardId: String): Unit = { @@ -137,7 +137,7 @@ private[kinesis] class KinesisTestUtils(streamShardCount: Int = 2) extends Loggi * Expose a Python friendly API. */ def pushData(testData: java.util.List[Int]): Unit = { - pushData(testData.asScala, aggregate = false) + pushData(testData.asScala.toSeq, aggregate = false) } def deleteStream(): Unit = { @@ -289,6 +289,6 @@ private[kinesis] class SimpleDataGenerator( sentSeqNumbers += ((num, seqNumber)) } - shardIdToSeqNumbers.toMap + shardIdToSeqNumbers.mapValues(_.toSeq).toMap } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala index af84498d5e47e..c76eb7c29dd94 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala @@ -72,6 +72,6 @@ private[kinesis] class KPLDataGenerator(regionName: String) extends KinesisDataG Futures.addCallback(future, kinesisCallBack, ThreadUtils.sameThreadExecutorService) } producer.flushSync() - shardIdToSeqNumbers.toMap + shardIdToSeqNumbers.mapValues(_.toSeq).toMap } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala index 2c7b9c58e6fa6..12d950096b4c2 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala @@ -47,8 +47,8 @@ abstract class KinesisBackedBlockRDDTests(aggregateTestData: Boolean) require(shardIdToDataAndSeqNumbers.size > 1, "Need data to be sent to multiple shards") shardIds = shardIdToDataAndSeqNumbers.keySet.toSeq - shardIdToData = shardIdToDataAndSeqNumbers.mapValues { _.map { _._1 }} - shardIdToSeqNumbers = shardIdToDataAndSeqNumbers.mapValues { _.map { _._2 }} + shardIdToData = shardIdToDataAndSeqNumbers.mapValues(_.map(_._1)).toMap + shardIdToSeqNumbers = shardIdToDataAndSeqNumbers.mapValues(_.map(_._2)).toMap shardIdToRange = shardIdToSeqNumbers.map { case (shardId, seqNumbers) => val seqNumRange = SequenceNumberRange( testUtils.streamName, shardId, seqNumbers.head, seqNumbers.last, seqNumbers.size) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala index fd8f6979c9e65..da7fe7cdda328 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala @@ -62,7 +62,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( .withoutLabel(SPARK_EXECUTOR_INACTIVE_LABEL, "true") .list() .getItems - .asScala) + .asScala.toSeq) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index d68dc3ebef5d8..5c192c690eba5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -131,7 +131,7 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul try { val snapshots = new ArrayList[ExecutorPodsSnapshot]() snapshotsBuffer.drainTo(snapshots) - onNewSnapshots(snapshots.asScala) + onNewSnapshots(snapshots.asScala.toSeq) } catch { case NonFatal(e) => logWarning("Exception when notifying snapshot subscriber.", e) } finally { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala index 9ac7e0222054a..6dc052a5e6894 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala @@ -35,7 +35,7 @@ class DeterministicExecutorPodsSnapshotsStore extends ExecutorPodsSnapshotsStore override def stop(): Unit = {} override def notifySubscribers(): Unit = { - subscribers.foreach(_(snapshotsBuffer)) + subscribers.foreach(_(snapshotsBuffer.toSeq)) snapshotsBuffer.clear() } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala index 7d76a22e6d363..cce842ce62f01 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala @@ -47,6 +47,6 @@ object ProcessUtils extends Logging { assert(proc.exitValue == 0, s"Failed to execute ${fullCommand.mkString(" ")}" + s"${if (dumpErrors) "\n" + outputLines.mkString("\n")}") - outputLines + outputLines.toSeq } } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 26939ef23eaab..e5c18539a01ae 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -383,13 +383,13 @@ private[spark] class MesosClusterScheduler( taskId.split(s"${RETRY_SEP}").head } - private def adjust[A, B](m: collection.Map[A, B], k: A, default: B)(f: B => B) = { + private def adjust[A, B](m: Map[A, B], k: A, default: B)(f: B => B) = { m.updated(k, f(m.getOrElse(k, default))) } private def getDriverEnvironment(desc: MesosDriverDescription): Environment = { // TODO(mgummelt): Don't do this here. This should be passed as a --conf - val commandEnv = adjust(desc.command.environment, "SPARK_SUBMIT_OPTS", "")( + val commandEnv = adjust(desc.command.environment.toMap, "SPARK_SUBMIT_OPTS", "")( v => s"$v -D${config.DRIVER_FRAMEWORK_ID.key}=${getDriverFrameworkID(desc)}" ) @@ -686,14 +686,14 @@ private[spark] class MesosClusterScheduler( } scheduleTasks( - copyBuffer(driversToRetry), + copyBuffer(driversToRetry).toSeq, removeFromPendingRetryDrivers, currentOffers, tasks) // Then we walk through the queued drivers and try to schedule them. scheduleTasks( - copyBuffer(queuedDrivers), + copyBuffer(queuedDrivers).toSeq, removeFromQueuedDrivers, currentOffers, tasks) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 5e7a29ac6d344..32cd50298bc6c 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -491,8 +491,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( val tasks = new mutable.HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil) // offerID -> resources - val remainingResources = mutable.Map(offers.map(offer => - (offer.getId.getValue, offer.getResourcesList)): _*) + val remainingResources = mutable.Map[String, JList[Resource]]() + remainingResources ++= offers.map(offer => (offer.getId.getValue, offer.getResourcesList)) var launchTasks = true diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 5784ee314aa17..2be8835f77e36 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -379,7 +379,7 @@ trait MesosSchedulerUtils extends Logging { } else { v.split(',').toSet } - ) + ).toMap } catch { case NonFatal(e) => throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index bb37bbd2d8046..2433c16c34af6 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -146,7 +146,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val resources = taskInfo.getResourcesList assert(scheduler.getResource(resources, "cpus") == 1.5) assert(scheduler.getResource(resources, "mem") == 1200) - val resourcesSeq: Seq[Resource] = resources.asScala + val resourcesSeq: Seq[Resource] = resources.asScala.toSeq val cpus = resourcesSeq.filter(_.getName == "cpus").toList assert(cpus.size == 2) assert(cpus.exists(_.getRole() == "role2")) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index 36ed84858dbfb..67ecf3242f52d 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala @@ -267,7 +267,8 @@ class MesosFineGrainedSchedulerBackendSuite properties = new Properties(), resources = immutable.Map.empty[String, ResourceInformation], ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) + when(taskScheduler.resourceOffers( + expectedWorkerOffers.toIndexedSeq)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(2) val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) @@ -379,7 +380,8 @@ class MesosFineGrainedSchedulerBackendSuite properties = new Properties(), resources = immutable.Map.empty[String, ResourceInformation], ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) + when(taskScheduler.resourceOffers( + expectedWorkerOffers.toIndexedSeq)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(1) val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 7b121194d1b31..1045fb089c017 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -553,7 +553,7 @@ private[spark] class Client( } // Propagate the local URIs to the containers using the configuration. - sparkConf.set(SPARK_JARS, localJars) + sparkConf.set(SPARK_JARS, localJars.toSeq) case None => // No configuration, so fall back to uploading local jar files. @@ -628,7 +628,7 @@ private[spark] class Client( } } if (cachedSecondaryJarLinks.nonEmpty) { - sparkConf.set(SECONDARY_JARS, cachedSecondaryJarLinks) + sparkConf.set(SECONDARY_JARS, cachedSecondaryJarLinks.toSeq) } if (isClusterMode && args.primaryPyFile != null) { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index e6e0ea38ade94..e02fbd0c91495 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -91,11 +91,11 @@ private[spark] class ClientDistributedCacheManager() extends Logging { * Writes down information about cached files needed in executors to the given configuration. */ def updateConfiguration(conf: SparkConf): Unit = { - conf.set(CACHED_FILES, distCacheEntries.map(_.uri.toString)) - conf.set(CACHED_FILES_SIZES, distCacheEntries.map(_.size)) - conf.set(CACHED_FILES_TIMESTAMPS, distCacheEntries.map(_.modTime)) - conf.set(CACHED_FILES_VISIBILITIES, distCacheEntries.map(_.visibility.name())) - conf.set(CACHED_FILES_TYPES, distCacheEntries.map(_.resType.name())) + conf.set(CACHED_FILES, distCacheEntries.map(_.uri.toString).toSeq) + conf.set(CACHED_FILES_SIZES, distCacheEntries.map(_.size).toSeq) + conf.set(CACHED_FILES_TIMESTAMPS, distCacheEntries.map(_.modTime).toSeq) + conf.set(CACHED_FILES_VISIBILITIES, distCacheEntries.map(_.visibility.name()).toSeq) + conf.set(CACHED_FILES_TYPES, distCacheEntries.map(_.resType.name()).toSeq) } /** diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index cd0e7d5c87bc8..dc093235288a9 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -296,7 +296,7 @@ private[yarn] class YarnAllocator( val profResource = rpIdToYarnResource.get(id) val result = amClient.getMatchingRequests(getContainerPriority(id), location, profResource) .asScala.flatMap(_.asScala) - allContainerRequests(id) = result + allContainerRequests(id) = result.toSeq } allContainerRequests.toMap } @@ -426,13 +426,13 @@ private[yarn] class YarnAllocator( getNumExecutorsStarting, allocateResponse.getAvailableResources)) - handleAllocatedContainers(allocatedContainers.asScala) + handleAllocatedContainers(allocatedContainers.asScala.toSeq) } val completedContainers = allocateResponse.getCompletedContainersStatuses() if (completedContainers.size > 0) { logDebug("Completed %d containers".format(completedContainers.size)) - processCompletedContainers(completedContainers.asScala) + processCompletedContainers(completedContainers.asScala.toSeq) logDebug("Finished processing %d completed containers. Current running executor count: %d." .format(completedContainers.size, getNumExecutorsRunning)) } @@ -960,7 +960,7 @@ private[yarn] class YarnAllocator( } } - (localityMatched, localityUnMatched, localityFree) + (localityMatched.toSeq, localityUnMatched.toSeq, localityFree.toSeq) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala index 669e39fb7c1c7..ce46ffa06f0fe 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala @@ -73,8 +73,8 @@ private[spark] object YarnCoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv, ResourceProfile) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env, resourceProfile) => new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, - arguments.resourcesFileOpt, resourceProfile) + arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath.toSeq, + env, arguments.resourcesFileOpt, resourceProfile) } val backendArgs = CoarseGrainedExecutorBackend.parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index 18d2950a02745..bbfc1b83379aa 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -37,7 +37,7 @@ private[hive] trait SparkOperation extends Operation with Logging { protected var statementId = getHandle().getHandleIdentifier().getPublicId().toString() - protected def cleanup(): Unit = Unit // noop by default + protected def cleanup(): Unit = () // noop by default abstract override def run(): Unit = { withLocalProperties { From 40ef01283d6615b968d76d208113d9b6ecc9aecf Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 19 Jul 2020 11:02:37 +0900 Subject: [PATCH 223/384] [SPARK-29802][BUILD] Use python3 in build scripts ### What changes were proposed in this pull request? Use `/usr/bin/env python3` consistently instead of `/usr/bin/env python` in build scripts, to reliably select Python 3. ### Why are the changes needed? Scripts no longer work with Python 2. ### Does this PR introduce _any_ user-facing change? No, should be all build system changes. ### How was this patch tested? Existing tests / NA Closes #29151 from srowen/SPARK-29909.2. Authored-by: Sean Owen Signed-off-by: HyukjinKwon --- dev/create-release/generate-contributors.py | 2 +- dev/create-release/releaseutils.py | 2 +- dev/create-release/translate-contributors.py | 2 +- dev/github_jira_sync.py | 2 +- dev/merge_spark_pr.py | 2 +- python/pyspark/find_spark_home.py | 2 +- python/setup.py | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index d9135173419ae..57775dde9dd67 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 241b7ed539ae9..9f1dffbd09ac4 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py index be30e6ad30b24..554acc8b8a379 100755 --- a/dev/create-release/translate-contributors.py +++ b/dev/create-release/translate-contributors.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with diff --git a/dev/github_jira_sync.py b/dev/github_jira_sync.py index b90afeebc5238..d916e2dbd8223 100755 --- a/dev/github_jira_sync.py +++ b/dev/github_jira_sync.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index b42429d7175b1..2e376fb343bbe 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/python/pyspark/find_spark_home.py b/python/pyspark/find_spark_home.py index 920c04009dd11..011c3f5b10a6b 100755 --- a/python/pyspark/find_spark_home.py +++ b/python/pyspark/find_spark_home.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/python/setup.py b/python/setup.py index 294a34c7a1916..6a96ee743b597 100755 --- a/python/setup.py +++ b/python/setup.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more From c7a68a920df433f682b3716f761ff0a2208e18f7 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 19 Jul 2020 11:11:42 +0900 Subject: [PATCH 224/384] [SPARK-32344][SQL] Unevaluable expr is set to FIRST/LAST ignoreNullsExpr in distinct aggregates ### What changes were proposed in this pull request? This PR intends to fix a bug of distinct FIRST/LAST aggregates in v2.4.6/v3.0.0/master; ``` scala> sql("SELECT FIRST(DISTINCT v) FROM VALUES 1, 2, 3 t(v)").show() ... Caused by: java.lang.UnsupportedOperationException: Cannot evaluate expression: false#37 at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.eval(Expression.scala:258) at org.apache.spark.sql.catalyst.expressions.AttributeReference.eval(namedExpressions.scala:226) at org.apache.spark.sql.catalyst.expressions.aggregate.First.ignoreNulls(First.scala:68) at org.apache.spark.sql.catalyst.expressions.aggregate.First.updateExpressions$lzycompute(First.scala:82) at org.apache.spark.sql.catalyst.expressions.aggregate.First.updateExpressions(First.scala:81) at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$15.apply(HashAggregateExec.scala:268) ``` A root cause of this bug is that the `Aggregation` strategy replaces a foldable boolean `ignoreNullsExpr` expr with a `Unevaluable` expr (`AttributeReference`) for distinct FIRST/LAST aggregate functions. But, this operation cannot be allowed because the `Analyzer` has checked that it must be foldabe; https://github.com/apache/spark/blob/ffdbbae1d465fe2c710d020de62ca1a6b0b924d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala#L74-L76 So, this PR proposes to change a vriable for `IGNORE NULLS` from `Expression` to `Boolean` to avoid the case. ### Why are the changes needed? Bugfix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added a test in `DataFrameAggregateSuite`. Closes #29143 from maropu/SPARK-32344. Authored-by: Takeshi Yamamuro Signed-off-by: HyukjinKwon --- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../expressions/aggregate/First.scala | 28 +++++++++----- .../catalyst/expressions/aggregate/Last.scala | 19 +++++----- .../optimizer/RewriteDistinctAggregates.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 4 +- ...stSuite.scala => FirstLastTestSuite.scala} | 18 +++++++-- .../parser/ExpressionParserSuite.scala | 8 ++-- .../org/apache/spark/sql/functions.scala | 4 +- .../sql-functions/sql-expression-schema.md | 8 ++-- .../results/postgreSQL/window_part1.sql.out | 12 +++--- .../results/postgreSQL/window_part2.sql.out | 38 +++++++++---------- .../spark/sql/DataFrameAggregateSuite.scala | 7 ++++ 12 files changed, 89 insertions(+), 63 deletions(-) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/{LastTestSuite.scala => FirstLastTestSuite.scala} (84%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 023ef2ee17473..a77d4f1c6243b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -791,9 +791,9 @@ class Analyzer( // AggregateFunction's with the exception of First and Last in their default mode // (which we handle) and possibly some Hive UDAF's. case First(expr, _) => - First(ifExpr(expr), Literal(true)) + First(ifExpr(expr), true) case Last(expr, _) => - Last(ifExpr(expr), Literal(true)) + Last(ifExpr(expr), true) case a: AggregateFunction => a.withNewChildren(a.children.map(ifExpr)) }.transform { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala index df806edbfda05..65fd43c924d08 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckSuccess import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -49,12 +50,16 @@ import org.apache.spark.sql.types._ """, group = "agg_funcs", since = "2.0.0") -case class First(child: Expression, ignoreNullsExpr: Expression) +case class First(child: Expression, ignoreNulls: Boolean) extends DeclarativeAggregate with ExpectsInputTypes { - def this(child: Expression) = this(child, Literal.create(false, BooleanType)) + def this(child: Expression) = this(child, false) - override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil + def this(child: Expression, ignoreNullsExpr: Expression) = { + this(child, FirstLast.validateIgnoreNullExpr(ignoreNullsExpr, "first")) + } + + override def children: Seq[Expression] = child :: Nil override def nullable: Boolean = true @@ -71,16 +76,11 @@ case class First(child: Expression, ignoreNullsExpr: Expression) val defaultCheck = super.checkInputDataTypes() if (defaultCheck.isFailure) { defaultCheck - } else if (!ignoreNullsExpr.foldable) { - TypeCheckFailure( - s"The second argument of First must be a boolean literal, but got: ${ignoreNullsExpr.sql}") } else { TypeCheckSuccess } } - private def ignoreNulls: Boolean = ignoreNullsExpr.eval().asInstanceOf[Boolean] - private lazy val first = AttributeReference("first", child.dataType)() private lazy val valueSet = AttributeReference("valueSet", BooleanType)() @@ -120,3 +120,11 @@ case class First(child: Expression, ignoreNullsExpr: Expression) override def toString: String = s"$prettyName($child)${if (ignoreNulls) " ignore nulls"}" } + +object FirstLast { + def validateIgnoreNullExpr(exp: Expression, funcName: String): Boolean = exp match { + case Literal(b: Boolean, BooleanType) => b + case _ => throw new AnalysisException( + s"The second argument in $funcName should be a boolean literal.") + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index e55bced192f34..8d17a48a69f6f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckSuccess import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -49,12 +49,16 @@ import org.apache.spark.sql.types._ """, group = "agg_funcs", since = "2.0.0") -case class Last(child: Expression, ignoreNullsExpr: Expression) +case class Last(child: Expression, ignoreNulls: Boolean) extends DeclarativeAggregate with ExpectsInputTypes { - def this(child: Expression) = this(child, Literal.create(false, BooleanType)) + def this(child: Expression) = this(child, false) - override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil + def this(child: Expression, ignoreNullsExpr: Expression) = { + this(child, FirstLast.validateIgnoreNullExpr(ignoreNullsExpr, "last")) + } + + override def children: Seq[Expression] = child :: Nil override def nullable: Boolean = true @@ -71,16 +75,11 @@ case class Last(child: Expression, ignoreNullsExpr: Expression) val defaultCheck = super.checkInputDataTypes() if (defaultCheck.isFailure) { defaultCheck - } else if (!ignoreNullsExpr.foldable) { - TypeCheckFailure( - s"The second argument of Last must be a boolean literal, but got: ${ignoreNullsExpr.sql}") } else { TypeCheckSuccess } } - private def ignoreNulls: Boolean = ignoreNullsExpr.eval().asInstanceOf[Boolean] - private lazy val last = AttributeReference("last", child.dataType)() private lazy val valueSet = AttributeReference("valueSet", BooleanType)() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index e5571069a7c41..15aa02ff677de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -257,7 +257,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { // Select the result of the first aggregate in the last aggregate. val result = AggregateExpression( - aggregate.First(evalWithinGroup(regularGroupId, operator.toAttribute), Literal(true)), + aggregate.First(evalWithinGroup(regularGroupId, operator.toAttribute), true), mode = Complete, isDistinct = false) 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 740892dfdb1c6..13e528e7968ca 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 @@ -1535,7 +1535,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitFirst(ctx: FirstContext): Expression = withOrigin(ctx) { val ignoreNullsExpr = ctx.IGNORE != null - First(expression(ctx.expression), Literal(ignoreNullsExpr)).toAggregateExpression() + First(expression(ctx.expression), ignoreNullsExpr).toAggregateExpression() } /** @@ -1543,7 +1543,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitLast(ctx: LastContext): Expression = withOrigin(ctx) { val ignoreNullsExpr = ctx.IGNORE != null - Last(expression(ctx.expression), Literal(ignoreNullsExpr)).toAggregateExpression() + Last(expression(ctx.expression), ignoreNullsExpr).toAggregateExpression() } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala similarity index 84% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala index ba36bc074e154..bb6672e1046da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala @@ -17,14 +17,15 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Literal} import org.apache.spark.sql.types.IntegerType -class LastTestSuite extends SparkFunSuite { +class FirstLastTestSuite extends SparkFunSuite { val input = AttributeReference("input", IntegerType, nullable = true)() - val evaluator = DeclarativeAggregateEvaluator(Last(input, Literal(false)), Seq(input)) - val evaluatorIgnoreNulls = DeclarativeAggregateEvaluator(Last(input, Literal(true)), Seq(input)) + val evaluator = DeclarativeAggregateEvaluator(Last(input, false), Seq(input)) + val evaluatorIgnoreNulls = DeclarativeAggregateEvaluator(Last(input, true), Seq(input)) test("empty buffer") { assert(evaluator.initialize() === InternalRow(null, false)) @@ -106,4 +107,15 @@ class LastTestSuite extends SparkFunSuite { val m1 = evaluatorIgnoreNulls.merge(p1, p2) assert(evaluatorIgnoreNulls.eval(m1) === InternalRow(1)) } + + test("SPARK-32344: correct error handling for a type mismatch") { + val msg1 = intercept[AnalysisException] { + new First(input, Literal(1, IntegerType)) + }.getMessage + assert(msg1.contains("The second argument in first should be a boolean literal")) + val msg2 = intercept[AnalysisException] { + new Last(input, Literal(1, IntegerType)) + }.getMessage + assert(msg2.contains("The second argument in last should be a boolean literal")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 41652cab2675b..bcbdf5df57d0e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -785,10 +785,10 @@ class ExpressionParserSuite extends AnalysisTest { } test("SPARK-19526 Support ignore nulls keywords for first and last") { - assertEqual("first(a ignore nulls)", First('a, Literal(true)).toAggregateExpression()) - assertEqual("first(a)", First('a, Literal(false)).toAggregateExpression()) - assertEqual("last(a ignore nulls)", Last('a, Literal(true)).toAggregateExpression()) - assertEqual("last(a)", Last('a, Literal(false)).toAggregateExpression()) + assertEqual("first(a ignore nulls)", First('a, true).toAggregateExpression()) + assertEqual("first(a)", First('a, false).toAggregateExpression()) + assertEqual("last(a ignore nulls)", Last('a, true).toAggregateExpression()) + assertEqual("last(a)", Last('a, false).toAggregateExpression()) } test("timestamp literals") { 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 f9e6e2f7b359c..d8e39d416d877 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 @@ -461,7 +461,7 @@ object functions { * @since 2.0.0 */ def first(e: Column, ignoreNulls: Boolean): Column = withAggregateFunction { - new First(e.expr, Literal(ignoreNulls)) + First(e.expr, ignoreNulls) } /** @@ -586,7 +586,7 @@ object functions { * @since 2.0.0 */ def last(e: Column, ignoreNulls: Boolean): Column = withAggregateFunction { - new Last(e.expr, Literal(ignoreNulls)) + new Last(e.expr, ignoreNulls) } /** 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 8898a11ec08fb..e8169b9f06ce2 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 @@ -315,12 +315,12 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.CountMinSketchAgg | count_min_sketch | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.aggregate.CovPopulation | covar_pop | SELECT covar_pop(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.CovSample | covar_samp | SELECT covar_samp(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.First | first_value | SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.First | first | SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.First | first_value | SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.First | first | SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus | approx_count_distinct | SELECT approx_count_distinct(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Kurtosis | kurtosis | SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Max | max | SELECT max(col) FROM VALUES (10), (50), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.MaxBy | max_by | SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Min | min | SELECT min(col) FROM VALUES (10), (-1), (20) AS tab(col) | struct | diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out index 2b1de87a6be5e..76567b689445a 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out @@ -270,7 +270,7 @@ struct +struct -- !query output 0 0 0 0 0 0 @@ -287,7 +287,7 @@ struct +struct -- !query output 0 4 0 1 1 1 @@ -306,7 +306,7 @@ SELECT last(ten) OVER (PARTITION BY four), ten, four FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s ORDER BY four, ten -- !query schema -struct +struct -- !query output 4 0 0 4 0 0 @@ -476,7 +476,7 @@ sum(ten) over (partition by four order by ten), last(ten) over (partition by four order by ten) FROM (select distinct ten, four from tenk1) ss -- !query schema -struct +struct -- !query output 0 0 0 0 0 2 2 2 @@ -506,7 +506,7 @@ sum(ten) over (partition by four order by ten range between unbounded preceding last(ten) over (partition by four order by ten range between unbounded preceding and current row) FROM (select distinct ten, four from tenk1) ss -- !query schema -struct +struct -- !query output 0 0 0 0 0 2 2 2 @@ -536,7 +536,7 @@ sum(ten) over (partition by four order by ten range between unbounded preceding last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) FROM (select distinct ten, four from tenk1) ss -- !query schema -struct +struct -- !query output 0 0 20 8 0 2 20 8 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out index f41659a196ae1..ccddf9db172a6 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -101,7 +101,7 @@ from window w as (order by ss.id asc nulls first range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 1 3 2 2 1 4 @@ -123,7 +123,7 @@ from window w as (order by ss.id asc nulls last range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 1 3 2 2 1 4 @@ -145,7 +145,7 @@ from window w as (order by ss.id desc nulls first range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 3 1 2 2 4 1 @@ -167,7 +167,7 @@ from window w as (order by ss.id desc nulls last range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 3 1 2 2 4 1 @@ -182,7 +182,7 @@ NULL 43 42 43 select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) from range(32764, 32767) x -- !query schema -struct +struct -- !query output 32764 32766 32765 32766 @@ -193,7 +193,7 @@ struct +struct -- !query output -32766 -32766 @@ -202,7 +202,7 @@ struct +struct -- !query output 2147483644 2147483646 2147483645 2147483646 @@ -213,7 +213,7 @@ struct +struct -- !query output -2147483646 -2147483646 @@ -272,7 +272,7 @@ from numerics window w as (order by f_float4 range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -289,7 +289,7 @@ from numerics window w as (order by f_float4 range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -306,7 +306,7 @@ from numerics window w as (order by f_float4 range between 'inf' preceding and 'inf' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 1 7 @@ -323,7 +323,7 @@ from numerics window w as (order by f_float4 range between 1.1 preceding and 'NaN' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 2 7 @@ -340,7 +340,7 @@ from numerics window w as (order by f_float8 range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -357,7 +357,7 @@ from numerics window w as (order by f_float8 range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -374,7 +374,7 @@ from numerics window w as (order by f_float8 range between 'inf' preceding and 'inf' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 1 7 @@ -391,7 +391,7 @@ from numerics window w as (order by f_float8 range between 1.1 preceding and 'NaN' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 2 7 @@ -408,7 +408,7 @@ from numerics window w as (order by f_numeric range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 -3 1 1 2 -1 2 3 @@ -425,7 +425,7 @@ from numerics window w as (order by f_numeric range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3 1 1 2 -1 2 3 @@ -442,7 +442,7 @@ from numerics window w as (order by f_numeric range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3 1 1 2 -1 2 3 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 09f30bb5e2c77..54327b38c100b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -1037,6 +1037,13 @@ class DataFrameAggregateSuite extends QueryTest val groupBy = df.groupBy("b").agg(count("*")) checkAnswer(groupBy, Row(null, 1) :: Row(Row(null), 1) :: Row(Row(1.0), 1) :: Nil) } + + test("SPARK-32344: Unevaluable's set to FIRST/LAST ignoreNullsExpr in distinct aggregates") { + val queryTemplate = (agg: String) => + s"SELECT $agg(DISTINCT v) FROM (SELECT v FROM VALUES 1, 2, 3 t(v) ORDER BY v)" + checkAnswer(sql(queryTemplate("FIRST")), Row(1)) + checkAnswer(sql(queryTemplate("LAST")), Row(3)) + } } case class B(c: Option[Double]) From 026b0b926dfd40038f2cee932f38b917eb25b77e Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sun, 19 Jul 2020 12:00:23 +0900 Subject: [PATCH 225/384] [SPARK-32253][INFRA] Show errors only for the sbt tests of github actions ### What changes were proposed in this pull request? Make the test result log of github action more readable by showing errors from SBT only. 1. Add "--error" flag to sbt in github action to set the log level as "ERROR" 2. Show only failed test cases in stderr output of github action. According to https://www.scalatest.org/user_guide/using_the_runner, with SBT option `-eNCXEHLOPQMDF ` we can drop all the following events: ``` N - drop TestStarting events C - drop TestSucceeded events X - drop TestIgnored events E - drop TestPending events H - drop SuiteStarting events L - drop SuiteCompleted events O - drop InfoProvided events P - drop ScopeOpened events Q - drop ScopeClosed events R - drop ScopePending events M - drop MarkupProvided events ``` and enable the following two mode: ``` D - show all durations F - show full stack traces ``` ### Why are the changes needed? Currently, the output of github action is very long and we have to scroll down to find the failed test cases. Even more, the log may be truncated. In such a case, we will have to wait until all the jobs are completed and then download all the raw logs. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Before changes, all the warnings in compiling are shown: ![image](https://user-images.githubusercontent.com/1097932/87846810-98ec8900-c887-11ea-913b-164b84df62cd.png) as well as all the passed and ignored test cases: ![image](https://user-images.githubusercontent.com/1097932/87846834-ca655480-c887-11ea-9c29-977f802e4c82.png) After changes, sbt test only shows the summary for a successful job: ![image](https://user-images.githubusercontent.com/1097932/87846961-e74e5780-c888-11ea-82d5-cf1da1740181.png) ![image](https://user-images.githubusercontent.com/1097932/87745273-5735e280-c7a2-11ea-8ac9-b4b0e3cb458d.png) If there is a test failure, a full stack track is shown as well as a test failure summary at the end of test log: ![image](https://user-images.githubusercontent.com/1097932/87751143-3aa1a680-c7b2-11ea-9d09-52637a322270.png) ![image](https://user-images.githubusercontent.com/1097932/87752404-1f846600-c7b5-11ea-8106-8ddaf3cc3f7e.png) Closes #29133 from gengliangwang/shortLog. Authored-by: Gengliang Wang Signed-off-by: HyukjinKwon --- dev/run-tests.py | 3 +++ project/SparkBuild.scala | 7 +++++++ 2 files changed, 10 insertions(+) diff --git a/dev/run-tests.py b/dev/run-tests.py index 065a27c0e853b..8e29f89c3a0d0 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -653,6 +653,9 @@ def main(): # If we're running the tests in Github Actions, attempt to detect and test # only the affected modules. if test_env == "github_actions": + # Set the log level of sbt as ERROR to make the output more readable. + if build_tool == "sbt": + extra_profiles.append("--error") if os.environ["GITHUB_BASE_REF"] != "": # Pull requests changed_files = identify_changed_files_from_git_commits( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d19b514d662fa..198405a1d29ca 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1027,6 +1027,13 @@ object TestSettings { }.getOrElse(Nil): _*), // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), + // Show only the failed test cases with full stack traces in github action to make the log more + // readable. + // Check https://www.scalatest.org/user_guide/using_the_runner for the details of options . + testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, + sys.env.get("GITHUB_ACTIONS").map { _ => + Seq("-eNCXEHLOPQMDF") + }.getOrElse(Nil): _*), testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), // Required to detect Junit tests for each project, see also https://github.com/sbt/junit-interface/issues/35 crossPaths := false, From 0aca1a6ed4cd7d0e5d231a40e7f3cc0331a3776a Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sun, 19 Jul 2020 12:11:26 -0700 Subject: [PATCH 226/384] [SPARK-32276][SQL] Remove redundant sorts before repartition nodes ### What changes were proposed in this pull request? This PR proposes to remove redundant sorts before repartition nodes whenever the data is ordered after the repartitioning. ### Why are the changes needed? It looks like our `EliminateSorts` rule can be extended further to remove sorts before repartition nodes that don't affect the final output ordering. It seems safe to perform the following rewrites: - `Sort -> Repartition -> Sort -> Scan` as `Sort -> Repartition -> Scan` - `Sort -> Repartition -> Project -> Sort -> Scan` as `Sort -> Repartition -> Project -> Scan` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? More test cases. Closes #29089 from aokolnychyi/spark-32276. Authored-by: Anton Okolnychyi Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/Optimizer.scala | 19 +- ...EliminateSortsBeforeRepartitionSuite.scala | 193 ++++++++++++++++++ 2 files changed, 206 insertions(+), 6 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 33da482c4eea4..a0e21ed86a71e 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 @@ -966,14 +966,19 @@ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { } /** - * Removes Sort operation. This can happen: + * Removes Sort operations if they don't affect the final output ordering. + * Note that changes in the final output ordering may affect the file size (SPARK-32318). + * This rule handles the following cases: * 1) if the sort order is empty or the sort order does not have any reference * 2) if the child is already sorted - * 3) if there is another Sort operator separated by 0...n Project/Filter operators - * 4) if the Sort operator is within Join separated by 0...n Project/Filter operators only, - * and the Join conditions is deterministic - * 5) if the Sort operator is within GroupBy separated by 0...n Project/Filter operators only, - * and the aggregate function is order irrelevant + * 3) if there is another Sort operator separated by 0...n Project, Filter, Repartition or + * RepartitionByExpression (with deterministic expressions) operators + * 4) if the Sort operator is within Join separated by 0...n Project, Filter, Repartition or + * RepartitionByExpression (with deterministic expressions) operators only and the Join condition + * is deterministic + * 5) if the Sort operator is within GroupBy separated by 0...n Project, Filter, Repartition or + * RepartitionByExpression (with deterministic expressions) operators only and the aggregate + * function is order irrelevant */ object EliminateSorts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -999,6 +1004,8 @@ object EliminateSorts extends Rule[LogicalPlan] { private def canEliminateSort(plan: LogicalPlan): Boolean = plan match { case p: Project => p.projectList.forall(_.deterministic) case f: Filter => f.condition.deterministic + case r: RepartitionByExpression => r.partitionExpressions.forall(_.deterministic) + case _: Repartition => true case _ => false } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala new file mode 100644 index 0000000000000..9f031358611b1 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class EliminateSortsBeforeRepartitionSuite extends PlanTest { + + val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) + val analyzer = new Analyzer(catalog, conf) + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val anotherTestRelation = LocalRelation('d.int, 'e.int) + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Default", FixedPoint(10), + FoldablePropagation, + LimitPushDown) :: + Batch("Eliminate Sorts", Once, + EliminateSorts) :: + Batch("Collapse Project", Once, + CollapseProject) :: Nil + } + + def repartition(plan: LogicalPlan): LogicalPlan = plan.repartition(10) + + test("sortBy") { + val plan = testRelation.select('a, 'b).sortBy('a.asc, 'b.desc) + val optimizedPlan = testRelation.select('a, 'b) + checkRepartitionCases(plan, optimizedPlan) + } + + test("sortBy with projection") { + val plan = testRelation.sortBy('a.asc, 'b.asc).select('a + 1 as "a", 'b + 2 as "b") + val optimizedPlan = testRelation.select('a + 1 as "a", 'b + 2 as "b") + checkRepartitionCases(plan, optimizedPlan) + } + + test("sortBy with projection and filter") { + val plan = testRelation.sortBy('a.asc, 'b.asc).select('a, 'b).where('a === 10) + val optimizedPlan = testRelation.select('a, 'b).where('a === 10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("sortBy with limit") { + val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + val optimizedPlan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("sortBy with non-deterministic projection") { + val plan = testRelation.sortBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + val optimizedPlan = testRelation.sortBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy") { + val plan = testRelation.select('a, 'b).orderBy('a.asc, 'b.asc) + val optimizedPlan = testRelation.select('a, 'b) + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy with projection") { + val plan = testRelation.orderBy('a.asc, 'b.asc).select('a + 1 as "a", 'b + 2 as "b") + val optimizedPlan = testRelation.select('a + 1 as "a", 'b + 2 as "b") + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy with projection and filter") { + val plan = testRelation.orderBy('a.asc, 'b.asc).select('a, 'b).where('a === 10) + val optimizedPlan = testRelation.select('a, 'b).where('a === 10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy with limit") { + val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + val optimizedPlan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy with non-deterministic projection") { + val plan = testRelation.orderBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + val optimizedPlan = testRelation.orderBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + checkRepartitionCases(plan, optimizedPlan) + } + + test("additional coalesce and sortBy") { + val plan = testRelation.sortBy('a.asc, 'b.asc).coalesce(1) + val optimizedPlan = testRelation.coalesce(1) + checkRepartitionCases(plan, optimizedPlan) + } + + test("additional projection, repartition and sortBy") { + val plan = testRelation.sortBy('a.asc, 'b.asc).repartition(100).select('a + 1 as "a") + val optimizedPlan = testRelation.repartition(100).select('a + 1 as "a") + checkRepartitionCases(plan, optimizedPlan) + } + + test("additional filter, distribute and sortBy") { + val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) + val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("join") { + val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) + val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) + val anotherPlan = anotherTestRelation.select('d) + val joinPlan = plan.join(anotherPlan) + val optimizedJoinPlan = optimize(joinPlan) + val correctJoinPlan = analyze(optimizedPlan.join(anotherPlan)) + comparePlans(optimizedJoinPlan, correctJoinPlan) + } + + test("aggregate") { + val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) + val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) + val aggPlan = plan.groupBy('a)(sum('b)) + val optimizedAggPlan = optimize(aggPlan) + val correctAggPlan = analyze(optimizedPlan.groupBy('a)(sum('b))) + comparePlans(optimizedAggPlan, correctAggPlan) + } + + protected def checkRepartitionCases(plan: LogicalPlan, optimizedPlan: LogicalPlan): Unit = { + // cannot remove sortBy before repartition without sortBy/orderBy + val planWithRepartition = repartition(plan) + val optimizedPlanWithRepartition = optimize(planWithRepartition) + val correctPlanWithRepartition = analyze(planWithRepartition) + comparePlans(optimizedPlanWithRepartition, correctPlanWithRepartition) + + // can remove sortBy before repartition with sortBy + val planWithRepartitionAndSortBy = planWithRepartition.sortBy('a.asc) + val optimizedPlanWithRepartitionAndSortBy = optimize(planWithRepartitionAndSortBy) + val correctPlanWithRepartitionAndSortBy = analyze(repartition(optimizedPlan).sortBy('a.asc)) + comparePlans(optimizedPlanWithRepartitionAndSortBy, correctPlanWithRepartitionAndSortBy) + + // can remove sortBy before repartition with orderBy + val planWithRepartitionAndOrderBy = planWithRepartition.orderBy('a.asc) + val optimizedPlanWithRepartitionAndOrderBy = optimize(planWithRepartitionAndOrderBy) + val correctPlanWithRepartitionAndOrderBy = analyze(repartition(optimizedPlan).orderBy('a.asc)) + comparePlans(optimizedPlanWithRepartitionAndOrderBy, correctPlanWithRepartitionAndOrderBy) + } + + private def analyze(plan: LogicalPlan): LogicalPlan = { + analyzer.execute(plan) + } + + private def optimize(plan: LogicalPlan): LogicalPlan = { + Optimize.execute(analyzer.execute(plan)) + } +} + +class EliminateSortsBeforeRepartitionByExprsSuite extends EliminateSortsBeforeRepartitionSuite { + override def repartition(plan: LogicalPlan): LogicalPlan = plan.distribute('a)(10) + + test("sortBy before repartition with non-deterministic expressions") { + val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) + checkRepartitionCases(plan = planWithRepartition, optimizedPlan = planWithRepartition) + } + + test("orderBy before repartition with non-deterministic expressions") { + val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) + checkRepartitionCases(plan = planWithRepartition, optimizedPlan = planWithRepartition) + } +} + +class EliminateSortsBeforeCoalesceSuite extends EliminateSortsBeforeRepartitionSuite { + override def repartition(plan: LogicalPlan): LogicalPlan = plan.coalesce(1) +} From 32a0451376ab775fdd4ac364388e46179d9ee550 Mon Sep 17 00:00:00 2001 From: Igor Dvorzhak Date: Sun, 19 Jul 2020 12:19:36 -0700 Subject: [PATCH 227/384] [MINOR][DOCS] Fix links to Cloud Storage connectors docs Closes #29155 from medb/patch-1. Authored-by: Igor Dvorzhak Signed-off-by: Dongjoon Hyun --- docs/cloud-integration.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 749d026528017..1eaa8ab4eeee0 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -163,7 +163,7 @@ different stores and connectors when renaming directories: | Amazon S3 | s3a | Unsafe | O(data) | | Azure Storage | wasb | Safe | O(files) | | Azure Datalake Gen 2 | abfs | Safe | O(1) | -| Google GCS | gs | Safe | O(1) | +| Google Cloud Storage | gs | Safe | O(1) | As storing temporary files can run up charges; delete directories called `"_temporary"` on a regular basis. @@ -253,9 +253,9 @@ Here is the documentation on the standard connectors both from Apache and the cl * [Azure Blob Storage and Azure Datalake Gen 2](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). * [Azure Data Lake Gen 1](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). * [Hadoop-AWS module (Hadoop 3.x)](https://hadoop.apache.org/docs/current3/hadoop-aws/tools/hadoop-aws/index.html). -* [Amazon S3 via S3A and S3N (Hadoop 2.x)](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). -* [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon -* [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google +* [Amazon S3 via S3A and S3N (Hadoop 2.x)](https://hadoop.apache.org/docs/current2/hadoop-aws/tools/hadoop-aws/index.html). +* [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon. +* [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/dataproc/docs/concepts/connectors/cloud-storage). From Google. * [The Azure Blob Filesystem driver (ABFS)](https://docs.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-abfs-driver) -* IBM Cloud Object Storage connector for Apache Spark: [Stocator](https://github.com/CODAIT/stocator), [IBM Object Storage](https://www.ibm.com/cloud/object-storage). From IBM +* IBM Cloud Object Storage connector for Apache Spark: [Stocator](https://github.com/CODAIT/stocator), [IBM Object Storage](https://www.ibm.com/cloud/object-storage). From IBM. From ef3cad17a6c31707e537f3d7be9183fe26f1c60b Mon Sep 17 00:00:00 2001 From: zero323 Date: Mon, 20 Jul 2020 10:42:33 +0900 Subject: [PATCH 228/384] [SPARK-29157][SQL][PYSPARK] Add DataFrameWriterV2 to Python API ### What changes were proposed in this pull request? - Adds `DataFramWriterV2` class. - Adds `writeTo` method to `pyspark.sql.DataFrame`. - Adds related SQL partitioning functions (`years`, `months`, ..., `bucket`). ### Why are the changes needed? Feature parity. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added new unit tests. TODO: Should we test against `org.apache.spark.sql.connector.InMemoryTableCatalog`? If so, how to expose it in Python tests? Closes #27331 from zero323/SPARK-29157. Authored-by: zero323 Signed-off-by: HyukjinKwon --- python/pyspark/sql/dataframe.py | 18 ++- python/pyspark/sql/functions.py | 112 ++++++++++++++++ python/pyspark/sql/readwriter.py | 141 +++++++++++++++++++- python/pyspark/sql/tests/test_readwriter.py | 36 +++++ 4 files changed, 305 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 023fbeabcbabc..1027918adbe15 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -29,7 +29,7 @@ from pyspark.traceback_utils import SCCallSiteSync from pyspark.sql.types import _parse_datatype_json_string from pyspark.sql.column import Column, _to_seq, _to_list, _to_java_column -from pyspark.sql.readwriter import DataFrameWriter +from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from pyspark.sql.streaming import DataStreamWriter from pyspark.sql.types import * from pyspark.sql.pandas.conversion import PandasConversionMixin @@ -2240,6 +2240,22 @@ def inputFiles(self): sinceversion=1.4, doc=":func:`drop_duplicates` is an alias for :func:`dropDuplicates`.") + @since(3.1) + def writeTo(self, table): + """ + Create a write configuration builder for v2 sources. + + This builder is used to configure and execute write operations. + + For example, to append or create or replace existing tables. + + >>> df.writeTo("catalog.db.table").append() # doctest: +SKIP + >>> df.writeTo( # doctest: +SKIP + ... "catalog.db.table" + ... ).partitionedBy("col").createOrReplace() + """ + return DataFrameWriterV2(self, table) + def _to_scala_map(sc, jm): """ diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 5a352104c4eca..3ca4edafa6873 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -3322,6 +3322,118 @@ def map_zip_with(col1, col2, f): return _invoke_higher_order_function("MapZipWith", [col1, col2], [f]) +# ---------------------- Partition transform functions -------------------------------- + +@since(3.1) +def years(col): + """ + Partition transform function: A transform for timestamps and dates + to partition data into years. + + >>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP + ... years("ts") + ... ).createOrReplace() + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.years(_to_java_column(col))) + + +@since(3.1) +def months(col): + """ + Partition transform function: A transform for timestamps and dates + to partition data into months. + + >>> df.writeTo("catalog.db.table").partitionedBy( + ... months("ts") + ... ).createOrReplace() # doctest: +SKIP + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.months(_to_java_column(col))) + + +@since(3.1) +def days(col): + """ + Partition transform function: A transform for timestamps and dates + to partition data into days. + + >>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP + ... days("ts") + ... ).createOrReplace() + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.days(_to_java_column(col))) + + +@since(3.1) +def hours(col): + """ + Partition transform function: A transform for timestamps + to partition data into hours. + + >>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP + ... hours("ts") + ... ).createOrReplace() + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.hours(_to_java_column(col))) + + +@since(3.1) +def bucket(numBuckets, col): + """ + Partition transform function: A transform for any type that partitions + by a hash of the input column. + + >>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP + ... bucket(42, "ts") + ... ).createOrReplace() + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + if not isinstance(numBuckets, (int, Column)): + raise TypeError( + "numBuckets should be a Column or and int, got {}".format(type(numBuckets)) + ) + + sc = SparkContext._active_spark_context + numBuckets = ( + _create_column_from_literal(numBuckets) + if isinstance(numBuckets, int) + else _to_java_column(numBuckets) + ) + return Column(sc._jvm.functions.bucket(numBuckets, _to_java_column(col))) + + # ---------------------------- User Defined Function ---------------------------------- @since(1.3) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index a83aece2e485d..6925adf567fb6 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -18,7 +18,7 @@ from py4j.java_gateway import JavaClass from pyspark import RDD, since -from pyspark.sql.column import _to_seq +from pyspark.sql.column import _to_seq, _to_java_column from pyspark.sql.types import * from pyspark.sql import utils from pyspark.sql.utils import to_str @@ -1075,6 +1075,145 @@ def jdbc(self, url, table, mode=None, properties=None): self.mode(mode)._jwrite.jdbc(url, table, jprop) +class DataFrameWriterV2(object): + """ + Interface used to write a class:`pyspark.sql.dataframe.DataFrame` + to external storage using the v2 API. + + .. versionadded:: 3.1.0 + """ + + def __init__(self, df, table): + self._df = df + self._spark = df.sql_ctx + self._jwriter = df._jdf.writeTo(table) + + @since(3.1) + def using(self, provider): + """ + Specifies a provider for the underlying output data source. + Spark's default catalog supports "parquet", "json", etc. + """ + self._jwriter.using(provider) + return self + + @since(3.1) + def option(self, key, value): + """ + Add a write option. + """ + self._jwriter.option(key, to_str(value)) + return self + + @since(3.1) + def options(self, **options): + """ + Add write options. + """ + options = {k: to_str(v) for k, v in options.items()} + self._jwriter.options(options) + return self + + @since(3.1) + def tableProperty(self, property, value): + """ + Add table property. + """ + self._jwriter.tableProperty(property, value) + return self + + @since(3.1) + def partitionedBy(self, col, *cols): + """ + Partition the output table created by `create`, `createOrReplace`, or `replace` using + the given columns or transforms. + + When specified, the table data will be stored by these values for efficient reads. + + For example, when a table is partitioned by day, it may be stored + in a directory layout like: + + * `table/day=2019-06-01/` + * `table/day=2019-06-02/` + + Partitioning is one of the most widely used techniques to optimize physical data layout. + It provides a coarse-grained index for skipping unnecessary data reads when queries have + predicates on the partitioned columns. In order for partitioning to work well, the number + of distinct values in each column should typically be less than tens of thousands. + + `col` and `cols` support only the following functions: + + * :py:func:`pyspark.sql.functions.years` + * :py:func:`pyspark.sql.functions.months` + * :py:func:`pyspark.sql.functions.days` + * :py:func:`pyspark.sql.functions.hours` + * :py:func:`pyspark.sql.functions.bucket` + + """ + col = _to_java_column(col) + cols = _to_seq(self._spark._sc, [_to_java_column(c) for c in cols]) + return self + + @since(3.1) + def create(self): + """ + Create a new table from the contents of the data frame. + + The new table's schema, partition layout, properties, and other configuration will be + based on the configuration set on this writer. + """ + self._jwriter.create() + + @since(3.1) + def replace(self): + """ + Replace an existing table with the contents of the data frame. + + The existing table's schema, partition layout, properties, and other configuration will be + replaced with the contents of the data frame and the configuration set on this writer. + """ + self._jwriter.replace() + + @since(3.1) + def createOrReplace(self): + """ + Create a new table or replace an existing table with the contents of the data frame. + + The output table's schema, partition layout, properties, + and other configuration will be based on the contents of the data frame + and the configuration set on this writer. + If the table exists, its configuration and data will be replaced. + """ + self._jwriter.createOrReplace() + + @since(3.1) + def append(self): + """ + Append the contents of the data frame to the output table. + """ + self._jwriter.append() + + @since(3.1) + def overwrite(self, condition): + """ + Overwrite rows matching the given filter condition with the contents of the data frame in + the output table. + """ + condition = _to_java_column(column) + self._jwriter.overwrite(condition) + + @since(3.1) + def overwritePartitions(self): + """ + Overwrite all partition for which the data frame contains at least one row with the contents + of the data frame in the output table. + + This operation is equivalent to Hive's `INSERT OVERWRITE ... PARTITION`, which replaces + partitions dynamically depending on the contents of the data frame. + """ + self._jwriter.overwritePartitions() + + def _test(): import doctest import os diff --git a/python/pyspark/sql/tests/test_readwriter.py b/python/pyspark/sql/tests/test_readwriter.py index 2530cc2ebf224..8e34d3865c9d8 100644 --- a/python/pyspark/sql/tests/test_readwriter.py +++ b/python/pyspark/sql/tests/test_readwriter.py @@ -19,6 +19,8 @@ import shutil import tempfile +from pyspark.sql.functions import col +from pyspark.sql.readwriter import DataFrameWriterV2 from pyspark.sql.types import * from pyspark.testing.sqlutils import ReusedSQLTestCase @@ -163,6 +165,40 @@ def test_insert_into(self): self.assertEqual(6, self.spark.sql("select * from test_table").count()) +class ReadwriterV2Tests(ReusedSQLTestCase): + def test_api(self): + df = self.df + writer = df.writeTo("testcat.t") + self.assertIsInstance(writer, DataFrameWriterV2) + self.assertIsInstance(writer.option("property", "value"), DataFrameWriterV2) + self.assertIsInstance(writer.options(property="value"), DataFrameWriterV2) + self.assertIsInstance(writer.using("source"), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy("id"), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(col("id")), DataFrameWriterV2) + self.assertIsInstance(writer.tableProperty("foo", "bar"), DataFrameWriterV2) + + def test_partitioning_functions(self): + import datetime + from pyspark.sql.functions import years, months, days, hours, bucket + + df = self.spark.createDataFrame( + [(1, datetime.datetime(2000, 1, 1), "foo")], + ("id", "ts", "value") + ) + + writer = df.writeTo("testcat.t") + + self.assertIsInstance(writer.partitionedBy(years("ts")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(months("ts")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(days("ts")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(hours("ts")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(bucket(11, "id")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(bucket(11, col("id"))), DataFrameWriterV2) + self.assertIsInstance( + writer.partitionedBy(bucket(3, "id"), hours(col("ts"))), DataFrameWriterV2 + ) + + if __name__ == "__main__": import unittest from pyspark.sql.tests.test_readwriter import * From a4ca355af8556e8c5948e492ef70ef0b48416dc4 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 19 Jul 2020 21:33:13 -0700 Subject: [PATCH 229/384] [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What is changed? This pull request adds the ability to migrate shuffle files during Spark's decommissioning. The design document associated with this change is at https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE . To allow this change the `MapOutputTracker` has been extended to allow the location of shuffle files to be updated with `updateMapOutput`. When a shuffle block is put, a block update message will be sent which triggers the `updateMapOutput`. Instead of rejecting remote puts of shuffle blocks `BlockManager` delegates the storage of shuffle blocks to it's shufflemanager's resolver (if supported). A new, experimental, trait is added for shuffle resolvers to indicate they handle remote putting of blocks. The existing block migration code is moved out into a separate file, and a producer/consumer model is introduced for migrating shuffle files from the host as quickly as possible while not overwhelming other executors. ### Why are the changes needed? Recomputting shuffle blocks can be expensive, we should take advantage of our decommissioning time to migrate these blocks. ### Does this PR introduce any user-facing change? This PR introduces two new configs parameters, `spark.storage.decommission.shuffleBlocks.enabled` & `spark.storage.decommission.rddBlocks.enabled` that control which blocks should be migrated during storage decommissioning. ### How was this patch tested? New unit test & expansion of the Spark on K8s decom test to assert that decommisioning with shuffle block migration means that the results are not recomputed even when the original executor is terminated. This PR is a cleaned-up version of the previous WIP PR I made https://github.com/apache/spark/pull/28331 (thanks to attilapiros for his very helpful reviewing on it :)). Closes #28708 from holdenk/SPARK-20629-copy-shuffle-data-when-nodes-are-being-shutdown-cleaned-up. Lead-authored-by: Holden Karau Co-authored-by: Holden Karau Co-authored-by: “attilapiros” Co-authored-by: Attila Zsolt Piros Signed-off-by: Holden Karau --- .../org/apache/spark/MapOutputTracker.scala | 38 +- .../scala/org/apache/spark/SparkEnv.scala | 3 +- .../spark/internal/config/package.scala | 23 ++ .../netty/NettyBlockTransferService.scala | 5 +- .../apache/spark/scheduler/MapStatus.scala | 15 +- .../cluster/StandaloneSchedulerBackend.scala | 2 +- .../shuffle/IndexShuffleBlockResolver.scala | 99 +++++- .../spark/shuffle/MigratableResolver.scala | 48 +++ .../spark/shuffle/ShuffleBlockInfo.scala | 28 ++ .../org/apache/spark/storage/BlockId.scala | 5 +- .../apache/spark/storage/BlockManager.scala | 154 +++----- .../storage/BlockManagerDecommissioner.scala | 330 ++++++++++++++++++ .../storage/BlockManagerMasterEndpoint.scala | 26 +- ...nDecommissionedBlockManagerException.scala | 21 ++ .../scheduler/WorkerDecommissionSuite.scala | 2 +- .../sort/IndexShuffleBlockResolverSuite.scala | 3 +- .../apache/spark/storage/BlockIdSuite.scala | 4 +- ...kManagerDecommissionIntegrationSuite.scala | 229 ++++++++++++ .../BlockManagerDecommissionSuite.scala | 106 ------ .../BlockManagerDecommissionUnitSuite.scala | 92 +++++ .../BlockManagerReplicationSuite.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 93 ++++- .../integrationtest/DecommissionSuite.scala | 13 +- .../k8s/integrationtest/KubernetesSuite.scala | 35 +- .../tests/decommissioning.py | 27 +- .../streaming/ReceivedBlockHandlerSuite.scala | 2 +- 26 files changed, 1150 insertions(+), 255 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala create mode 100644 core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala create mode 100644 core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 32251df6f4bbe..64102ccc05882 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -49,7 +49,7 @@ import org.apache.spark.util._ * * All public methods of this class are thread-safe. */ -private class ShuffleStatus(numPartitions: Int) { +private class ShuffleStatus(numPartitions: Int) extends Logging { private val (readLock, writeLock) = { val lock = new ReentrantReadWriteLock() @@ -121,12 +121,28 @@ private class ShuffleStatus(numPartitions: Int) { mapStatuses(mapIndex) = status } + /** + * Update the map output location (e.g. during migration). + */ + def updateMapOutput(mapId: Long, bmAddress: BlockManagerId): Unit = withWriteLock { + val mapStatusOpt = mapStatuses.find(_.mapId == mapId) + mapStatusOpt match { + case Some(mapStatus) => + logInfo(s"Updating map output for ${mapId} to ${bmAddress}") + mapStatus.updateLocation(bmAddress) + invalidateSerializedMapOutputStatusCache() + case None => + logError(s"Asked to update map output ${mapId} for untracked map status.") + } + } + /** * Remove the map output which was served by the specified block manager. * This is a no-op if there is no registered map output or if the registered output is from a * different block manager. */ def removeMapOutput(mapIndex: Int, bmAddress: BlockManagerId): Unit = withWriteLock { + logDebug(s"Removing existing map output ${mapIndex} ${bmAddress}") if (mapStatuses(mapIndex) != null && mapStatuses(mapIndex).location == bmAddress) { _numAvailableOutputs -= 1 mapStatuses(mapIndex) = null @@ -139,6 +155,7 @@ private class ShuffleStatus(numPartitions: Int) { * outputs which are served by an external shuffle server (if one exists). */ def removeOutputsOnHost(host: String): Unit = withWriteLock { + logDebug(s"Removing outputs for host ${host}") removeOutputsByFilter(x => x.host == host) } @@ -148,6 +165,7 @@ private class ShuffleStatus(numPartitions: Int) { * still registered with that execId. */ def removeOutputsOnExecutor(execId: String): Unit = withWriteLock { + logDebug(s"Removing outputs for execId ${execId}") removeOutputsByFilter(x => x.executorId == execId) } @@ -265,7 +283,7 @@ private[spark] class MapOutputTrackerMasterEndpoint( override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = context.senderAddress.hostPort - logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) + logInfo(s"Asked to send map output locations for shuffle ${shuffleId} to ${hostPort}") tracker.post(new GetMapOutputMessage(shuffleId, context)) case StopMapOutputTracker => @@ -465,6 +483,15 @@ private[spark] class MapOutputTrackerMaster( } } + def updateMapOutput(shuffleId: Int, mapId: Long, bmAddress: BlockManagerId): Unit = { + shuffleStatuses.get(shuffleId) match { + case Some(shuffleStatus) => + shuffleStatus.updateMapOutput(mapId, bmAddress) + case None => + logError(s"Asked to update map output for unknown shuffle ${shuffleId}") + } + } + def registerMapOutput(shuffleId: Int, mapIndex: Int, status: MapStatus): Unit = { shuffleStatuses(shuffleId).addMapOutput(mapIndex, status) } @@ -745,7 +772,12 @@ private[spark] class MapOutputTrackerMaster( override def stop(): Unit = { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() - sendTracker(StopMapOutputTracker) + try { + sendTracker(StopMapOutputTracker) + } catch { + case e: SparkException => + logError("Could not tell tracker we are stopping.", e) + } trackerEndpoint = null shuffleStatuses.clear() } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8ba1739831803..d543359f4dedf 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -367,7 +367,8 @@ object SparkEnv extends Logging { externalShuffleClient } else { None - }, blockManagerInfo)), + }, blockManagerInfo, + mapOutputTracker.asInstanceOf[MapOutputTrackerMaster])), registerOrLookupEndpoint( BlockManagerMaster.DRIVER_HEARTBEAT_ENDPOINT_NAME, new BlockManagerMasterHeartbeatEndpoint(rpcEnv, isLocal, blockManagerInfo)), diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index f0b292b71eef7..e1b598e670495 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -420,6 +420,29 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED = + ConfigBuilder("spark.storage.decommission.shuffleBlocks.enabled") + .doc("Whether to transfer shuffle blocks during block manager decommissioning. Requires " + + "a migratable shuffle resolver (like sort based shuffe)") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + + private[spark] val STORAGE_DECOMMISSION_SHUFFLE_MAX_THREADS = + ConfigBuilder("spark.storage.decommission.shuffleBlocks.maxThreads") + .doc("Maximum number of threads to use in migrating shuffle files.") + .version("3.1.0") + .intConf + .checkValue(_ > 0, "The maximum number of threads should be positive") + .createWithDefault(8) + + private[spark] val STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED = + ConfigBuilder("spark.storage.decommission.rddBlocks.enabled") + .doc("Whether to transfer RDD blocks during block manager decommissioning.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + private[spark] val STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK = ConfigBuilder("spark.storage.decommission.maxReplicationFailuresPerBlock") .internal() diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 3de7377f99202..5d9cea068b097 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -168,7 +168,10 @@ private[spark] class NettyBlockTransferService( // Everything else is encoded using our binary protocol. val metadata = JavaUtils.bufferToArray(serializer.newInstance().serialize((level, classTag))) - val asStream = blockData.size() > conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) + // We always transfer shuffle blocks as a stream for simplicity with the receiving code since + // they are always written to disk. Otherwise we check the block size. + val asStream = (blockData.size() > conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) || + blockId.isShuffle) val callback = new RpcResponseCallback { override def onSuccess(response: ByteBuffer): Unit = { logTrace(s"Successfully uploaded block $blockId${if (asStream) " as stream" else ""}") diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 7f8893ff3b9d8..0af3a2e171906 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -30,12 +30,15 @@ import org.apache.spark.util.Utils /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the - * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. + * task has shuffle files stored on as well as the sizes of outputs for each reducer, for passing + * on to the reduce tasks. */ private[spark] sealed trait MapStatus { - /** Location where this task was run. */ + /** Location where this task output is. */ def location: BlockManagerId + def updateLocation(newLoc: BlockManagerId): Unit + /** * Estimated size for the reduce block, in bytes. * @@ -126,6 +129,10 @@ private[spark] class CompressedMapStatus( override def location: BlockManagerId = loc + override def updateLocation(newLoc: BlockManagerId): Unit = { + loc = newLoc + } + override def getSizeForBlock(reduceId: Int): Long = { MapStatus.decompressSize(compressedSizes(reduceId)) } @@ -178,6 +185,10 @@ private[spark] class HighlyCompressedMapStatus private ( override def location: BlockManagerId = loc + override def updateLocation(newLoc: BlockManagerId): Unit = { + loc = newLoc + } + override def getSizeForBlock(reduceId: Int): Long = { assert(hugeBlockSizes != null) if (emptyBlocks.contains(reduceId)) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index ec1299a924b5c..4024b44bdfd2f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -44,7 +44,7 @@ private[spark] class StandaloneSchedulerBackend( with StandaloneAppClientListener with Logging { - private var client: StandaloneAppClient = null + private[spark] var client: StandaloneAppClient = null private val stopping = new AtomicBoolean(false) private val launcherBackend = new LauncherBackend() { override protected def conf: SparkConf = sc.conf diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index af2c82e771970..0d0dad6d77ac1 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle import java.io._ +import java.nio.ByteBuffer import java.nio.channels.Channels import java.nio.file.Files @@ -25,8 +26,10 @@ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.client.StreamCallbackWithID import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExecutorDiskUtils +import org.apache.spark.serializer.SerializerManager import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -44,9 +47,10 @@ import org.apache.spark.util.Utils // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData(). private[spark] class IndexShuffleBlockResolver( conf: SparkConf, - _blockManager: BlockManager = null) + // var for testing + var _blockManager: BlockManager = null) extends ShuffleBlockResolver - with Logging { + with Logging with MigratableResolver { private lazy val blockManager = Option(_blockManager).getOrElse(SparkEnv.get.blockManager) @@ -55,6 +59,19 @@ private[spark] class IndexShuffleBlockResolver( def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None) + /** + * Get the shuffle files that are stored locally. Used for block migrations. + */ + override def getStoredShuffles(): Seq[ShuffleBlockInfo] = { + val allBlocks = blockManager.diskBlockManager.getAllBlocks() + allBlocks.flatMap { + case ShuffleIndexBlockId(shuffleId, mapId, _) => + Some(ShuffleBlockInfo(shuffleId, mapId)) + case _ => + None + } + } + /** * Get the shuffle data file. * @@ -148,6 +165,82 @@ private[spark] class IndexShuffleBlockResolver( } } + /** + * Write a provided shuffle block as a stream. Used for block migrations. + * ShuffleBlockBatchIds must contain the full range represented in the ShuffleIndexBlock. + * Requires the caller to delete any shuffle index blocks where the shuffle block fails to + * put. + */ + override def putShuffleBlockAsStream(blockId: BlockId, serializerManager: SerializerManager): + StreamCallbackWithID = { + val file = blockId match { + case ShuffleIndexBlockId(shuffleId, mapId, _) => + getIndexFile(shuffleId, mapId) + case ShuffleDataBlockId(shuffleId, mapId, _) => + getDataFile(shuffleId, mapId) + case _ => + throw new IllegalStateException(s"Unexpected shuffle block transfer ${blockId} as " + + s"${blockId.getClass().getSimpleName()}") + } + val fileTmp = Utils.tempFileWith(file) + val channel = Channels.newChannel( + serializerManager.wrapStream(blockId, + new FileOutputStream(fileTmp))) + + new StreamCallbackWithID { + + override def getID: String = blockId.name + + override def onData(streamId: String, buf: ByteBuffer): Unit = { + while (buf.hasRemaining) { + channel.write(buf) + } + } + + override def onComplete(streamId: String): Unit = { + logTrace(s"Done receiving shuffle block $blockId, now storing on local disk.") + channel.close() + val diskSize = fileTmp.length() + this.synchronized { + if (file.exists()) { + file.delete() + } + if (!fileTmp.renameTo(file)) { + throw new IOException(s"fail to rename file ${fileTmp} to ${file}") + } + } + blockManager.reportBlockStatus(blockId, BlockStatus(StorageLevel.DISK_ONLY, 0, diskSize)) + } + + override def onFailure(streamId: String, cause: Throwable): Unit = { + // the framework handles the connection itself, we just need to do local cleanup + logWarning(s"Error while uploading $blockId", cause) + channel.close() + fileTmp.delete() + } + } + } + + /** + * Get the index & data block for migration. + */ + def getMigrationBlocks(shuffleBlockInfo: ShuffleBlockInfo): List[(BlockId, ManagedBuffer)] = { + val shuffleId = shuffleBlockInfo.shuffleId + val mapId = shuffleBlockInfo.mapId + // Load the index block + val indexFile = getIndexFile(shuffleId, mapId) + val indexBlockId = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + val indexFileSize = indexFile.length() + val indexBlockData = new FileSegmentManagedBuffer(transportConf, indexFile, 0, indexFileSize) + + // Load the data block + val dataFile = getDataFile(shuffleId, mapId) + val dataBlockId = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + val dataBlockData = new FileSegmentManagedBuffer(transportConf, dataFile, 0, dataFile.length()) + List((indexBlockId, indexBlockData), (dataBlockId, dataBlockData)) + } + + /** * Write an index file with the offsets of each block, plus a final offset at the end for the * end of the output file. This will be used by getBlockData to figure out where each block @@ -169,7 +262,7 @@ private[spark] class IndexShuffleBlockResolver( val dataFile = getDataFile(shuffleId, mapId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. - synchronized { + this.synchronized { val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) if (existingLengths != null) { // Another attempt for the same task has already written our map outputs successfully, diff --git a/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala new file mode 100644 index 0000000000000..3851fa6eca19b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.client.StreamCallbackWithID +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.storage.BlockId + +/** + * :: Experimental :: + * An experimental trait to allow Spark to migrate shuffle blocks. + */ +@Experimental +@Since("3.1.0") +trait MigratableResolver { + /** + * Get the shuffle ids that are stored locally. Used for block migrations. + */ + def getStoredShuffles(): Seq[ShuffleBlockInfo] + + /** + * Write a provided shuffle block as a stream. Used for block migrations. + */ + def putShuffleBlockAsStream(blockId: BlockId, serializerManager: SerializerManager): + StreamCallbackWithID + + /** + * Get the blocks for migration for a particular shuffle and map. + */ + def getMigrationBlocks(shuffleBlockInfo: ShuffleBlockInfo): List[(BlockId, ManagedBuffer)] +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala new file mode 100644 index 0000000000000..99ceee81d349d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * An experimental case class used by MigratableResolver to return the shuffleId and mapId in a + * type safe way. + */ +@Experimental +case class ShuffleBlockInfo(shuffleId: Int, mapId: Long) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 68ed3aa5b062f..7b084e73c92f9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -38,7 +38,10 @@ sealed abstract class BlockId { // convenience methods def asRDDId: Option[RDDBlockId] = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None def isRDD: Boolean = isInstanceOf[RDDBlockId] - def isShuffle: Boolean = isInstanceOf[ShuffleBlockId] || isInstanceOf[ShuffleBlockBatchId] + def isShuffle: Boolean = { + (isInstanceOf[ShuffleBlockId] || isInstanceOf[ShuffleBlockBatchId] || + isInstanceOf[ShuffleDataBlockId] || isInstanceOf[ShuffleIndexBlockId]) + } def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId] override def toString: String = name diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6eec288015380..47af854b6e8ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -24,6 +24,7 @@ import java.nio.channels.Channels import java.util.Collections import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.HashMap import scala.concurrent.{ExecutionContext, Future} @@ -53,6 +54,7 @@ import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.serializer.{SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.{MigratableResolver, ShuffleManager, ShuffleWriteMetricsReporter} import org.apache.spark.shuffle.{ShuffleManager, ShuffleWriteMetricsReporter} import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock import org.apache.spark.storage.memory._ @@ -242,8 +244,8 @@ private[spark] class BlockManager( private var blockReplicationPolicy: BlockReplicationPolicy = _ - private var blockManagerDecommissioning: Boolean = false - private var decommissionManager: Option[BlockManagerDecommissionManager] = None + // This is volatile since if it's defined we should not accept remote blocks. + @volatile private var decommissioner: Option[BlockManagerDecommissioner] = None // A DownloadFileManager used to track all the files of remote blocks which are above the // specified memory threshold. Files will be deleted automatically based on weak reference. @@ -254,6 +256,15 @@ private[spark] class BlockManager( var hostLocalDirManager: Option[HostLocalDirManager] = None + @inline final private def isDecommissioning() = { + decommissioner.isDefined + } + // This is a lazy val so someone can migrating RDDs even if they don't have a MigratableResolver + // for shuffles. Used in BlockManagerDecommissioner & block puts. + private[storage] lazy val migratableResolver: MigratableResolver = { + shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] + } + /** * Abstraction for storing blocks from bytes, whether they start in memory or on disk. * @@ -364,7 +375,7 @@ private[spark] class BlockManager( ThreadUtils.awaitReady(replicationFuture, Duration.Inf) } catch { case NonFatal(t) => - throw new Exception("Error occurred while waiting for replication to finish", t) + throw new SparkException("Error occurred while waiting for replication to finish", t) } } if (blockWasSuccessfullyStored) { @@ -617,6 +628,7 @@ private[spark] class BlockManager( */ override def getLocalBlockData(blockId: BlockId): ManagedBuffer = { if (blockId.isShuffle) { + logInfo(s"Getting local shuffle block ${blockId}") shuffleManager.shuffleBlockResolver.getBlockData(blockId) } else { getLocalBytes(blockId) match { @@ -650,6 +662,23 @@ private[spark] class BlockManager( blockId: BlockId, level: StorageLevel, classTag: ClassTag[_]): StreamCallbackWithID = { + + if (isDecommissioning()) { + throw new BlockSavedOnDecommissionedBlockManagerException(blockId) + } + + if (blockId.isShuffle) { + logDebug(s"Putting shuffle block ${blockId}") + try { + return migratableResolver.putShuffleBlockAsStream(blockId, serializerManager) + } catch { + case e: ClassCastException => throw new SparkException( + s"Unexpected shuffle block ${blockId} with unsupported shuffle " + + s"resolver ${shuffleManager.shuffleBlockResolver}") + } + } + logDebug(s"Putting regular block ${blockId}") + // All other blocks val (_, tmpFile) = diskBlockManager.createTempLocalBlock() val channel = new CountingWritableChannel( Channels.newChannel(serializerManager.wrapForEncryption(new FileOutputStream(tmpFile)))) @@ -720,7 +749,7 @@ private[spark] class BlockManager( * it is still valid). This ensures that update in master will compensate for the increase in * memory on the storage endpoint. */ - private def reportBlockStatus( + private[spark] def reportBlockStatus( blockId: BlockId, status: BlockStatus, droppedMemorySize: Long = 0L): Unit = { @@ -1285,6 +1314,9 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + if (isDecommissioning()) { + throw new BlockSavedOnDecommissionedBlockManagerException(blockId) + } val putBlockInfo = { val newInfo = new BlockInfo(level, classTag, tellMaster) @@ -1540,7 +1572,7 @@ private[spark] class BlockManager( /** * Get peer block managers in the system. */ - private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { + private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { peerFetchLock.synchronized { val cachedPeersTtl = conf.get(config.STORAGE_CACHED_PEERS_TTL) // milliseconds val diff = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lastPeerFetchTimeNs) @@ -1779,59 +1811,19 @@ private[spark] class BlockManager( blocksToRemove.size } - def decommissionBlockManager(): Unit = { - if (!blockManagerDecommissioning) { - logInfo("Starting block manager decommissioning process") - blockManagerDecommissioning = true - decommissionManager = Some(new BlockManagerDecommissionManager(conf)) - decommissionManager.foreach(_.start()) - } else { - logDebug("Block manager already in decommissioning state") + def decommissionBlockManager(): Unit = synchronized { + decommissioner match { + case None => + logInfo("Starting block manager decommissioning process...") + decommissioner = Some(new BlockManagerDecommissioner(conf, this)) + decommissioner.foreach(_.start()) + case Some(_) => + logDebug("Block manager already in decommissioning state") } } - /** - * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers - * Visible for testing - */ - def decommissionRddCacheBlocks(): Unit = { - val replicateBlocksInfo = master.getReplicateInfoForRDDBlocks(blockManagerId) - - if (replicateBlocksInfo.nonEmpty) { - logInfo(s"Need to replicate ${replicateBlocksInfo.size} blocks " + - "for block manager decommissioning") - } else { - logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") - return - } - - // Maximum number of storage replication failure which replicateBlock can handle - val maxReplicationFailures = conf.get( - config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) - - // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) - // so that we end up prioritize them over each other - val blocksFailedReplication = replicateBlocksInfo.map { - case ReplicateBlock(blockId, existingReplicas, maxReplicas) => - val replicatedSuccessfully = replicateBlock( - blockId, - existingReplicas.toSet, - maxReplicas, - maxReplicationFailures = Some(maxReplicationFailures)) - if (replicatedSuccessfully) { - logInfo(s"Block $blockId offloaded successfully, Removing block now") - removeBlock(blockId) - logInfo(s"Block $blockId removed") - } else { - logWarning(s"Failed to offload block $blockId") - } - (blockId, replicatedSuccessfully) - }.filterNot(_._2).map(_._1) - if (blocksFailedReplication.nonEmpty) { - logWarning("Blocks failed replication in cache decommissioning " + - s"process: ${blocksFailedReplication.mkString(",")}") - } - } + private[storage] def getMigratableRDDBlocks(): Seq[ReplicateBlock] = + master.getReplicateInfoForRDDBlocks(blockManagerId) /** * Remove all blocks belonging to the given broadcast. @@ -1901,58 +1893,8 @@ private[spark] class BlockManager( data.dispose() } - /** - * Class to handle block manager decommissioning retries - * It creates a Thread to retry offloading all RDD cache blocks - */ - private class BlockManagerDecommissionManager(conf: SparkConf) { - @volatile private var stopped = false - private val sleepInterval = conf.get( - config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) - - private val blockReplicationThread = new Thread { - override def run(): Unit = { - var failures = 0 - while (blockManagerDecommissioning - && !stopped - && !Thread.interrupted() - && failures < 20) { - try { - logDebug("Attempting to replicate all cached RDD blocks") - decommissionRddCacheBlocks() - logInfo("Attempt to replicate all cached blocks done") - Thread.sleep(sleepInterval) - } catch { - case _: InterruptedException => - logInfo("Interrupted during migration, will not refresh migrations.") - stopped = true - case NonFatal(e) => - failures += 1 - logError("Error occurred while trying to replicate cached RDD blocks" + - s" for block manager decommissioning (failure count: $failures)", e) - } - } - } - } - blockReplicationThread.setDaemon(true) - blockReplicationThread.setName("block-replication-thread") - - def start(): Unit = { - logInfo("Starting block replication thread") - blockReplicationThread.start() - } - - def stop(): Unit = { - if (!stopped) { - stopped = true - logInfo("Stopping block replication thread") - blockReplicationThread.interrupt() - } - } - } - def stop(): Unit = { - decommissionManager.foreach(_.stop()) + decommissioner.foreach(_.stop()) blockTransferService.close() if (blockStoreClient ne blockTransferService) { // Closing should be idempotent, but maybe not for the NioBlockTransferService. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala new file mode 100644 index 0000000000000..1cc7ef6a25f92 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -0,0 +1,330 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.util.concurrent.ExecutorService + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.control.NonFatal + +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.shuffle.{MigratableResolver, ShuffleBlockInfo} +import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock +import org.apache.spark.util.ThreadUtils + +/** + * Class to handle block manager decommissioning retries. + * It creates a Thread to retry offloading all RDD cache and Shuffle blocks + */ +private[storage] class BlockManagerDecommissioner( + conf: SparkConf, + bm: BlockManager) extends Logging { + + private val maxReplicationFailuresForDecommission = + conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) + + /** + * This runnable consumes any shuffle blocks in the queue for migration. This part of a + * producer/consumer where the main migration loop updates the queue of blocks to be migrated + * periodically. On migration failure, the current thread will reinsert the block for another + * thread to consume. Each thread migrates blocks to a different particular executor to avoid + * distribute the blocks as quickly as possible without overwhelming any particular executor. + * + * There is no preference for which peer a given block is migrated to. + * This is notable different than the RDD cache block migration (further down in this file) + * which uses the existing priority mechanism for determining where to replicate blocks to. + * Generally speaking cache blocks are less impactful as they normally represent narrow + * transformations and we normally have less cache present than shuffle data. + * + * The producer/consumer model is chosen for shuffle block migration to maximize + * the chance of migrating all shuffle blocks before the executor is forced to exit. + */ + private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable { + @volatile var running = true + override def run(): Unit = { + var migrating: Option[(ShuffleBlockInfo, Int)] = None + logInfo(s"Starting migration thread for ${peer}") + // Once a block fails to transfer to an executor stop trying to transfer more blocks + try { + while (running && !Thread.interrupted()) { + migrating = Option(shufflesToMigrate.poll()) + migrating match { + case None => + logDebug("Nothing to migrate") + // Nothing to do right now, but maybe a transfer will fail or a new block + // will finish being committed. + val SLEEP_TIME_SECS = 1 + Thread.sleep(SLEEP_TIME_SECS * 1000L) + case Some((shuffleBlockInfo, retryCount)) => + if (retryCount < maxReplicationFailuresForDecommission) { + logInfo(s"Trying to migrate shuffle ${shuffleBlockInfo} to ${peer}") + val blocks = + bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo) + logDebug(s"Got migration sub-blocks ${blocks}") + blocks.foreach { case (blockId, buffer) => + logDebug(s"Migrating sub-block ${blockId}") + bm.blockTransferService.uploadBlockSync( + peer.host, + peer.port, + peer.executorId, + blockId, + buffer, + StorageLevel.DISK_ONLY, + null)// class tag, we don't need for shuffle + logDebug(s"Migrated sub block ${blockId}") + } + logInfo(s"Migrated ${shuffleBlockInfo} to ${peer}") + } else { + logError(s"Skipping block ${shuffleBlockInfo} because it has failed ${retryCount}") + } + } + } + // This catch is intentionally outside of the while running block. + // if we encounter errors migrating to an executor we want to stop. + } catch { + case e: Exception => + migrating match { + case Some((shuffleMap, retryCount)) => + logError(s"Error during migration, adding ${shuffleMap} back to migration queue", e) + shufflesToMigrate.add((shuffleMap, retryCount + 1)) + case None => + logError(s"Error while waiting for block to migrate", e) + } + } + } + } + + // Shuffles which are either in queue for migrations or migrated + private val migratingShuffles = mutable.HashSet[ShuffleBlockInfo]() + + // Shuffles which are queued for migration & number of retries so far. + private[storage] val shufflesToMigrate = + new java.util.concurrent.ConcurrentLinkedQueue[(ShuffleBlockInfo, Int)]() + + // Set if we encounter an error attempting to migrate and stop. + @volatile private var stopped = false + + private val migrationPeers = + mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]() + + private lazy val rddBlockMigrationExecutor = + ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission-rdd") + + private val rddBlockMigrationRunnable = new Runnable { + val sleepInterval = conf.get(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) + + override def run(): Unit = { + assert(conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) + while (!stopped && !Thread.interrupted()) { + logInfo("Iterating on migrating from the block manager.") + try { + logDebug("Attempting to replicate all cached RDD blocks") + decommissionRddCacheBlocks() + logInfo("Attempt to replicate all cached blocks done") + logInfo(s"Waiting for ${sleepInterval} before refreshing migrations.") + Thread.sleep(sleepInterval) + } catch { + case e: InterruptedException => + logInfo("Interrupted during migration, will not refresh migrations.") + stopped = true + case NonFatal(e) => + logError("Error occurred while trying to replicate for block manager decommissioning.", + e) + stopped = true + } + } + } + } + + private lazy val shuffleBlockMigrationRefreshExecutor = + ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission-shuffle") + + private val shuffleBlockMigrationRefreshRunnable = new Runnable { + val sleepInterval = conf.get(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) + + override def run() { + assert(conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) + while (!stopped && !Thread.interrupted()) { + try { + logDebug("Attempting to replicate all shuffle blocks") + refreshOffloadingShuffleBlocks() + logInfo("Done starting workers to migrate shuffle blocks") + Thread.sleep(sleepInterval) + } catch { + case e: InterruptedException => + logInfo("Interrupted during migration, will not refresh migrations.") + stopped = true + case NonFatal(e) => + logError("Error occurred while trying to replicate for block manager decommissioning.", + e) + stopped = true + } + } + } + } + + lazy val shuffleMigrationPool = ThreadUtils.newDaemonCachedThreadPool( + "migrate-shuffles", + conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_MAX_THREADS)) + + /** + * Tries to offload all shuffle blocks that are registered with the shuffle service locally. + * Note: this does not delete the shuffle files in-case there is an in-progress fetch + * but rather shadows them. + * Requires an Indexed based shuffle resolver. + * Note: if called in testing please call stopOffloadingShuffleBlocks to avoid thread leakage. + */ + private[storage] def refreshOffloadingShuffleBlocks(): Unit = { + // Update the queue of shuffles to be migrated + logInfo("Offloading shuffle blocks") + val localShuffles = bm.migratableResolver.getStoredShuffles().toSet + val newShufflesToMigrate = localShuffles.diff(migratingShuffles).toSeq + shufflesToMigrate.addAll(newShufflesToMigrate.map(x => (x, 0)).asJava) + migratingShuffles ++= newShufflesToMigrate + + // Update the threads doing migrations + val livePeerSet = bm.getPeers(false).toSet + val currentPeerSet = migrationPeers.keys.toSet + val deadPeers = currentPeerSet.diff(livePeerSet) + val newPeers = livePeerSet.diff(currentPeerSet) + migrationPeers ++= newPeers.map { peer => + logDebug(s"Starting thread to migrate shuffle blocks to ${peer}") + val runnable = new ShuffleMigrationRunnable(peer) + shuffleMigrationPool.submit(runnable) + (peer, runnable) + } + // A peer may have entered a decommissioning state, don't transfer any new blocks + deadPeers.foreach { peer => + migrationPeers.get(peer).foreach(_.running = false) + } + } + + /** + * Stop migrating shuffle blocks. + */ + private[storage] def stopOffloadingShuffleBlocks(): Unit = { + logInfo("Stopping offloading shuffle blocks.") + // Stop as gracefully as possible. + migrationPeers.values.foreach{ _.running = false } + shuffleMigrationPool.shutdown() + shuffleMigrationPool.shutdownNow() + } + + /** + * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers + * Visible for testing + */ + private[storage] def decommissionRddCacheBlocks(): Unit = { + val replicateBlocksInfo = bm.getMigratableRDDBlocks() + + if (replicateBlocksInfo.nonEmpty) { + logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + + "for block manager decommissioning") + } else { + logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") + return + } + + // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) + // so that we end up prioritize them over each other + val blocksFailedReplication = replicateBlocksInfo.map { replicateBlock => + val replicatedSuccessfully = migrateBlock(replicateBlock) + (replicateBlock.blockId, replicatedSuccessfully) + }.filterNot(_._2).map(_._1) + if (blocksFailedReplication.nonEmpty) { + logWarning("Blocks failed replication in cache decommissioning " + + s"process: ${blocksFailedReplication.mkString(",")}") + } + } + + private def migrateBlock(blockToReplicate: ReplicateBlock): Boolean = { + val replicatedSuccessfully = bm.replicateBlock( + blockToReplicate.blockId, + blockToReplicate.replicas.toSet, + blockToReplicate.maxReplicas, + maxReplicationFailures = Some(maxReplicationFailuresForDecommission)) + if (replicatedSuccessfully) { + logInfo(s"Block ${blockToReplicate.blockId} offloaded successfully, Removing block now") + bm.removeBlock(blockToReplicate.blockId) + logInfo(s"Block ${blockToReplicate.blockId} removed") + } else { + logWarning(s"Failed to offload block ${blockToReplicate.blockId}") + } + replicatedSuccessfully + } + + def start(): Unit = { + logInfo("Starting block migration thread") + if (conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) { + rddBlockMigrationExecutor.submit(rddBlockMigrationRunnable) + } + if (conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) { + shuffleBlockMigrationRefreshExecutor.submit(shuffleBlockMigrationRefreshRunnable) + } + if (!conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED) && + !conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) { + logError(s"Storage decommissioning attempted but neither " + + s"${config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key} or " + + s"${config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key} is enabled ") + stopped = true + } + } + + def stop(): Unit = { + if (stopped) { + return + } else { + stopped = true + } + try { + rddBlockMigrationExecutor.shutdown() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + try { + shuffleBlockMigrationRefreshExecutor.shutdown() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + try { + stopOffloadingShuffleBlocks() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + logInfo("Forcing block migrations threads to stop") + try { + rddBlockMigrationExecutor.shutdownNow() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + try { + shuffleBlockMigrationRefreshExecutor.shutdownNow() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + logInfo("Stopped storage decommissioner") + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 2a4817797a87c..a3d42348befaa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -29,7 +29,7 @@ import scala.util.control.NonFatal import com.google.common.cache.CacheBuilder -import org.apache.spark.SparkConf +import org.apache.spark.{MapOutputTrackerMaster, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.{config, Logging} import org.apache.spark.network.shuffle.ExternalBlockStoreClient @@ -50,7 +50,8 @@ class BlockManagerMasterEndpoint( conf: SparkConf, listenerBus: LiveListenerBus, externalBlockStoreClient: Option[ExternalBlockStoreClient], - blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo]) + blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo], + mapOutputTracker: MapOutputTrackerMaster) extends IsolatedRpcEndpoint with Logging { // Mapping from executor id to the block manager's local disk directories. @@ -162,7 +163,8 @@ class BlockManagerMasterEndpoint( context.reply(true) case DecommissionBlockManagers(executorIds) => - decommissionBlockManagers(executorIds.flatMap(blockManagerIdByExecutor.get)) + val bmIds = executorIds.flatMap(blockManagerIdByExecutor.get) + decommissionBlockManagers(bmIds) context.reply(true) case GetReplicateInfoForRDDBlocks(blockManagerId) => @@ -539,6 +541,24 @@ class BlockManagerMasterEndpoint( storageLevel: StorageLevel, memSize: Long, diskSize: Long): Boolean = { + logDebug(s"Updating block info on master ${blockId} for ${blockManagerId}") + + if (blockId.isShuffle) { + blockId match { + case ShuffleIndexBlockId(shuffleId, mapId, _) => + // Don't update the map output on just the index block + logDebug(s"Received shuffle index block update for ${shuffleId} ${mapId}, ignoring.") + return true + case ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) => + logDebug(s"Received shuffle data block update for ${shuffleId} ${mapId}, updating.") + mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) + return true + case _ => + logDebug(s"Unexpected shuffle block type ${blockId}" + + s"as ${blockId.getClass().getSimpleName()}") + return false + } + } if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala b/core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala new file mode 100644 index 0000000000000..4684d9c67754d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +class BlockSavedOnDecommissionedBlockManagerException(blockId: BlockId) + extends Exception(s"Block $blockId cannot be saved on decommissioned executor") diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala index 148d20ee659a2..cd3ab4db77f85 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala @@ -58,7 +58,7 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { }) TestUtils.waitUntilExecutorsUp(sc = sc, numExecutors = 2, - timeout = 10000) // 10s + timeout = 30000) // 30s val sleepyRdd = input.mapPartitions{ x => Thread.sleep(5000) // 5s x diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index 27bb06b4e0636..725a1d90557a2 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -27,7 +27,7 @@ import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleBlockInfo} import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -48,6 +48,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(diskBlockManager.getFile(any[BlockId])).thenAnswer( (invocation: InvocationOnMock) => new File(tempDir, invocation.getArguments.head.toString)) + when(diskBlockManager.localDirs).thenReturn(Array(tempDir)) } override def afterEach(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index ef7b13875540f..d7009e6ed5799 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -87,7 +87,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 4) assert(id.mapId === 5) assert(id.reduceId === 6) - assert(!id.isShuffle) + assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } @@ -100,7 +100,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 7) assert(id.mapId === 8) assert(id.reduceId === 9) - assert(!id.isShuffle) + assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala new file mode 100644 index 0000000000000..afcb38bc38836 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.util.concurrent.Semaphore + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ + +import org.scalatest.concurrent.Eventually + +import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.util.{ResetSystemProperties, ThreadUtils} + +class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalSparkContext + with ResetSystemProperties with Eventually { + + val numExecs = 3 + val numParts = 3 + + test(s"verify that an already running task which is going to cache data succeeds " + + s"on a decommissioned executor") { + runDecomTest(true, false, true) + } + + test(s"verify that shuffle blocks are migrated") { + runDecomTest(false, true, false) + } + + test(s"verify that both migrations can work at the same time.") { + runDecomTest(true, true, false) + } + + private def runDecomTest(persist: Boolean, shuffle: Boolean, migrateDuring: Boolean) = { + + val master = s"local-cluster[${numExecs}, 1, 1024]" + val conf = new SparkConf().setAppName("test").setMaster(master) + .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) + .set(config.STORAGE_DECOMMISSION_ENABLED, true) + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, persist) + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, shuffle) + // Just replicate blocks as fast as we can during testing, there isn't another + // workload we need to worry about. + .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 1L) + + sc = new SparkContext(master, "test", conf) + + // Wait for the executors to start + TestUtils.waitUntilExecutorsUp(sc = sc, + numExecutors = numExecs, + timeout = 60000) // 60s + + val input = sc.parallelize(1 to numParts, numParts) + val accum = sc.longAccumulator("mapperRunAccumulator") + input.count() + + // Create a new RDD where we have sleep in each partition, we are also increasing + // the value of accumulator in each partition + val baseRdd = input.mapPartitions { x => + if (migrateDuring) { + Thread.sleep(1000) + } + accum.add(1) + x.map(y => (y, y)) + } + val testRdd = shuffle match { + case true => baseRdd.reduceByKey(_ + _) + case false => baseRdd + } + + // Listen for the job & block updates + val taskStartSem = new Semaphore(0) + val broadcastSem = new Semaphore(0) + val executorRemovedSem = new Semaphore(0) + val taskEndEvents = ArrayBuffer.empty[SparkListenerTaskEnd] + val blocksUpdated = ArrayBuffer.empty[SparkListenerBlockUpdated] + sc.addSparkListener(new SparkListener { + + override def onExecutorRemoved(execRemoved: SparkListenerExecutorRemoved): Unit = { + executorRemovedSem.release() + } + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + taskStartSem.release() + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + taskEndEvents.append(taskEnd) + } + + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + // Once broadcast start landing on the executors we're good to proceed. + // We don't only use task start as it can occur before the work is on the executor. + if (blockUpdated.blockUpdatedInfo.blockId.isBroadcast) { + broadcastSem.release() + } + blocksUpdated.append(blockUpdated) + } + }) + + + // Cache the RDD lazily + if (persist) { + testRdd.persist() + } + + // Start the computation of RDD - this step will also cache the RDD + val asyncCount = testRdd.countAsync() + + // Wait for the job to have started. + taskStartSem.acquire(1) + // Wait for each executor + driver to have it's broadcast info delivered. + broadcastSem.acquire((numExecs + 1)) + + // Make sure the job is either mid run or otherwise has data to migrate. + if (migrateDuring) { + // Give Spark a tiny bit to start executing after the broadcast blocks land. + // For me this works at 100, set to 300 for system variance. + Thread.sleep(300) + } else { + ThreadUtils.awaitResult(asyncCount, 15.seconds) + } + + // Decommission one of the executors. + val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] + val execs = sched.getExecutorIds() + assert(execs.size == numExecs, s"Expected ${numExecs} executors but found ${execs.size}") + + val execToDecommission = execs.head + logDebug(s"Decommissioning executor ${execToDecommission}") + sched.decommissionExecutor(execToDecommission) + + // Wait for job to finish. + val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 15.seconds) + assert(asyncCountResult === numParts) + // All tasks finished, so accum should have been increased numParts times. + assert(accum.value === numParts) + + sc.listenerBus.waitUntilEmpty() + if (shuffle) { + // mappers & reducers which succeeded + assert(taskEndEvents.count(_.reason == Success) === 2 * numParts, + s"Expected ${2 * numParts} tasks got ${taskEndEvents.size} (${taskEndEvents})") + } else { + // only mappers which executed successfully + assert(taskEndEvents.count(_.reason == Success) === numParts, + s"Expected ${numParts} tasks got ${taskEndEvents.size} (${taskEndEvents})") + } + + // Wait for our respective blocks to have migrated + eventually(timeout(30.seconds), interval(10.milliseconds)) { + if (persist) { + // One of our blocks should have moved. + val rddUpdates = blocksUpdated.filter { update => + val blockId = update.blockUpdatedInfo.blockId + blockId.isRDD} + val blockLocs = rddUpdates.map { update => + (update.blockUpdatedInfo.blockId.name, + update.blockUpdatedInfo.blockManagerId)} + val blocksToManagers = blockLocs.groupBy(_._1).mapValues(_.size) + assert(!blocksToManagers.filter(_._2 > 1).isEmpty, + s"We should have a block that has been on multiple BMs in rdds:\n ${rddUpdates} from:\n" + + s"${blocksUpdated}\n but instead we got:\n ${blocksToManagers}") + } + // If we're migrating shuffles we look for any shuffle block updates + // as there is no block update on the initial shuffle block write. + if (shuffle) { + val numDataLocs = blocksUpdated.filter { update => + val blockId = update.blockUpdatedInfo.blockId + blockId.isInstanceOf[ShuffleDataBlockId] + }.size + val numIndexLocs = blocksUpdated.filter { update => + val blockId = update.blockUpdatedInfo.blockId + blockId.isInstanceOf[ShuffleIndexBlockId] + }.size + assert(numDataLocs === 1, s"Expect shuffle data block updates in ${blocksUpdated}") + assert(numIndexLocs === 1, s"Expect shuffle index block updates in ${blocksUpdated}") + } + } + + // Since the RDD is cached or shuffled so further usage of same RDD should use the + // cached data. Original RDD partitions should not be recomputed i.e. accum + // should have same value like before + assert(testRdd.count() === numParts) + assert(accum.value === numParts) + + val storageStatus = sc.env.blockManager.master.getStorageStatus + val execIdToBlocksMapping = storageStatus.map( + status => (status.blockManagerId.executorId, status.blocks)).toMap + // No cached blocks should be present on executor which was decommissioned + assert(execIdToBlocksMapping(execToDecommission).keys.filter(_.isRDD).toSeq === Seq(), + "Cache blocks should be migrated") + if (persist) { + // There should still be all the RDD blocks cached + assert(execIdToBlocksMapping.values.flatMap(_.keys).count(_.isRDD) === numParts) + } + + // Make the executor we decommissioned exit + sched.client.killExecutors(List(execToDecommission)) + + // Wait for the executor to be removed + executorRemovedSem.acquire(1) + + // Since the RDD is cached or shuffled so further usage of same RDD should use the + // cached data. Original RDD partitions should not be recomputed i.e. accum + // should have same value like before + assert(testRdd.count() === numParts) + assert(accum.value === numParts) + + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala deleted file mode 100644 index 7456ca7f02a2e..0000000000000 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala +++ /dev/null @@ -1,106 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - -import java.util.concurrent.Semaphore - -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration._ - -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite, Success} -import org.apache.spark.internal.config -import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend -import org.apache.spark.util.{ResetSystemProperties, ThreadUtils} - -class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext - with ResetSystemProperties { - - override def beforeEach(): Unit = { - val conf = new SparkConf().setAppName("test") - .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) - .set(config.STORAGE_DECOMMISSION_ENABLED, true) - - sc = new SparkContext("local-cluster[2, 1, 1024]", "test", conf) - } - - test(s"verify that an already running task which is going to cache data succeeds " + - s"on a decommissioned executor") { - // Create input RDD with 10 partitions - val input = sc.parallelize(1 to 10, 10) - val accum = sc.longAccumulator("mapperRunAccumulator") - // Do a count to wait for the executors to be registered. - input.count() - - // Create a new RDD where we have sleep in each partition, we are also increasing - // the value of accumulator in each partition - val sleepyRdd = input.mapPartitions { x => - Thread.sleep(500) - accum.add(1) - x - } - - // Listen for the job - val sem = new Semaphore(0) - val taskEndEvents = ArrayBuffer.empty[SparkListenerTaskEnd] - sc.addSparkListener(new SparkListener { - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { - sem.release() - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - taskEndEvents.append(taskEnd) - } - }) - - // Cache the RDD lazily - sleepyRdd.persist() - - // Start the computation of RDD - this step will also cache the RDD - val asyncCount = sleepyRdd.countAsync() - - // Wait for the job to have started - sem.acquire(1) - - // Give Spark a tiny bit to start the tasks after the listener says hello - Thread.sleep(100) - // Decommission one of the executor - val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] - val execs = sched.getExecutorIds() - assert(execs.size == 2, s"Expected 2 executors but found ${execs.size}") - val execToDecommission = execs.head - sched.decommissionExecutor(execToDecommission) - - // Wait for job to finish - val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 6.seconds) - assert(asyncCountResult === 10) - // All 10 tasks finished, so accum should have been increased 10 times - assert(accum.value === 10) - - // All tasks should be successful, nothing should have failed - sc.listenerBus.waitUntilEmpty() - assert(taskEndEvents.size === 10) // 10 mappers - assert(taskEndEvents.map(_.reason).toSet === Set(Success)) - - // Since the RDD is cached, so further usage of same RDD should use the - // cached data. Original RDD partitions should not be recomputed i.e. accum - // should have same value like before - assert(sleepyRdd.count() === 10) - assert(accum.value === 10) - } -} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala new file mode 100644 index 0000000000000..5ff1ff05cc4e0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.concurrent.duration._ + +import org.mockito.{ArgumentMatchers => mc} +import org.mockito.Mockito.{mock, times, verify, when} +import org.scalatest._ +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.shuffle.{MigratableResolver, ShuffleBlockInfo} +import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock + +class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { + + private val bmPort = 12345 + + private val sparkConf = new SparkConf(false) + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, true) + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, true) + + private def registerShuffleBlocks( + mockMigratableShuffleResolver: MigratableResolver, + ids: Set[(Int, Long, Int)]): Unit = { + + when(mockMigratableShuffleResolver.getStoredShuffles()) + .thenReturn(ids.map(triple => ShuffleBlockInfo(triple._1, triple._2)).toSeq) + + ids.foreach { case (shuffleId: Int, mapId: Long, reduceId: Int) => + when(mockMigratableShuffleResolver.getMigrationBlocks(mc.any())) + .thenReturn(List( + (ShuffleIndexBlockId(shuffleId, mapId, reduceId), mock(classOf[ManagedBuffer])), + (ShuffleDataBlockId(shuffleId, mapId, reduceId), mock(classOf[ManagedBuffer])))) + } + } + + test("test shuffle and cached rdd migration without any error") { + val blockTransferService = mock(classOf[BlockTransferService]) + val bm = mock(classOf[BlockManager]) + + val storedBlockId1 = RDDBlockId(0, 0) + val storedBlock1 = + new ReplicateBlock(storedBlockId1, Seq(BlockManagerId("replicaHolder", "host1", bmPort)), 1) + + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1))) + when(bm.getPeers(mc.any())) + .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345))) + + when(bm.blockTransferService).thenReturn(blockTransferService) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq(storedBlock1)) + + val bmDecomManager = new BlockManagerDecommissioner(sparkConf, bm) + + try { + bmDecomManager.start() + + eventually(timeout(5.second), interval(10.milliseconds)) { + assert(bmDecomManager.shufflesToMigrate.isEmpty == true) + verify(bm, times(1)).replicateBlock( + mc.eq(storedBlockId1), mc.any(), mc.any(), mc.eq(Some(3))) + verify(blockTransferService, times(2)) + .uploadBlockSync(mc.eq("host2"), mc.eq(bmPort), mc.eq("exec2"), mc.any(), mc.any(), + mc.eq(StorageLevel.DISK_ONLY), mc.isNull()) + } + } finally { + bmDecomManager.stop() + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 660bfcfc48267..d18d84dfaa9e5 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -103,7 +103,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), + new LiveListenerBus(conf), None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) allStores.clear() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index dc1c7cd52d466..62bb4d9f0d1fb 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.io.File import java.nio.ByteBuffer +import java.nio.file.Files import scala.collection.JavaConverters._ import scala.collection.mutable @@ -50,10 +51,11 @@ import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, Transpo import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExecutorDiskUtils, ExternalBlockStoreClient} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEnv} -import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerBlockUpdated} +import org.apache.spark.scheduler.{LiveListenerBus, MapStatus, SparkListenerBlockUpdated} import org.apache.spark.scheduler.cluster.{CoarseGrainedClusterMessages, CoarseGrainedSchedulerBackend} import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager} +import org.apache.spark.shuffle.{ShuffleBlockResolver, ShuffleManager} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util._ @@ -61,7 +63,7 @@ import org.apache.spark.util.io.ChunkedByteBuffer class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach with PrivateMethodTester with LocalSparkContext with ResetSystemProperties - with EncryptionFunSuite with TimeLimits { + with EncryptionFunSuite with TimeLimits with BeforeAndAfterAll { import BlockManagerSuite._ @@ -70,6 +72,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE var conf: SparkConf = null val allStores = ArrayBuffer[BlockManager]() + val sortShuffleManagers = ArrayBuffer[SortShuffleManager]() var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null var liveListenerBus: LiveListenerBus = null @@ -97,12 +100,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE .set(Network.RPC_ASK_TIMEOUT, "5s") } + private def makeSortShuffleManager(): SortShuffleManager = { + val newMgr = new SortShuffleManager(new SparkConf(false)) + sortShuffleManagers += newMgr + newMgr + } + private def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, master: BlockManagerMaster = this.master, transferService: Option[BlockTransferService] = Option.empty, - testConf: Option[SparkConf] = None): BlockManager = { + testConf: Option[SparkConf] = None, + shuffleManager: ShuffleManager = shuffleManager): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) bmConf.set(TEST_MEMORY, maxMem) bmConf.set(MEMORY_OFFHEAP_SIZE, maxMem) @@ -153,7 +163,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE liveListenerBus = spy(new LiveListenerBus(conf)) master = spy(new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - liveListenerBus, None, blockManagerInfo)), + liveListenerBus, None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true)) @@ -166,6 +176,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf = null allStores.foreach(_.stop()) allStores.clear() + sortShuffleManagers.foreach(_.stop()) + sortShuffleManagers.clear() rpcEnv.shutdown() rpcEnv.awaitTermination() rpcEnv = null @@ -176,6 +188,17 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } + override def afterAll(): Unit = { + try { + // Cleanup the reused items. + Option(bcastManager).foreach(_.stop()) + Option(mapOutputTracker).foreach(_.stop()) + Option(shuffleManager).foreach(_.stop()) + } finally { + super.afterAll() + } + } + private def stopBlockManager(blockManager: BlockManager): Unit = { allStores -= blockManager blockManager.stop() @@ -1815,6 +1838,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE verify(liveListenerBus, never()).post(SparkListenerBlockUpdated(BlockUpdatedInfo(updateInfo))) } + test("we reject putting blocks when we have the wrong shuffle resolver") { + val badShuffleManager = mock(classOf[ShuffleManager]) + val badShuffleResolver = mock(classOf[ShuffleBlockResolver]) + when(badShuffleManager.shuffleBlockResolver).thenReturn(badShuffleResolver) + val shuffleBlockId = ShuffleDataBlockId(0, 0, 0) + val bm = makeBlockManager(100, "exec1", shuffleManager = badShuffleManager) + val message = "message" + val exception = intercept[SparkException] { + bm.putBlockDataAsStream(shuffleBlockId, StorageLevel.DISK_ONLY, ClassTag(message.getClass)) + } + assert(exception.getMessage.contains("unsupported shuffle resolver")) + } + test("test decommission block manager should not be part of peers") { val exec1 = "exec1" val exec2 = "exec2" @@ -1846,7 +1882,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).contains(store1.blockManagerId)) - store1.decommissionRddCacheBlocks() + val decomManager = new BlockManagerDecommissioner(conf, store1) + decomManager.decommissionRddCacheBlocks() assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).toSet === Set(store2.blockManagerId, store3.blockManagerId)) @@ -1866,13 +1903,57 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId)) assert(master.getLocations(blockIdSmall) === Seq(store1.blockManagerId)) - store1.decommissionRddCacheBlocks() + val decomManager = new BlockManagerDecommissioner(conf, store1) + decomManager.decommissionRddCacheBlocks() // Smaller block offloaded to store2 assert(master.getLocations(blockIdSmall) === Seq(store2.blockManagerId)) // Larger block still present in store1 as it can't be offloaded assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId)) } + test("test migration of shuffle blocks during decommissioning") { + val shuffleManager1 = makeSortShuffleManager() + val bm1 = makeBlockManager(3500, "exec1", shuffleManager = shuffleManager1) + shuffleManager1.shuffleBlockResolver._blockManager = bm1 + + val shuffleManager2 = makeSortShuffleManager() + val bm2 = makeBlockManager(3500, "exec2", shuffleManager = shuffleManager2) + shuffleManager2.shuffleBlockResolver._blockManager = bm2 + + val blockSize = 5 + val shuffleDataBlockContent = Array[Byte](0, 1, 2, 3, 4) + val shuffleData = ShuffleDataBlockId(0, 0, 0) + Files.write(bm1.diskBlockManager.getFile(shuffleData).toPath(), shuffleDataBlockContent) + val shuffleIndexBlockContent = Array[Byte](5, 6, 7, 8, 9) + val shuffleIndex = ShuffleIndexBlockId(0, 0, 0) + Files.write(bm1.diskBlockManager.getFile(shuffleIndex).toPath(), shuffleIndexBlockContent) + + mapOutputTracker.registerShuffle(0, 1) + val decomManager = new BlockManagerDecommissioner(conf, bm1) + try { + mapOutputTracker.registerMapOutput(0, 0, MapStatus(bm1.blockManagerId, Array(blockSize), 0)) + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm1.blockManagerId) + + val env = mock(classOf[SparkEnv]) + when(env.conf).thenReturn(conf) + SparkEnv.set(env) + + decomManager.refreshOffloadingShuffleBlocks() + + eventually(timeout(1.second), interval(10.milliseconds)) { + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm2.blockManagerId) + } + assert(Files.readAllBytes(bm2.diskBlockManager.getFile(shuffleData).toPath()) + === shuffleDataBlockContent) + assert(Files.readAllBytes(bm2.diskBlockManager.getFile(shuffleIndex).toPath()) + === shuffleIndexBlockContent) + } finally { + mapOutputTracker.unregisterShuffle(0) + // Avoid thread leak + decomManager.stopOffloadingShuffleBlocks() + } + } + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { var numCalls = 0 var tempFileManager: DownloadFileManager = null diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala index becf9415c7506..fd67a03160afb 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.deploy.k8s.integrationtest +import org.apache.spark.internal.config import org.apache.spark.internal.config.Worker private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => @@ -28,18 +29,28 @@ private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => .set(Worker.WORKER_DECOMMISSION_ENABLED.key, "true") .set("spark.kubernetes.pyspark.pythonVersion", "3") .set("spark.kubernetes.container.image", pyImage) + .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true") + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true") + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true") + // Ensure we have somewhere to migrate our data too + .set("spark.executor.instances", "3") + // The default of 30 seconds is fine, but for testing we just want to get this done fast. + .set("spark.storage.decommission.replicationReattemptInterval", "1") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_DECOMISSIONING, mainClass = "", expectedLogOnCompletion = Seq( "Finished waiting, stopping Spark", - "decommissioning executor"), + "decommissioning executor", + "Final accumulator value is: 100"), appArgs = Array.empty[String], driverPodChecker = doBasicDriverPyPodCheck, executorPodChecker = doBasicExecutorPyPodCheck, appLocator = appLocator, isJVM = false, + pyFiles = None, + executorPatience = None, decommissioningTest = true) } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 65a2f1ff79697..ebf71e8cb83e4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -42,7 +42,8 @@ import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite - with DepsTestsSuite with DecommissionSuite with RTestsSuite with Logging with Eventually + // TODO(SPARK-32354): Fix and re-enable the R tests. + with DepsTestsSuite with DecommissionSuite /* with RTestsSuite */ with Logging with Eventually with Matchers { @@ -325,21 +326,36 @@ class KubernetesSuite extends SparkFunSuite val result = checkPodReady(namespace, name) result shouldBe (true) } - // Look for the string that indicates we're good to clean up - // on the driver + // Look for the string that indicates we're good to trigger decom on the driver logDebug("Waiting for first collect...") Eventually.eventually(TIMEOUT, INTERVAL) { assert(kubernetesTestComponents.kubernetesClient .pods() .withName(driverPodName) .getLog - .contains("Waiting to give nodes time to finish."), + .contains("Waiting to give nodes time to finish migration, decom exec 1."), "Decommission test did not complete first collect.") } // Delete the pod to simulate cluster scale down/migration. - val pod = kubernetesTestComponents.kubernetesClient.pods().withName(name) + // This will allow the pod to remain up for the grace period + val pod = kubernetesTestComponents.kubernetesClient.pods() + .withName(name) pod.delete() logDebug(s"Triggered pod decom/delete: $name deleted") + // Look for the string that indicates we should force kill the first + // Executor. This simulates the pod being fully lost. + logDebug("Waiting for second collect...") + Eventually.eventually(TIMEOUT, INTERVAL) { + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPodName) + .getLog + .contains("Waiting some more, please kill exec 1."), + "Decommission test did not complete second collect.") + } + logDebug("Force deleting") + val podNoGrace = pod.withGracePeriod(0) + podNoGrace.delete() } case Action.DELETED | Action.ERROR => execPods.remove(name) @@ -365,9 +381,10 @@ class KubernetesSuite extends SparkFunSuite .get(0) driverPodChecker(driverPod) - // If we're testing decommissioning we delete all the executors, but we should have - // an executor at some point. - Eventually.eventually(patienceTimeout, patienceInterval) { + + // If we're testing decommissioning we an executors, but we should have an executor + // at some point. + Eventually.eventually(TIMEOUT, patienceInterval) { execPods.values.nonEmpty should be (true) } execWatcher.close() @@ -482,6 +499,6 @@ private[spark] object KubernetesSuite { val SPARK_DFS_READ_WRITE_TEST = "org.apache.spark.examples.DFSReadWriteTest" val SPARK_REMOTE_MAIN_CLASS: String = "org.apache.spark.examples.SparkRemoteFileTest" val SPARK_DRIVER_MAIN_CLASS: String = "org.apache.spark.examples.DriverSubmissionTest" - val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) + val TIMEOUT = PatienceConfiguration.Timeout(Span(3, Minutes)) val INTERVAL = PatienceConfiguration.Interval(Span(1, Seconds)) } diff --git a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py index f68f24d49763d..d34e61611461c 100644 --- a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py +++ b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py @@ -31,14 +31,29 @@ .appName("PyMemoryTest") \ .getOrCreate() sc = spark._sc - rdd = sc.parallelize(range(10)) - rdd.collect() - print("Waiting to give nodes time to finish.") - time.sleep(5) + acc = sc.accumulator(0) + + def addToAcc(x): + acc.add(1) + return x + + initialRdd = sc.parallelize(range(100), 5) + accRdd = initialRdd.map(addToAcc) + # Trigger a shuffle so there are shuffle blocks to migrate + rdd = accRdd.map(lambda x: (x, x)).groupByKey() rdd.collect() - print("Waiting some more....") - time.sleep(10) + print("1st accumulator value is: " + str(acc.value)) + print("Waiting to give nodes time to finish migration, decom exec 1.") + print("...") + time.sleep(30) + rdd.count() + print("Waiting some more, please kill exec 1.") + print("...") + time.sleep(30) + print("Executor node should be deleted now") + rdd.count() rdd.collect() + print("Final accumulator value is: " + str(acc.value)) print("Finished waiting, stopping Spark.") spark.stop() print("Done, exiting Python") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 0976494b6d094..558e2c99e0442 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -91,7 +91,7 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), + new LiveListenerBus(conf), None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) From c2afe1c0b97fe2ae1fd03326968afdadb6806a97 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 20 Jul 2020 20:49:22 +0900 Subject: [PATCH 230/384] [SPARK-32366][DOC] Fix doc link of datetime pattern in 3.0 migration guide ### What changes were proposed in this pull request? In http://spark.apache.org/docs/latest/sql-migration-guide.html#query-engine, there is a invalid reference for datetime reference "sql-ref-datetime-pattern.md". We should fix the link as http://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html. ![image](https://user-images.githubusercontent.com/1097932/87916920-fff57380-ca28-11ea-9028-99b9f9ebdfa4.png) Also, it is nice to add url for [DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) ### Why are the changes needed? Fix migration guide doc ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Build the doc in local env and check it: ![image](https://user-images.githubusercontent.com/1097932/87919723-13a2d900-ca2d-11ea-9923-a29b4cefaf3c.png) Closes #29162 from gengliangwang/fixDoc. Authored-by: Gengliang Wang Signed-off-by: HyukjinKwon --- docs/sql-migration-guide.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index d3138ae319160..46d330e095865 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -148,7 +148,8 @@ license: | - In Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and so on. Spark 3.0 uses Java 8 API classes from the `java.time` packages that are based on [ISO chronology](https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and below, those operations are performed using the hybrid calendar ([Julian + Gregorian](https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: - * Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions when patterns specified by users is used for parsing and formatting. In Spark 3.0, we define our own pattern strings in `sql-ref-datetime-pattern.md`, which is implemented via `java.time.format.DateTimeFormatter` under the hood. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and below, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in [simpleDateFormat](https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html). The old behavior can be restored by setting `spark.sql.legacy.timeParserPolicy` to `LEGACY`. + * Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions when patterns specified by users is used for parsing and formatting. In Spark 3.0, we define our own pattern strings in [Datetime Patterns for Formatting and Parsing](sql-ref-datetime-pattern.html), + which is implemented via [DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) under the hood. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and below, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in [SimpleDateFormat](https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html). The old behavior can be restored by setting `spark.sql.legacy.timeParserPolicy` to `LEGACY`. * The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. From d0c83f372b1aa87327eacc16b131c57fccf6c205 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 20 Jul 2020 14:17:31 +0000 Subject: [PATCH 231/384] [SPARK-32302][SQL] Partially push down disjunctive predicates through Join/Partitions ### What changes were proposed in this pull request? In https://github.com/apache/spark/pull/28733 and #28805, CNF conversion is used to push down disjunctive predicates through join and partitions pruning. It's a good improvement, however, converting all the predicates in CNF can lead to a very long result, even with grouping functions over expressions. For example, for the following predicate ``` (p0 = '1' AND p1 = '1') OR (p0 = '2' AND p1 = '2') OR (p0 = '3' AND p1 = '3') OR (p0 = '4' AND p1 = '4') OR (p0 = '5' AND p1 = '5') OR (p0 = '6' AND p1 = '6') OR (p0 = '7' AND p1 = '7') OR (p0 = '8' AND p1 = '8') OR (p0 = '9' AND p1 = '9') OR (p0 = '10' AND p1 = '10') OR (p0 = '11' AND p1 = '11') OR (p0 = '12' AND p1 = '12') OR (p0 = '13' AND p1 = '13') OR (p0 = '14' AND p1 = '14') OR (p0 = '15' AND p1 = '15') OR (p0 = '16' AND p1 = '16') OR (p0 = '17' AND p1 = '17') OR (p0 = '18' AND p1 = '18') OR (p0 = '19' AND p1 = '19') OR (p0 = '20' AND p1 = '20') ``` will be converted into a long query(130K characters) in Hive metastore, and there will be error: ``` javax.jdo.JDOException: Exception thrown when executing query : SELECT DISTINCT 'org.apache.hadoop.hive.metastore.model.MPartition' AS NUCLEUS_TYPE,A0.CREATE_TIME,A0.LAST_ACCESS_TIME,A0.PART_NAME,A0.PART_ID,A0.PART_NAME AS NUCORDER0 FROM PARTITIONS A0 LEFT OUTER JOIN TBLS B0 ON A0.TBL_ID = B0.TBL_ID LEFT OUTER JOIN DBS C0 ON B0.DB_ID = C0.DB_ID WHERE B0.TBL_NAME = ? AND C0."NAME" = ? AND ((((((A0.PART_NAME LIKE '%/p1=1' ESCAPE '\' ) OR (A0.PART_NAME LIKE '%/p1=2' ESCAPE '\' )) OR (A0.PART_NAME LIKE '%/p1=3' ESCAPE '\' )) OR ((A0.PART_NAME LIKE '%/p1=4' ESCAPE '\' ) O ... ``` Essentially, we just need to traverse predicate and extract the convertible sub-predicates like what we did in https://github.com/apache/spark/pull/24598. There is no need to maintain the CNF result set. ### Why are the changes needed? A better implementation for pushing down disjunctive and complex predicates. The pushed down predicates is always equal or shorter than the CNF result. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit tests Closes #29101 from gengliangwang/pushJoin. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/predicates.scala | 153 +++++------------- .../sql/catalyst/optimizer/Optimizer.scala | 8 +- ...la => PushExtraPredicateThroughJoin.scala} | 42 +++-- .../apache/spark/sql/internal/SQLConf.scala | 15 -- .../ConjunctiveNormalFormPredicateSuite.scala | 128 --------------- ...xtractPredicatesWithinOutputSetSuite.scala | 82 ++++++++++ .../optimizer/FilterPushdownSuite.scala | 62 +++---- .../PruneFileSourcePartitions.scala | 8 +- .../execution/PruneHiveTablePartitions.scala | 9 +- .../PruneHiveTablePartitionsSuite.scala | 25 +++ 10 files changed, 208 insertions(+), 324 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/{PushCNFPredicateThroughJoin.scala => PushExtraPredicateThroughJoin.scala} (59%) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExtractPredicatesWithinOutputSetSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 527618b8e2c5a..aa5cf4758564b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -202,125 +202,50 @@ trait PredicateHelper extends Logging { } /** - * Convert an expression into conjunctive normal form. - * Definition and algorithm: https://en.wikipedia.org/wiki/Conjunctive_normal_form - * CNF can explode exponentially in the size of the input expression when converting [[Or]] - * clauses. Use a configuration [[SQLConf.MAX_CNF_NODE_COUNT]] to prevent such cases. - * - * @param condition to be converted into CNF. - * @return the CNF result as sequence of disjunctive expressions. If the number of expressions - * exceeds threshold on converting `Or`, `Seq.empty` is returned. + * Returns a filter that its reference is a subset of `outputSet` and it contains the maximum + * constraints from `condition`. This is used for predicate pushdown. + * When there is no such filter, `None` is returned. */ - protected def conjunctiveNormalForm( + protected def extractPredicatesWithinOutputSet( condition: Expression, - groupExpsFunc: Seq[Expression] => Seq[Expression]): Seq[Expression] = { - val postOrderNodes = postOrderTraversal(condition) - val resultStack = new mutable.Stack[Seq[Expression]] - val maxCnfNodeCount = SQLConf.get.maxCnfNodeCount - // Bottom up approach to get CNF of sub-expressions - while (postOrderNodes.nonEmpty) { - val cnf = postOrderNodes.pop() match { - case _: And => - val right = resultStack.pop() - val left = resultStack.pop() - left ++ right - case _: Or => - // For each side, there is no need to expand predicates of the same references. - // So here we can aggregate predicates of the same qualifier as one single predicate, - // for reducing the size of pushed down predicates and corresponding codegen. - val right = groupExpsFunc(resultStack.pop()) - val left = groupExpsFunc(resultStack.pop()) - // Stop the loop whenever the result exceeds the `maxCnfNodeCount` - if (left.size * right.size > maxCnfNodeCount) { - logInfo(s"As the result size exceeds the threshold $maxCnfNodeCount. " + - "The CNF conversion is skipped and returning Seq.empty now. To avoid this, you can " + - s"raise the limit ${SQLConf.MAX_CNF_NODE_COUNT.key}.") - return Seq.empty - } else { - for { x <- left; y <- right } yield Or(x, y) - } - case other => other :: Nil + outputSet: AttributeSet): Option[Expression] = condition match { + case And(left, right) => + val leftResultOptional = extractPredicatesWithinOutputSet(left, outputSet) + val rightResultOptional = extractPredicatesWithinOutputSet(right, outputSet) + (leftResultOptional, rightResultOptional) match { + case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) + case (Some(leftResult), None) => Some(leftResult) + case (None, Some(rightResult)) => Some(rightResult) + case _ => None } - resultStack.push(cnf) - } - if (resultStack.length != 1) { - logWarning("The length of CNF conversion result stack is supposed to be 1. There might " + - "be something wrong with CNF conversion.") - return Seq.empty - } - resultStack.top - } - - /** - * Convert an expression to conjunctive normal form when pushing predicates through Join, - * when expand predicates, we can group by the qualifier avoiding generate unnecessary - * expression to control the length of final result since there are multiple tables. - * - * @param condition condition need to be converted - * @return the CNF result as sequence of disjunctive expressions. If the number of expressions - * exceeds threshold on converting `Or`, `Seq.empty` is returned. - */ - def CNFWithGroupExpressionsByQualifier(condition: Expression): Seq[Expression] = { - conjunctiveNormalForm(condition, (expressions: Seq[Expression]) => - expressions.groupBy(_.references.map(_.qualifier)).map(_._2.reduceLeft(And)).toSeq) - } - - /** - * Convert an expression to conjunctive normal form for predicate pushdown and partition pruning. - * When expanding predicates, this method groups expressions by their references for reducing - * the size of pushed down predicates and corresponding codegen. In partition pruning strategies, - * we split filters by [[splitConjunctivePredicates]] and partition filters by judging if it's - * references is subset of partCols, if we combine expressions group by reference when expand - * predicate of [[Or]], it won't impact final predicate pruning result since - * [[splitConjunctivePredicates]] won't split [[Or]] expression. - * - * @param condition condition need to be converted - * @return the CNF result as sequence of disjunctive expressions. If the number of expressions - * exceeds threshold on converting `Or`, `Seq.empty` is returned. - */ - def CNFWithGroupExpressionsByReference(condition: Expression): Seq[Expression] = { - conjunctiveNormalForm(condition, (expressions: Seq[Expression]) => - expressions.groupBy(e => AttributeSet(e.references)).map(_._2.reduceLeft(And)).toSeq) - } - /** - * Iterative post order traversal over a binary tree built by And/Or clauses with two stacks. - * For example, a condition `(a And b) Or c`, the postorder traversal is - * (`a`,`b`, `And`, `c`, `Or`). - * Following is the complete algorithm. After step 2, we get the postorder traversal in - * the second stack. - * 1. Push root to first stack. - * 2. Loop while first stack is not empty - * 2.1 Pop a node from first stack and push it to second stack - * 2.2 Push the children of the popped node to first stack - * - * @param condition to be traversed as binary tree - * @return sub-expressions in post order traversal as a stack. - * The first element of result stack is the leftmost node. - */ - private def postOrderTraversal(condition: Expression): mutable.Stack[Expression] = { - val stack = new mutable.Stack[Expression] - val result = new mutable.Stack[Expression] - stack.push(condition) - while (stack.nonEmpty) { - val node = stack.pop() - node match { - case Not(a And b) => stack.push(Or(Not(a), Not(b))) - case Not(a Or b) => stack.push(And(Not(a), Not(b))) - case Not(Not(a)) => stack.push(a) - case a And b => - result.push(node) - stack.push(a) - stack.push(b) - case a Or b => - result.push(node) - stack.push(a) - stack.push(b) - case _ => - result.push(node) + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // condition: (a1 AND a2) OR (b1 AND b2), + // outputSet: AttributeSet(a1, b1) + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + case Or(left, right) => + for { + lhs <- extractPredicatesWithinOutputSet(left, outputSet) + rhs <- extractPredicatesWithinOutputSet(right, outputSet) + } yield Or(lhs, rhs) + + // Here we assume all the `Not` operators is already below all the `And` and `Or` operators + // after the optimization rule `BooleanSimplification`, so that we don't need to handle the + // `Not` operators here. + case other => + if (other.references.subsetOf(outputSet)) { + Some(other) + } else { + None } - } - result } } 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 a0e21ed86a71e..79d00d32c9307 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 @@ -51,8 +51,7 @@ abstract class Optimizer(catalogManager: CatalogManager) override protected val excludedOnceBatches: Set[String] = Set( "PartitionPruning", - "Extract Python UDFs", - "Push CNF predicate through join") + "Extract Python UDFs") protected def fixedPoint = FixedPoint( @@ -123,8 +122,9 @@ abstract class Optimizer(catalogManager: CatalogManager) rulesWithoutInferFiltersFromConstraints: _*) :: // Set strategy to Once to avoid pushing filter every time because we do not change the // join condition. - Batch("Push CNF predicate through join", Once, - PushCNFPredicateThroughJoin) :: Nil + Batch("Push extra predicate through join", fixedPoint, + PushExtraPredicateThroughJoin, + PushDownPredicates) :: Nil } val batches = (Batch("Eliminate Distinct", Once, EliminateDistinct) :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushExtraPredicateThroughJoin.scala similarity index 59% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushExtraPredicateThroughJoin.scala index 47e9527ead7c3..0ba2ce3106061 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushExtraPredicateThroughJoin.scala @@ -17,18 +17,20 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.{And, Expression, PredicateHelper} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNodeTag /** - * Try converting join condition to conjunctive normal form expression so that more predicates may - * be able to be pushed down. + * Try pushing down disjunctive join condition into left and right child. * To avoid expanding the join condition, the join condition will be kept in the original form even * when predicate pushdown happens. */ -object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { +object PushExtraPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { + + private val processedJoinConditionTag = TreeNodeTag[Expression]("processedJoinCondition") private def canPushThrough(joinType: JoinType): Boolean = joinType match { case _: InnerLike | LeftSemi | RightOuter | LeftOuter | LeftAnti | ExistenceJoin(_) => true @@ -38,22 +40,28 @@ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelpe def apply(plan: LogicalPlan): LogicalPlan = plan transform { case j @ Join(left, right, joinType, Some(joinCondition), hint) if canPushThrough(joinType) => - val predicates = CNFWithGroupExpressionsByQualifier(joinCondition) - if (predicates.isEmpty) { + val alreadyProcessed = j.getTagValue(processedJoinConditionTag).exists { condition => + condition.semanticEquals(joinCondition) + } + + lazy val filtersOfBothSide = splitConjunctivePredicates(joinCondition).filter { f => + f.deterministic && f.references.nonEmpty && + !f.references.subsetOf(left.outputSet) && !f.references.subsetOf(right.outputSet) + } + lazy val leftExtraCondition = + filtersOfBothSide.flatMap(extractPredicatesWithinOutputSet(_, left.outputSet)) + lazy val rightExtraCondition = + filtersOfBothSide.flatMap(extractPredicatesWithinOutputSet(_, right.outputSet)) + + if (alreadyProcessed || (leftExtraCondition.isEmpty && rightExtraCondition.isEmpty)) { j } else { - val pushDownCandidates = predicates.filter(_.deterministic) - lazy val leftFilterConditions = - pushDownCandidates.filter(_.references.subsetOf(left.outputSet)) - lazy val rightFilterConditions = - pushDownCandidates.filter(_.references.subsetOf(right.outputSet)) - lazy val newLeft = - leftFilterConditions.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) + leftExtraCondition.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) lazy val newRight = - rightFilterConditions.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) + rightExtraCondition.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) - joinType match { + val newJoin = joinType match { case _: InnerLike | LeftSemi => Join(newLeft, newRight, joinType, Some(joinCondition), hint) case RightOuter => @@ -63,6 +71,8 @@ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelpe case other => throw new IllegalStateException(s"Unexpected join type: $other") } - } + newJoin.setTagValue(processedJoinConditionTag, joinCondition) + newJoin + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e95ef3d77d549..4a1299d82f7bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -545,19 +545,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val MAX_CNF_NODE_COUNT = - buildConf("spark.sql.optimizer.maxCNFNodeCount") - .internal() - .doc("Specifies the maximum allowable number of conjuncts in the result of CNF " + - "conversion. If the conversion exceeds the threshold, an empty sequence is returned. " + - "For example, CNF conversion of (a && b) || (c && d) generates " + - "four conjuncts (a || c) && (a || d) && (b || c) && (b || d).") - .version("3.1.0") - .intConf - .checkValue(_ >= 0, - "The depth of the maximum rewriting conjunction normal form must be positive.") - .createWithDefault(128) - val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals") .internal() .doc("When true, string literals (including regex patterns) remain escaped in our SQL " + @@ -2954,8 +2941,6 @@ class SQLConf extends Serializable with Logging { def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED) - def maxCnfNodeCount: Int = getConf(MAX_CNF_NODE_COUNT) - def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS) def fileCompressionFactor: Double = getConf(FILE_COMPRESSION_FACTOR) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala deleted file mode 100644 index 793abccd79405..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.BooleanType - -class ConjunctiveNormalFormPredicateSuite extends SparkFunSuite with PredicateHelper with PlanTest { - private val a = AttributeReference("A", BooleanType)(exprId = ExprId(1)).withQualifier(Seq("ta")) - private val b = AttributeReference("B", BooleanType)(exprId = ExprId(2)).withQualifier(Seq("tb")) - private val c = AttributeReference("C", BooleanType)(exprId = ExprId(3)).withQualifier(Seq("tc")) - private val d = AttributeReference("D", BooleanType)(exprId = ExprId(4)).withQualifier(Seq("td")) - private val e = AttributeReference("E", BooleanType)(exprId = ExprId(5)).withQualifier(Seq("te")) - private val f = AttributeReference("F", BooleanType)(exprId = ExprId(6)).withQualifier(Seq("tf")) - private val g = AttributeReference("G", BooleanType)(exprId = ExprId(7)).withQualifier(Seq("tg")) - private val h = AttributeReference("H", BooleanType)(exprId = ExprId(8)).withQualifier(Seq("th")) - private val i = AttributeReference("I", BooleanType)(exprId = ExprId(9)).withQualifier(Seq("ti")) - private val j = AttributeReference("J", BooleanType)(exprId = ExprId(10)).withQualifier(Seq("tj")) - private val a1 = - AttributeReference("a1", BooleanType)(exprId = ExprId(11)).withQualifier(Seq("ta")) - private val a2 = - AttributeReference("a2", BooleanType)(exprId = ExprId(12)).withQualifier(Seq("ta")) - private val b1 = - AttributeReference("b1", BooleanType)(exprId = ExprId(12)).withQualifier(Seq("tb")) - - // Check CNF conversion with expected expression, assuming the input has non-empty result. - private def checkCondition(input: Expression, expected: Expression): Unit = { - val cnf = CNFWithGroupExpressionsByQualifier(input) - assert(cnf.nonEmpty) - val result = cnf.reduceLeft(And) - assert(result.semanticEquals(expected)) - } - - test("Keep non-predicated expressions") { - checkCondition(a, a) - checkCondition(Literal(1), Literal(1)) - } - - test("Conversion of Not") { - checkCondition(!a, !a) - checkCondition(!(!a), a) - checkCondition(!(!(a && b)), a && b) - checkCondition(!(!(a || b)), a || b) - checkCondition(!(a || b), !a && !b) - checkCondition(!(a && b), !a || !b) - } - - test("Conversion of And") { - checkCondition(a && b, a && b) - checkCondition(a && b && c, a && b && c) - checkCondition(a && (b || c), a && (b || c)) - checkCondition((a || b) && c, (a || b) && c) - checkCondition(a && b && c && d, a && b && c && d) - } - - test("Conversion of Or") { - checkCondition(a || b, a || b) - checkCondition(a || b || c, a || b || c) - checkCondition(a || b || c || d, a || b || c || d) - checkCondition((a && b) || c, (a || c) && (b || c)) - checkCondition((a && b) || (c && d), (a || c) && (a || d) && (b || c) && (b || d)) - } - - test("More complex cases") { - checkCondition(a && !(b || c), a && !b && !c) - checkCondition((a && b) || !(c && d), (a || !c || !d) && (b || !c || !d)) - checkCondition(a || b || c && d, (a || b || c) && (a || b || d)) - checkCondition(a || (b && c || d), (a || b || d) && (a || c || d)) - checkCondition(a && !(b && c || d && e), a && (!b || !c) && (!d || !e)) - checkCondition(((a && b) || c) || (d || e), (a || c || d || e) && (b || c || d || e)) - - checkCondition( - (a && b && c) || (d && e && f), - (a || d) && (a || e) && (a || f) && (b || d) && (b || e) && (b || f) && - (c || d) && (c || e) && (c || f) - ) - } - - test("Aggregate predicate of same qualifiers to avoid expanding") { - checkCondition(((a && b && a1) || c), ((a && a1) || c) && (b ||c)) - checkCondition(((a && a1 && b) || c), ((a && a1) || c) && (b ||c)) - checkCondition(((b && d && a && a1) || c), ((a && a1) || c) && (b ||c) && (d || c)) - checkCondition(((b && a2 && d && a && a1) || c), ((a2 && a && a1) || c) && (b ||c) && (d || c)) - checkCondition(((b && d && a && a1 && b1) || c), - ((a && a1) || c) && ((b && b1) ||c) && (d || c)) - checkCondition((a && a1) || (b && b1), (a && a1) || (b && b1)) - checkCondition((a && a1 && c) || (b && b1), ((a && a1) || (b && b1)) && (c || (b && b1))) - } - - test("Return Seq.empty when exceeding MAX_CNF_NODE_COUNT") { - // The following expression contains 36 conjunctive sub-expressions in CNF - val input = (a && b && c) || (d && e && f) || (g && h && i && j) - // The following expression contains 9 conjunctive sub-expressions in CNF - val input2 = (a && b && c) || (d && e && f) - Seq(8, 9, 10, 35, 36, 37).foreach { maxCount => - withSQLConf(SQLConf.MAX_CNF_NODE_COUNT.key -> maxCount.toString) { - if (maxCount < 36) { - assert(CNFWithGroupExpressionsByQualifier(input).isEmpty) - } else { - assert(CNFWithGroupExpressionsByQualifier(input).nonEmpty) - } - if (maxCount < 9) { - assert(CNFWithGroupExpressionsByQualifier(input2).isEmpty) - } else { - assert(CNFWithGroupExpressionsByQualifier(input2).nonEmpty) - } - } - } - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExtractPredicatesWithinOutputSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExtractPredicatesWithinOutputSetSuite.scala new file mode 100644 index 0000000000000..ed141ef923e0a --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExtractPredicatesWithinOutputSetSuite.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.types.BooleanType + +class ExtractPredicatesWithinOutputSetSuite + extends SparkFunSuite + with PredicateHelper + with PlanTest { + private val a = AttributeReference("A", BooleanType)(exprId = ExprId(1)) + private val b = AttributeReference("B", BooleanType)(exprId = ExprId(2)) + private val c = AttributeReference("C", BooleanType)(exprId = ExprId(3)) + private val d = AttributeReference("D", BooleanType)(exprId = ExprId(4)) + private val e = AttributeReference("E", BooleanType)(exprId = ExprId(5)) + private val f = AttributeReference("F", BooleanType)(exprId = ExprId(6)) + private val g = AttributeReference("G", BooleanType)(exprId = ExprId(7)) + private val h = AttributeReference("H", BooleanType)(exprId = ExprId(8)) + private val i = AttributeReference("I", BooleanType)(exprId = ExprId(9)) + + private def checkCondition( + input: Expression, + convertibleAttributes: Seq[Attribute], + expected: Option[Expression]): Unit = { + val result = extractPredicatesWithinOutputSet(input, AttributeSet(convertibleAttributes)) + if (expected.isEmpty) { + assert(result.isEmpty) + } else { + assert(result.isDefined && result.get.semanticEquals(expected.get)) + } + } + + test("Convertible conjunctive predicates") { + checkCondition(a && b, Seq(a, b), Some(a && b)) + checkCondition(a && b, Seq(a), Some(a)) + checkCondition(a && b, Seq(b), Some(b)) + checkCondition(a && b && c, Seq(a, c), Some(a && c)) + checkCondition(a && b && c && d, Seq(b, c), Some(b && c)) + } + + test("Convertible disjunctive predicates") { + checkCondition(a || b, Seq(a, b), Some(a || b)) + checkCondition(a || b, Seq(a), None) + checkCondition(a || b, Seq(b), None) + checkCondition(a || b || c, Seq(a, c), None) + checkCondition(a || b || c || d, Seq(a, b, d), None) + checkCondition(a || b || c || d, Seq(d, c, b, a), Some(a || b || c || d)) + } + + test("Convertible complex predicates") { + checkCondition((a && b) || (c && d), Seq(a, c), Some(a || c)) + checkCondition((a && b) || (c && d), Seq(a, b), None) + checkCondition((a && b) || (c && d), Seq(a, c, d), Some(a || (c && d))) + checkCondition((a && b && c) || (d && e && f), Seq(a, c, d, f), Some((a && c) || (d && f))) + checkCondition((a && b) || (c && d) || (e && f) || (g && h), Seq(a, c, e, g), + Some(a || c || e || g)) + checkCondition((a && b) || (c && d) || (e && f) || (g && h), Seq(a, e, g), None) + checkCondition((a || b) || (c && d) || (e && f) || (g && h), Seq(a, c, e, g), None) + checkCondition((a || b) || (c && d) || (e && f) || (g && h), Seq(a, b, c, e, g), + Some(a || b || c || e || g)) + checkCondition((a && b && c) || (d && e && f) || (g && h && i), Seq(b, e, h), Some(b || e || h)) + checkCondition((a && b && c) || (d && e && f) || (g && h && i), Seq(b, e, d), None) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index bb7e9d04c12d9..cf92e25ccab48 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -33,9 +33,6 @@ class FilterPushdownSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { - override protected val excludedOnceBatches: Set[String] = - Set("Push CNF predicate through join") - val batches = Batch("Subqueries", Once, EliminateSubqueryAliases) :: @@ -45,8 +42,9 @@ class FilterPushdownSuite extends PlanTest { BooleanSimplification, PushPredicateThroughJoin, CollapseProject) :: - Batch("Push CNF predicate through join", Once, - PushCNFPredicateThroughJoin) :: Nil + Batch("Push extra predicate through join", FixedPoint(10), + PushExtraPredicateThroughJoin, + PushDownPredicates) :: Nil } val attrA = 'a.int @@ -60,7 +58,7 @@ class FilterPushdownSuite extends PlanTest { val simpleDisjunctivePredicate = ("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11) - val expectedCNFPredicatePushDownResult = { + val expectedPredicatePushDownResult = { val left = testRelation.where(('a > 3 || 'a > 1)).subquery('x) val right = testRelation.where('a > 13 || 'a > 11).subquery('y) left.join(right, condition = Some("x.b".attr === "y.b".attr @@ -1247,17 +1245,17 @@ class FilterPushdownSuite extends PlanTest { comparePlans(Optimize.execute(query.analyze), expected) } - test("inner join: rewrite filter predicates to conjunctive normal form") { + test("push down filter predicates through inner join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) val originalQuery = x.join(y).where(("x.b".attr === "y.b".attr) && (simpleDisjunctivePredicate)) val optimized = Optimize.execute(originalQuery.analyze) - comparePlans(optimized, expectedCNFPredicatePushDownResult) + comparePlans(optimized, expectedPredicatePushDownResult) } - test("inner join: rewrite join predicates to conjunctive normal form") { + test("push down join predicates through inner join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1265,10 +1263,10 @@ class FilterPushdownSuite extends PlanTest { x.join(y, condition = Some(("x.b".attr === "y.b".attr) && (simpleDisjunctivePredicate))) val optimized = Optimize.execute(originalQuery.analyze) - comparePlans(optimized, expectedCNFPredicatePushDownResult) + comparePlans(optimized, expectedPredicatePushDownResult) } - test("inner join: rewrite complex join predicates to conjunctive normal form") { + test("push down complex predicates through inner join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1288,7 +1286,7 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("inner join: rewrite join predicates(with NOT predicate) to conjunctive normal form") { + test("push down predicates(with NOT predicate) through inner join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1308,7 +1306,7 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("left join: rewrite join predicates to conjunctive normal form") { + test("push down predicates through left join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1327,7 +1325,7 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("right join: rewrite join predicates to conjunctive normal form") { + test("push down predicates through right join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1346,7 +1344,7 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("inner join: rewrite to conjunctive normal form avoid generating too many predicates") { + test("SPARK-32302: avoid generating too many predicates") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1364,30 +1362,20 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test(s"Disable rewrite to CNF by setting ${SQLConf.MAX_CNF_NODE_COUNT.key}=0") { + test("push down predicate through multiple joins") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) + val z = testRelation.subquery('z) + val xJoinY = x.join(y, condition = Some("x.b".attr === "y.b".attr)) + val originalQuery = z.join(xJoinY, + condition = Some("x.a".attr === "z.a".attr && simpleDisjunctivePredicate)) - val originalQuery = - x.join(y, condition = Some(("x.b".attr === "y.b".attr) - && ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) - || (("y.a".attr > 2) && ("y.c".attr < 1))))) - - Seq(0, 10).foreach { count => - withSQLConf(SQLConf.MAX_CNF_NODE_COUNT.key -> count.toString) { - val optimized = Optimize.execute(originalQuery.analyze) - val (left, right) = if (count == 0) { - (testRelation.subquery('x), testRelation.subquery('y)) - } else { - (testRelation.subquery('x), - testRelation.where('c <= 5 || ('a > 2 && 'c < 1)).subquery('y)) - } - val correctAnswer = left.join(right, condition = Some("x.b".attr === "y.b".attr - && ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) - || (("y.a".attr > 2) && ("y.c".attr < 1))))).analyze - - comparePlans(optimized, correctAnswer) - } - } + val optimized = Optimize.execute(originalQuery.analyze) + val left = x.where('a > 3 || 'a > 1) + val right = y.where('a > 13 || 'a > 11) + val correctAnswer = z.join(left.join(right, + condition = Some("x.b".attr === "y.b".attr && simpleDisjunctivePredicate)), + condition = Some("x.a".attr === "z.a".attr)).analyze + comparePlans(optimized, correctAnswer) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 576a826faf894..0c56e7675da6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -56,8 +56,10 @@ private[sql] object PruneFileSourcePartitions val (partitionFilters, dataFilters) = normalizedFilters.partition(f => f.references.subsetOf(partitionSet) ) + val extraPartitionFilter = + dataFilters.flatMap(extractPredicatesWithinOutputSet(_, partitionSet)) - (ExpressionSet(partitionFilters), dataFilters) + (ExpressionSet(partitionFilters ++ extraPartitionFilter), dataFilters) } private def rebuildPhysicalOperation( @@ -88,10 +90,8 @@ private[sql] object PruneFileSourcePartitions _, _)) if filters.nonEmpty && fsRelation.partitionSchemaOption.isDefined => - val predicates = CNFWithGroupExpressionsByReference(filters.reduceLeft(And)) - val finalPredicates = if (predicates.nonEmpty) predicates else filters val (partitionKeyFilters, _) = getPartitionKeyFiltersAndDataFilters( - fsRelation.sparkSession, logicalRelation, partitionSchema, finalPredicates, + fsRelation.sparkSession, logicalRelation, partitionSchema, filters, logicalRelation.output) if (partitionKeyFilters.nonEmpty) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala index c4885f2842597..f6aff10cbc147 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala @@ -54,9 +54,8 @@ private[sql] class PruneHiveTablePartitions(session: SparkSession) val normalizedFilters = DataSourceStrategy.normalizeExprs( filters.filter(f => f.deterministic && !SubqueryExpression.hasSubquery(f)), relation.output) val partitionColumnSet = AttributeSet(relation.partitionCols) - ExpressionSet(normalizedFilters.filter { f => - !f.references.isEmpty && f.references.subsetOf(partitionColumnSet) - }) + ExpressionSet( + normalizedFilters.flatMap(extractPredicatesWithinOutputSet(_, partitionColumnSet))) } /** @@ -103,9 +102,7 @@ private[sql] class PruneHiveTablePartitions(session: SparkSession) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case op @ PhysicalOperation(projections, filters, relation: HiveTableRelation) if filters.nonEmpty && relation.isPartitioned && relation.prunedPartitions.isEmpty => - val predicates = CNFWithGroupExpressionsByReference(filters.reduceLeft(And)) - val finalPredicates = if (predicates.nonEmpty) predicates else filters - val partitionKeyFilters = getPartitionKeyFilters(finalPredicates, relation) + val partitionKeyFilters = getPartitionKeyFilters(filters, relation) if (partitionKeyFilters.nonEmpty) { val newPartitions = prunePartitions(relation, partitionKeyFilters) val newTableMeta = updateTableMeta(relation.tableMeta, newPartitions) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala index c29e889c3a941..06aea084330fa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala @@ -55,6 +55,31 @@ class PruneHiveTablePartitionsSuite extends PrunePartitionSuiteBase { } } + test("Avoid generating too many predicates in partition pruning") { + withTempView("temp") { + withTable("t") { + sql( + s""" + |CREATE TABLE t(i INT, p0 INT, p1 INT) + |USING $format + |PARTITIONED BY (p0, p1)""".stripMargin) + + spark.range(0, 10, 1).selectExpr("id as col") + .createOrReplaceTempView("temp") + + for (part <- (0 to 25)) { + sql( + s""" + |INSERT OVERWRITE TABLE t PARTITION (p0='$part', p1='$part') + |SELECT col FROM temp""".stripMargin) + } + val scale = 20 + val predicate = (1 to scale).map(i => s"(p0 = '$i' AND p1 = '$i')").mkString(" OR ") + assertPrunedPartitions(s"SELECT * FROM t WHERE $predicate", scale) + } + } + } + override def getScanExecPartitionSize(plan: SparkPlan): Long = { plan.collectFirst { case p: HiveTableScanExec => p From e0ecb66f53058f999d0de6005d08b744b020fa7f Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 20 Jul 2020 14:25:51 +0000 Subject: [PATCH 232/384] [SPARK-31869][SQL] BroadcastHashJoinExec can utilize the build side for its output partitioning ### What changes were proposed in this pull request? Currently, the `BroadcastHashJoinExec`'s `outputPartitioning` only uses the streamed side's `outputPartitioning`. However, if the join type of `BroadcastHashJoinExec` is an inner-like join, the build side's info (the join keys) can be added to `BroadcastHashJoinExec`'s `outputPartitioning`. For example, ```Scala spark.conf.set("spark.sql.autoBroadcastJoinThreshold", "500") val t1 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i1", "j1") val t2 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i2", "j2") val t3 = (0 until 20).map(i => (i % 7, i % 11)).toDF("i3", "j3") val t4 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i4", "j4") // join1 is a sort merge join. val join1 = t1.join(t2, t1("i1") === t2("i2")) // join2 is a broadcast join where t3 is broadcasted. val join2 = join1.join(t3, join1("i1") === t3("i3")) // Join on the column from the broadcasted side (i3). val join3 = join2.join(t4, join2("i3") === t4("i4")) join3.explain ``` You see that `Exchange hashpartitioning(i2#103, 200)` is introduced because there is no output partitioning info from the build side. ``` == Physical Plan == *(6) SortMergeJoin [i3#29], [i4#40], Inner :- *(4) Sort [i3#29 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(i3#29, 200), true, [id=#55] : +- *(3) BroadcastHashJoin [i1#7], [i3#29], Inner, BuildRight : :- *(3) SortMergeJoin [i1#7], [i2#18], Inner : : :- *(1) Sort [i1#7 ASC NULLS FIRST], false, 0 : : : +- Exchange hashpartitioning(i1#7, 200), true, [id=#28] : : : +- LocalTableScan [i1#7, j1#8] : : +- *(2) Sort [i2#18 ASC NULLS FIRST], false, 0 : : +- Exchange hashpartitioning(i2#18, 200), true, [id=#29] : : +- LocalTableScan [i2#18, j2#19] : +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#34] : +- LocalTableScan [i3#29, j3#30] +- *(5) Sort [i4#40 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(i4#40, 200), true, [id=#39] +- LocalTableScan [i4#40, j4#41] ``` This PR proposes to introduce output partitioning for the build side for `BroadcastHashJoinExec` if the streamed side has a `HashPartitioning` or a collection of `HashPartitioning`s. There is a new internal config `spark.sql.execution.broadcastHashJoin.outputPartitioningExpandLimit`, which can limit the number of partitioning a `HashPartitioning` can expand to. It can be set to "0" to disable this feature. ### Why are the changes needed? To remove unnecessary shuffle. ### Does this PR introduce _any_ user-facing change? Yes, now the shuffle in the above example can be eliminated: ``` == Physical Plan == *(5) SortMergeJoin [i3#108], [i4#119], Inner :- *(3) Sort [i3#108 ASC NULLS FIRST], false, 0 : +- *(3) BroadcastHashJoin [i1#86], [i3#108], Inner, BuildRight : :- *(3) SortMergeJoin [i1#86], [i2#97], Inner : : :- *(1) Sort [i1#86 ASC NULLS FIRST], false, 0 : : : +- Exchange hashpartitioning(i1#86, 200), true, [id=#120] : : : +- LocalTableScan [i1#86, j1#87] : : +- *(2) Sort [i2#97 ASC NULLS FIRST], false, 0 : : +- Exchange hashpartitioning(i2#97, 200), true, [id=#121] : : +- LocalTableScan [i2#97, j2#98] : +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#126] : +- LocalTableScan [i3#108, j3#109] +- *(4) Sort [i4#119 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(i4#119, 200), true, [id=#130] +- LocalTableScan [i4#119, j4#120] ``` ### How was this patch tested? Added new tests. Closes #28676 from imback82/broadcast_join_output. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 14 ++ .../joins/BroadcastHashJoinExec.scala | 79 ++++++- .../spark/sql/execution/joins/HashJoin.scala | 21 +- .../joins/ShuffledHashJoinExec.scala | 3 +- .../adaptive/AdaptiveQueryExecSuite.scala | 3 +- .../execution/joins/BroadcastJoinSuite.scala | 218 +++++++++++++++++- 6 files changed, 322 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4a1299d82f7bc..89c41f31ff234 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2658,6 +2658,17 @@ object SQLConf { .checkValue(_ > 0, "The difference must be positive.") .createWithDefault(4) + val BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT = + buildConf("spark.sql.execution.broadcastHashJoin.outputPartitioningExpandLimit") + .internal() + .doc("The maximum number of partitionings that a HashPartitioning can be expanded to. " + + "This configuration is applicable only for BroadcastHashJoin inner joins and can be " + + "set to '0' to disable this feature.") + .version("3.1.0") + .intConf + .checkValue(_ >= 0, "The value must be non-negative.") + .createWithDefault(8) + /** * Holds information about keys that have been deprecated. * @@ -2966,6 +2977,9 @@ class SQLConf extends Serializable with Logging { LegacyBehaviorPolicy.withName(getConf(SQLConf.LEGACY_TIME_PARSER_POLICY)) } + def broadcastHashJoinOutputPartitioningExpandLimit: Int = + getConf(BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT) + /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two * identifiers are equal. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 707ed1402d1ae..71faad9829a42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import scala.collection.mutable + import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -26,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, HashPartitioning, Partitioning, PartitioningCollection, UnspecifiedDistribution} import org.apache.spark.sql.execution.{CodegenSupport, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.{BooleanType, LongType} @@ -51,7 +53,7 @@ case class BroadcastHashJoinExec( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { - val mode = HashedRelationBroadcastMode(buildKeys) + val mode = HashedRelationBroadcastMode(buildBoundKeys) buildSide match { case BuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil @@ -60,6 +62,73 @@ case class BroadcastHashJoinExec( } } + override lazy val outputPartitioning: Partitioning = { + joinType match { + case _: InnerLike if sqlContext.conf.broadcastHashJoinOutputPartitioningExpandLimit > 0 => + streamedPlan.outputPartitioning match { + case h: HashPartitioning => expandOutputPartitioning(h) + case c: PartitioningCollection => expandOutputPartitioning(c) + case other => other + } + case _ => streamedPlan.outputPartitioning + } + } + + // An one-to-many mapping from a streamed key to build keys. + private lazy val streamedKeyToBuildKeyMapping = { + val mapping = mutable.Map.empty[Expression, Seq[Expression]] + streamedKeys.zip(buildKeys).foreach { + case (streamedKey, buildKey) => + val key = streamedKey.canonicalized + mapping.get(key) match { + case Some(v) => mapping.put(key, v :+ buildKey) + case None => mapping.put(key, Seq(buildKey)) + } + } + mapping.toMap + } + + // Expands the given partitioning collection recursively. + private def expandOutputPartitioning( + partitioning: PartitioningCollection): PartitioningCollection = { + PartitioningCollection(partitioning.partitionings.flatMap { + case h: HashPartitioning => expandOutputPartitioning(h).partitionings + case c: PartitioningCollection => Seq(expandOutputPartitioning(c)) + case other => Seq(other) + }) + } + + // Expands the given hash partitioning by substituting streamed keys with build keys. + // For example, if the expressions for the given partitioning are Seq("a", "b", "c") + // where the streamed keys are Seq("b", "c") and the build keys are Seq("x", "y"), + // the expanded partitioning will have the following expressions: + // Seq("a", "b", "c"), Seq("a", "b", "y"), Seq("a", "x", "c"), Seq("a", "x", "y"). + // The expanded expressions are returned as PartitioningCollection. + private def expandOutputPartitioning(partitioning: HashPartitioning): PartitioningCollection = { + val maxNumCombinations = sqlContext.conf.broadcastHashJoinOutputPartitioningExpandLimit + var currentNumCombinations = 0 + + def generateExprCombinations( + current: Seq[Expression], + accumulated: Seq[Expression]): Seq[Seq[Expression]] = { + if (currentNumCombinations >= maxNumCombinations) { + Nil + } else if (current.isEmpty) { + currentNumCombinations += 1 + Seq(accumulated) + } else { + val buildKeysOpt = streamedKeyToBuildKeyMapping.get(current.head.canonicalized) + generateExprCombinations(current.tail, accumulated :+ current.head) ++ + buildKeysOpt.map(_.flatMap(b => generateExprCombinations(current.tail, accumulated :+ b))) + .getOrElse(Nil) + } + } + + PartitioningCollection( + generateExprCombinations(partitioning.expressions, Nil) + .map(HashPartitioning(_, partitioning.numPartitions))) + } + protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") @@ -135,13 +204,13 @@ case class BroadcastHashJoinExec( ctx: CodegenContext, input: Seq[ExprCode]): (ExprCode, String) = { ctx.currentVars = input - if (streamedKeys.length == 1 && streamedKeys.head.dataType == LongType) { + if (streamedBoundKeys.length == 1 && streamedBoundKeys.head.dataType == LongType) { // generate the join key as Long - val ev = streamedKeys.head.genCode(ctx) + val ev = streamedBoundKeys.head.genCode(ctx) (ev, ev.isNull) } else { // generate the join key as UnsafeRow - val ev = GenerateUnsafeProjection.createCode(ctx, streamedKeys) + val ev = GenerateUnsafeProjection.createCode(ctx, streamedBoundKeys) (ev, s"${ev.value}.anyNull()") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index c7c3e1672f034..7c3c53b0fa54c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -62,21 +62,30 @@ trait HashJoin extends BaseJoinExec { protected lazy val (buildKeys, streamedKeys) = { require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), "Join keys from two sides should have same types") - val lkeys = bindReferences(HashJoin.rewriteKeyExpr(leftKeys), left.output) - val rkeys = bindReferences(HashJoin.rewriteKeyExpr(rightKeys), right.output) buildSide match { - case BuildLeft => (lkeys, rkeys) - case BuildRight => (rkeys, lkeys) + case BuildLeft => (leftKeys, rightKeys) + case BuildRight => (rightKeys, leftKeys) } } + @transient private lazy val (buildOutput, streamedOutput) = { + buildSide match { + case BuildLeft => (left.output, right.output) + case BuildRight => (right.output, left.output) + } + } + + @transient protected lazy val buildBoundKeys = + bindReferences(HashJoin.rewriteKeyExpr(buildKeys), buildOutput) + @transient protected lazy val streamedBoundKeys = + bindReferences(HashJoin.rewriteKeyExpr(streamedKeys), streamedOutput) protected def buildSideKeyGenerator(): Projection = - UnsafeProjection.create(buildKeys) + UnsafeProjection.create(buildBoundKeys) protected def streamSideKeyGenerator(): UnsafeProjection = - UnsafeProjection.create(streamedKeys) + UnsafeProjection.create(streamedBoundKeys) @transient private[this] lazy val boundCondition = if (condition.isDefined) { Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 2b7cd65e7d96f..1120850fdddaf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -55,7 +55,8 @@ case class ShuffledHashJoinExec( val buildTime = longMetric("buildTime") val start = System.nanoTime() val context = TaskContext.get() - val relation = HashedRelation(iter, buildKeys, taskMemoryManager = context.taskMemoryManager()) + val relation = HashedRelation( + iter, buildBoundKeys, taskMemoryManager = context.taskMemoryManager()) buildTime += NANOSECONDS.toMillis(System.nanoTime() - start) buildDataSize += relation.estimatedSize // This relation is usually used until the end of task. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index c696d3f648ed1..511e0cf0b3817 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -557,7 +557,8 @@ class AdaptiveQueryExecSuite withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", + SQLConf.BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT.key -> "0") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM testData " + "join testData2 t2 ON key = t2.a " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index f7d5a899df1c9..7ff945f5cbfb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -21,13 +21,15 @@ import scala.reflect.ClassTag import org.apache.spark.AccumulatorSuite import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, BitwiseOr, Cast, Literal, ShiftLeft} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BitwiseAnd, BitwiseOr, Cast, Expression, Literal, ShiftLeft} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.BROADCAST -import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection} +import org.apache.spark.sql.execution.{DummySparkPlan, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.exchange.EnsureRequirements +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchangeExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -415,6 +417,216 @@ abstract class BroadcastJoinSuiteBase extends QueryTest with SQLTestUtils assert(e.getMessage.contains(s"Could not execute broadcast in $timeout secs.")) } } + + test("broadcast join where streamed side's output partitioning is HashPartitioning") { + withTable("t1", "t3") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val df1 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i1", "j1") + val df2 = (0 until 20).map(i => (i % 7, i % 11)).toDF("i2", "j2") + val df3 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i3", "j3") + df1.write.format("parquet").bucketBy(8, "i1", "j1").saveAsTable("t1") + df3.write.format("parquet").bucketBy(8, "i3", "j3").saveAsTable("t3") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + // join1 is a broadcast join where df2 is broadcasted. Note that output partitioning on the + // streamed side (t1) is HashPartitioning (bucketed files). + val join1 = t1.join(df2, t1("i1") === df2("i2") && t1("j1") === df2("j2")) + val plan1 = join1.queryExecution.executedPlan + assert(collect(plan1) { case e: ShuffleExchangeExec => e }.isEmpty) + val broadcastJoins = collect(plan1) { case b: BroadcastHashJoinExec => b } + assert(broadcastJoins.size == 1) + assert(broadcastJoins(0).outputPartitioning.isInstanceOf[PartitioningCollection]) + val p = broadcastJoins(0).outputPartitioning.asInstanceOf[PartitioningCollection] + assert(p.partitionings.size == 4) + // Verify all the combinations of output partitioning. + Seq(Seq(t1("i1"), t1("j1")), + Seq(t1("i1"), df2("j2")), + Seq(df2("i2"), t1("j1")), + Seq(df2("i2"), df2("j2"))).foreach { expected => + val expectedExpressions = expected.map(_.expr) + assert(p.partitionings.exists { + case h: HashPartitioning => expressionsEqual(h.expressions, expectedExpressions) + }) + } + + // Join on the column from the broadcasted side (i2, j2) and make sure output partitioning + // is maintained by checking no shuffle exchange is introduced. + val join2 = join1.join(t3, join1("i2") === t3("i3") && join1("j2") === t3("j3")) + val plan2 = join2.queryExecution.executedPlan + assert(collect(plan2) { case s: SortMergeJoinExec => s }.size == 1) + assert(collect(plan2) { case b: BroadcastHashJoinExec => b }.size == 1) + assert(collect(plan2) { case e: ShuffleExchangeExec => e }.isEmpty) + + // Validate the data with broadcast join off. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = join1.join(t3, join1("i2") === t3("i3") && join1("j2") === t3("j3")) + checkAnswer(join2, df) + } + } + } + } + + test("broadcast join where streamed side's output partitioning is PartitioningCollection") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val t1 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i1", "j1") + val t2 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i2", "j2") + val t3 = (0 until 20).map(i => (i % 7, i % 11)).toDF("i3", "j3") + val t4 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i4", "j4") + + // join1 is a sort merge join (shuffle on the both sides). + val join1 = t1.join(t2, t1("i1") === t2("i2")) + val plan1 = join1.queryExecution.executedPlan + assert(collect(plan1) { case s: SortMergeJoinExec => s }.size == 1) + assert(collect(plan1) { case e: ShuffleExchangeExec => e }.size == 2) + + // join2 is a broadcast join where t3 is broadcasted. Note that output partitioning on the + // streamed side (join1) is PartitioningCollection (sort merge join) + val join2 = join1.join(t3, join1("i1") === t3("i3")) + val plan2 = join2.queryExecution.executedPlan + assert(collect(plan2) { case s: SortMergeJoinExec => s }.size == 1) + assert(collect(plan2) { case e: ShuffleExchangeExec => e }.size == 2) + val broadcastJoins = collect(plan2) { case b: BroadcastHashJoinExec => b } + assert(broadcastJoins.size == 1) + assert(broadcastJoins(0).outputPartitioning.isInstanceOf[PartitioningCollection]) + val p = broadcastJoins(0).outputPartitioning.asInstanceOf[PartitioningCollection] + assert(p.partitionings.size == 3) + // Verify all the combinations of output partitioning. + Seq(Seq(t1("i1")), Seq(t2("i2")), Seq(t3("i3"))).foreach { expected => + val expectedExpressions = expected.map(_.expr) + assert(p.partitionings.exists { + case h: HashPartitioning => expressionsEqual(h.expressions, expectedExpressions) + }) + } + + // Join on the column from the broadcasted side (i3) and make sure output partitioning + // is maintained by checking no shuffle exchange is introduced. Note that one extra + // ShuffleExchangeExec is from t4, not from join2. + val join3 = join2.join(t4, join2("i3") === t4("i4")) + val plan3 = join3.queryExecution.executedPlan + assert(collect(plan3) { case s: SortMergeJoinExec => s }.size == 2) + assert(collect(plan3) { case b: BroadcastHashJoinExec => b }.size == 1) + assert(collect(plan3) { case e: ShuffleExchangeExec => e }.size == 3) + + // Validate the data with broadcast join off. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = join2.join(t4, join2("i3") === t4("i4")) + checkAnswer(join3, df) + } + } + } + + test("BroadcastHashJoinExec output partitioning scenarios for inner join") { + val l1 = AttributeReference("l1", LongType)() + val l2 = AttributeReference("l2", LongType)() + val l3 = AttributeReference("l3", LongType)() + val r1 = AttributeReference("r1", LongType)() + val r2 = AttributeReference("r2", LongType)() + val r3 = AttributeReference("r3", LongType)() + + // Streamed side has a HashPartitioning. + var bhj = BroadcastHashJoinExec( + leftKeys = Seq(l2, l3), + rightKeys = Seq(r1, r2), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(l1, l2, l3), 1)), + right = DummySparkPlan()) + var expected = PartitioningCollection(Seq( + HashPartitioning(Seq(l1, l2, l3), 1), + HashPartitioning(Seq(l1, l2, r2), 1), + HashPartitioning(Seq(l1, r1, l3), 1), + HashPartitioning(Seq(l1, r1, r2), 1))) + assert(bhj.outputPartitioning === expected) + + // Streamed side has a PartitioningCollection. + bhj = BroadcastHashJoinExec( + leftKeys = Seq(l1, l2, l3), + rightKeys = Seq(r1, r2, r3), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(l1, l2), 1), HashPartitioning(Seq(l3), 1)))), + right = DummySparkPlan()) + expected = PartitioningCollection(Seq( + HashPartitioning(Seq(l1, l2), 1), + HashPartitioning(Seq(l1, r2), 1), + HashPartitioning(Seq(r1, l2), 1), + HashPartitioning(Seq(r1, r2), 1), + HashPartitioning(Seq(l3), 1), + HashPartitioning(Seq(r3), 1))) + assert(bhj.outputPartitioning === expected) + + // Streamed side has a nested PartitioningCollection. + bhj = BroadcastHashJoinExec( + leftKeys = Seq(l1, l2, l3), + rightKeys = Seq(r1, r2, r3), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + PartitioningCollection(Seq(HashPartitioning(Seq(l1), 1), HashPartitioning(Seq(l2), 1))), + HashPartitioning(Seq(l3), 1)))), + right = DummySparkPlan()) + expected = PartitioningCollection(Seq( + PartitioningCollection(Seq( + HashPartitioning(Seq(l1), 1), + HashPartitioning(Seq(r1), 1), + HashPartitioning(Seq(l2), 1), + HashPartitioning(Seq(r2), 1))), + HashPartitioning(Seq(l3), 1), + HashPartitioning(Seq(r3), 1))) + assert(bhj.outputPartitioning === expected) + + // One-to-mapping case ("l1" = "r1" AND "l1" = "r2") + bhj = BroadcastHashJoinExec( + leftKeys = Seq(l1, l1), + rightKeys = Seq(r1, r2), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(l1, l2), 1)), + right = DummySparkPlan()) + expected = PartitioningCollection(Seq( + HashPartitioning(Seq(l1, l2), 1), + HashPartitioning(Seq(r1, l2), 1), + HashPartitioning(Seq(r2, l2), 1))) + assert(bhj.outputPartitioning === expected) + } + + test("BroadcastHashJoinExec output partitioning size should be limited with a config") { + val l1 = AttributeReference("l1", LongType)() + val l2 = AttributeReference("l2", LongType)() + val r1 = AttributeReference("r1", LongType)() + val r2 = AttributeReference("r2", LongType)() + + val expected = Seq( + HashPartitioning(Seq(l1, l2), 1), + HashPartitioning(Seq(l1, r2), 1), + HashPartitioning(Seq(r1, l2), 1), + HashPartitioning(Seq(r1, r2), 1)) + + Seq(1, 2, 3, 4).foreach { limit => + withSQLConf( + SQLConf.BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT.key -> s"$limit") { + val bhj = BroadcastHashJoinExec( + leftKeys = Seq(l1, l2), + rightKeys = Seq(r1, r2), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(l1, l2), 1)), + right = DummySparkPlan()) + assert(bhj.outputPartitioning === PartitioningCollection(expected.take(limit))) + } + } + } + + private def expressionsEqual(l: Seq[Expression], r: Seq[Expression]): Boolean = { + l.length == r.length && l.zip(r).forall { case (e1, e2) => e1.semanticEquals(e2) } + } } class BroadcastJoinSuite extends BroadcastJoinSuiteBase with DisableAdaptiveExecutionSuite From fe07521c9efd9ce0913eee0d42b0ffd98b1225ec Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Mon, 20 Jul 2020 14:38:43 +0000 Subject: [PATCH 233/384] [SPARK-32330][SQL] Preserve shuffled hash join build side partitioning ### What changes were proposed in this pull request? Currently `ShuffledHashJoin.outputPartitioning` inherits from `HashJoin.outputPartitioning`, which only preserves stream side partitioning (`HashJoin.scala`): ``` override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning ``` This loses build side partitioning information, and causes extra shuffle if there's another join / group-by after this join. Example: ``` withSQLConf( SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50", SQLConf.SHUFFLE_PARTITIONS.key -> "2", SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { val df1 = spark.range(10).select($"id".as("k1")) val df2 = spark.range(30).select($"id".as("k2")) Seq("inner", "cross").foreach(joinType => { val plan = df1.join(df2, $"k1" === $"k2", joinType).groupBy($"k1").count() .queryExecution.executedPlan assert(plan.collect { case _: ShuffledHashJoinExec => true }.size === 1) // No extra shuffle before aggregate assert(plan.collect { case _: ShuffleExchangeExec => true }.size === 2) }) } ``` Current physical plan (having an extra shuffle on `k1` before aggregate) ``` *(4) HashAggregate(keys=[k1#220L], functions=[count(1)], output=[k1#220L, count#235L]) +- Exchange hashpartitioning(k1#220L, 2), true, [id=#117] +- *(3) HashAggregate(keys=[k1#220L], functions=[partial_count(1)], output=[k1#220L, count#239L]) +- *(3) Project [k1#220L] +- ShuffledHashJoin [k1#220L], [k2#224L], Inner, BuildLeft :- Exchange hashpartitioning(k1#220L, 2), true, [id=#109] : +- *(1) Project [id#218L AS k1#220L] : +- *(1) Range (0, 10, step=1, splits=2) +- Exchange hashpartitioning(k2#224L, 2), true, [id=#111] +- *(2) Project [id#222L AS k2#224L] +- *(2) Range (0, 30, step=1, splits=2) ``` Ideal physical plan (no shuffle on `k1` before aggregate) ``` *(3) HashAggregate(keys=[k1#220L], functions=[count(1)], output=[k1#220L, count#235L]) +- *(3) HashAggregate(keys=[k1#220L], functions=[partial_count(1)], output=[k1#220L, count#239L]) +- *(3) Project [k1#220L] +- ShuffledHashJoin [k1#220L], [k2#224L], Inner, BuildLeft :- Exchange hashpartitioning(k1#220L, 2), true, [id=#107] : +- *(1) Project [id#218L AS k1#220L] : +- *(1) Range (0, 10, step=1, splits=2) +- Exchange hashpartitioning(k2#224L, 2), true, [id=#109] +- *(2) Project [id#222L AS k2#224L] +- *(2) Range (0, 30, step=1, splits=2) ``` This can be fixed by overriding `outputPartitioning` method in `ShuffledHashJoinExec`, similar to `SortMergeJoinExec`. In addition, also fix one typo in `HashJoin`, as that code path is shared between broadcast hash join and shuffled hash join. ### Why are the changes needed? To avoid shuffle (for queries having multiple joins or group-by), for saving CPU and IO. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit test in `JoinSuite`. Closes #29130 from c21/shj. Authored-by: Cheng Su Signed-off-by: Wenchen Fan --- .../spark/sql/execution/joins/HashJoin.scala | 2 +- .../joins/ShuffledHashJoinExec.scala | 5 +-- .../sql/execution/joins/ShuffledJoin.scala | 43 +++++++++++++++++++ .../execution/joins/SortMergeJoinExec.scala | 17 +------- .../org/apache/spark/sql/JoinSuite.scala | 18 ++++++++ 5 files changed, 66 insertions(+), 19 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 7c3c53b0fa54c..8d9ba54f6568d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -215,7 +215,7 @@ trait HashJoin extends BaseJoinExec { existenceJoin(streamedIter, hashed) case x => throw new IllegalArgumentException( - s"BroadcastHashJoin should not take $x as the JoinType") + s"HashJoin should not take $x as the JoinType") } val resultProj = createResultProjection diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 1120850fdddaf..3b398dd7120c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -40,15 +40,14 @@ case class ShuffledHashJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan) - extends HashJoin { + extends HashJoin with ShuffledJoin { override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"), "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) - override def requiredChildDistribution: Seq[Distribution] = - HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + override def outputPartitioning: Partitioning = super[ShuffledJoin].outputPartitioning private def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = { val buildDataSize = longMetric("buildDataSize") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala new file mode 100644 index 0000000000000..7035ddc35be9c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.apache.spark.sql.catalyst.plans.{FullOuter, InnerLike, LeftExistence, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution, Partitioning, PartitioningCollection, UnknownPartitioning} + +/** + * Holds common logic for join operators by shuffling two child relations + * using the join keys. + */ +trait ShuffledJoin extends BaseJoinExec { + override def requiredChildDistribution: Seq[Distribution] = { + HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + } + + override def outputPartitioning: Partitioning = joinType match { + case _: InnerLike => + PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning)) + case LeftOuter => left.outputPartitioning + case RightOuter => right.outputPartitioning + case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) + case LeftExistence(_) => left.outputPartitioning + case x => + throw new IllegalArgumentException( + s"ShuffledJoin should not take $x as the JoinType") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 2c57956de5bca..b9f6684447dd8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -41,7 +41,7 @@ case class SortMergeJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan, - isSkewJoin: Boolean = false) extends BaseJoinExec with CodegenSupport { + isSkewJoin: Boolean = false) extends ShuffledJoin with CodegenSupport { override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -72,26 +72,13 @@ case class SortMergeJoinExec( } } - override def outputPartitioning: Partitioning = joinType match { - case _: InnerLike => - PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning)) - // For left and right outer joins, the output is partitioned by the streamed input's join keys. - case LeftOuter => left.outputPartitioning - case RightOuter => right.outputPartitioning - case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) - case LeftExistence(_) => left.outputPartitioning - case x => - throw new IllegalArgumentException( - s"${getClass.getSimpleName} should not take $x as the JoinType") - } - override def requiredChildDistribution: Seq[Distribution] = { if (isSkewJoin) { // We re-arrange the shuffle partitions to deal with skew join, and the new children // partitioning doesn't satisfy `HashClusteredDistribution`. UnspecifiedDistribution :: UnspecifiedDistribution :: Nil } else { - HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + super.requiredChildDistribution } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index f24da6df67ca0..b4f626270cfc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrd import org.apache.spark.sql.catalyst.plans.logical.Filter import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf @@ -1086,4 +1087,21 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan assert(df2.join(df1, "id").collect().isEmpty) } } + + test("SPARK-32330: Preserve shuffled hash join build side partitioning") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { + val df1 = spark.range(10).select($"id".as("k1")) + val df2 = spark.range(30).select($"id".as("k2")) + Seq("inner", "cross").foreach(joinType => { + val plan = df1.join(df2, $"k1" === $"k2", joinType).groupBy($"k1").count() + .queryExecution.executedPlan + assert(plan.collect { case _: ShuffledHashJoinExec => true }.size === 1) + // No extra shuffle before aggregate + assert(plan.collect { case _: ShuffleExchangeExec => true }.size === 2) + }) + } + } } From ffdca8285ef7c7bd0da2622a81d9c21ada035794 Mon Sep 17 00:00:00 2001 From: maruilei Date: Mon, 20 Jul 2020 13:48:57 -0700 Subject: [PATCH 234/384] [SPARK-32367][K8S][TESTS] Correct the spelling of parameter in KubernetesTestComponents ### What changes were proposed in this pull request? Correct the spelling of parameter 'spark.executor.instances' in KubernetesTestComponents ### Why are the changes needed? Parameter spelling error ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Test is not needed. Closes #29164 from merrily01/SPARK-32367. Authored-by: maruilei Signed-off-by: Dongjoon Hyun --- .../deploy/k8s/integrationtest/KubernetesTestComponents.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 4cfda8ab9ba77..af980f0494369 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -69,7 +69,7 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .set("spark.master", s"k8s://${kubernetesClient.getMasterUrl}") .set("spark.kubernetes.namespace", namespace) .set("spark.executor.cores", "1") - .set("spark.executors.instances", "1") + .set("spark.executor.instances", "1") .set("spark.app.name", "spark-test-app") .set(IS_TESTING.key, "false") .set(UI_ENABLED.key, "true") From 133c5edc807ca87825f61dd9a5d36018620033ee Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 20 Jul 2020 13:56:00 -0700 Subject: [PATCH 235/384] [SPARK-32368][SQL] pathGlobFilter, recursiveFileLookup and basePath should respect case insensitivity ### What changes were proposed in this pull request? This PR proposes to make the datasource options at `PartitioningAwareFileIndex` respect case insensitivity consistently: - `pathGlobFilter` - `recursiveFileLookup ` - `basePath` ### Why are the changes needed? To support consistent case insensitivity in datasource options. ### Does this PR introduce _any_ user-facing change? Yes, now users can also use case insensitive options such as `PathglobFilter`. ### How was this patch tested? Unittest were added. It reuses existing tests and adds extra clues to make it easier to track when the test is broken. Closes #29165 from HyukjinKwon/SPARK-32368. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../datasources/PartitioningAwareFileIndex.scala | 11 +++++++---- .../spark/sql/FileBasedDataSourceSuite.scala | 16 +++++++++------- .../execution/datasources/FileIndexSuite.scala | 16 +++++++++------- 3 files changed, 25 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 2e09c729529a6..5341e22f5e670 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -56,14 +56,17 @@ abstract class PartitioningAwareFileIndex( protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] - protected lazy val pathGlobFilter = parameters.get("pathGlobFilter").map(new GlobFilter(_)) + private val caseInsensitiveMap = CaseInsensitiveMap(parameters) + + protected lazy val pathGlobFilter: Option[GlobFilter] = + caseInsensitiveMap.get("pathGlobFilter").map(new GlobFilter(_)) protected def matchGlobPattern(file: FileStatus): Boolean = { pathGlobFilter.forall(_.accept(file.getPath)) } - protected lazy val recursiveFileLookup = { - parameters.getOrElse("recursiveFileLookup", "false").toBoolean + protected lazy val recursiveFileLookup: Boolean = { + caseInsensitiveMap.getOrElse("recursiveFileLookup", "false").toBoolean } override def listFiles( @@ -215,7 +218,7 @@ abstract class PartitioningAwareFileIndex( * and the returned DataFrame will have the column of `something`. */ private def basePaths: Set[Path] = { - parameters.get(BASE_PATH_PARAM).map(new Path(_)) match { + caseInsensitiveMap.get(BASE_PATH_PARAM).map(new Path(_)) match { case Some(userDefinedBasePath) => val fs = userDefinedBasePath.getFileSystem(hadoopConf) if (!fs.isDirectory(userDefinedBasePath)) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 231a8f2aa7ddd..e9bff64d72fc3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -633,13 +633,15 @@ class FileBasedDataSourceSuite extends QueryTest assert(fileList.toSet === expectedFileList.toSet) - val fileList2 = spark.read.format("binaryFile") - .option("recursiveFileLookup", true) - .option("pathGlobFilter", "*.bin") - .load(dataPath) - .select("path").collect().map(_.getString(0)) - - assert(fileList2.toSet === expectedFileList.filter(_.endsWith(".bin")).toSet) + withClue("SPARK-32368: 'recursiveFileLookup' and 'pathGlobFilter' can be case insensitive") { + val fileList2 = spark.read.format("binaryFile") + .option("RecuRsivefileLookup", true) + .option("PaThglobFilter", "*.bin") + .load(dataPath) + .select("path").collect().map(_.getString(0)) + + assert(fileList2.toSet === expectedFileList.filter(_.endsWith(".bin")).toSet) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index 040996276063b..02be8c9221704 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -367,13 +367,15 @@ class FileIndexSuite extends SharedSparkSession { val wrongBasePath = new File(dir, "unknown") // basePath must be a directory wrongBasePath.mkdir() - val parameters = Map("basePath" -> wrongBasePath.getCanonicalPath) - val fileIndex = new InMemoryFileIndex(spark, Seq(path), parameters, None) - val msg = intercept[IllegalArgumentException] { - // trigger inferPartitioning() - fileIndex.partitionSpec() - }.getMessage - assert(msg === s"Wrong basePath ${wrongBasePath.getCanonicalPath} for the root path: $path") + withClue("SPARK-32368: 'basePath' can be case insensitive") { + val parameters = Map("bAsepAtH" -> wrongBasePath.getCanonicalPath) + val fileIndex = new InMemoryFileIndex(spark, Seq(path), parameters, None) + val msg = intercept[IllegalArgumentException] { + // trigger inferPartitioning() + fileIndex.partitionSpec() + }.getMessage + assert(msg === s"Wrong basePath ${wrongBasePath.getCanonicalPath} for the root path: $path") + } } } From 7d65caebec7da6ee7131b8121e103a9f8ff40d6e Mon Sep 17 00:00:00 2001 From: Nik Vanderhoof Date: Mon, 20 Jul 2020 17:48:07 -0700 Subject: [PATCH 236/384] [SPARK-32338][SQL] Overload slice to accept Column for start and length ### What changes were proposed in this pull request? Add an overload for the `slice` function that can accept Columns for the `start` and `length` parameters. ### Why are the changes needed? This will allow users to take slices of arrays based on the length of the arrays, or via data in other columns. ```scala df.select(slice(x, 4, size(x) - 4)) ``` ### Does this PR introduce _any_ user-facing change? Yes, before the `slice` method would only accept Ints for the start and length parameters, now we can pass in Columns and/or Ints. ### How was this patch tested? I've extended the existing tests for slice but using combinations of Column and Ints. Closes #29138 from nvander1/SPARK-32338. Authored-by: Nik Vanderhoof Signed-off-by: Takuya UESHIN --- .../scala/org/apache/spark/sql/functions.scala | 18 ++++++++++++++++-- .../spark/sql/DataFrameFunctionsSuite.scala | 2 ++ 2 files changed, 18 insertions(+), 2 deletions(-) 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 d8e39d416d877..fe0057c3d588b 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 @@ -3402,8 +3402,22 @@ object functions { * @group collection_funcs * @since 2.4.0 */ - def slice(x: Column, start: Int, length: Int): Column = withExpr { - Slice(x.expr, Literal(start), Literal(length)) + def slice(x: Column, start: Int, length: Int): Column = + slice(x, lit(start), lit(length)) + + /** + * Returns an array containing all the elements in `x` from index `start` (or starting from the + * end if `start` is negative) with the specified `length`. + * + * @param x the array column to be sliced + * @param start the starting index + * @param length the length of the slice + * + * @group collection_funcs + * @since 3.1.0 + */ + def slice(x: Column, start: Column, length: Column): Column = withExpr { + Slice(x.expr, start.expr, length.expr) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index cb11519497747..585a835024e19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -936,10 +936,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val answer = Seq(Row(Seq(2, 3)), Row(Seq(5))) checkAnswer(df.select(slice(df("x"), 2, 2)), answer) + checkAnswer(df.select(slice(df("x"), lit(2), lit(2))), answer) checkAnswer(df.selectExpr("slice(x, 2, 2)"), answer) val answerNegative = Seq(Row(Seq(3)), Row(Seq(5))) checkAnswer(df.select(slice(df("x"), -1, 1)), answerNegative) + checkAnswer(df.select(slice(df("x"), lit(-1), lit(1))), answerNegative) checkAnswer(df.selectExpr("slice(x, -1, 1)"), answerNegative) } From 02114f96d64ec5be23fc61be6f6b32df7ad48a6c Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Mon, 20 Jul 2020 20:34:51 -0700 Subject: [PATCH 237/384] [SPARK-32365][SQL] Add a boundary condition for negative index in regexp_extract ### What changes were proposed in this pull request? The current implement of regexp_extract will throws a unprocessed exception show below: SELECT regexp_extract('1a 2b 14m', 'd+' -1) ``` java.lang.IndexOutOfBoundsException: No group -1 java.util.regex.Matcher.group(Matcher.java:538) org.apache.spark.sql.catalyst.expressions.RegExpExtract.nullSafeEval(regexpExpressions.scala:455) org.apache.spark.sql.catalyst.expressions.TernaryExpression.eval(Expression.scala:704) org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52) org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45) ``` ### Why are the changes needed? Fix a bug `java.lang.IndexOutOfBoundsException: No group -1` ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? new UT Closes #29161 from beliefer/regexp_extract-group-not-allow-less-than-zero. Authored-by: gengjiaan Signed-off-by: Dongjoon Hyun --- .../expressions/regexpExpressions.scala | 12 ++++++++++- .../expressions/RegexpExpressionsSuite.scala | 6 ++++++ .../sql-tests/inputs/regexp-functions.sql | 2 ++ .../results/regexp-functions.sql.out | 20 ++++++++++++++++++- 4 files changed, 38 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index 28924fac48eef..1af1636e1df75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -412,7 +412,9 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio object RegExpExtract { def checkGroupIndex(groupCount: Int, groupIndex: Int): Unit = { - if (groupCount < groupIndex) { + if (groupIndex < 0) { + throw new IllegalArgumentException("The specified group index cannot be less than zero") + } else if (groupCount < groupIndex) { throw new IllegalArgumentException( s"Regex group count is $groupCount, but the specified group index is $groupIndex") } @@ -426,6 +428,14 @@ object RegExpExtract { */ @ExpressionDescription( usage = "_FUNC_(str, regexp[, idx]) - Extracts a group that matches `regexp`.", + arguments = """ + Arguments: + * str - a string expression. + * regexp - a string representing a regular expression. + The regex string should be a Java regular expression. + * idx - an integer expression that representing the group index. The group index should be + non-negative. If `idx` is not specified, the default group index value is 1. + """, examples = """ Examples: > SELECT _FUNC_('100-200', '(\\d+)-(\\d+)', 1); diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index ad9492a8d3ab1..cab81f85fda06 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -303,6 +303,8 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row8 = create_row("100-200", "(\\d+)-(\\d+)", 3) val row9 = create_row("100-200", "(\\d+).*", 2) val row10 = create_row("100-200", "\\d+", 1) + val row11 = create_row("100-200", "(\\d+)-(\\d+)", -1) + val row12 = create_row("100-200", "\\d+", -1) checkExceptionInExpression[IllegalArgumentException]( expr, row8, "Regex group count is 2, but the specified group index is 3") @@ -310,6 +312,10 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { expr, row9, "Regex group count is 1, but the specified group index is 2") checkExceptionInExpression[IllegalArgumentException]( expr, row10, "Regex group count is 0, but the specified group index is 1") + checkExceptionInExpression[IllegalArgumentException]( + expr, row11, "The specified group index cannot be less than zero") + checkExceptionInExpression[IllegalArgumentException]( + expr, row12, "The specified group index cannot be less than zero") // Test escaping of arguments GenerateUnsafeProjection.generate( diff --git a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql index c0827a3cba39b..8a531be30d896 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql @@ -3,7 +3,9 @@ SELECT regexp_extract('1a 2b 14m', '\\d+'); SELECT regexp_extract('1a 2b 14m', '\\d+', 0); SELECT regexp_extract('1a 2b 14m', '\\d+', 1); SELECT regexp_extract('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract('1a 2b 14m', '\\d+', -1); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)'); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1); diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out index c92c1ddca774f..64aa6053d8d70 100644 --- a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 8 +-- Number of queries: 10 -- !query @@ -37,6 +37,15 @@ java.lang.IllegalArgumentException Regex group count is 0, but the specified group index is 2 +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + -- !query SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)') -- !query schema @@ -67,3 +76,12 @@ SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2) struct -- !query output a + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero \ No newline at end of file From 8a1c24bb0364313f20382e2d14d5670b111a5742 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 21 Jul 2020 12:47:45 +0900 Subject: [PATCH 238/384] [SPARK-32362][SQL][TEST] AdaptiveQueryExecSuite misses verifying AE results ### What changes were proposed in this pull request? Verify results for `AdaptiveQueryExecSuite` ### Why are the changes needed? `AdaptiveQueryExecSuite` misses verifying AE results ```scala QueryTest.sameRows(result.toSeq, df.collect().toSeq) ``` Even the results are different, no fail. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Exists unit tests. Closes #29158 from LantaoJin/SPARK-32362. Authored-by: LantaoJin Signed-off-by: HyukjinKwon --- .../spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 511e0cf0b3817..03471fb047260 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -68,7 +68,7 @@ class AdaptiveQueryExecSuite val result = dfAdaptive.collect() withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { val df = sql(query) - QueryTest.sameRows(result.toSeq, df.collect().toSeq) + checkAnswer(df, result) } val planAfter = dfAdaptive.queryExecution.executedPlan assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) From 1267d80db6abaa130384b8e7b514c39aec3a8c77 Mon Sep 17 00:00:00 2001 From: Brandon Date: Tue, 21 Jul 2020 13:42:19 +0900 Subject: [PATCH 239/384] [MINOR][DOCS] add link for Debugging your Application in running-on-yarn.html#launching-spark-on-yarn ### What changes were proposed in this pull request? add link for Debugging your Application in `running-on-yarn.html#launching-spark-on-yar` ### Why are the changes needed? Currrently on running-on-yarn.html page launching-spark-on-yarn section, it mentions to refer for Debugging your Application. It is better to add a direct link for it to save reader time to find the section ![image](https://user-images.githubusercontent.com/20021316/87867542-80cc5500-c9c0-11ea-8560-5ddcb5a308bc.png) ### Does this PR introduce _any_ user-facing change? Yes. Docs changes. 1. add link for Debugging your Application in `running-on-yarn.html#launching-spark-on-yarn` section Updated behavior: ![image](https://user-images.githubusercontent.com/20021316/87867534-6eeab200-c9c0-11ea-94ee-d3fa58157156.png) 2. update Spark Properties link to anchor link only ### How was this patch tested? manual test has been performed to test the updated Closes #29154 from brandonJY/patch-1. Authored-by: Brandon Signed-off-by: HyukjinKwon --- docs/running-on-yarn.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index eeeb71b022c52..36d8f0b7ab8df 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -60,7 +60,7 @@ For example: examples/jars/spark-examples*.jar \ 10 -The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Debugging your Application" section below for how to see driver and executor logs. +The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the [Debugging your Application](#debugging-your-application) section below for how to see driver and executor logs. To launch a Spark application in `client` mode, do the same, but replace `cluster` with `client`. The following shows how you can run `spark-shell` in `client` mode: @@ -96,7 +96,7 @@ in your application jar. To build Spark yourself, refer to [Building Spark](building-spark.html). -To make Spark runtime jars accessible from YARN side, you can specify `spark.yarn.archive` or `spark.yarn.jars`. For details please refer to [Spark Properties](running-on-yarn.html#spark-properties). If neither `spark.yarn.archive` nor `spark.yarn.jars` is specified, Spark will create a zip file with all jars under `$SPARK_HOME/jars` and upload it to the distributed cache. +To make Spark runtime jars accessible from YARN side, you can specify `spark.yarn.archive` or `spark.yarn.jars`. For details please refer to [Spark Properties](#spark-properties). If neither `spark.yarn.archive` nor `spark.yarn.jars` is specified, Spark will create a zip file with all jars under `$SPARK_HOME/jars` and upload it to the distributed cache. # Configuration From 8c7d6f9733751503f80d5a1b2463904dfefd6843 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 20 Jul 2020 22:20:16 -0700 Subject: [PATCH 240/384] [SPARK-32377][SQL] CaseInsensitiveMap should be deterministic for addition ### What changes were proposed in this pull request? This PR aims to fix `CaseInsensitiveMap` to be deterministic for addition. ### Why are the changes needed? ```scala import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap var m = CaseInsensitiveMap(Map.empty[String, String]) Seq(("paTh", "1"), ("PATH", "2"), ("Path", "3"), ("patH", "4"), ("path", "5")).foreach { kv => m = (m + kv).asInstanceOf[CaseInsensitiveMap[String]] println(m.get("path")) } ``` **BEFORE** ``` Some(1) Some(2) Some(3) Some(4) Some(1) ``` **AFTER** ``` Some(1) Some(2) Some(3) Some(4) Some(5) ``` ### Does this PR introduce _any_ user-facing change? Yes, but this is a bug fix on non-deterministic behavior. ### How was this patch tested? Pass the newly added test case. Closes #29172 from dongjoon-hyun/SPARK-32377. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/util/CaseInsensitiveMap.scala | 2 +- .../spark/sql/catalyst/util/CaseInsensitiveMap.scala | 2 +- .../spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala | 8 ++++++++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index 699b42bd6d60d..aff294e5d6305 100644 --- a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -40,7 +40,7 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) override def +[B1 >: T](kv: (String, B1)): Map[String, B1] = { - new CaseInsensitiveMap(originalMap + kv) + new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(kv._1)) + kv) } override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index 1f8cc6c0f83a2..a74636a80f1ce 100644 --- a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -40,7 +40,7 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) override def updated[B1 >: T](key: String, value: B1): Map[String, B1] = { - new CaseInsensitiveMap[B1](originalMap + (key -> value)) + new CaseInsensitiveMap[B1](originalMap.filter(!_._1.equalsIgnoreCase(key)) + (key -> value)) } override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala index a8bb1d0afdb87..db8b7403cb60e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala @@ -44,4 +44,12 @@ class CaseInsensitiveMapSuite extends SparkFunSuite { assert(m == Map("a" -> "b", "foo" -> "bar", "x" -> "y")) shouldBeSerializable(m) } + + test("SPARK-32377: CaseInsensitiveMap should be deterministic for addition") { + var m = CaseInsensitiveMap(Map.empty[String, String]) + Seq(("paTh", "1"), ("PATH", "2"), ("Path", "3"), ("patH", "4"), ("path", "5")).foreach { kv => + m = (m + kv).asInstanceOf[CaseInsensitiveMap[String]] + assert(m.get("path").contains(kv._2)) + } + } } From 4da93b00d7c8b6dd35ae37ece584aac1d7793c33 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 21 Jul 2020 22:49:14 +0900 Subject: [PATCH 241/384] [SPARK-32363][PYTHON][BUILD] Fix flakiness in pip package testing in Jenkins ### What changes were proposed in this pull request? This PR proposes: - Don't use `--user` in pip packaging test - Pull `source` out of the subshell, and place it first. - Exclude user sitepackages in Python path during pip installation test to address the flakiness of the pip packaging test in Jenkins. (I think) #29116 caused this flakiness given my observation in the Jenkins log. I had to work around by specifying `--user` but it turned out that it does not properly work in old Conda on Jenkins for some reasons. Therefore, reverting this change back. (I think) the installation at user site-packages affects other environments created by Conda in the old Conda version that Jenkins has. Seems it fails to isolate the environments for some reasons. So, it excludes user sitepackages in the Python path during the test. In addition, #29116 also added some fallback logics of `conda (de)activate` and `source (de)activate` because Conda prefers to use `conda (de)activate` now per the official documentation and `source (de)activate` doesn't work for some reasons in certain environments (see also https://github.com/conda/conda/issues/7980). The problem was that `source` loads things to the current shell so does not affect the current shell. Therefore, this PR pulls `source` out of the subshell. Disclaimer: I made the analysis purely based on Jenkins machine's log in this PR. It may have a different reason I missed during my observation. ### Why are the changes needed? To make the build and tests pass in Jenkins. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Jenkins tests should test it out. Closes #29117 from HyukjinKwon/debug-conda. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/run-pip-tests | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 5fd0be7476f29..be96ed9dda20f 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -63,11 +63,15 @@ fi PYSPARK_VERSION=$(python3 -c "exec(open('python/pyspark/version.py').read());print(__version__)") PYSPARK_DIST="$FWDIR/python/dist/pyspark-$PYSPARK_VERSION.tar.gz" # The pip install options we use for all the pip commands -PIP_OPTIONS="--user --upgrade --no-cache-dir --force-reinstall " +PIP_OPTIONS="--upgrade --no-cache-dir --force-reinstall" # Test both regular user and edit/dev install modes. PIP_COMMANDS=("pip install $PIP_OPTIONS $PYSPARK_DIST" "pip install $PIP_OPTIONS -e python/") +# Jenkins has PySpark installed under user sitepackages shared for some reasons. +# In this test, explicitly exclude user sitepackages to prevent side effects +export PYTHONNOUSERSITE=1 + for python in "${PYTHON_EXECS[@]}"; do for install_command in "${PIP_COMMANDS[@]}"; do echo "Testing pip installation with python $python" @@ -81,7 +85,7 @@ for python in "${PYTHON_EXECS[@]}"; do source "$CONDA_PREFIX/etc/profile.d/conda.sh" fi conda create -y -p "$VIRTUALENV_PATH" python=$python numpy pandas pip setuptools - conda activate "$VIRTUALENV_PATH" || (echo "Falling back to 'source activate'" && source activate "$VIRTUALENV_PATH") + source activate "$VIRTUALENV_PATH" || (echo "Falling back to 'conda activate'" && conda activate "$VIRTUALENV_PATH") else mkdir -p "$VIRTUALENV_PATH" virtualenv --python=$python "$VIRTUALENV_PATH" @@ -96,8 +100,6 @@ for python in "${PYTHON_EXECS[@]}"; do cd "$FWDIR"/python # Delete the egg info file if it exists, this can cache the setup file. rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" - # Also, delete the symbolic link if exists. It can be left over from the previous editable mode installation. - python3 -c "from distutils.sysconfig import get_python_lib; import os; f = os.path.join(get_python_lib(), 'pyspark.egg-link'); os.unlink(f) if os.path.isfile(f) else 0" python3 setup.py sdist @@ -116,7 +118,6 @@ for python in "${PYTHON_EXECS[@]}"; do cd / echo "Run basic sanity check on pip installed version with spark-submit" - export PATH="$(python3 -m site --user-base)/bin:$PATH" spark-submit "$FWDIR"/dev/pip-sanity-check.py echo "Run basic sanity check with import based" python3 "$FWDIR"/dev/pip-sanity-check.py @@ -127,7 +128,7 @@ for python in "${PYTHON_EXECS[@]}"; do # conda / virtualenv environments need to be deactivated differently if [ -n "$USE_CONDA" ]; then - conda deactivate || (echo "Falling back to 'source deactivate'" && source deactivate) + source deactivate || (echo "Falling back to 'conda deactivate'" && conda deactivate) else deactivate fi From 0432379f9923768a767566e9ac5a4021cfe8d052 Mon Sep 17 00:00:00 2001 From: Stijn De Haes Date: Tue, 21 Jul 2020 16:34:30 -0700 Subject: [PATCH 242/384] [SPARK-24266][K8S] Restart the watcher when we receive a version changed from k8s ### What changes were proposed in this pull request? Restart the watcher when it failed with a HTTP_GONE code from the kubernetes api. Which means a resource version has changed. For more relevant information see here: https://github.com/fabric8io/kubernetes-client/issues/1075 ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Running spark-submit to a k8s cluster. Not sure how to make an automated test for this. If someone can help me out that would be great. Closes #28423 from stijndehaes/bugfix/k8s-submit-resource-version-change. Authored-by: Stijn De Haes Signed-off-by: Holden Karau --- .../submit/KubernetesClientApplication.scala | 52 ++++++++++++------- .../k8s/submit/LoggingPodStatusWatcher.scala | 35 ++++++++++--- .../spark/deploy/k8s/submit/ClientSuite.scala | 1 + 3 files changed, 61 insertions(+), 27 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index b4155fed8aa24..713d35dcf64f5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -21,9 +21,11 @@ import java.util.{Collections, UUID} import java.util.Properties import io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.{KubernetesClient, Watch} +import io.fabric8.kubernetes.client.Watcher.Action import scala.collection.mutable import scala.util.control.NonFatal +import util.control.Breaks._ import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkApplication @@ -127,25 +129,37 @@ private[spark] class Client( .endSpec() .build() val driverPodName = resolvedDriverPod.getMetadata.getName - Utils.tryWithResource( - kubernetesClient - .pods() - .withName(driverPodName) - .watch(watcher)) { _ => - val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) - try { - val otherKubernetesResources = - resolvedDriverSpec.driverKubernetesResources ++ Seq(configMap) - addDriverOwnerReference(createdDriverPod, otherKubernetesResources) - kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() - } catch { - case NonFatal(e) => - kubernetesClient.pods().delete(createdDriverPod) - throw e - } - val sId = Seq(conf.namespace, driverPodName).mkString(":") - watcher.watchOrStop(sId) + var watch: Watch = null + val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) + try { + val otherKubernetesResources = resolvedDriverSpec.driverKubernetesResources ++ Seq(configMap) + addDriverOwnerReference(createdDriverPod, otherKubernetesResources) + kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() + } catch { + case NonFatal(e) => + kubernetesClient.pods().delete(createdDriverPod) + throw e + } + val sId = Seq(conf.namespace, driverPodName).mkString(":") + breakable { + while (true) { + val podWithName = kubernetesClient + .pods() + .withName(driverPodName) + // Reset resource to old before we start the watch, this is important for race conditions + watcher.reset() + watch = podWithName.watch(watcher) + + // Send the latest pod state we know to the watcher to make sure we didn't miss anything + watcher.eventReceived(Action.MODIFIED, podWithName.get()) + + // Break the while loop if the pod is completed or we don't want to wait + if(watcher.watchOrStop(sId)) { + watch.close() + break + } + } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index ce3c80c0f85b1..aa27a9ef508ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action +import java.net.HttpURLConnection.HTTP_GONE import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.KubernetesDriverConf @@ -26,7 +27,8 @@ import org.apache.spark.deploy.k8s.KubernetesUtils._ import org.apache.spark.internal.Logging private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { - def watchOrStop(submissionId: String): Unit + def watchOrStop(submissionId: String): Boolean + def reset(): Unit } /** @@ -42,10 +44,16 @@ private[k8s] class LoggingPodStatusWatcherImpl(conf: KubernetesDriverConf) private var podCompleted = false + private var resourceTooOldReceived = false + private var pod = Option.empty[Pod] private def phase: String = pod.map(_.getStatus.getPhase).getOrElse("unknown") + override def reset(): Unit = { + resourceTooOldReceived = false + } + override def eventReceived(action: Action, pod: Pod): Unit = { this.pod = Option(pod) action match { @@ -62,7 +70,12 @@ private[k8s] class LoggingPodStatusWatcherImpl(conf: KubernetesDriverConf) override def onClose(e: KubernetesClientException): Unit = { logDebug(s"Stopping watching application $appId with last-observed phase $phase") - closeWatch() + if(e != null && e.getCode == HTTP_GONE) { + resourceTooOldReceived = true + logDebug(s"Got HTTP Gone code, resource version changed in k8s api: $e") + } else { + closeWatch() + } } private def logLongStatus(): Unit = { @@ -78,20 +91,26 @@ private[k8s] class LoggingPodStatusWatcherImpl(conf: KubernetesDriverConf) this.notifyAll() } - override def watchOrStop(sId: String): Unit = if (conf.get(WAIT_FOR_APP_COMPLETION)) { + override def watchOrStop(sId: String): Boolean = if (conf.get(WAIT_FOR_APP_COMPLETION)) { logInfo(s"Waiting for application ${conf.appName} with submission ID $sId to finish...") val interval = conf.get(REPORT_INTERVAL) synchronized { - while (!podCompleted) { + while (!podCompleted && !resourceTooOldReceived) { wait(interval) logInfo(s"Application status for $appId (phase: $phase)") } } - logInfo( - pod.map { p => s"Container final statuses:\n\n${containersDescription(p)}" } - .getOrElse("No containers were found in the driver pod.")) - logInfo(s"Application ${conf.appName} with submission ID $sId finished") + + if(podCompleted) { + logInfo( + pod.map { p => s"Container final statuses:\n\n${containersDescription(p)}" } + .getOrElse("No containers were found in the driver pod.")) + logInfo(s"Application ${conf.appName} with submission ID $sId finished") + } + podCompleted } else { logInfo(s"Deployed Spark application ${conf.appName} with submission ID $sId into Kubernetes") + // Always act like the application has completed since we don't want to wait for app completion + true } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 5d49ac0bbaafa..d9ec3feb526ee 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -136,6 +136,7 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { createdResourcesArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) when(podOperations.create(FULL_EXPECTED_POD)).thenReturn(POD_WITH_OWNER_REFERENCE) when(namedPods.watch(loggingPodStatusWatcher)).thenReturn(mock[Watch]) + when(loggingPodStatusWatcher.watchOrStop(kconf.namespace + ":" + POD_NAME)).thenReturn(true) doReturn(resourceList) .when(kubernetesClient) .resourceList(createdResourcesArgumentCaptor.capture()) From 39181ff209354a20b829609483fe8301e1e4a9d4 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Wed, 22 Jul 2020 08:56:26 +0900 Subject: [PATCH 243/384] [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable ### What changes were proposed in this pull request? Based on a follow up comment in https://github.com/apache/spark/pull/28123, where we can coalesce buckets for shuffled hash join as well. The note here is we only coalesce the buckets from shuffled hash join stream side (i.e. the side not building hash map), so we don't need to worry about OOM when coalescing multiple buckets in one task for building hash map. > If you refactor some codes with changing classes, showing the class hierarchy will help reviewers. Refactor existing physical plan rule `CoalesceBucketsInSortMergeJoin` to `CoalesceBucketsInJoin`, for covering shuffled hash join as well. Refactor existing unit test `CoalesceBucketsInSortMergeJoinSuite` to `CoalesceBucketsInJoinSuite`, for covering shuffled hash join as well. ### Why are the changes needed? Avoid shuffle for joining different bucketed tables, is also useful for shuffled hash join. In production, we are seeing users to use shuffled hash join to join bucketed tables (set `spark.sql.join.preferSortMergeJoin`=false, to avoid sort), and this can help avoid shuffle if number of buckets are not same. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit tests in `CoalesceBucketsInJoinSuite` for verifying shuffled hash join physical plan. ### Performance number per request from maropu I was looking at TPCDS per suggestion from maropu. But I found most of queries from TPCDS are doing aggregate, and only several ones are doing join. None of input tables are bucketed. So I took the approach to test a modified version of `TPCDS q93` as ``` SELECT ss_ticket_number, sr_ticket_number FROM store_sales JOIN store_returns ON ss_ticket_number = sr_ticket_number ``` And make `store_sales` and `store_returns` to be bucketed tables. Physical query plan without coalesce: ``` ShuffledHashJoin [ss_ticket_number#109L], [sr_ticket_number#120L], Inner, BuildLeft :- Exchange hashpartitioning(ss_ticket_number#109L, 4), true, [id=#67] : +- *(1) Project [ss_ticket_number#109L] : +- *(1) Filter isnotnull(ss_ticket_number#109L) : +- *(1) ColumnarToRow : +- FileScan parquet default.store_sales[ss_ticket_number#109L] Batched: true, DataFilters: [isnotnull(ss_ticket_number#109L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_sales], PartitionFilters: [], PushedFilters: [IsNotNull(ss_ticket_number)], ReadSchema: struct, SelectedBucketsCount: 2 out of 2 +- *(2) Project [sr_returned_date_sk#111L, sr_return_time_sk#112L, sr_item_sk#113L, sr_customer_sk#114L, sr_cdemo_sk#115L, sr_hdemo_sk#116L, sr_addr_sk#117L, sr_store_sk#118L, sr_reason_sk#119L, sr_ticket_number#120L, sr_return_quantity#121L, sr_return_amt#122, sr_return_tax#123, sr_return_amt_inc_tax#124, sr_fee#125, sr_return_ship_cost#126, sr_refunded_cash#127, sr_reversed_charge#128, sr_store_credit#129, sr_net_loss#130] +- *(2) Filter isnotnull(sr_ticket_number#120L) +- *(2) ColumnarToRow +- FileScan parquet default.store_returns[sr_returned_date_sk#111L,sr_return_time_sk#112L,sr_item_sk#113L,sr_customer_sk#114L,sr_cdemo_sk#115L,sr_hdemo_sk#116L,sr_addr_sk#117L,sr_store_sk#118L,sr_reason_sk#119L,sr_ticket_number#120L,sr_return_quantity#121L,sr_return_amt#122,sr_return_tax#123,sr_return_amt_inc_tax#124,sr_fee#125,sr_return_ship_cost#126,sr_refunded_cash#127,sr_reversed_charge#128,sr_store_credit#129,sr_net_loss#130] Batched: true, DataFilters: [isnotnull(sr_ticket_number#120L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_returns], PartitionFilters: [], PushedFilters: [IsNotNull(sr_ticket_number)], ReadSchema: struct, SelectedBucketsCount: 2 out of 2 +- *(2) Project [sr_returned_date_sk#111L, sr_return_time_sk#112L, sr_item_sk#113L, sr_customer_sk#114L, sr_cdemo_sk#115L, sr_hdemo_sk#116L, sr_addr_sk#117L, sr_store_sk#118L, sr_reason_sk#119L, sr_ticket_number#120L, sr_return_quantity#121L, sr_return_amt#122, sr_return_tax#123, sr_return_amt_inc_tax#124, sr_fee#125, sr_return_ship_cost#126, sr_refunded_cash#127, sr_reversed_charge#128, sr_store_credit#129, sr_net_loss#130] +- *(2) Filter isnotnull(sr_ticket_number#120L) +- *(2) ColumnarToRow +- FileScan parquet default.store_returns[sr_returned_date_sk#111L,sr_return_time_sk#112L,sr_item_sk#113L,sr_customer_sk#114L,sr_cdemo_sk#115L,sr_hdemo_sk#116L,sr_addr_sk#117L,sr_store_sk#118L,sr_reason_sk#119L,sr_ticket_number#120L,sr_return_quantity#121L,sr_return_amt#122,sr_return_tax#123,sr_return_amt_inc_tax#124,sr_fee#125,sr_return_ship_cost#126,sr_refunded_cash#127,sr_reversed_charge#128,sr_store_credit#129,sr_net_loss#130] Batched: true, DataFilters: [isnotnull(sr_ticket_number#120L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_returns], PartitionFilters: [], PushedFilters: [IsNotNull(sr_ticket_number)], ReadSchema: struct Signed-off-by: Takeshi Yamamuro --- .../apache/spark/sql/internal/SQLConf.scala | 22 ++- .../spark/sql/execution/QueryExecution.scala | 4 +- .../bucketing/CoalesceBucketsInJoin.scala | 177 ++++++++++++++++++ .../CoalesceBucketsInSortMergeJoin.scala | 132 ------------- .../org/apache/spark/sql/ExplainSuite.scala | 2 +- ...scala => CoalesceBucketsInJoinSuite.scala} | 136 +++++++++++--- .../spark/sql/sources/BucketedReadSuite.scala | 14 +- 7 files changed, 309 insertions(+), 178 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala rename sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/{CoalesceBucketsInSortMergeJoinSuite.scala => CoalesceBucketsInJoinSuite.scala} (55%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 89c41f31ff234..c68d7ccab4d10 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2638,21 +2638,24 @@ object SQLConf { .booleanConf .createWithDefault(true) - val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED = - buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled") + val COALESCE_BUCKETS_IN_JOIN_ENABLED = + buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled") .doc("When true, if two bucketed tables with the different number of buckets are joined, " + "the side with a bigger number of buckets will be coalesced to have the same number " + - "of buckets as the other side. Bucket coalescing is applied only to sort-merge joins " + - "and only when the bigger number of buckets is divisible by the smaller number of buckets.") + "of buckets as the other side. Bigger number of buckets is divisible by the smaller " + + "number of buckets. Bucket coalescing is applied to sort-merge joins and " + + "shuffled hash join. Note: Coalescing bucketed table can avoid unnecessary shuffling " + + "in join, but it also reduces parallelism and could possibly cause OOM for " + + "shuffled hash join.") .version("3.1.0") .booleanConf .createWithDefault(false) - val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO = - buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio") + val COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO = + buildConf("spark.sql.bucketing.coalesceBucketsInJoin.maxBucketRatio") .doc("The ratio of the number of two buckets being coalesced should be less than or " + "equal to this value for bucket coalescing to be applied. This configuration only " + - s"has an effect when '${COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key}' is set to true.") + s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set to true.") .version("3.1.0") .intConf .checkValue(_ > 0, "The difference must be positive.") @@ -3269,6 +3272,11 @@ class SQLConf extends Serializable with Logging { def metadataCacheTTL: Long = getConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS) + def coalesceBucketsInJoinEnabled: Boolean = getConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED) + + def coalesceBucketsInJoinMaxBucketRatio: Int = + getConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 791e432269632..e4b9322016cf0 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 @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.adaptive.{AdaptiveExecutionContext, InsertAdaptiveSparkPlan} -import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInSortMergeJoin +import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInJoin import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} @@ -332,7 +332,7 @@ object QueryExecution { // as the original plan is hidden behind `AdaptiveSparkPlanExec`. adaptiveExecutionRule.toSeq ++ Seq( - CoalesceBucketsInSortMergeJoin(sparkSession.sessionState.conf), + CoalesceBucketsInJoin(sparkSession.sessionState.conf), PlanDynamicPruningFilters(sparkSession), PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala new file mode 100644 index 0000000000000..22f308f331449 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.bucketing + +import scala.annotation.tailrec + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.joins.{BaseJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf + +/** + * This rule coalesces one side of the `SortMergeJoin` and `ShuffledHashJoin` + * if the following conditions are met: + * - Two bucketed tables are joined. + * - Join keys match with output partition expressions on their respective sides. + * - The larger bucket number is divisible by the smaller bucket number. + * - COALESCE_BUCKETS_IN_JOIN_ENABLED is set to true. + * - The ratio of the number of buckets is less than the value set in + * COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO. + */ +case class CoalesceBucketsInJoin(conf: SQLConf) extends Rule[SparkPlan] { + private def updateNumCoalescedBucketsInScan( + plan: SparkPlan, + numCoalescedBuckets: Int): SparkPlan = { + plan transformUp { + case f: FileSourceScanExec => + f.copy(optionalNumCoalescedBuckets = Some(numCoalescedBuckets)) + } + } + + private def updateNumCoalescedBuckets( + join: BaseJoinExec, + numLeftBuckets: Int, + numRightBucket: Int, + numCoalescedBuckets: Int): BaseJoinExec = { + if (numCoalescedBuckets != numLeftBuckets) { + val leftCoalescedChild = + updateNumCoalescedBucketsInScan(join.left, numCoalescedBuckets) + join match { + case j: SortMergeJoinExec => j.copy(left = leftCoalescedChild) + case j: ShuffledHashJoinExec => j.copy(left = leftCoalescedChild) + } + } else { + val rightCoalescedChild = + updateNumCoalescedBucketsInScan(join.right, numCoalescedBuckets) + join match { + case j: SortMergeJoinExec => j.copy(right = rightCoalescedChild) + case j: ShuffledHashJoinExec => j.copy(right = rightCoalescedChild) + } + } + } + + private def isCoalesceSHJStreamSide( + join: ShuffledHashJoinExec, + numLeftBuckets: Int, + numRightBucket: Int, + numCoalescedBuckets: Int): Boolean = { + if (numCoalescedBuckets == numLeftBuckets) { + join.buildSide != BuildRight + } else { + join.buildSide != BuildLeft + } + } + + def apply(plan: SparkPlan): SparkPlan = { + if (!conf.coalesceBucketsInJoinEnabled) { + return plan + } + + plan transform { + case ExtractJoinWithBuckets(join, numLeftBuckets, numRightBuckets) + if math.max(numLeftBuckets, numRightBuckets) / math.min(numLeftBuckets, numRightBuckets) <= + conf.coalesceBucketsInJoinMaxBucketRatio => + val numCoalescedBuckets = math.min(numLeftBuckets, numRightBuckets) + join match { + case j: SortMergeJoinExec => + updateNumCoalescedBuckets(j, numLeftBuckets, numRightBuckets, numCoalescedBuckets) + case j: ShuffledHashJoinExec + // Only coalesce the buckets for shuffled hash join stream side, + // to avoid OOM for build side. + if isCoalesceSHJStreamSide(j, numLeftBuckets, numRightBuckets, numCoalescedBuckets) => + updateNumCoalescedBuckets(j, numLeftBuckets, numRightBuckets, numCoalescedBuckets) + case other => other + } + case other => other + } + } +} + +/** + * An extractor that extracts `SortMergeJoinExec` and `ShuffledHashJoin`, + * where both sides of the join have the bucketed tables, + * are consisted of only the scan operation, + * and numbers of buckets are not equal but divisible. + */ +object ExtractJoinWithBuckets { + @tailrec + private def hasScanOperation(plan: SparkPlan): Boolean = plan match { + case f: FilterExec => hasScanOperation(f.child) + case p: ProjectExec => hasScanOperation(p.child) + case _: FileSourceScanExec => true + case _ => false + } + + private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { + plan.collectFirst { + case f: FileSourceScanExec if f.relation.bucketSpec.nonEmpty && + f.optionalNumCoalescedBuckets.isEmpty => + f.relation.bucketSpec.get + } + } + + /** + * The join keys should match with expressions for output partitioning. Note that + * the ordering does not matter because it will be handled in `EnsureRequirements`. + */ + private def satisfiesOutputPartitioning( + keys: Seq[Expression], + partitioning: Partitioning): Boolean = { + partitioning match { + case HashPartitioning(exprs, _) if exprs.length == keys.length => + exprs.forall(e => keys.exists(_.semanticEquals(e))) + case _ => false + } + } + + private def isApplicable(j: BaseJoinExec): Boolean = { + (j.isInstanceOf[SortMergeJoinExec] || + j.isInstanceOf[ShuffledHashJoinExec]) && + hasScanOperation(j.left) && + hasScanOperation(j.right) && + satisfiesOutputPartitioning(j.leftKeys, j.left.outputPartitioning) && + satisfiesOutputPartitioning(j.rightKeys, j.right.outputPartitioning) + } + + private def isDivisible(numBuckets1: Int, numBuckets2: Int): Boolean = { + val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) + // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller + // number of buckets because bucket id is calculated by modding the total number of buckets. + numBuckets1 != numBuckets2 && large % small == 0 + } + + def unapply(plan: SparkPlan): Option[(BaseJoinExec, Int, Int)] = { + plan match { + case j: BaseJoinExec if isApplicable(j) => + val leftBucket = getBucketSpec(j.left) + val rightBucket = getBucketSpec(j.right) + if (leftBucket.isDefined && rightBucket.isDefined && + isDivisible(leftBucket.get.numBuckets, rightBucket.get.numBuckets)) { + Some(j, leftBucket.get.numBuckets, rightBucket.get.numBuckets) + } else { + None + } + case _ => None + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala deleted file mode 100644 index 3bb0597ecd87c..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.bucketing - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} -import org.apache.spark.sql.execution.joins.SortMergeJoinExec -import org.apache.spark.sql.internal.SQLConf - -/** - * This rule coalesces one side of the `SortMergeJoin` if the following conditions are met: - * - Two bucketed tables are joined. - * - Join keys match with output partition expressions on their respective sides. - * - The larger bucket number is divisible by the smaller bucket number. - * - COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED is set to true. - * - The ratio of the number of buckets is less than the value set in - * COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO. - */ -case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] { - private def mayCoalesce(numBuckets1: Int, numBuckets2: Int, conf: SQLConf): Option[Int] = { - assert(numBuckets1 != numBuckets2) - val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) - // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller - // number of buckets because bucket id is calculated by modding the total number of buckets. - if (large % small == 0 && - large / small <= conf.getConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO)) { - Some(small) - } else { - None - } - } - - private def updateNumCoalescedBuckets(plan: SparkPlan, numCoalescedBuckets: Int): SparkPlan = { - plan.transformUp { - case f: FileSourceScanExec => - f.copy(optionalNumCoalescedBuckets = Some(numCoalescedBuckets)) - } - } - - def apply(plan: SparkPlan): SparkPlan = { - if (!conf.getConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED)) { - return plan - } - - plan transform { - case ExtractSortMergeJoinWithBuckets(smj, numLeftBuckets, numRightBuckets) - if numLeftBuckets != numRightBuckets => - mayCoalesce(numLeftBuckets, numRightBuckets, conf).map { numCoalescedBuckets => - if (numCoalescedBuckets != numLeftBuckets) { - smj.copy(left = updateNumCoalescedBuckets(smj.left, numCoalescedBuckets)) - } else { - smj.copy(right = updateNumCoalescedBuckets(smj.right, numCoalescedBuckets)) - } - }.getOrElse(smj) - case other => other - } - } -} - -/** - * An extractor that extracts `SortMergeJoinExec` where both sides of the join have the bucketed - * tables and are consisted of only the scan operation. - */ -object ExtractSortMergeJoinWithBuckets { - private def isScanOperation(plan: SparkPlan): Boolean = plan match { - case f: FilterExec => isScanOperation(f.child) - case p: ProjectExec => isScanOperation(p.child) - case _: FileSourceScanExec => true - case _ => false - } - - private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { - plan.collectFirst { - case f: FileSourceScanExec if f.relation.bucketSpec.nonEmpty && - f.optionalNumCoalescedBuckets.isEmpty => - f.relation.bucketSpec.get - } - } - - /** - * The join keys should match with expressions for output partitioning. Note that - * the ordering does not matter because it will be handled in `EnsureRequirements`. - */ - private def satisfiesOutputPartitioning( - keys: Seq[Expression], - partitioning: Partitioning): Boolean = { - partitioning match { - case HashPartitioning(exprs, _) if exprs.length == keys.length => - exprs.forall(e => keys.exists(_.semanticEquals(e))) - case _ => false - } - } - - private def isApplicable(s: SortMergeJoinExec): Boolean = { - isScanOperation(s.left) && - isScanOperation(s.right) && - satisfiesOutputPartitioning(s.leftKeys, s.left.outputPartitioning) && - satisfiesOutputPartitioning(s.rightKeys, s.right.outputPartitioning) - } - - def unapply(plan: SparkPlan): Option[(SortMergeJoinExec, Int, Int)] = { - plan match { - case s: SortMergeJoinExec if isApplicable(s) => - val leftBucket = getBucketSpec(s.left) - val rightBucket = getBucketSpec(s.right) - if (leftBucket.isDefined && rightBucket.isDefined) { - Some(s, leftBucket.get.numBuckets, rightBucket.get.numBuckets) - } else { - None - } - case _ => None - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index 70303792fdf1a..e43a4147ceb63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -347,7 +347,7 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite test("Coalesced bucket info should be a part of explain string") { withTable("t1", "t2") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { Seq(1, 2).toDF("i").write.bucketBy(8, "i").saveAsTable("t1") Seq(2, 3).toDF("i").write.bucketBy(4, "i").saveAsTable("t2") val df1 = spark.table("t1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoinSuite.scala similarity index 55% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoinSuite.scala index 6a70045c55e64..89aee37a4246f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoinSuite.scala @@ -19,17 +19,21 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.optimizer.BuildLeft +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.execution.{BinaryExecNode, FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, PartitionSpec} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types.{IntegerType, StructType} -class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkSession { +class CoalesceBucketsInJoinSuite extends SQLTestUtils with SharedSparkSession { + private val SORT_MERGE_JOIN = "sortMergeJoin" + private val SHUFFLED_HASH_JOIN = "shuffledHashJoin" + private val BROADCAST_HASH_JOIN = "broadcastHashJoin" + case class RelationSetting( cols: Seq[Attribute], numBuckets: Int, @@ -47,11 +51,16 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS rightKeys: Seq[Attribute], leftRelation: RelationSetting, rightRelation: RelationSetting, - isSortMergeJoin: Boolean) + joinOperator: String, + shjBuildSide: Option[BuildSide]) object JoinSetting { - def apply(l: RelationSetting, r: RelationSetting, isSortMergeJoin: Boolean): JoinSetting = { - JoinSetting(l.cols, r.cols, l, r, isSortMergeJoin) + def apply( + l: RelationSetting, + r: RelationSetting, + joinOperator: String = SORT_MERGE_JOIN, + shjBuildSide: Option[BuildSide] = None): JoinSetting = { + JoinSetting(l.cols, r.cols, l, r, joinOperator, shjBuildSide) } } @@ -73,17 +82,24 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS leftRelation = setting.rightRelation, rightRelation = setting.leftRelation) - Seq(setting, swappedSetting).foreach { case s => + val settings = if (setting.joinOperator != SHUFFLED_HASH_JOIN) { + Seq(setting, swappedSetting) + } else { + Seq(setting) + } + settings.foreach { s => val lScan = newFileSourceScanExec(s.leftRelation) val rScan = newFileSourceScanExec(s.rightRelation) - val join = if (s.isSortMergeJoin) { + val join = if (s.joinOperator == SORT_MERGE_JOIN) { SortMergeJoinExec(s.leftKeys, s.rightKeys, Inner, None, lScan, rScan) + } else if (s.joinOperator == SHUFFLED_HASH_JOIN) { + ShuffledHashJoinExec(s.leftKeys, s.rightKeys, Inner, s.shjBuildSide.get, None, lScan, rScan) } else { BroadcastHashJoinExec( s.leftKeys, s.rightKeys, Inner, BuildLeft, None, lScan, rScan) } - val plan = CoalesceBucketsInSortMergeJoin(spark.sessionState.conf)(join) + val plan = CoalesceBucketsInJoin(spark.sessionState.conf)(join) def verify(expected: Option[Int], subPlan: SparkPlan): Unit = { val coalesced = subPlan.collect { @@ -91,7 +107,7 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS f.optionalNumCoalescedBuckets.get } if (expected.isDefined) { - assert(coalesced.size == 1 && coalesced(0) == expected.get) + assert(coalesced.size == 1 && coalesced.head == expected.get) } else { assert(coalesced.isEmpty) } @@ -103,46 +119,73 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS } test("bucket coalescing - basic") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, Some(4)), joinOperator = SORT_MERGE_JOIN)) run(JoinSetting( - RelationSetting(4, None), RelationSetting(8, Some(4)), isSortMergeJoin = true)) + RelationSetting(4, None), RelationSetting(8, Some(4)), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { - run(JoinSetting(RelationSetting(4, None), RelationSetting(8, None), isSortMergeJoin = true)) + + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "false") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, None), joinOperator = SORT_MERGE_JOIN)) + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } } - test("bucket coalescing should work only for sort merge join") { + test("bucket coalescing should work only for sort merge join and shuffled hash join") { Seq(true, false).foreach { enabled => - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> enabled.toString) { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { run(JoinSetting( - RelationSetting(4, None), RelationSetting(8, None), isSortMergeJoin = false)) + RelationSetting(4, None), RelationSetting(8, None), joinOperator = BROADCAST_HASH_JOIN)) } } } + test("bucket coalescing shouldn't be applied to shuffled hash join build side") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildRight))) + } + } + test("bucket coalescing shouldn't be applied when the number of buckets are the same") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { - run(JoinSetting(RelationSetting(8, None), RelationSetting(8, None), isSortMergeJoin = true)) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(8, None), RelationSetting(8, None), joinOperator = SORT_MERGE_JOIN)) + run(JoinSetting( + RelationSetting(8, None), RelationSetting(8, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } } test("number of bucket is not divisible by other number of bucket") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { - run(JoinSetting(RelationSetting(3, None), RelationSetting(8, None), isSortMergeJoin = true)) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(3, None), RelationSetting(8, None), joinOperator = SORT_MERGE_JOIN)) + run(JoinSetting( + RelationSetting(3, None), RelationSetting(8, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } } test("the ratio of the number of buckets is greater than max allowed") { - withSQLConf( - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { - run(JoinSetting(RelationSetting(4, None), RelationSetting(16, None), isSortMergeJoin = true)) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO.key -> "2") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(16, None), joinOperator = SORT_MERGE_JOIN)) + run(JoinSetting( + RelationSetting(4, None), RelationSetting(16, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } } test("join keys should match with output partitioning") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { val lCols = Seq( AttributeReference("l1", IntegerType)(), AttributeReference("l2", IntegerType)()) @@ -160,7 +203,16 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS rightKeys = Seq(rCols.head), leftRelation = lRel, rightRelation = rRel, - isSortMergeJoin = true)) + joinOperator = SORT_MERGE_JOIN, + shjBuildSide = None)) + + run(JoinSetting( + leftKeys = Seq(lCols.head), + rightKeys = Seq(rCols.head), + leftRelation = lRel, + rightRelation = rRel, + joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) // The following should not be coalesced because join keys do not match with output // partitioning (more expressions). @@ -169,7 +221,16 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS rightKeys = rCols :+ AttributeReference("r3", IntegerType)(), leftRelation = lRel, rightRelation = rRel, - isSortMergeJoin = true)) + joinOperator = SORT_MERGE_JOIN, + shjBuildSide = None)) + + run(JoinSetting( + leftKeys = lCols :+ AttributeReference("l3", IntegerType)(), + rightKeys = rCols :+ AttributeReference("r3", IntegerType)(), + leftRelation = lRel, + rightRelation = rRel, + joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) // The following will be coalesced since ordering should not matter because it will be // adjusted in `EnsureRequirements`. @@ -178,7 +239,24 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS rightKeys = rCols.reverse, leftRelation = lRel, rightRelation = RelationSetting(rCols, 8, Some(4)), - isSortMergeJoin = true)) + joinOperator = SORT_MERGE_JOIN, + shjBuildSide = None)) + + run(JoinSetting( + leftKeys = lCols.reverse, + rightKeys = rCols.reverse, + leftRelation = lRel, + rightRelation = RelationSetting(rCols, 8, Some(4)), + joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) + + run(JoinSetting( + leftKeys = rCols.reverse, + rightKeys = lCols.reverse, + leftRelation = RelationSetting(rCols, 8, Some(4)), + rightRelation = lRel, + joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildRight))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index b6767eb3132ea..98886d271e977 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -876,7 +876,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } test("bucket coalescing eliminates shuffle") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. // Currently, sort will be introduced for the side that is coalesced. val testSpec1 = BucketedTableTestSpec( @@ -911,7 +911,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "false") { // Coalescing buckets is disabled by a config. run( BucketedTableTestSpec( @@ -921,8 +921,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } withSQLConf( - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO.key -> "2") { // Coalescing buckets is not applied because the ratio of the number of buckets (3) // is greater than max allowed (2). run( @@ -932,7 +932,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { run( // Coalescing buckets is not applied because the bigger number of buckets (8) is not // divisible by the smaller number of buckets (7). @@ -950,7 +950,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { withSQLConf( SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { def verify( query: String, expectedNumShuffles: Int, @@ -964,7 +964,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } if (expectedCoalescedNumBuckets.isDefined) { assert(scans.length == 1) - assert(scans(0).optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + assert(scans.head.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) } else { assert(scans.isEmpty) } From 7b9d7551a6de840d6ec8bc031ed8c314bf33b252 Mon Sep 17 00:00:00 2001 From: Baohe Zhang Date: Wed, 22 Jul 2020 13:27:34 +0900 Subject: [PATCH 244/384] [SPARK-32350][CORE] Add batch-write on LevelDB to improve performance of HybridStore ### What changes were proposed in this pull request? The idea is to improve the performance of HybridStore by adding batch write support to LevelDB. #28412 introduces HybridStore. HybridStore will write data to InMemoryStore at first and use a background thread to dump data to LevelDB once the writing to InMemoryStore is completed. In the comments section of #28412 , mridulm mentioned using batch writing can improve the performance of this dumping process and he wrote the code of writeAll(). ### Why are the changes needed? I did the comparison of the HybridStore switching time between one-by-one write and batch write on an HDD disk. When the disk is free, the batch-write has around 25% improvement, and when the disk is 100% busy, the batch-write has 7x - 10x improvement. when the disk is at 0% utilization: | log size, jobs and tasks per job | original switching time, with write() | switching time with writeAll() | | ---------------------------------- | ------------------------------------- | ------------------------------ | | 133m, 400 jobs, 100 tasks per job | 16s | 13s | | 265m, 400 jobs, 200 tasks per job | 30s | 23s | | 1.3g, 1000 jobs, 400 tasks per job | 136s | 108s | when the disk is at 100% utilization: | log size, jobs and tasks per job | original switching time, with write() | switching time with writeAll() | | --------------------------------- | ------------------------------------- | ------------------------------ | | 133m, 400 jobs, 100 tasks per job | 116s | 17s | | 265m, 400 jobs, 200 tasks per job | 251s | 26s | I also ran some write related benchmarking tests on LevelDBBenchmark.java and measured the total time of writing 1024 objects. The tests were conducted when the disk is at 0% utilization. | Benchmark test | with write(), ms | with writeAll(), ms | | ------------------------ | ---------------- | ------------------- | | randomUpdatesIndexed | 213.06 | 157.356 | | randomUpdatesNoIndex | 57.869 | 35.439 | | randomWritesIndexed | 298.854 | 229.274 | | randomWritesNoIndex | 66.764 | 38.361 | | sequentialUpdatesIndexed | 87.019 | 56.219 | | sequentialUpdatesNoIndex | 61.851 | 41.942 | | sequentialWritesIndexed | 94.044 | 56.534 | | sequentialWritesNoIndex | 118.345 | 66.483 | ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually tested. Closes #29149 from baohe-zhang/SPARK-32350. Authored-by: Baohe Zhang Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../apache/spark/util/kvstore/LevelDB.java | 73 ++++++++++++++++--- .../spark/deploy/history/HybridStore.scala | 9 ++- 2 files changed, 66 insertions(+), 16 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java index 98f33b70fea23..121dfbd4f6838 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java @@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; @@ -153,24 +154,72 @@ public void write(Object value) throws Exception { try (WriteBatch batch = db().createWriteBatch()) { byte[] data = serializer.serialize(value); synchronized (ti) { - Object existing; - try { - existing = get(ti.naturalIndex().entityKey(null, value), value.getClass()); - } catch (NoSuchElementException e) { - existing = null; - } + updateBatch(batch, value, data, value.getClass(), ti.naturalIndex(), ti.indices()); + db().write(batch); + } + } + } + + public void writeAll(List values) throws Exception { + Preconditions.checkArgument(values != null && !values.isEmpty(), + "Non-empty values required."); + + // Group by class, in case there are values from different classes in the values + // Typical usecase is for this to be a single class. + // A NullPointerException will be thrown if values contain null object. + for (Map.Entry, ? extends List> entry : + values.stream().collect(Collectors.groupingBy(Object::getClass)).entrySet()) { + + final Iterator valueIter = entry.getValue().iterator(); + final Iterator serializedValueIter; + + // Deserialize outside synchronized block + List list = new ArrayList<>(entry.getValue().size()); + for (Object value : values) { + list.add(serializer.serialize(value)); + } + serializedValueIter = list.iterator(); + + final Class klass = entry.getKey(); + final LevelDBTypeInfo ti = getTypeInfo(klass); - PrefixCache cache = new PrefixCache(value); - byte[] naturalKey = ti.naturalIndex().toKey(ti.naturalIndex().getValue(value)); - for (LevelDBTypeInfo.Index idx : ti.indices()) { - byte[] prefix = cache.getPrefix(idx); - idx.add(batch, value, existing, data, naturalKey, prefix); + synchronized (ti) { + final LevelDBTypeInfo.Index naturalIndex = ti.naturalIndex(); + final Collection indices = ti.indices(); + + try (WriteBatch batch = db().createWriteBatch()) { + while (valueIter.hasNext()) { + updateBatch(batch, valueIter.next(), serializedValueIter.next(), klass, + naturalIndex, indices); + } + db().write(batch); } - db().write(batch); } } } + private void updateBatch( + WriteBatch batch, + Object value, + byte[] data, + Class klass, + LevelDBTypeInfo.Index naturalIndex, + Collection indices) throws Exception { + Object existing; + try { + existing = get(naturalIndex.entityKey(null, value), klass); + } catch (NoSuchElementException e) { + existing = null; + } + + PrefixCache cache = new PrefixCache(value); + byte[] naturalKey = naturalIndex.toKey(naturalIndex.getValue(value)); + for (LevelDBTypeInfo.Index idx : indices) { + byte[] prefix = cache.getPrefix(idx); + idx.add(batch, value, existing, data, naturalKey, prefix); + } + } + @Override public void delete(Class type, Object naturalKey) throws Exception { Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala index 96db86f8e745a..08db2bd0766c3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala @@ -24,6 +24,8 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ +import com.google.common.collect.Lists; + import org.apache.spark.util.kvstore._ /** @@ -144,10 +146,9 @@ private[history] class HybridStore extends KVStore { backgroundThread = new Thread(() => { try { for (klass <- klassMap.keys().asScala) { - val it = inMemoryStore.view(klass).closeableIterator() - while (it.hasNext()) { - levelDB.write(it.next()) - } + val values = Lists.newArrayList( + inMemoryStore.view(klass).closeableIterator()) + levelDB.writeAll(values) } listener.onSwitchToLevelDBSuccess() shouldUseInMemoryStore.set(false) From b4a9606890b422183fad7937805addfc3b823cd8 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 21 Jul 2020 23:16:38 -0700 Subject: [PATCH 245/384] [SPARK-31922][CORE] logDebug "RpcEnv already stopped" error on LocalSparkCluster shutdown ### What changes were proposed in this pull request? Catch the `RpcEnvStoppedException` and log debug it when stop is called for a `LocalSparkCluster`. This PR also contains two small changes to fix the potential issues. ### Why are the changes needed? Currently, there's always "RpcEnv already stopped" error if we exit spark-shell with local-cluster mode: ``` 20/06/07 14:54:18 ERROR TransportRequestHandler: Error while invoking RpcHandler#receive() for one-way message. org.apache.spark.rpc.RpcEnvStoppedException: RpcEnv already stopped. at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:167) at org.apache.spark.rpc.netty.Dispatcher.postOneWayMessage(Dispatcher.scala:150) at org.apache.spark.rpc.netty.NettyRpcHandler.receive(NettyRpcEnv.scala:691) at org.apache.spark.network.server.TransportRequestHandler.processOneWayMessage(TransportRequestHandler.java:253) at org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:111) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:140) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:53) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:102) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) at org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:102) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:163) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:714) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:650) at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:576) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:493) at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989) at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:748) ``` When we call stop on `StandaloneSchedulerBackend`, the backend will firstly send `UnregisterApplication` to `Master` and then call stop on `LocalSparkCluster` immediately. On the other side, `Master` will send messages to `Worker` when it receives `UnregisterApplication`. However, the rpcEnv of the `Worker` has been already stoped by the backend. Therefore, the error message shows when the `Worker` tries to handle the messages. It's only an error on shutdown, users would not like to care about it. So we could hide it in debug log and this is also what we've done previously in #18547. ### Does this PR introduce _any_ user-facing change? Yes, users will not see the error message after this PR. ### How was this patch tested? Tested manually. Closes #28746 from Ngone51/fix-spark-31922. Authored-by: yi.wu Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/deploy/LocalSparkCluster.scala | 2 +- .../scala/org/apache/spark/rpc/netty/Dispatcher.scala | 11 +++++++++-- .../org/apache/spark/rpc/netty/NettyRpcEnv.scala | 2 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 2 +- 4 files changed, 12 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index fc849d7f4372f..33851d9145d0a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -76,8 +76,8 @@ class LocalSparkCluster( logInfo("Shutting down local Spark cluster.") // Stop the workers before the master so they don't get upset that it disconnected workerRpcEnvs.foreach(_.shutdown()) - masterRpcEnvs.foreach(_.shutdown()) workerRpcEnvs.foreach(_.awaitTermination()) + masterRpcEnvs.foreach(_.shutdown()) masterRpcEnvs.foreach(_.awaitTermination()) masterRpcEnvs.clear() workerRpcEnvs.clear() diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index 41d6d146a86d7..4a9f551646fc7 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -122,7 +122,7 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte while (iter.hasNext) { val name = iter.next postMessage(name, message, (e) => { e match { - case e: RpcEnvStoppedException => logDebug (s"Message $message dropped. ${e.getMessage}") + case e: RpcEnvStoppedException => logDebug(s"Message $message dropped. ${e.getMessage}") case e: Throwable => logWarning(s"Message $message dropped. ${e.getMessage}") }} )} @@ -147,7 +147,14 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte /** Posts a one-way message. */ def postOneWayMessage(message: RequestMessage): Unit = { postMessage(message.receiver.name, OneWayMessage(message.senderAddress, message.content), - (e) => throw e) + (e) => e match { + // SPARK-31922: in local cluster mode, there's always a RpcEnvStoppedException when + // stop is called due to some asynchronous message handling. We catch the exception + // and log it at debug level to avoid verbose error message when user stop a local + // cluster in spark shell. + case re: RpcEnvStoppedException => logDebug(s"Message $message dropped. ${re.getMessage}") + case _ => throw e + }) } /** diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index 3886cc5baa48e..fcb9fe422c0d4 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -213,7 +213,7 @@ private[netty] class NettyRpcEnv( def onFailure(e: Throwable): Unit = { if (!promise.tryFailure(e)) { e match { - case e : RpcEnvStoppedException => logDebug (s"Ignored failure: $e") + case e : RpcEnvStoppedException => logDebug(s"Ignored failure: $e") case _ => logWarning(s"Ignored failure: $e") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6b9b4d6fe57e0..b4b3efaaff88f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -562,7 +562,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } } - override def reviveOffers(): Unit = { + override def reviveOffers(): Unit = Utils.tryLogNonFatalError { driverEndpoint.send(ReviveOffers) } From 29b7eaa43812ecbf1a0a5a406d00fad02a0cc26f Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 22 Jul 2020 08:15:04 +0000 Subject: [PATCH 246/384] [MINOR][SQL] Fix warning message for ThriftCLIService.GetCrossReference and GetPrimaryKeys ### What changes were proposed in this pull request? This PR fixes the warning message for ThriftCLIService.GetCrossReference and GetPrimaryKeys ### Why are the changes needed? Although we haven't had our own implementation for these thrift APIs, but it still worth logging the right message when people call them wrongly. ### Does this PR introduce _any_ user-facing change? yes, the driver log for the thrift server will log the right message for the ThriftCLIService.GetCrossReference and GetPrimaryKeys APIs ### How was this patch tested? passing Jenkins. Closes #29184 from yaooqinn/minor. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../org/apache/hive/service/cli/thrift/ThriftCLIService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 914d6d3612596..ea9ed57410045 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -648,7 +648,7 @@ public TGetPrimaryKeysResp GetPrimaryKeys(TGetPrimaryKeysReq req) resp.setOperationHandle(opHandle.toTOperationHandle()); resp.setStatus(OK_STATUS); } catch (Exception e) { - LOG.warn("Error getting functions: ", e); + LOG.warn("Error getting primary keys: ", e); resp.setStatus(HiveSQLException.toTStatus(e)); } return resp; @@ -666,7 +666,7 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) resp.setOperationHandle(opHandle.toTOperationHandle()); resp.setStatus(OK_STATUS); } catch (Exception e) { - LOG.warn("Error getting functions: ", e); + LOG.warn("Error getting cross reference: ", e); resp.setStatus(HiveSQLException.toTStatus(e)); } return resp; From feca9edbddcf9098dfca2fd4ef37f6faed5d83ee Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 22 Jul 2020 08:32:01 +0000 Subject: [PATCH 247/384] [MINOR][SQL][TESTS] Create tables once in JDBC tests ### What changes were proposed in this pull request? In PR, I propose to create input tables once before executing tests in `JDBCSuite` and `JdbcRDDSuite`. Currently, the table are created before every test in the test suites. ### Why are the changes needed? This speed up the test suites up 30-40%. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Run the modified test suites Closes #29176 from MaxGekk/jdbc-suite-before-all. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala | 6 ++++-- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 05013fbc49b8e..a204502be74b6 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -26,7 +26,8 @@ import org.apache.spark.util.Utils class JdbcRDDSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { - before { + override def beforeAll(): Unit = { + super.beforeAll() Utils.classForName("org.apache.derby.jdbc.EmbeddedDriver") val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true") try { @@ -96,7 +97,7 @@ class JdbcRDDSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkCont assert(rdd.reduce(_ + _) === 5050) } - after { + override def afterAll(): Unit = { try { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;shutdown=true") } catch { @@ -104,5 +105,6 @@ class JdbcRDDSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkCont // Normal single database shutdown // https://db.apache.org/derby/docs/10.2/ref/rrefexcept71493.html } + super.afterAll() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 7c723f083642d..ea086162c9271 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -71,7 +71,8 @@ class JDBCSuite extends QueryTest } } - before { + override def beforeAll(): Unit = { + super.beforeAll() Utils.classForName("org.h2.Driver") // Extra properties that will be specified for our database. We need these to test // usage of parameters from OPTIONS clause in queries. @@ -273,8 +274,9 @@ class JDBCSuite extends QueryTest // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. } - after { + override def afterAll(): Unit = { conn.close() + super.afterAll() } // Check whether the tables are fetched in the expected degree of parallelism From 04bf3511f174ff3189a2c010770f21f716315529 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 22 Jul 2020 02:51:30 -0700 Subject: [PATCH 248/384] [SPARK-21117][SQL][FOLLOWUP] Define prettyName for WidthBucket ### What changes were proposed in this pull request? This PR is to define prettyName for `WidthBucket`. This comes from the gatorsmile's suggestion: https://github.com/apache/spark/pull/28764#discussion_r457802957 ### Why are the changes needed? For a better name. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #29183 from maropu/SPARK-21117-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../expressions/mathExpressions.scala | 1 + .../sql-functions/sql-expression-schema.md | 2 +- .../sql-tests/results/operators.sql.out | 24 +++++++++---------- .../results/postgreSQL/numeric.sql.out | 22 ++++++++--------- 4 files changed, 25 insertions(+), 24 deletions(-) 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 5c764956adf0a..7f6ad2467480d 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 @@ -1406,6 +1406,7 @@ case class WidthBucket( override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType, DoubleType, DoubleType, LongType) override def dataType: DataType = LongType override def nullable: Boolean = true + override def prettyName: String = "width_bucket" override protected def nullSafeEval(input: Any, min: Any, max: Any, numBucket: Any): Any = { WidthBucket.computeBucketNumber( 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 e8169b9f06ce2..edf2ede9e5a44 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 @@ -291,7 +291,7 @@ | org.apache.spark.sql.catalyst.expressions.Uuid | uuid | SELECT uuid() | struct | | org.apache.spark.sql.catalyst.expressions.WeekDay | weekday | SELECT weekday('2009-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.WeekOfYear | weekofyear | SELECT weekofyear('2008-02-20') | struct | -| org.apache.spark.sql.catalyst.expressions.WidthBucket | width_bucket | SELECT width_bucket(5.3, 0.2, 10.6, 5) | struct | +| org.apache.spark.sql.catalyst.expressions.WidthBucket | width_bucket | SELECT width_bucket(5.3, 0.2, 10.6, 5) | struct | | org.apache.spark.sql.catalyst.expressions.XxHash64 | xxhash64 | SELECT xxhash64('Spark', array(123), 2) | struct | | org.apache.spark.sql.catalyst.expressions.Year | year | SELECT year('2016-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.ZipWith | zip_with | SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)) | struct>> | diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index fc58bc784c82f..f678e0141eaa9 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -461,7 +461,7 @@ NULL NULL -- !query select width_bucket(5.35, 0.024, 10.06, 5) -- !query schema -struct +struct -- !query output 3 @@ -469,7 +469,7 @@ struct +struct -- !query output 3 @@ -477,7 +477,7 @@ struct +struct -- !query output 3 @@ -485,7 +485,7 @@ struct +struct -- !query output 2 @@ -493,7 +493,7 @@ struct +struct -- !query output NULL @@ -501,7 +501,7 @@ NULL -- !query select width_bucket(null, 0.024, 10.06, 5) -- !query schema -struct +struct -- !query output NULL @@ -509,7 +509,7 @@ NULL -- !query select width_bucket(5.35, null, 10.06, 5) -- !query schema -struct +struct -- !query output NULL @@ -517,7 +517,7 @@ NULL -- !query select width_bucket(5.35, 0.024, null, -5) -- !query schema -struct +struct -- !query output NULL @@ -525,7 +525,7 @@ NULL -- !query select width_bucket(5.35, 0.024, 10.06, null) -- !query schema -struct +struct -- !query output NULL @@ -533,7 +533,7 @@ NULL -- !query select width_bucket(5.35, 0.024, 10.06, -5) -- !query schema -struct +struct -- !query output NULL @@ -541,7 +541,7 @@ NULL -- !query select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L) -- !query schema -struct +struct -- !query output NULL @@ -549,6 +549,6 @@ NULL -- !query select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L - 1) -- !query schema -struct +struct -- !query output 4894746858139549697 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 3cade4f1f9558..d97853d5fc6d0 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 @@ -4426,7 +4426,7 @@ struct<> -- !query SELECT width_bucket(5.0, 3.0, 4.0, 0) -- !query schema -struct +struct -- !query output NULL @@ -4434,7 +4434,7 @@ NULL -- !query SELECT width_bucket(5.0, 3.0, 4.0, -5) -- !query schema -struct +struct -- !query output NULL @@ -4442,7 +4442,7 @@ NULL -- !query SELECT width_bucket(3.5, 3.0, 3.0, 888) -- !query schema -struct +struct -- !query output NULL @@ -4450,7 +4450,7 @@ NULL -- !query SELECT width_bucket(double(5.0), double(3.0), double(4.0), 0) -- !query schema -struct +struct -- !query output NULL @@ -4458,7 +4458,7 @@ NULL -- !query SELECT width_bucket(double(5.0), double(3.0), double(4.0), -5) -- !query schema -struct +struct -- !query output NULL @@ -4466,7 +4466,7 @@ NULL -- !query SELECT width_bucket(double(3.5), double(3.0), double(3.0), 888) -- !query schema -struct +struct -- !query output NULL @@ -4474,7 +4474,7 @@ NULL -- !query SELECT width_bucket('NaN', 3.0, 4.0, 888) -- !query schema -struct +struct -- !query output NULL @@ -4482,7 +4482,7 @@ NULL -- !query SELECT width_bucket(double(0), 'NaN', double(4.0), 888) -- !query schema -struct +struct -- !query output NULL @@ -4564,7 +4564,7 @@ struct +struct -- !query output NULL @@ -4572,7 +4572,7 @@ NULL -- !query SELECT width_bucket(double(0.0), 5, double('-Infinity'), 20) -- !query schema -struct +struct -- !query output NULL @@ -4581,7 +4581,7 @@ NULL SELECT width_bucket(double('Infinity'), 1, 10, 10), width_bucket(double('-Infinity'), 1, 10, 10) -- !query schema -struct +struct -- !query output 11 0 From e8c06af7d1ab19b6478ac542cf876259cac5e13a Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 22 Jul 2020 09:53:16 -0500 Subject: [PATCH 249/384] [SPARK-32003][CORE] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost ### What changes were proposed in this pull request? If an executor is lost, the `DAGScheduler` handles the executor loss by removing the executor but does not unregister its outputs if the external shuffle service is used. However, if the node on which the executor runs is lost, the shuffle service may not be able to serve the shuffle files. In such a case, when fetches from the executor's outputs fail in the same stage, the `DAGScheduler` again removes the executor and by right, should unregister its outputs. It doesn't because the epoch used to track the executor failure has not increased. We track the epoch for failed executors that result in lost file output separately, so we can unregister the outputs in this scenario. The idea to track a second epoch is due to Attila Zsolt Piros. ### Why are the changes needed? Without the changes, the loss of a node could require two stage attempts to recover instead of one. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. This test fails without the change and passes with it. Closes #28848 from wypoon/SPARK-32003. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/DAGScheduler.scala | 100 ++++++++++++------ .../spark/scheduler/DAGSchedulerSuite.scala | 98 ++++++++++++++--- 2 files changed, 148 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index cb024d0852d06..73c95d19387c2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -170,13 +170,34 @@ private[spark] class DAGScheduler( */ private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]] - // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with - // every task. When we detect a node failing, we note the current epoch number and failed - // executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask results. - // - // TODO: Garbage collect information about failure epochs when we know there are no more - // stray messages to detect. - private val failedEpoch = new HashMap[String, Long] + /** + * Tracks the latest epoch of a fully processed error related to the given executor. (We use + * the MapOutputTracker's epoch number, which is sent with every task.) + * + * When an executor fails, it can affect the results of many tasks, and we have to deal with + * all of them consistently. We don't simply ignore all future results from that executor, + * as the failures may have been transient; but we also don't want to "overreact" to follow- + * on errors we receive. Furthermore, we might receive notification of a task success, after + * we find out the executor has actually failed; we'll assume those successes are, in fact, + * simply delayed notifications and the results have been lost, if the tasks started in the + * same or an earlier epoch. In particular, we use this to control when we tell the + * BlockManagerMaster that the BlockManager has been lost. + */ + private val executorFailureEpoch = new HashMap[String, Long] + + /** + * Tracks the latest epoch of a fully processed error where shuffle files have been lost from + * the given executor. + * + * This is closely related to executorFailureEpoch. They only differ for the executor when + * there is an external shuffle service serving shuffle files and we haven't been notified that + * the entire worker has been lost. In that case, when an executor is lost, we do not update + * the shuffleFileLostEpoch; we wait for a fetch failure. This way, if only the executor + * fails, we do not unregister the shuffle data as it can still be served; but if there is + * a failure in the shuffle service (resulting in fetch failure), we unregister the shuffle + * data only once, even if we get many fetch failures. + */ + private val shuffleFileLostEpoch = new HashMap[String, Long] private [scheduler] val outputCommitCoordinator = env.outputCommitCoordinator @@ -1566,7 +1587,8 @@ private[spark] class DAGScheduler( val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) - if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { + if (executorFailureEpoch.contains(execId) && + smt.epoch <= executorFailureEpoch(execId)) { logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") } else { // The epoch of the task is acceptable (i.e., the task was launched after the most @@ -1912,12 +1934,8 @@ private[spark] class DAGScheduler( * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * * We will also assume that we've lost all shuffle blocks associated with the executor if the - * executor serves its own blocks (i.e., we're not using external shuffle), the entire executor - * process is lost (likely including the shuffle service), or a FetchFailed occurred, in which - * case we presume all shuffle data related to this executor to be lost. - * - * Optionally the epoch during which the failure was caught can be passed to avoid allowing - * stray fetch failures from possibly retriggering the detection of a node as lost. + * executor serves its own blocks (i.e., we're not using an external shuffle service), or the + * entire Standalone worker is lost. */ private[scheduler] def handleExecutorLost( execId: String, @@ -1933,29 +1951,44 @@ private[spark] class DAGScheduler( maybeEpoch = None) } + /** + * Handles removing an executor from the BlockManagerMaster as well as unregistering shuffle + * outputs for the executor or optionally its host. + * + * @param execId executor to be removed + * @param fileLost If true, indicates that we assume we've lost all shuffle blocks associated + * with the executor; this happens if the executor serves its own blocks (i.e., we're not + * using an external shuffle service), the entire Standalone worker is lost, or a FetchFailed + * occurred (in which case we presume all shuffle data related to this executor to be lost). + * @param hostToUnregisterOutputs (optional) executor host if we're unregistering all the + * outputs on the host + * @param maybeEpoch (optional) the epoch during which the failure was caught (this prevents + * reprocessing for follow-on fetch failures) + */ private def removeExecutorAndUnregisterOutputs( execId: String, fileLost: Boolean, hostToUnregisterOutputs: Option[String], maybeEpoch: Option[Long] = None): Unit = { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) - if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { - failedEpoch(execId) = currentEpoch - logInfo("Executor lost: %s (epoch %d)".format(execId, currentEpoch)) + logDebug(s"Considering removal of executor $execId; " + + s"fileLost: $fileLost, currentEpoch: $currentEpoch") + if (!executorFailureEpoch.contains(execId) || executorFailureEpoch(execId) < currentEpoch) { + executorFailureEpoch(execId) = currentEpoch + logInfo(s"Executor lost: $execId (epoch $currentEpoch)") blockManagerMaster.removeExecutor(execId) - if (fileLost) { - hostToUnregisterOutputs match { - case Some(host) => - logInfo("Shuffle files lost for host: %s (epoch %d)".format(host, currentEpoch)) - mapOutputTracker.removeOutputsOnHost(host) - case None => - logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch)) - mapOutputTracker.removeOutputsOnExecutor(execId) - } - clearCacheLocs() - - } else { - logDebug("Additional executor lost message for %s (epoch %d)".format(execId, currentEpoch)) + clearCacheLocs() + } + if (fileLost && + (!shuffleFileLostEpoch.contains(execId) || shuffleFileLostEpoch(execId) < currentEpoch)) { + shuffleFileLostEpoch(execId) = currentEpoch + hostToUnregisterOutputs match { + case Some(host) => + logInfo(s"Shuffle files lost for host: $host (epoch $currentEpoch)") + mapOutputTracker.removeOutputsOnHost(host) + case None => + logInfo(s"Shuffle files lost for executor: $execId (epoch $currentEpoch)") + mapOutputTracker.removeOutputsOnExecutor(execId) } } } @@ -1981,11 +2014,12 @@ private[spark] class DAGScheduler( } private[scheduler] def handleExecutorAdded(execId: String, host: String): Unit = { - // remove from failedEpoch(execId) ? - if (failedEpoch.contains(execId)) { + // remove from executorFailureEpoch(execId) ? + if (executorFailureEpoch.contains(execId)) { logInfo("Host added was in lost list earlier: " + host) - failedEpoch -= execId + executorFailureEpoch -= execId } + shuffleFileLostEpoch -= execId } private[scheduler] def handleStageCancellation(stageId: Int, reason: Option[String]): Unit = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 7013832757e38..664cfc88cc410 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -25,6 +25,9 @@ import scala.annotation.meta.param import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.util.control.NonFatal +import org.mockito.Mockito.spy +import org.mockito.Mockito.times +import org.mockito.Mockito.verify import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.exceptions.TestFailedException import org.scalatest.time.SpanSugar._ @@ -235,6 +238,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi var sparkListener: EventInfoRecordingListener = null + var blockManagerMaster: BlockManagerMaster = null var mapOutputTracker: MapOutputTrackerMaster = null var broadcastManager: BroadcastManager = null var securityMgr: SecurityManager = null @@ -248,17 +252,18 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null, null, conf, true) { - override def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = { - blockIds.map { - _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). - getOrElse(Seq()) - }.toIndexedSeq - } - override def removeExecutor(execId: String): Unit = { - // don't need to propagate to the driver, which we don't have - } + class MyBlockManagerMaster(conf: SparkConf) extends BlockManagerMaster(null, null, conf, true) { + override def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = { + blockIds.map { + _.asRDDId.map { id => (id.rddId -> id.splitIndex) + }.flatMap { key => cacheLocations.get(key) + }.getOrElse(Seq()) + }.toIndexedSeq } + override def removeExecutor(execId: String): Unit = { + // don't need to propagate to the driver, which we don't have + } + } /** The list of results that DAGScheduler has collected. */ val results = new HashMap[Int, Any]() @@ -276,6 +281,16 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def jobFailed(exception: Exception): Unit = { failure = exception } } + class MyMapOutputTrackerMaster( + conf: SparkConf, + broadcastManager: BroadcastManager) + extends MapOutputTrackerMaster(conf, broadcastManager, true) { + + override def sendTracker(message: Any): Unit = { + // no-op, just so we can stop this to avoid leaking threads + } + } + override def beforeEach(): Unit = { super.beforeEach() init(new SparkConf()) @@ -293,11 +308,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi results.clear() securityMgr = new SecurityManager(conf) broadcastManager = new BroadcastManager(true, conf, securityMgr) - mapOutputTracker = new MapOutputTrackerMaster(conf, broadcastManager, true) { - override def sendTracker(message: Any): Unit = { - // no-op, just so we can stop this to avoid leaking threads - } - } + mapOutputTracker = spy(new MyMapOutputTrackerMaster(conf, broadcastManager)) + blockManagerMaster = spy(new MyBlockManagerMaster(conf)) scheduler = new DAGScheduler( sc, taskScheduler, @@ -548,6 +560,56 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mapStatus2(2).location.host === "hostB") } + test("SPARK-32003: All shuffle files for executor should be cleaned up on fetch failure") { + // reset the test context with the right shuffle service config + afterEach() + val conf = new SparkConf() + conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") + init(conf) + + val shuffleMapRdd = new MyRDD(sc, 3, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(3)) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 3, List(shuffleDep), tracker = mapOutputTracker) + + submit(reduceRdd, Array(0, 1, 2)) + // Map stage completes successfully, + // two tasks are run on an executor on hostA and one on an executor on hostB + completeShuffleMapStageSuccessfully(0, 0, 3, Seq("hostA", "hostA", "hostB")) + // Now the executor on hostA is lost + runEvent(ExecutorLost("hostA-exec", ExecutorExited(-100, false, "Container marked as failed"))) + // Executor is removed but shuffle files are not unregistered + verify(blockManagerMaster, times(1)).removeExecutor("hostA-exec") + verify(mapOutputTracker, times(0)).removeOutputsOnExecutor("hostA-exec") + + // The MapOutputTracker has all the shuffle files + val mapStatuses = mapOutputTracker.shuffleStatuses(shuffleId).mapStatuses + assert(mapStatuses.count(_ != null) === 3) + assert(mapStatuses.count(s => s != null && s.location.executorId == "hostA-exec") === 2) + assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1) + + // Now a fetch failure from the lost executor occurs + complete(taskSets(1), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null) + )) + // blockManagerMaster.removeExecutor is not called again + // but shuffle files are unregistered + verify(blockManagerMaster, times(1)).removeExecutor("hostA-exec") + verify(mapOutputTracker, times(1)).removeOutputsOnExecutor("hostA-exec") + + // Shuffle files for hostA-exec should be lost + assert(mapStatuses.count(_ != null) === 1) + assert(mapStatuses.count(s => s != null && s.location.executorId == "hostA-exec") === 0) + assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1) + + // Additional fetch failure from the executor does not result in further call to + // mapOutputTracker.removeOutputsOnExecutor + complete(taskSets(1), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 1, 0, "ignored"), null) + )) + verify(mapOutputTracker, times(1)).removeOutputsOnExecutor("hostA-exec") + } + test("zero split job") { var numResults = 0 var failureReason: Option[Exception] = None @@ -765,8 +827,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null))) - // this will get called - // blockManagerMaster.removeExecutor("hostA-exec") + verify(blockManagerMaster, times(1)).removeExecutor("hostA-exec") // ask the scheduler to try it again scheduler.resubmitFailedStages() // have the 2nd attempt pass @@ -806,11 +867,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0)) completeShuffleMapStageSuccessfully(0, 0, 1) runEvent(ExecutorLost("hostA-exec", event)) + verify(blockManagerMaster, times(1)).removeExecutor("hostA-exec") if (expectFileLoss) { + verify(mapOutputTracker, times(1)).removeOutputsOnExecutor("hostA-exec") intercept[MetadataFetchFailedException] { mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0) } } else { + verify(mapOutputTracker, times(0)).removeOutputsOnExecutor("hostA-exec") assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) } From cd16a10475c110dbf5739a37e8f5f103b5541234 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 22 Jul 2020 07:58:45 -0700 Subject: [PATCH 250/384] [SPARK-32364][SQL] Use CaseInsensitiveMap for DataFrameReader/Writer options ### What changes were proposed in this pull request? When a user have multiple options like `path`, `paTH`, and `PATH` for the same key `path`, `option/options` is non-deterministic because `extraOptions` is `HashMap`. This PR aims to use `CaseInsensitiveMap` instead of `HashMap` to fix this bug fundamentally. ### Why are the changes needed? Like the following, DataFrame's `option/options` have been non-deterministic in terms of case-insensitivity because it stores the options at `extraOptions` which is using `HashMap` class. ```scala spark.read .option("paTh", "1") .option("PATH", "2") .option("Path", "3") .option("patH", "4") .load("5") ... org.apache.spark.sql.AnalysisException: Path does not exist: file:/.../1; ``` ### Does this PR introduce _any_ user-facing change? Yes. However, this is a bug fix for the indeterministic cases. ### How was this patch tested? Pass the Jenkins or GitHub Action with newly added test cases. Closes #29160 from dongjoon-hyun/SPARK-32364. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../catalyst/util/CaseInsensitiveMap.scala | 6 ++++- .../catalyst/util/CaseInsensitiveMap.scala | 6 ++++- .../apache/spark/sql/DataFrameReader.scala | 10 ++++----- .../apache/spark/sql/DataFrameWriter.scala | 5 +++-- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 3 ++- .../sql/test/DataFrameReaderWriterSuite.scala | 22 +++++++++++++++++++ 6 files changed, 42 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index aff294e5d6305..c013888d080f2 100644 --- a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -39,10 +39,14 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma override def contains(k: String): Boolean = keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) - override def +[B1 >: T](kv: (String, B1)): Map[String, B1] = { + override def +[B1 >: T](kv: (String, B1)): CaseInsensitiveMap[B1] = { new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(kv._1)) + kv) } + def ++(xs: TraversableOnce[(String, T)]): CaseInsensitiveMap[T] = { + xs.foldLeft(this)(_ + _) + } + override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator override def -(key: String): Map[String, T] = { diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index a74636a80f1ce..66a2f2549b460 100644 --- a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -39,10 +39,14 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma override def contains(k: String): Boolean = keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) - override def updated[B1 >: T](key: String, value: B1): Map[String, B1] = { + override def updated[B1 >: T](key: String, value: B1): CaseInsensitiveMap[B1] = { new CaseInsensitiveMap[B1](originalMap.filter(!_._1.equalsIgnoreCase(key)) + (key -> value)) } + def ++(xs: IterableOnce[(String, T)]): CaseInsensitiveMap[T] = { + xs.iterator.foldLeft(this) { (m, kv) => m.updated(kv._1, kv._2) } + } + override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator override def removed(key: String): Map[String, T] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index a7b3d08ec4df7..a291e5825815d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -31,7 +31,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} -import org.apache.spark.sql.catalyst.util.FailureSafeParser +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, FailureSafeParser} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsCatalogOptions, SupportsRead} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils @@ -238,7 +238,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { Some("paths" -> objectMapper.writeValueAsString(paths.toArray)) } - val finalOptions = sessionOptions ++ extraOptions.toMap ++ pathsOption + val finalOptions = sessionOptions ++ extraOptions.originalMap ++ pathsOption val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) val (table, catalog, ident) = provider match { case _: SupportsCatalogOptions if userSpecifiedSchema.nonEmpty => @@ -276,7 +276,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { paths = paths, userSpecifiedSchema = userSpecifiedSchema, className = source, - options = extraOptions.toMap).resolveRelation()) + options = extraOptions.originalMap).resolveRelation()) } /** @@ -290,7 +290,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { // properties should override settings in extraOptions. this.extraOptions ++= properties.asScala // explicit url and dbtable should override all - this.extraOptions += (JDBCOptions.JDBC_URL -> url, JDBCOptions.JDBC_TABLE_NAME -> table) + this.extraOptions ++= Seq(JDBCOptions.JDBC_URL -> url, JDBCOptions.JDBC_TABLE_NAME -> table) format("jdbc").load() } @@ -879,6 +879,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { private var userSpecifiedSchema: Option[StructType] = None - private val extraOptions = new scala.collection.mutable.HashMap[String, String] + private var extraOptions = CaseInsensitiveMap[String](Map.empty) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 9aef382470bcc..952f896d1a20f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchT import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateTableAsSelectStatement, InsertIntoStatement, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelectStatement} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.{CatalogPlugin, CatalogV2Implicits, CatalogV2Util, Identifier, SupportsCatalogOptions, Table, TableCatalog, TableProvider, V1Table} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} @@ -768,7 +769,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { // connectionProperties should override settings in extraOptions. this.extraOptions ++= connectionProperties.asScala // explicit url and dbtable should override all - this.extraOptions += ("url" -> url, "dbtable" -> table) + this.extraOptions ++= Seq("url" -> url, "dbtable" -> table) format("jdbc").save() } @@ -960,7 +961,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private var mode: SaveMode = SaveMode.ErrorIfExists - private val extraOptions = new scala.collection.mutable.HashMap[String, String] + private var extraOptions = CaseInsensitiveMap[String](Map.empty) private var partitioningColumns: Option[Seq[String]] = None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index ea086162c9271..0e3dd4447c3fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -1304,7 +1304,8 @@ class JDBCSuite extends QueryTest testJdbcOptions(new JDBCOptions(parameters)) testJdbcOptions(new JDBCOptions(CaseInsensitiveMap(parameters))) // test add/remove key-value from the case-insensitive map - var modifiedParameters = CaseInsensitiveMap(Map.empty) ++ parameters + var modifiedParameters = + (CaseInsensitiveMap(Map.empty) ++ parameters).asInstanceOf[Map[String, String]] testJdbcOptions(new JDBCOptions(modifiedParameters)) modifiedParameters -= "dbtable" assert(modifiedParameters.get("dbTAblE").isEmpty) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index fe0a8439acc2d..c7ca0125c8a0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -224,6 +224,28 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with assert(LastOptions.parameters("opt3") == "3") } + test("SPARK-32364: path argument of load function should override all existing options") { + spark.read + .format("org.apache.spark.sql.test") + .option("paTh", "1") + .option("PATH", "2") + .option("Path", "3") + .option("patH", "4") + .load("5") + assert(LastOptions.parameters("path") == "5") + } + + test("SPARK-32364: path argument of save function should override all existing options") { + Seq(1).toDF.write + .format("org.apache.spark.sql.test") + .option("paTh", "1") + .option("PATH", "2") + .option("Path", "3") + .option("patH", "4") + .save("5") + assert(LastOptions.parameters("path") == "5") + } + test("pass partitionBy as options") { Seq(1).toDF.write .format("org.apache.spark.sql.test") From 184074de2286475f60d3fd8a2299ac21b8893256 Mon Sep 17 00:00:00 2001 From: ulysses Date: Wed, 22 Jul 2020 19:05:50 +0000 Subject: [PATCH 251/384] [SPARK-31999][SQL] Add REFRESH FUNCTION command ### What changes were proposed in this pull request? In Hive mode, permanent functions are shared with Hive metastore so that functions may be modified by other Hive client. With in long-lived spark scene, it's hard to update the change of function. Here are 2 reasons: * Spark cache the function in memory using `FunctionRegistry`. * User may not know the location or classname of udf when using `replace function`. Note that we use v2 command code path to add new command. ### Why are the changes needed? Give a easy way to make spark function registry sync with Hive metastore. Then we can call ``` refresh function functionName ``` ### Does this PR introduce _any_ user-facing change? Yes, new command. ### How was this patch tested? New UT. Closes #28840 from ulysses-you/SPARK-31999. Authored-by: ulysses Signed-off-by: Wenchen Fan --- docs/_data/menu-sql.yaml | 2 + docs/sql-ref-syntax-aux-cache-cache-table.md | 1 + docs/sql-ref-syntax-aux-cache-clear-cache.md | 1 + ...l-ref-syntax-aux-cache-refresh-function.md | 60 +++++++++++++++++++ .../sql-ref-syntax-aux-cache-refresh-table.md | 1 + docs/sql-ref-syntax-aux-cache-refresh.md | 1 + .../sql-ref-syntax-aux-cache-uncache-table.md | 1 + docs/sql-ref-syntax-aux-cache.md | 3 +- docs/sql-ref-syntax.md | 1 + .../spark/sql/catalyst/parser/SqlBase.g4 | 1 + .../sql/catalyst/analysis/Analyzer.scala | 6 ++ .../catalyst/analysis/v2ResolutionPlans.scala | 20 +++++++ .../sql/catalyst/catalog/SessionCatalog.scala | 8 +++ .../sql/catalyst/parser/AstBuilder.scala | 5 ++ .../catalyst/plans/logical/v2Commands.scala | 7 +++ .../catalog/CatalogV2Implicits.scala | 10 +++- .../sql/connector/catalog/LookupCatalog.scala | 29 ++++++++- .../sql/catalyst/parser/DDLParserSuite.scala | 11 +++- .../analysis/ResolveSessionCatalog.scala | 30 ++-------- .../sql/execution/command/functions.scala | 39 ++++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 13 ++++ .../sql/execution/command/DDLSuite.scala | 47 ++++++++++++++- 22 files changed, 265 insertions(+), 32 deletions(-) create mode 100644 docs/sql-ref-syntax-aux-cache-refresh-function.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 2d26326a00b0e..ef5a53a69eda6 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -208,6 +208,8 @@ url: sql-ref-syntax-aux-cache-clear-cache.html - text: REFRESH TABLE url: sql-ref-syntax-aux-cache-refresh-table.html + - text: REFRESH FUNCTION + url: sql-ref-syntax-aux-cache-refresh-function.html - text: REFRESH url: sql-ref-syntax-aux-cache-refresh.html - text: DESCRIBE diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md index fdef3d657dfa3..8829016fc17ac 100644 --- a/docs/sql-ref-syntax-aux-cache-cache-table.md +++ b/docs/sql-ref-syntax-aux-cache-cache-table.md @@ -80,3 +80,4 @@ CACHE TABLE testCache OPTIONS ('storageLevel' 'DISK_ONLY') SELECT * FROM testDat * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-cache-clear-cache.md b/docs/sql-ref-syntax-aux-cache-clear-cache.md index a27cd83c146a3..aae4e39600375 100644 --- a/docs/sql-ref-syntax-aux-cache-clear-cache.md +++ b/docs/sql-ref-syntax-aux-cache-clear-cache.md @@ -41,3 +41,4 @@ CLEAR CACHE; * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-cache-refresh-function.md b/docs/sql-ref-syntax-aux-cache-refresh-function.md new file mode 100644 index 0000000000000..d91fc062eb714 --- /dev/null +++ b/docs/sql-ref-syntax-aux-cache-refresh-function.md @@ -0,0 +1,60 @@ +--- +layout: global +title: REFRESH FUNCTION +displayTitle: REFRESH FUNCTION +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +### Description + +`REFRESH FUNCTION` statement invalidates the cached function entry, which includes a class name +and resource location of the given function. The invalidated cache is populated right away. +Note that `REFRESH FUNCTION` only works for permanent functions. Refreshing native functions or temporary functions will cause an exception. + +### Syntax + +```sql +REFRESH FUNCTION function_identifier +``` + +### Parameters + +* **function_identifier** + + Specifies a function name, which is either a qualified or unqualified name. If no database identifier is provided, uses the current database. + + **Syntax:** `[ database_name. ] function_name` + +### Examples + +```sql +-- The cached entry of the function will be refreshed +-- The function is resolved from the current database as the function name is unqualified. +REFRESH FUNCTION func1; + +-- The cached entry of the function will be refreshed +-- The function is resolved from tempDB database as the function name is qualified. +REFRESH FUNCTION db1.func1; +``` + +### Related Statements + +* [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) +* [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) +* [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) +* [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) +* [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-cache-refresh-table.md b/docs/sql-ref-syntax-aux-cache-refresh-table.md index 8d4a804f88671..cc35c0451d5cb 100644 --- a/docs/sql-ref-syntax-aux-cache-refresh-table.md +++ b/docs/sql-ref-syntax-aux-cache-refresh-table.md @@ -57,3 +57,4 @@ REFRESH TABLE tempDB.view1; * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-cache-refresh.md b/docs/sql-ref-syntax-aux-cache-refresh.md index b10e6fb47aaf7..715bdcac3b6f1 100644 --- a/docs/sql-ref-syntax-aux-cache-refresh.md +++ b/docs/sql-ref-syntax-aux-cache-refresh.md @@ -54,3 +54,4 @@ REFRESH "hdfs://path/to/table"; * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md index 96a691e4c3931..4456378cdee15 100644 --- a/docs/sql-ref-syntax-aux-cache-uncache-table.md +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -50,3 +50,4 @@ UNCACHE TABLE t1; * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md index 0ccb1c61a0da5..17a13e67e532e 100644 --- a/docs/sql-ref-syntax-aux-cache.md +++ b/docs/sql-ref-syntax-aux-cache.md @@ -23,4 +23,5 @@ license: | * [UNCACHE TABLE statement](sql-ref-syntax-aux-cache-uncache-table.html) * [CLEAR CACHE statement](sql-ref-syntax-aux-cache-clear-cache.html) * [REFRESH TABLE statement](sql-ref-syntax-aux-cache-refresh-table.html) - * [REFRESH statement](sql-ref-syntax-aux-cache-refresh.html) \ No newline at end of file + * [REFRESH statement](sql-ref-syntax-aux-cache-refresh.html) + * [REFRESH FUNCTION statement](sql-ref-syntax-aux-cache-refresh-function.html) diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index 4bf1858428d98..290523a2b1ee1 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -83,6 +83,7 @@ Spark SQL is Apache Spark's module for working with structured data. The SQL Syn * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) + * [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) * [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) * [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index bc7e982830584..73dfdcc263a7d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -229,6 +229,7 @@ statement comment=(STRING | NULL) #commentNamespace | COMMENT ON TABLE multipartIdentifier IS comment=(STRING | NULL) #commentTable | REFRESH TABLE multipartIdentifier #refreshTable + | REFRESH FUNCTION multipartIdentifier #refreshFunction | REFRESH (STRING | .*?) #refreshResource | CACHE LAZY? TABLE multipartIdentifier (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a77d4f1c6243b..cfc31858d47a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1886,11 +1886,17 @@ class Analyzer( } /** + * Replaces [[UnresolvedFunc]]s with concrete [[LogicalPlan]]s. * Replaces [[UnresolvedFunction]]s with concrete [[Expression]]s. */ object ResolveFunctions extends Rule[LogicalPlan] { val trimWarningEnabled = new AtomicBoolean(true) def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + // Resolve functions with concrete relations from v2 catalog. + case UnresolvedFunc(multipartIdent) => + val funcIdent = parseSessionCatalogFunctionIdentifier(multipartIdent, "function lookup") + ResolvedFunc(Identifier.of(funcIdent.database.toArray, funcIdent.funcName)) + case q: LogicalPlan => q transformExpressions { case u if !u.childrenResolved => u // Skip until children are resolved. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index f3d40c6d36cc3..a16763f2cf943 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.catalog.CatalogFunction import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsNamespaces, Table, TableCatalog} @@ -50,6 +51,15 @@ case class UnresolvedTableOrView(multipartIdentifier: Seq[String]) extends LeafN override def output: Seq[Attribute] = Nil } +/** + * Holds the name of a function that has yet to be looked up in a catalog. It will be resolved to + * [[ResolvedFunc]] during analysis. + */ +case class UnresolvedFunc(multipartIdentifier: Seq[String]) extends LeafNode { + override lazy val resolved: Boolean = false + override def output: Seq[Attribute] = Nil +} + /** * A plan containing resolved namespace. */ @@ -74,3 +84,13 @@ case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: T case class ResolvedView(identifier: Identifier) extends LeafNode { override def output: Seq[Attribute] = Nil } + +/** + * A plan containing resolved function. + */ +// TODO: create a generic representation for v1, v2 function, after we add function +// support to v2 catalog. For now we only need the identifier to fallback to v1 command. +case class ResolvedFunc(identifier: Identifier) + extends LeafNode { + override def output: Seq[Attribute] = Nil +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 2b3f05f61b483..ec0c34d4c7960 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1351,6 +1351,14 @@ class SessionCatalog( functionRegistry.registerFunction(func, info, builder) } + /** + * Unregister a temporary or permanent function from a session-specific [[FunctionRegistry]] + * Return true if function exists. + */ + def unregisterFunction(name: FunctionIdentifier): Boolean = { + functionRegistry.dropFunction(name) + } + /** * Drop a temporary function. */ 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 13e528e7968ca..5663741bae505 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 @@ -3660,6 +3660,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx.REPLACE != null) } + override def visitRefreshFunction(ctx: RefreshFunctionContext): LogicalPlan = withOrigin(ctx) { + val functionIdentifier = visitMultipartIdentifier(ctx.multipartIdentifier) + RefreshFunction(UnresolvedFunc(functionIdentifier)) + } + override def visitCommentNamespace(ctx: CommentNamespaceContext): LogicalPlan = withOrigin(ctx) { val comment = ctx.comment.getType match { case SqlBaseParser.NULL => "" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index b4120d9f64cc5..137fc70397642 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -516,3 +516,10 @@ case class CommentOnNamespace(child: LogicalPlan, comment: String) extends Comma case class CommentOnTable(child: LogicalPlan, comment: String) extends Command { override def children: Seq[LogicalPlan] = child :: Nil } + +/** + * The logical plan of the REFRESH FUNCTION command that works for v2 catalogs. + */ +case class RefreshFunction(child: LogicalPlan) extends Command { + override def children: Seq[LogicalPlan] = child :: Nil +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index d90804f4b6ff6..2ee760d4f60b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.connector.catalog import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.connector.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} @@ -107,6 +107,14 @@ private[sql] object CatalogV2Implicits { throw new AnalysisException( s"$quoted is not a valid TableIdentifier as it has more than 2 name parts.") } + + def asFunctionIdentifier: FunctionIdentifier = ident.namespace() match { + case ns if ns.isEmpty => FunctionIdentifier(ident.name()) + case Array(dbName) => FunctionIdentifier(ident.name(), Some(dbName)) + case _ => + throw new AnalysisException( + s"$quoted is not a valid FunctionIdentifier as it has more than 2 name parts.") + } } implicit class MultipartIdentifierHelper(parts: Seq[String]) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 10c15747ec4ce..b84bf3e2786bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.connector.catalog import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} /** @@ -155,4 +155,31 @@ private[sql] trait LookupCatalog extends Logging { None } } + + // TODO: move function related v2 statements to the new framework. + def parseSessionCatalogFunctionIdentifier( + nameParts: Seq[String], + sql: String): FunctionIdentifier = { + if (nameParts.length == 1 && catalogManager.v1SessionCatalog.isTempFunction(nameParts.head)) { + return FunctionIdentifier(nameParts.head) + } + + nameParts match { + case SessionCatalogAndIdentifier(_, ident) => + if (nameParts.length == 1) { + // If there is only one name part, it means the current catalog is the session catalog. + // Here we don't fill the default database, to keep the error message unchanged for + // v1 commands. + FunctionIdentifier(nameParts.head, None) + } else { + ident.namespace match { + case Array(db) => FunctionIdentifier(ident.name, Some(db)) + case _ => + throw new AnalysisException(s"Unsupported function name '$ident'") + } + } + + case _ => throw new AnalysisException(s"$sql is only supported in v1 catalog") + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index e802449a69743..47f21a0a1910d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedFunc, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView} import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, BucketSpec, FileResource, FunctionResource, FunctionResourceType, JarResource} import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ @@ -2109,6 +2109,15 @@ class DDLParserSuite extends AnalysisTest { "Operation not allowed: CREATE FUNCTION with resource type 'other'") } + test("REFRESH FUNCTION") { + parseCompare("REFRESH FUNCTION c", + RefreshFunction(UnresolvedFunc(Seq("c")))) + parseCompare("REFRESH FUNCTION b.c", + RefreshFunction(UnresolvedFunc(Seq("b", "c")))) + parseCompare("REFRESH FUNCTION a.b.c", + RefreshFunction(UnresolvedFunc(Seq("a", "b", "c")))) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index bc3f38a35834d..5717013b2eba2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -622,33 +622,11 @@ class ResolveSessionCatalog( CreateFunctionCommand(database, function, className, resources, isTemp, ignoreIfExists, replace) } - } - - // TODO: move function related v2 statements to the new framework. - private def parseSessionCatalogFunctionIdentifier( - nameParts: Seq[String], - sql: String): FunctionIdentifier = { - if (nameParts.length == 1 && isTempFunction(nameParts.head)) { - return FunctionIdentifier(nameParts.head) - } - - nameParts match { - case SessionCatalogAndIdentifier(_, ident) => - if (nameParts.length == 1) { - // If there is only one name part, it means the current catalog is the session catalog. - // Here we don't fill the default database, to keep the error message unchanged for - // v1 commands. - FunctionIdentifier(nameParts.head, None) - } else { - ident.namespace match { - case Array(db) => FunctionIdentifier(ident.name, Some(db)) - case _ => - throw new AnalysisException(s"Unsupported function name '$ident'") - } - } - case _ => throw new AnalysisException(s"$sql is only supported in v1 catalog") - } + case RefreshFunction(ResolvedFunc(identifier)) => + // Fallback to v1 command + val funcIdentifier = identifier.asFunctionIdentifier + RefreshFunctionCommand(funcIdentifier.database, funcIdentifier.funcName) } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = tableName match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 6fdc7f4a58195..252d188ff8fe2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -236,6 +236,45 @@ case class ShowFunctionsCommand( } } + +/** + * A command for users to refresh the persistent function. + * The syntax of using this command in SQL is: + * {{{ + * REFRESH FUNCTION functionName + * }}} + */ +case class RefreshFunctionCommand( + databaseName: Option[String], + functionName: String) + extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + if (FunctionRegistry.builtin.functionExists(FunctionIdentifier(functionName))) { + throw new AnalysisException(s"Cannot refresh builtin function $functionName") + } + if (catalog.isTemporaryFunction(FunctionIdentifier(functionName, databaseName))) { + throw new AnalysisException(s"Cannot refresh temporary function $functionName") + } + + val identifier = FunctionIdentifier( + functionName, Some(databaseName.getOrElse(catalog.getCurrentDatabase))) + // we only refresh the permanent function. + if (catalog.isPersistentFunction(identifier)) { + // register overwrite function. + val func = catalog.getFunctionMetadata(identifier) + catalog.registerFunction(func, true) + } else { + // clear cached function and throw exception + catalog.unregisterFunction(identifier) + throw new NoSuchFunctionException(identifier.database.get, identifier.funcName) + } + + Seq.empty[Row] + } +} + object FunctionsCommand { // operators that do not have corresponding functions. // They should be handled `DescribeFunctionCommand`, `ShowFunctionsCommand` diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 85aea3ce41ecc..d6c24e47e8652 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2229,6 +2229,19 @@ class DataSourceV2SQLSuite "The namespace in session catalog must have exactly one name part: default.ns1.ns2.fun")) } + test("REFRESH FUNCTION: only support session catalog") { + val e = intercept[AnalysisException] { + sql("REFRESH FUNCTION testcat.ns1.ns2.fun") + } + assert(e.message.contains("function lookup is only supported in v1 catalog")) + + val e1 = intercept[AnalysisException] { + sql("REFRESH FUNCTION default.ns1.ns2.fun") + } + assert(e1.message.contains( + "The namespace in session catalog must have exactly one name part: default.ns1.ns2.fun")) + } + test("global temp view should not be masked by v2 catalog") { val globalTempDB = spark.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) spark.conf.set(s"spark.sql.catalog.$globalTempDB", classOf[InMemoryTableCatalog].getName) 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 e4709e469dca3..faafcb721008f 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 @@ -28,8 +28,8 @@ import org.apache.spark.{SparkException, SparkFiles} import org.apache.spark.internal.config import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} -import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchFunctionException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER @@ -3030,6 +3030,49 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } } + + test("REFRESH FUNCTION") { + val msg = intercept[AnalysisException] { + sql("REFRESH FUNCTION md5") + }.getMessage + assert(msg.contains("Cannot refresh builtin function")) + + withUserDefinedFunction("func1" -> true) { + sql("CREATE TEMPORARY FUNCTION func1 AS 'test.org.apache.spark.sql.MyDoubleAvg'") + val msg = intercept[AnalysisException] { + sql("REFRESH FUNCTION func1") + }.getMessage + assert(msg.contains("Cannot refresh temporary function")) + } + + withUserDefinedFunction("func1" -> false) { + intercept[NoSuchFunctionException] { + sql("REFRESH FUNCTION func1") + } + + val func = FunctionIdentifier("func1", Some("default")) + sql("CREATE FUNCTION func1 AS 'test.org.apache.spark.sql.MyDoubleAvg'") + assert(!spark.sessionState.catalog.isRegisteredFunction(func)) + sql("REFRESH FUNCTION func1") + assert(spark.sessionState.catalog.isRegisteredFunction(func)) + + spark.sessionState.catalog.externalCatalog.dropFunction("default", "func1") + assert(spark.sessionState.catalog.isRegisteredFunction(func)) + intercept[NoSuchFunctionException] { + sql("REFRESH FUNCTION func1") + } + assert(!spark.sessionState.catalog.isRegisteredFunction(func)) + + val function = CatalogFunction(func, "test.non.exists.udf", Seq.empty) + spark.sessionState.catalog.createFunction(function, false) + assert(!spark.sessionState.catalog.isRegisteredFunction(func)) + val err = intercept[AnalysisException] { + sql("REFRESH FUNCTION func1") + }.getMessage + assert(err.contains("Can not load class")) + assert(!spark.sessionState.catalog.isRegisteredFunction(func)) + } + } } object FakeLocalFsFileSystem { From b151194299f5ba15e0d9d8d7d2980fd164fe0822 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 23 Jul 2020 10:12:13 +0900 Subject: [PATCH 252/384] [SPARK-32392][SQL] Reduce duplicate error log for executing sql statement operation in thrift server ### What changes were proposed in this pull request? This PR removes the duplicated error log which has been logged in `org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation#execute` but logged again in `runInternal`. Besides, the log4j configuration for SparkExecuteStatementOperation is turned off because it's not very friendly for Jenkins ### Why are the changes needed? remove the duplicated error log for better user experience ### Does this PR introduce _any_ user-facing change? Yes, less log in thrift server's driver log ### How was this patch tested? locally verified the result in target/unit-test.log Closes #29189 from yaooqinn/SPARK-32392. Authored-by: Kent Yao Signed-off-by: Takeshi Yamamuro --- .../hive/thriftserver/SparkExecuteStatementOperation.scala | 4 +--- sql/hive-thriftserver/src/test/resources/log4j.properties | 2 ++ 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 57ed15a76a893..f2d53a77d6816 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -218,9 +218,7 @@ private[hive] class SparkExecuteStatementOperation( execute() } } catch { - case e: HiveSQLException => - setOperationException(e) - log.error("Error running hive query: ", e) + case e: HiveSQLException => setOperationException(e) } } } diff --git a/sql/hive-thriftserver/src/test/resources/log4j.properties b/sql/hive-thriftserver/src/test/resources/log4j.properties index 21975ba818142..a73946c6dc5f3 100644 --- a/sql/hive-thriftserver/src/test/resources/log4j.properties +++ b/sql/hive-thriftserver/src/test/resources/log4j.properties @@ -63,3 +63,5 @@ log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR # Parquet related logging log4j.logger.org.apache.parquet.CorruptStatistics=ERROR log4j.logger.parquet.CorruptStatistics=ERROR + +log4j.logger.org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation=OFF From 46169823c0e3f1a3877623f37359266ef3758f95 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Jul 2020 03:07:30 +0000 Subject: [PATCH 253/384] [SPARK-30616][SQL][FOLLOW-UP] Use only config key name in the config doc ### What changes were proposed in this pull request? This is a follow-up of #28852. This PR to use only config name; otherwise the doc for the config entry shows the entire details of the referring configs. ### Why are the changes needed? The doc for the newly introduced config entry shows the entire details of the referring configs. ### Does this PR introduce _any_ user-facing change? The doc for the config entry will show only the referring config keys. ### How was this patch tested? Existing tests. Closes #29194 from ueshin/issues/SPARK-30616/fup. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/internal/StaticSQLConf.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 9bc910b5e1cdf..10f54f856a193 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -232,9 +232,9 @@ object StaticSQLConf { .doc("Time-to-live (TTL) value for the metadata caches: partition file metadata cache and " + "session catalog cache. This configuration only has an effect when this value having " + "a positive value (> 0). It also requires setting " + - s"${StaticSQLConf.CATALOG_IMPLEMENTATION} to `hive`, setting " + - s"${SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE} > 0 and setting " + - s"${SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS} to `true` " + + s"'${StaticSQLConf.CATALOG_IMPLEMENTATION.key}' to `hive`, setting " + + s"'${SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key}' > 0 and setting " + + s"'${SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key}' to `true` " + "to be applied to the partition file metadata cache.") .version("3.1.0") .timeConf(TimeUnit.SECONDS) From 182566bf57a6691599c0dc986cd71b3d8f400e22 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Thu, 23 Jul 2020 03:10:45 +0000 Subject: [PATCH 254/384] [SPARK-32237][SQL] Resolve hint in CTE ### What changes were proposed in this pull request? This PR is to move `Substitution` rule before `Hints` rule in `Analyzer` to avoid hint in CTE not working. ### Why are the changes needed? Below SQL in Spark3.0 will throw AnalysisException, but it works in Spark2.x ```sql WITH cte AS (SELECT /*+ REPARTITION(3) */ T.id, T.data FROM $t1 T) SELECT cte.id, cte.data FROM cte ``` ``` Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve '`cte.id`' given input columns: [cte.data, cte.id]; line 3 pos 7; 'Project ['cte.id, 'cte.data] +- SubqueryAlias cte +- Project [id#21L, data#22] +- SubqueryAlias T +- SubqueryAlias testcat.ns1.ns2.tbl +- RelationV2[id#21L, data#22] testcat.ns1.ns2.tbl 'Project ['cte.id, 'cte.data] +- SubqueryAlias cte +- Project [id#21L, data#22] +- SubqueryAlias T +- SubqueryAlias testcat.ns1.ns2.tbl +- RelationV2[id#21L, data#22] testcat.ns1.ns2.tbl ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add a unit test Closes #29062 from LantaoJin/SPARK-32237. Authored-by: LantaoJin Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 10 ++++---- .../sql/catalyst/analysis/AnalysisSuite.scala | 25 ++++++++++++++++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 12 +++++++++ 3 files changed, 41 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index cfc31858d47a5..a9a6fcd3bdb28 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -200,6 +200,11 @@ class Analyzer( val postHocResolutionRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( + Batch("Substitution", fixedPoint, + CTESubstitution, + WindowsSubstitution, + EliminateUnions, + new SubstituteUnresolvedOrdinals(conf)), Batch("Disable Hints", Once, new ResolveHints.DisableHints(conf)), Batch("Hints", fixedPoint, @@ -207,11 +212,6 @@ class Analyzer( new ResolveHints.ResolveCoalesceHints(conf)), Batch("Simple Sanity Check", Once, LookupFunctions), - Batch("Substitution", fixedPoint, - CTESubstitution, - WindowsSubstitution, - EliminateUnions, - new SubstituteUnresolvedOrdinals(conf)), Batch("Resolution", fixedPoint, ResolveTableValuedFunctions :: ResolveNamespace(catalogManager) :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index c0be49af2107d..77e427d93daeb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -26,7 +26,7 @@ import org.apache.log4j.Level import org.scalatest.Matchers import org.apache.spark.api.python.PythonEvalType -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -895,4 +895,27 @@ class AnalysisSuite extends AnalysisTest with Matchers { assertAnalysisError(testRelation2.select(RowNumber() + 1), Seq("Window function row_number() requires an OVER clause.")) } + + test("SPARK-32237: Hint in CTE") { + val plan = With( + Project( + Seq(UnresolvedAttribute("cte.a")), + UnresolvedRelation(TableIdentifier("cte")) + ), + Seq( + ( + "cte", + SubqueryAlias( + AliasIdentifier("cte"), + UnresolvedHint( + "REPARTITION", + Seq(Literal(3)), + Project(testRelation.output, testRelation) + ) + ) + ) + ) + ) + assertAnalysisSuccess(plan) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 989f304b1f07f..2bddaee0f179b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3560,6 +3560,18 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-32237: Hint in CTE") { + withTable("t") { + sql("CREATE TABLE t USING PARQUET AS SELECT 1 AS id") + checkAnswer( + sql(s""" + |WITH cte AS (SELECT /*+ REPARTITION(3) */ * FROM t) + |SELECT * FROM cte + """.stripMargin), + Row(1) :: Nil) + } + } } case class Foo(bar: Option[String]) From f8d29d371cdf0b8be6a48a9124ffbc3c0794f32a Mon Sep 17 00:00:00 2001 From: Devesh Agrawal Date: Wed, 22 Jul 2020 21:04:06 -0700 Subject: [PATCH 255/384] [SPARK-32217] Plumb whether a worker would also be decommissioned along with executor ### What changes were proposed in this pull request? This PR is a giant plumbing PR that plumbs an `ExecutorDecommissionInfo` along with the DecommissionExecutor message. ### Why are the changes needed? The primary motivation is to know whether a decommissioned executor would also be loosing shuffle files -- and thus it is important to know whether the host would also be decommissioned. In the absence of this PR, the existing code assumes that decommissioning an executor does not loose the whole host with it, and thus does not clear the shuffle state if external shuffle service is enabled. While this may hold in some cases (like K8s decommissioning an executor pod, or YARN container preemption), it does not hold in others like when the cluster is managed by a Standalone Scheduler (Master). This is similar to the existing `workerLost` field in the `ExecutorProcessLost` message. In the future, this `ExecutorDecommissionInfo` can be embellished for knowing how long the executor has to live for scenarios like Cloud spot kills (or Yarn preemption) and the like. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tweaked an existing unit test in `AppClientSuite` Closes #29032 from agrawaldevesh/plumb_decom_info. Authored-by: Devesh Agrawal Signed-off-by: Holden Karau --- .../deploy/client/StandaloneAppClient.scala | 4 ++- .../client/StandaloneAppClientListener.scala | 4 ++- .../apache/spark/deploy/master/Master.scala | 5 +++- .../CoarseGrainedExecutorBackend.scala | 16 +++++++---- .../scheduler/ExecutorDecommissionInfo.scala | 28 +++++++++++++++++++ .../spark/scheduler/TaskScheduler.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 3 +- .../cluster/CoarseGrainedClusterMessage.scala | 4 ++- .../CoarseGrainedSchedulerBackend.scala | 22 ++++++++------- .../cluster/StandaloneSchedulerBackend.scala | 6 ++-- .../spark/deploy/client/AppClientSuite.scala | 15 ++++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 8 ++++-- .../ExternalClusterManagerSuite.scala | 4 ++- .../WorkerDecommissionExtendedSuite.scala | 2 +- .../scheduler/WorkerDecommissionSuite.scala | 2 +- ...kManagerDecommissionIntegrationSuite.scala | 2 +- 16 files changed, 92 insertions(+), 35 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index eedf5e969e291..a6da8393bf405 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -31,6 +31,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging import org.apache.spark.rpc._ +import org.apache.spark.scheduler.ExecutorDecommissionInfo import org.apache.spark.util.{RpcUtils, ThreadUtils} /** @@ -181,7 +182,8 @@ private[spark] class StandaloneAppClient( if (ExecutorState.isFinished(state)) { listener.executorRemoved(fullId, message.getOrElse(""), exitStatus, workerLost) } else if (state == ExecutorState.DECOMMISSIONED) { - listener.executorDecommissioned(fullId, message.getOrElse("")) + listener.executorDecommissioned(fullId, + ExecutorDecommissionInfo(message.getOrElse(""), isHostDecommissioned = workerLost)) } case WorkerRemoved(id, host, message) => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala index 2e38a6847891d..e72f7e976bb0a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.client +import org.apache.spark.scheduler.ExecutorDecommissionInfo + /** * Callbacks invoked by deploy client when various events happen. There are currently five events: * connecting to the cluster, disconnecting, being given an executor, having an executor removed @@ -39,7 +41,7 @@ private[spark] trait StandaloneAppClientListener { def executorRemoved( fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit - def executorDecommissioned(fullId: String, message: String): Unit + def executorDecommissioned(fullId: String, decommissionInfo: ExecutorDecommissionInfo): Unit def workerRemoved(workerId: String, host: String, message: String): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 0070df1d66dee..220e1c963d5ea 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -908,7 +908,10 @@ private[deploy] class Master( logInfo("Telling app of decommission executors") exec.application.driver.send(ExecutorUpdated( exec.id, ExecutorState.DECOMMISSIONED, - Some("worker decommissioned"), None, workerLost = false)) + Some("worker decommissioned"), None, + // workerLost is being set to true here to let the driver know that the host (aka. worker) + // is also being decommissioned. + workerLost = true)) exec.state = ExecutorState.DECOMMISSIONED exec.application.removeExecutor(exec) } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e072d7919450e..def125bb6bfb6 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -40,7 +40,7 @@ import org.apache.spark.resource.ResourceProfile import org.apache.spark.resource.ResourceProfile._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc._ -import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} +import org.apache.spark.scheduler.{ExecutorDecommissionInfo, ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, SignalUtils, ThreadUtils, Utils} @@ -166,11 +166,15 @@ private[spark] class CoarseGrainedExecutorBackend( exitExecutor(1, "Received LaunchTask command but executor was null") } else { if (decommissioned) { - logError("Asked to launch a task while decommissioned.") + val msg = "Asked to launch a task while decommissioned." + logError(msg) driver match { case Some(endpoint) => logInfo("Sending DecommissionExecutor to driver.") - endpoint.send(DecommissionExecutor(executorId)) + endpoint.send( + DecommissionExecutor( + executorId, + ExecutorDecommissionInfo(msg, isHostDecommissioned = false))) case _ => logError("No registered driver to send Decommission to.") } @@ -259,12 +263,14 @@ private[spark] class CoarseGrainedExecutorBackend( } private def decommissionSelf(): Boolean = { - logInfo("Decommissioning self w/sync") + val msg = "Decommissioning self w/sync" + logInfo(msg) try { decommissioned = true // Tell master we are are decommissioned so it stops trying to schedule us if (driver.nonEmpty) { - driver.get.askSync[Boolean](DecommissionExecutor(executorId)) + driver.get.askSync[Boolean](DecommissionExecutor( + executorId, ExecutorDecommissionInfo(msg, false))) } else { logError("No driver to message decommissioning.") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala new file mode 100644 index 0000000000000..a82b5d38afe9f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +/** + * Provides more detail when an executor is being decommissioned. + * @param message Human readable reason for why the decommissioning is happening. + * @param isHostDecommissioned Whether the host (aka the `node` or `worker` in other places) is + * being decommissioned too. Used to infer if the shuffle data might + * be lost even if the external shuffle service is enabled. + */ +private[spark] +case class ExecutorDecommissionInfo(message: String, isHostDecommissioned: Boolean) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 08f9f3c256e69..b29458c481413 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -101,7 +101,7 @@ private[spark] trait TaskScheduler { /** * Process a decommissioning executor. */ - def executorDecommission(executorId: String): Unit + def executorDecommission(executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit /** * Process a lost executor diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 12bd93286d736..28e138ea9b79c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -912,7 +912,8 @@ private[spark] class TaskSchedulerImpl( } } - override def executorDecommission(executorId: String): Unit = { + override def executorDecommission( + executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = { rootPool.executorDecommission(executorId) backend.reviveOffers() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index bb929c27b6a65..91485f01bf007 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.scheduler.ExecutorDecommissionInfo import org.apache.spark.scheduler.ExecutorLossReason import org.apache.spark.util.SerializableBuffer @@ -94,7 +95,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: ExecutorLossReason) extends CoarseGrainedClusterMessage - case class DecommissionExecutor(executorId: String) extends CoarseGrainedClusterMessage + case class DecommissionExecutor(executorId: String, decommissionInfo: ExecutorDecommissionInfo) + extends CoarseGrainedClusterMessage case class RemoveWorker(workerId: String, host: String, message: String) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index b4b3efaaff88f..8fbefae58af14 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -191,9 +191,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) - case DecommissionExecutor(executorId) => - logError(s"Received decommission executor message ${executorId}.") - decommissionExecutor(executorId) + case DecommissionExecutor(executorId, decommissionInfo) => + logError(s"Received decommission executor message ${executorId}: $decommissionInfo") + decommissionExecutor(executorId, decommissionInfo) case RemoveWorker(workerId, host, message) => removeWorker(workerId, host, message) @@ -272,9 +272,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp removeWorker(workerId, host, message) context.reply(true) - case DecommissionExecutor(executorId) => - logError(s"Received decommission executor message ${executorId}.") - decommissionExecutor(executorId) + case DecommissionExecutor(executorId, decommissionInfo) => + logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.") + decommissionExecutor(executorId, decommissionInfo) context.reply(true) case RetrieveSparkAppConfig(resourceProfileId) => @@ -422,7 +422,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Mark a given executor as decommissioned and stop making resource offers for it. */ - private def decommissionExecutor(executorId: String): Boolean = { + private def decommissionExecutor( + executorId: String, decommissionInfo: ExecutorDecommissionInfo): Boolean = { val shouldDisable = CoarseGrainedSchedulerBackend.this.synchronized { // Only bother decommissioning executors which are alive. if (isExecutorActive(executorId)) { @@ -436,7 +437,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp if (shouldDisable) { logInfo(s"Starting decommissioning executor $executorId.") try { - scheduler.executorDecommission(executorId) + scheduler.executorDecommission(executorId, decommissionInfo) } catch { case e: Exception => logError(s"Unexpected error during decommissioning ${e.toString}", e) @@ -590,10 +591,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Called by subclasses when notified of a decommissioning executor. */ - private[spark] def decommissionExecutor(executorId: String): Unit = { + private[spark] def decommissionExecutor( + executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = { if (driverEndpoint != null) { logInfo("Propagating executor decommission to driver.") - driverEndpoint.send(DecommissionExecutor(executorId)) + driverEndpoint.send(DecommissionExecutor(executorId, decommissionInfo)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 4024b44bdfd2f..d921af602b254 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -174,10 +174,10 @@ private[spark] class StandaloneSchedulerBackend( removeExecutor(fullId.split("/")(1), reason) } - override def executorDecommissioned(fullId: String, message: String) { + override def executorDecommissioned(fullId: String, decommissionInfo: ExecutorDecommissionInfo) { logInfo("Asked to decommission executor") - decommissionExecutor(fullId.split("/")(1)) - logInfo("Executor %s decommissioned: %s".format(fullId, message)) + decommissionExecutor(fullId.split("/")(1), decommissionInfo) + logInfo("Executor %s decommissioned: %s".format(fullId, decommissionInfo)) } override def workerRemoved(workerId: String, host: String, message: String): Unit = { diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index a3e39d7f53728..e091bd05c2dc8 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.client import java.io.Closeable -import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} import scala.concurrent.duration._ @@ -32,6 +32,7 @@ import org.apache.spark.deploy.master.{ApplicationInfo, Master} import org.apache.spark.deploy.worker.Worker import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.RpcEnv +import org.apache.spark.scheduler.ExecutorDecommissionInfo import org.apache.spark.util.Utils /** @@ -126,7 +127,10 @@ class AppClientSuite // Decommissioning is async. eventually(timeout(1.seconds), interval(10.millis)) { // We only record decommissioning for the executor we've requested - assert(ci.listener.execDecommissionedList.size === 1) + assert(ci.listener.execDecommissionedMap.size === 1) + val decommissionInfo = ci.listener.execDecommissionedMap.get(executorId) + assert(decommissionInfo != null && decommissionInfo.isHostDecommissioned, + s"$executorId should have been decommissioned along with its worker") } // Send request to kill executor, verify request was made @@ -215,7 +219,7 @@ class AppClientSuite val deadReasonList = new ConcurrentLinkedQueue[String]() val execAddedList = new ConcurrentLinkedQueue[String]() val execRemovedList = new ConcurrentLinkedQueue[String]() - val execDecommissionedList = new ConcurrentLinkedQueue[String]() + val execDecommissionedMap = new ConcurrentHashMap[String, ExecutorDecommissionInfo]() def connected(id: String): Unit = { connectedIdList.add(id) @@ -245,8 +249,9 @@ class AppClientSuite execRemovedList.add(id) } - def executorDecommissioned(id: String, message: String): Unit = { - execDecommissionedList.add(id) + def executorDecommissioned(id: String, decommissionInfo: ExecutorDecommissionInfo): Unit = { + val previousDecommissionInfo = execDecommissionedMap.putIfAbsent(id, decommissionInfo) + assert(previousDecommissionInfo === null, s"Expected no previous decommission info for $id") } def workerRemoved(workerId: String, host: String, message: String): Unit = {} diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 664cfc88cc410..c82a5ef1111bb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -172,10 +172,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 - override def executorDecommission(executorId: String) = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None + override def executorDecommission( + executorId: String, + decommissionInfo: ExecutorDecommissionInfo): Unit = {} } /** @@ -777,10 +779,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, executorUpdates: Map[(Int, Int), ExecutorMetrics]): Boolean = true - override def executorDecommission(executorId: String): Unit = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None + override def executorDecommission( + executorId: String, + decommissionInfo: ExecutorDecommissionInfo): Unit = {} } val noKillScheduler = new DAGScheduler( sc, diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 7ead51bc691fb..b2a5f77b4b04c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -90,7 +90,6 @@ private class DummyTaskScheduler extends TaskScheduler { override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = {} override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} override def defaultParallelism(): Int = 2 - override def executorDecommission(executorId: String): Unit = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None @@ -99,4 +98,7 @@ private class DummyTaskScheduler extends TaskScheduler { accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, executorMetrics: Map[(Int, Int), ExecutorMetrics]): Boolean = true + override def executorDecommission( + executorId: String, + decommissionInfo: ExecutorDecommissionInfo): Unit = {} } diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala index 4de5aaeab5c51..d95deb1f5f327 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala @@ -65,7 +65,7 @@ class WorkerDecommissionExtendedSuite extends SparkFunSuite with LocalSparkConte val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] sc.getExecutorIds().tail.foreach { id => - sched.decommissionExecutor(id) + sched.decommissionExecutor(id, ExecutorDecommissionInfo("", false)) assert(rdd3.sortByKey().collect().length === 100) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala index cd3ab4db77f85..3c34070e8bb97 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala @@ -73,7 +73,7 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { // decom.sh message passing is tested manually. val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] val execs = sched.getExecutorIds() - execs.foreach(execId => sched.decommissionExecutor(execId)) + execs.foreach(execId => sched.decommissionExecutor(execId, ExecutorDecommissionInfo("", false))) val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 20.seconds) assert(asyncCountResult === 10) // Try and launch task after decommissioning, this should fail diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index afcb38bc38836..57410103dd080 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -146,7 +146,7 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS val execToDecommission = execs.head logDebug(s"Decommissioning executor ${execToDecommission}") - sched.decommissionExecutor(execToDecommission) + sched.decommissionExecutor(execToDecommission, ExecutorDecommissionInfo("", false)) // Wait for job to finish. val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 15.seconds) From 7b66882c9d90fc060206de52735a8a80c0cf104f Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Jul 2020 13:53:50 +0900 Subject: [PATCH 256/384] [SPARK-32338][SQL][PYSPARK][FOLLOW-UP] Update slice to accept Column for start and length ### What changes were proposed in this pull request? This is a follow-up of #29138 which added overload `slice` function to accept `Column` for `start` and `length` in Scala. This PR is updating the equivalent Python function to accept `Column` as well. ### Why are the changes needed? Now that Scala version accepts `Column`, Python version should also accept it. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will also be able to pass Column object to `start` and `length` parameter in `slice` function. ### How was this patch tested? Added tests. Closes #29195 from ueshin/issues/SPARK-32338/slice. Authored-by: Takuya UESHIN Signed-off-by: HyukjinKwon --- python/pyspark/sql/functions.py | 6 +++++- python/pyspark/sql/tests/test_functions.py | 10 ++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 3ca4edafa6873..d7ac6d89c045a 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2068,7 +2068,11 @@ def slice(x, start, length): [Row(sliced=[2, 3]), Row(sliced=[5])] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.slice(_to_java_column(x), start, length)) + return Column(sc._jvm.functions.slice( + _to_java_column(x), + start._jc if isinstance(start, Column) else start, + length._jc if isinstance(length, Column) else length + )) @since(2.4) diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 7dcc19f3ba45d..02180daf081ec 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -292,6 +292,16 @@ def test_input_file_name_reset_for_rdd(self): for result in results: self.assertEqual(result[0], '') + def test_slice(self): + from pyspark.sql.functions import slice, lit + + df = self.spark.createDataFrame([([1, 2, 3],), ([4, 5],)], ['x']) + + self.assertEquals( + df.select(slice(df.x, 2, 2).alias("sliced")).collect(), + df.select(slice(df.x, lit(2), lit(2)).alias("sliced")).collect(), + ) + def test_array_repeat(self): from pyspark.sql.functions import array_repeat, lit From a71233f89d79c4d9a9c806b9ec0c082ddc81e204 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 23 Jul 2020 21:14:36 +0900 Subject: [PATCH 257/384] [SPARK-32389][TESTS] Add all hive.execution suite in the parallel test group ### What changes were proposed in this pull request? Add a new parallel test group for all `hive.execution` suites. ### Why are the changes needed? Base on the tests, it can reduce the Jenkins testing time. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. Closes #28977 from xuanyuanking/parallelTest. Authored-by: Yuanjian Li Signed-off-by: HyukjinKwon --- project/SparkBuild.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 198405a1d29ca..c05e41cfcc0c2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -469,7 +469,6 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite", "org.apache.spark.sql.hive.HiveExternalCatalogSuite", "org.apache.spark.sql.hive.StatisticsSuite", - "org.apache.spark.sql.hive.execution.HiveCompatibilitySuite", "org.apache.spark.sql.hive.client.VersionsSuite", "org.apache.spark.sql.hive.client.HiveClientVersions", "org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite", @@ -486,9 +485,14 @@ object SparkParallelTestGrouping { ) private val DEFAULT_TEST_GROUP = "default_test_group" + private val HIVE_EXECUTION_TEST_GROUP = "hive_execution_test_group" private def testNameToTestGroup(name: String): String = name match { case _ if testsWhichShouldRunInTheirOwnDedicatedJvm.contains(name) => name + // Different with the cases in testsWhichShouldRunInTheirOwnDedicatedJvm, here we are grouping + // all suites of `org.apache.spark.sql.hive.execution.*` into a single group, instead of + // launching one JVM per suite. + case _ if name.contains("org.apache.spark.sql.hive.execution") => HIVE_EXECUTION_TEST_GROUP case _ => DEFAULT_TEST_GROUP } From aed8dbab1d6725eb17f743c300451fcbdbfa3e97 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 23 Jul 2020 06:28:08 -0700 Subject: [PATCH 258/384] [SPARK-32364][SQL][FOLLOWUP] Add toMap to return originalMap and documentation ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/29160. We already removed the indeterministicity. This PR aims the following for the existing code base. 1. Add an explicit document to `DataFrameReader/DataFrameWriter`. 2. Add `toMap` to `CaseInsensitiveMap` in order to return `originalMap: Map[String, T]` because it's more consistent with the existing `case-sensitive key names` behavior for the existing code pattern like `AppendData.byName(..., extraOptions.toMap)`. Previously, it was `HashMap.toMap`. 3. During (2), we need to change the following to keep the original logic using `CaseInsensitiveMap.++`. ```scala - val params = extraOptions.toMap ++ connectionProperties.asScala.toMap + val params = extraOptions ++ connectionProperties.asScala ``` 4. Additionally, use `.toMap` in the following because `dsOptions.asCaseSensitiveMap()` is used later. ```scala - val options = sessionOptions ++ extraOptions + val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap val dsOptions = new CaseInsensitiveStringMap(options.asJava) ``` ### Why are the changes needed? `extraOptions.toMap` is used in several places (e.g. `DataFrameReader`) to hand over `Map[String, T]`. In this case, `CaseInsensitiveMap[T] private (val originalMap: Map[String, T])` had better return `originalMap`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins or GitHub Action with the existing tests and newly add test case at `JDBCSuite`. Closes #29191 from dongjoon-hyun/SPARK-32364-3. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../catalyst/util/CaseInsensitiveMap.scala | 2 ++ .../catalyst/util/CaseInsensitiveMap.scala | 2 ++ .../apache/spark/sql/DataFrameReader.scala | 20 +++++++++++++++++- .../apache/spark/sql/DataFrameWriter.scala | 20 +++++++++++++++++- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 21 +++++++++++++++++++ 5 files changed, 63 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index c013888d080f2..14b8f620017f6 100644 --- a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -52,6 +52,8 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma override def -(key: String): Map[String, T] = { new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(key))) } + + def toMap: Map[String, T] = originalMap } object CaseInsensitiveMap { diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index 66a2f2549b460..1db4b7a4f227b 100644 --- a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -52,6 +52,8 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma override def removed(key: String): Map[String, T] = { new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(key))) } + + def toMap: Map[String, T] = originalMap } object CaseInsensitiveMap { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index a291e5825815d..5b78690b3267c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -94,6 +94,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds an input option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
    *
  • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -121,6 +124,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds an input option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Boolean): DataFrameReader = option(key, value.toString) @@ -128,6 +134,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds an input option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Long): DataFrameReader = option(key, value.toString) @@ -135,6 +144,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds an input option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Double): DataFrameReader = option(key, value.toString) @@ -142,6 +154,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * (Scala-specific) Adds input options for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
      *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -169,6 +184,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds input options for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
        *
      • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -361,7 +379,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { connectionProperties: Properties): DataFrame = { assertNoSpecifiedSchema("jdbc") // connectionProperties should override settings in extraOptions. - val params = extraOptions.toMap ++ connectionProperties.asScala.toMap + val params = extraOptions ++ connectionProperties.asScala val options = new JDBCOptions(url, table, params) val parts: Array[Partition] = predicates.zipWithIndex.map { case (part, i) => JDBCPartition(part, i) : Partition diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 952f896d1a20f..f463166a9f268 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -105,6 +105,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds an output option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
          *
        • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -132,6 +135,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds an output option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Boolean): DataFrameWriter[T] = option(key, value.toString) @@ -139,6 +145,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds an output option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Long): DataFrameWriter[T] = option(key, value.toString) @@ -146,6 +155,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds an output option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Double): DataFrameWriter[T] = option(key, value.toString) @@ -153,6 +165,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * (Scala-specific) Adds output options for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
            *
          • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -180,6 +195,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds output options for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
              *
            • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -288,7 +306,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val provider = maybeV2Provider.get val sessionOptions = DataSourceV2Utils.extractSessionConfigs( provider, df.sparkSession.sessionState.conf) - val options = sessionOptions ++ extraOptions + val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap val dsOptions = new CaseInsensitiveStringMap(options.asJava) def getTable: Table = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 0e3dd4447c3fb..23cf25a4030c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -21,6 +21,8 @@ import java.math.BigDecimal import java.sql.{Date, DriverManager, SQLException, Timestamp} import java.util.{Calendar, GregorianCalendar, Properties} +import scala.collection.JavaConverters._ + import org.h2.jdbc.JdbcSQLException import org.scalatest.{BeforeAndAfter, PrivateMethodTester} @@ -1718,4 +1720,23 @@ class JDBCSuite extends QueryTest jdbcDF = sqlContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts) checkAnswer(jdbcDF, Row("mary", 2) :: Nil) } + + test("SPARK-32364: JDBCOption constructor") { + val extraOptions = CaseInsensitiveMap[String](Map("UrL" -> "url1", "dBTable" -> "table1")) + val connectionProperties = new Properties() + connectionProperties.put("url", "url2") + connectionProperties.put("dbtable", "table2") + + // connection property should override the options in extraOptions + val params = extraOptions ++ connectionProperties.asScala + assert(params.size == 2) + assert(params.get("uRl").contains("url2")) + assert(params.get("DbtaBle").contains("table2")) + + // JDBCOptions constructor parameter should overwrite the existing conf + val options = new JDBCOptions(url, "table3", params) + assert(options.asProperties.size == 2) + assert(options.asProperties.get("url") == url) + assert(options.asProperties.get("dbtable") == "table3") + } } From aa54dcf193a2149182da779191cf12f087305726 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 23 Jul 2020 14:02:38 +0000 Subject: [PATCH 259/384] [SPARK-32251][SQL][TESTS][FOLLOWUP] improve SQL keyword test ### What changes were proposed in this pull request? Improve the `SQLKeywordSuite` so that: 1. it checks keywords under default mode as well 2. it checks if there are typos in the doc (found one and fixed in this PR) ### Why are the changes needed? better test coverage ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? N/A Closes #29200 from cloud-fan/test. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- docs/sql-ref-ansi-compliance.md | 2 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 + .../spark/sql/catalyst/SQLKeywordSuite.scala | 46 ++++++++++++++++--- 3 files changed, 43 insertions(+), 7 deletions(-) diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index f083f880cc073..e786c0bc9aff8 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -264,7 +264,7 @@ Below is a list of all the keywords in Spark SQL. |MAP|non-reserved|non-reserved|non-reserved| |MATCHED|non-reserved|non-reserved|non-reserved| |MERGE|non-reserved|non-reserved|non-reserved| -|MINUS|not-reserved|strict-non-reserved|non-reserved| +|MINUS|non-reserved|strict-non-reserved|non-reserved| |MINUTE|reserved|non-reserved|reserved| |MONTH|reserved|non-reserved|reserved| |MSCK|non-reserved|non-reserved|non-reserved| diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 73dfdcc263a7d..b9c46616caf34 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -1226,6 +1226,7 @@ strictNonReserved ; nonReserved +//--DEFAULT-NON-RESERVED-START : ADD | AFTER | ALL @@ -1466,6 +1467,7 @@ nonReserved | WITH | YEAR | ZONE +//--DEFAULT-NON-RESERVED-END ; // NOTE: If you add a new token in the list below, you should update the list of keywords diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala index 3d41d02b23df5..082b01173e7b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala @@ -38,7 +38,7 @@ trait SQLKeywordUtils extends SQLHelper { } // each element is an array of 4 string: the keyword name, reserve or not in Spark ANSI mode, - // Spark non-ANSI mode, and the SQL standard. + // Spark default mode, and the SQL standard. val keywordsInDoc: Array[Array[String]] = { val docPath = { java.nio.file.Paths.get(sparkHome, "docs", "sql-ref-ansi-compliance.md").toFile @@ -135,6 +135,19 @@ trait SQLKeywordUtils extends SQLHelper { } val reservedKeywordsInAnsiMode = allCandidateKeywords -- nonReservedKeywordsInAnsiMode + + val nonReservedKeywordsInDefaultMode: Set[String] = { + val kwDef = """\s*[\|:]\s*([A-Z_]+)\s*""".r + parseAntlrGrammars("//--DEFAULT-NON-RESERVED-START", "//--DEFAULT-NON-RESERVED-END") { + // Parses a pattern, e.g., ` | AFTER` + case kwDef(symbol) => + if (symbolsToExpandIntoDifferentLiterals.contains(symbol)) { + symbolsToExpandIntoDifferentLiterals(symbol) + } else { + symbol :: Nil + } + } + } } class SQLKeywordSuite extends SparkFunSuite with SQLKeywordUtils { @@ -146,11 +159,32 @@ class SQLKeywordSuite extends SparkFunSuite with SQLKeywordUtils { } } - test("Spark keywords are documented correctly") { - val reservedKeywordsInDoc = keywordsInDoc.filter(_.apply(1) == "reserved").map(_.head).toSet - if (reservedKeywordsInAnsiMode != reservedKeywordsInDoc) { - val misImplemented = (reservedKeywordsInDoc -- reservedKeywordsInAnsiMode).toSeq.sorted - fail("Some keywords are documented as reserved but actually not: " + + test("Spark keywords are documented correctly under ANSI mode") { + // keywords under ANSI mode should either be reserved or non-reserved. + keywordsInDoc.map(_.apply(1)).foreach { desc => + assert(desc == "reserved" || desc == "non-reserved") + } + + val nonReservedInDoc = keywordsInDoc.filter(_.apply(1) == "non-reserved").map(_.head).toSet + if (nonReservedKeywordsInAnsiMode != nonReservedInDoc) { + val misImplemented = ((nonReservedInDoc -- nonReservedKeywordsInAnsiMode) ++ + (nonReservedKeywordsInAnsiMode -- nonReservedInDoc)).toSeq.sorted + fail("Some keywords are documented and implemented inconsistently: " + + misImplemented.mkString(", ")) + } + } + + test("Spark keywords are documented correctly under default mode") { + // keywords under default mode should either be strict-non-reserved or non-reserved. + keywordsInDoc.map(_.apply(2)).foreach { desc => + assert(desc == "strict-non-reserved" || desc == "non-reserved") + } + + val nonReservedInDoc = keywordsInDoc.filter(_.apply(2) == "non-reserved").map(_.head).toSet + if (nonReservedKeywordsInDefaultMode != nonReservedInDoc) { + val misImplemented = ((nonReservedInDoc -- nonReservedKeywordsInDefaultMode) ++ + (nonReservedKeywordsInDefaultMode -- nonReservedInDoc)).toSeq.sorted + fail("Some keywords are documented and implemented inconsistently: " + misImplemented.mkString(", ")) } } From a8e3de36e7d543f1c7923886628ac3178f45f512 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 23 Jul 2020 14:24:47 +0000 Subject: [PATCH 260/384] [SPARK-32280][SPARK-32372][SQL] ResolveReferences.dedupRight should only rewrite attributes for ancestor nodes of the conflict plan MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR refactors `ResolveReferences.dedupRight` to make sure it only rewrite attributes for ancestor nodes of the conflict plan. ### Why are the changes needed? This is a bug fix. ```scala sql("SELECT name, avg(age) as avg_age FROM person GROUP BY name") .createOrReplaceTempView("person_a") sql("SELECT p1.name, p2.avg_age FROM person p1 JOIN person_a p2 ON p1.name = p2.name") .createOrReplaceTempView("person_b") sql("SELECT * FROM person_a UNION SELECT * FROM person_b") .createOrReplaceTempView("person_c") sql("SELECT p1.name, p2.avg_age FROM person_c p1 JOIN person_c p2 ON p1.name = p2.name").show() ``` When executing the above query, we'll hit the error: ```scala [info] Failed to analyze query: org.apache.spark.sql.AnalysisException: Resolved attribute(s) avg_age#231 missing from name#223,avg_age#218,id#232,age#234,name#233 in operator !Project [name#233, avg_age#231]. Attribute(s) with the same name appear in the operation: avg_age. Please check if the right attribute(s) are used.;; ... ``` The plan below is the problematic plan which is the right plan of a `Join` operator. And, it has conflict plans comparing to the left plan. In this problematic plan, the first `Aggregate` operator (the one under the first child of `Union`) becomes a conflict plan compares to the left one and has a rewrite attribute pair as `avg_age#218` -> `avg_age#231`. With the current `dedupRight` logic, we'll first replace this `Aggregate` with a new one, and then rewrites the attribute `avg_age#218` from bottom to up. As you can see, projects with the attribute `avg_age#218` of the second child of the `Union` can also be replaced with `avg_age#231`(That means we also rewrite attributes for non-ancestor plans for the conflict plan). Ideally, the attribute `avg_age#218` in the second `Aggregate` operator (the one under the second child of `Union`) should also be replaced. But it didn't because it's an `Alias` while we only rewrite `Attribute` yet. Therefore, the project above the second `Aggregate` becomes unresolved. ```scala :
 : +- SubqueryAlias p2 +- SubqueryAlias person_c +- Distinct +- Union :- Project [name#233, avg_age#231] : +- SubqueryAlias person_a : +- Aggregate [name#233], [name#233, avg(cast(age#234 as bigint)) AS avg_age#231] : +- SubqueryAlias person : +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#232, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#233, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#234] : +- ExternalRDD [obj#165] +- Project [name#233 AS name#227, avg_age#231 AS avg_age#228] +- Project [name#233, avg_age#231] +- SubqueryAlias person_b +- !Project [name#233, avg_age#231] +- Join Inner, (name#233 = name#223) :- SubqueryAlias p1 : +- SubqueryAlias person : +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#232, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#233, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#234] : +- ExternalRDD [obj#165] +- SubqueryAlias p2 +- SubqueryAlias person_a +- Aggregate [name#223], [name#223, avg(cast(age#224 as bigint)) AS avg_age#218] +- SubqueryAlias person +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#222, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#223, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#224] +- ExternalRDD [obj#165] ``` ### Does this PR introduce _any_ user-facing change? Yes, users would no longer hit the error after this fix. ### How was this patch tested? Added test. Closes #29166 from Ngone51/impr-dedup. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 63 ++++++++++++++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 22 +++++++ 2 files changed, 75 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a9a6fcd3bdb28..2a849023ea13b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1192,11 +1192,24 @@ class Analyzer( if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => Seq((oldVersion, oldVersion.copy(projectList = newAliases(projectList)))) + // We don't need to search child plan recursively if the projectList of a Project + // is only composed of Alias and doesn't contain any conflicting attributes. + // Because, even if the child plan has some conflicting attributes, the attributes + // will be aliased to non-conflicting attributes by the Project at the end. + case _ @ Project(projectList, _) + if findAliases(projectList).size == projectList.size => + Nil + case oldVersion @ Aggregate(_, aggregateExpressions, _) if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => Seq((oldVersion, oldVersion.copy( aggregateExpressions = newAliases(aggregateExpressions)))) + // We don't search the child plan recursively for the same reason as the above Project. + case _ @ Aggregate(_, aggregateExpressions, _) + if findAliases(aggregateExpressions).size == aggregateExpressions.size => + Nil + case oldVersion @ FlatMapGroupsInPandas(_, _, output, _) if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) @@ -1237,20 +1250,50 @@ class Analyzer( if (conflictPlans.isEmpty) { right } else { - val attributeRewrites = AttributeMap(conflictPlans.flatMap { - case (oldRelation, newRelation) => oldRelation.output.zip(newRelation.output)}) - val conflictPlanMap = conflictPlans.toMap - // transformDown so that we can replace all the old Relations in one turn due to - // the reason that `conflictPlans` are also collected in pre-order. - right transformDown { - case r => conflictPlanMap.getOrElse(r, r) - } transformUp { - case other => other transformExpressions { + rewritePlan(right, conflictPlans.toMap)._1 + } + } + + private def rewritePlan(plan: LogicalPlan, conflictPlanMap: Map[LogicalPlan, LogicalPlan]) + : (LogicalPlan, Seq[(Attribute, Attribute)]) = { + if (conflictPlanMap.contains(plan)) { + // If the plan is the one that conflict the with left one, we'd + // just replace it with the new plan and collect the rewrite + // attributes for the parent node. + val newRelation = conflictPlanMap(plan) + newRelation -> plan.output.zip(newRelation.output) + } else { + val attrMapping = new mutable.ArrayBuffer[(Attribute, Attribute)]() + val newPlan = plan.mapChildren { child => + // If not, we'd rewrite child plan recursively until we find the + // conflict node or reach the leaf node. + val (newChild, childAttrMapping) = rewritePlan(child, conflictPlanMap) + attrMapping ++= childAttrMapping.filter { case (oldAttr, _) => + // `attrMapping` is not only used to replace the attributes of the current `plan`, + // but also to be propagated to the parent plans of the current `plan`. Therefore, + // the `oldAttr` must be part of either `plan.references` (so that it can be used to + // replace attributes of the current `plan`) or `plan.outputSet` (so that it can be + // used by those parent plans). + (plan.outputSet ++ plan.references).contains(oldAttr) + } + newChild + } + + if (attrMapping.isEmpty) { + newPlan -> attrMapping + } else { + assert(!attrMapping.groupBy(_._1.exprId) + .exists(_._2.map(_._2.exprId).distinct.length > 1), + "Found duplicate rewrite attributes") + val attributeRewrites = AttributeMap(attrMapping) + // Using attrMapping from the children plans to rewrite their parent node. + // Note that we shouldn't rewrite a node using attrMapping from its sibling nodes. + newPlan.transformExpressions { case a: Attribute => dedupAttr(a, attributeRewrites) case s: SubqueryExpression => s.withNewPlan(dedupOuterReferencesInSubquery(s.plan, attributeRewrites)) - } + } -> attrMapping } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 2bddaee0f179b..64d2567f84f1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3572,6 +3572,28 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark Row(1) :: Nil) } } + + test("SPARK-32372: ResolveReferences.dedupRight should only rewrite attributes for ancestor " + + "plans of the conflict plan") { + sql("SELECT name, avg(age) as avg_age FROM person GROUP BY name") + .createOrReplaceTempView("person_a") + sql("SELECT p1.name, p2.avg_age FROM person p1 JOIN person_a p2 ON p1.name = p2.name") + .createOrReplaceTempView("person_b") + sql("SELECT * FROM person_a UNION SELECT * FROM person_b") + .createOrReplaceTempView("person_c") + checkAnswer( + sql("SELECT p1.name, p2.avg_age FROM person_c p1 JOIN person_c p2 ON p1.name = p2.name"), + Row("jim", 20.0) :: Row("mike", 30.0) :: Nil) + } + + test("SPARK-32280: Avoid duplicate rewrite attributes when there're multiple JOINs") { + sql("SELECT 1 AS id").createOrReplaceTempView("A") + sql("SELECT id, 'foo' AS kind FROM A").createOrReplaceTempView("B") + sql("SELECT l.id as id FROM B AS l LEFT SEMI JOIN B AS r ON l.kind = r.kind") + .createOrReplaceTempView("C") + checkAnswer(sql("SELECT 0 FROM ( SELECT * FROM B JOIN C USING (id)) " + + "JOIN ( SELECT * FROM B JOIN C USING (id)) USING (id)"), Row(0)) + } } case class Foo(bar: Option[String]) From 35345e30e52ea05ab03683ade0b2e82146a150d2 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 23 Jul 2020 14:32:10 +0000 Subject: [PATCH 261/384] [SPARK-32374][SQL] Disallow setting properties when creating temporary views ### What changes were proposed in this pull request? Currently, you can specify properties when creating a temporary view. However, the specified properties are not used and can be misleading. This PR propose to disallow specifying properties when creating temporary views. ### Why are the changes needed? To avoid confusion by disallowing specifying unused properties. ### Does this PR introduce _any_ user-facing change? Yes, now if you create a temporary view with properties, the operation will fail: ``` scala> sql("CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1") org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE TEMPORARY VIEW ... TBLPROPERTIES (property_name = property_value, ...)(line 1, pos 0) == SQL == CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1 ^^^ ``` ### How was this patch tested? Added tests Closes #29167 from imback82/disable_properties_temp_view. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/AstBuilder.scala | 9 +++++++-- .../spark/sql/catalyst/parser/DDLParserSuite.scala | 14 ++++++++++++-- .../sql-tests/inputs/show-tblproperties.sql | 4 ++-- .../sql-tests/results/show-tblproperties.sql.out | 2 +- .../apache/spark/sql/execution/SQLViewSuite.scala | 12 +++++++++++- 5 files changed, 33 insertions(+), 8 deletions(-) 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 5663741bae505..d6ae89f49c57a 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 @@ -3515,6 +3515,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + val properties = ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues) + .getOrElse(Map.empty) + if (ctx.TEMPORARY != null && !properties.isEmpty) { + operationNotAllowed("TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW", ctx) + } + val viewType = if (ctx.TEMPORARY == null) { PersistedView } else if (ctx.GLOBAL != null) { @@ -3526,8 +3532,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging visitMultipartIdentifier(ctx.multipartIdentifier), userSpecifiedColumns, visitCommentSpecList(ctx.commentSpec()), - ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues) - .getOrElse(Map.empty), + properties, Option(source(ctx.query)), plan(ctx.query), ctx.EXISTS != null, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 47f21a0a1910d..02e086d5d7895 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1962,7 +1962,6 @@ class DDLParserSuite extends AnalysisTest { """ |CREATE OR REPLACE GLOBAL TEMPORARY VIEW a.b.c |(col1, col3 COMMENT 'hello') - |TBLPROPERTIES('prop1Key'="prop1Val") |COMMENT 'BLABLA' |AS SELECT * FROM tab1 """.stripMargin @@ -1971,7 +1970,7 @@ class DDLParserSuite extends AnalysisTest { Seq("a", "b", "c"), Seq("col1" -> None, "col3" -> Some("hello")), Some("BLABLA"), - Map("prop1Key" -> "prop1Val"), + Map(), Some("SELECT * FROM tab1"), parsePlan("SELECT * FROM tab1"), false, @@ -2003,6 +2002,17 @@ class DDLParserSuite extends AnalysisTest { intercept(sql2, "Found duplicate clauses: TBLPROPERTIES") } + test("SPARK-32374: create temporary view with properties not allowed") { + assertUnsupported( + sql = """ + |CREATE OR REPLACE TEMPORARY VIEW a.b.c + |(col1, col3 COMMENT 'hello') + |TBLPROPERTIES('prop1Key'="prop1Val") + |AS SELECT * FROM tab1 + """.stripMargin, + containsThesePhrases = Seq("TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW")) + } + test("SHOW TBLPROPERTIES table") { comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c"), diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql b/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql index 2861b2b43a113..0ba936dcad741 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql @@ -17,8 +17,8 @@ SHOW TBLPROPERTIES view("p3"); DROP VIEW view; --- create a temporary view with properties -CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1; +-- create a temporary view +CREATE TEMPORARY VIEW tv AS SELECT 1 AS c1; -- Properties for a temporary view should be empty SHOW TBLPROPERTIES tv; diff --git a/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out index 6984b34c365ec..eaaf894590d35 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out @@ -91,7 +91,7 @@ struct<> -- !query -CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1 +CREATE TEMPORARY VIEW tv AS SELECT 1 AS c1 -- !query schema struct<> -- !query output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 575efec364812..2e25c40cecd78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.internal.SQLConf.MAX_NESTED_VIEW_DEPTH import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -266,6 +267,16 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } + test("SPARK-32374: disallow setting properties for CREATE TEMPORARY VIEW") { + withTempView("myabcdview") { + val e = intercept[ParseException] { + sql("CREATE TEMPORARY VIEW myabcdview TBLPROPERTIES ('a' = 'b') AS SELECT * FROM jt") + } + assert(e.message.contains( + "Operation not allowed: TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW")) + } + } + test("correctly parse CREATE VIEW statement") { withView("testView") { sql( @@ -301,7 +312,6 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql( """CREATE TEMPORARY VIEW |testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') - |TBLPROPERTIES ('a' = 'b') |AS SELECT * FROM jt |""".stripMargin) checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) From e7fb67cd880511452b94c2077429868e72998c05 Mon Sep 17 00:00:00 2001 From: Venkata krishnan Sowrirajan Date: Thu, 23 Jul 2020 12:33:22 -0500 Subject: [PATCH 262/384] [SPARK-31418][SCHEDULER] Request more executors in case of dynamic allocation is enabled and a task becomes unschedulable due to spark's blacklisting feature ### What changes were proposed in this pull request? In this change, when dynamic allocation is enabled instead of aborting immediately when there is an unschedulable taskset due to blacklisting, pass an event saying `SparkListenerUnschedulableTaskSetAdded` which will be handled by `ExecutorAllocationManager` and request more executors needed to schedule the unschedulable blacklisted tasks. Once the event is sent, we start the abortTimer similar to [SPARK-22148][SPARK-15815] to abort in the case when no new executors launched either due to max executors reached or cluster manager is out of capacity. ### Why are the changes needed? This is an improvement. In the case when dynamic allocation is enabled, this would request more executors to schedule the unschedulable tasks instead of aborting the stage without even retrying upto spark.task.maxFailures times (in some cases not retrying at all). This is a potential issue with respect to Spark's Fault tolerance. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added unit tests both in ExecutorAllocationManagerSuite and TaskSchedulerImplSuite Closes #28287 from venkata91/SPARK-31418. Authored-by: Venkata krishnan Sowrirajan Signed-off-by: Thomas Graves --- .../apache/spark/SparkFirehoseListener.java | 10 + .../spark/ExecutorAllocationManager.scala | 55 +++++- .../apache/spark/scheduler/DAGScheduler.scala | 38 ++++ .../spark/scheduler/DAGSchedulerEvent.scala | 8 + .../spark/scheduler/SparkListener.scala | 30 +++ .../spark/scheduler/SparkListenerBus.scala | 4 + .../spark/scheduler/TaskSchedulerImpl.scala | 55 ++++-- .../ExecutorAllocationManagerSuite.scala | 175 +++++++++++++++++- .../scheduler/TaskSchedulerImplSuite.scala | 36 ++++ 9 files changed, 392 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 579e7ff320f5c..c0e72b57d48bd 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -162,6 +162,16 @@ public void onSpeculativeTaskSubmitted(SparkListenerSpeculativeTaskSubmitted spe onEvent(speculativeTask); } + public void onUnschedulableTaskSetAdded( + SparkListenerUnschedulableTaskSetAdded unschedulableTaskSetAdded) { + onEvent(unschedulableTaskSetAdded); + } + + public void onUnschedulableTaskSetRemoved( + SparkListenerUnschedulableTaskSetRemoved unschedulableTaskSetRemoved) { + onEvent(unschedulableTaskSetRemoved); + } + @Override public void onResourceProfileAdded(SparkListenerResourceProfileAdded event) { onEvent(event); diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 620a6fe2f9d72..85409d599ccaa 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -281,6 +281,7 @@ private[spark] class ExecutorAllocationManager( private def maxNumExecutorsNeededPerResourceProfile(rpId: Int): Int = { val pending = listener.totalPendingTasksPerResourceProfile(rpId) val pendingSpeculative = listener.pendingSpeculativeTasksPerResourceProfile(rpId) + val unschedulableTaskSets = listener.pendingUnschedulableTaskSetsPerResourceProfile(rpId) val running = listener.totalRunningTasksPerResourceProfile(rpId) val numRunningOrPendingTasks = pending + running val rp = resourceProfileManager.resourceProfileFromId(rpId) @@ -289,13 +290,27 @@ private[spark] class ExecutorAllocationManager( s" tasksperexecutor: $tasksPerExecutor") val maxNeeded = math.ceil(numRunningOrPendingTasks * executorAllocationRatio / tasksPerExecutor).toInt - if (tasksPerExecutor > 1 && maxNeeded == 1 && pendingSpeculative > 0) { + + val maxNeededWithSpeculationLocalityOffset = + if (tasksPerExecutor > 1 && maxNeeded == 1 && pendingSpeculative > 0) { // If we have pending speculative tasks and only need a single executor, allocate one more // to satisfy the locality requirements of speculation maxNeeded + 1 } else { maxNeeded } + + if (unschedulableTaskSets > 0) { + // Request additional executors to account for task sets having tasks that are unschedulable + // due to blacklisting when the active executor count has already reached the max needed + // which we would normally get. + val maxNeededForUnschedulables = math.ceil(unschedulableTaskSets * executorAllocationRatio / + tasksPerExecutor).toInt + math.max(maxNeededWithSpeculationLocalityOffset, + executorMonitor.executorCountWithResourceProfile(rpId) + maxNeededForUnschedulables) + } else { + maxNeededWithSpeculationLocalityOffset + } } private def totalRunningTasksPerResourceProfile(id: Int): Int = synchronized { @@ -622,6 +637,12 @@ private[spark] class ExecutorAllocationManager( private val resourceProfileIdToStageAttempt = new mutable.HashMap[Int, mutable.Set[StageAttempt]] + // Keep track of unschedulable task sets due to blacklisting. This is a Set of StageAttempt's + // because we'll only take the last unschedulable task in a taskset although there can be more. + // This is done in order to avoid costly loops in the scheduling. + // Check TaskSetManager#getCompletelyBlacklistedTaskIfAny for more details. + private val unschedulableTaskSets = new mutable.HashSet[StageAttempt] + // stageAttempt to tuple (the number of task with locality preferences, a map where each pair // is a node and the number of tasks that would like to be scheduled on that node, and // the resource profile id) map, @@ -789,6 +810,28 @@ private[spark] class ExecutorAllocationManager( } } + override def onUnschedulableTaskSetAdded( + unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded): Unit = { + val stageId = unschedulableTaskSetAdded.stageId + val stageAttemptId = unschedulableTaskSetAdded.stageAttemptId + val stageAttempt = StageAttempt(stageId, stageAttemptId) + allocationManager.synchronized { + unschedulableTaskSets.add(stageAttempt) + allocationManager.onSchedulerBacklogged() + } + } + + override def onUnschedulableTaskSetRemoved( + unschedulableTaskSetRemoved: SparkListenerUnschedulableTaskSetRemoved): Unit = { + val stageId = unschedulableTaskSetRemoved.stageId + val stageAttemptId = unschedulableTaskSetRemoved.stageAttemptId + val stageAttempt = StageAttempt(stageId, stageAttemptId) + allocationManager.synchronized { + // Clear unschedulableTaskSets since atleast one task becomes schedulable now + unschedulableTaskSets.remove(stageAttempt) + } + } + /** * An estimate of the total number of pending tasks remaining for currently running stages. Does * not account for tasks which may have failed and been resubmitted. @@ -829,6 +872,16 @@ private[spark] class ExecutorAllocationManager( numTotalTasks - numRunning } + /** + * Currently we only know when a task set has an unschedulable task, we don't know + * the exact number and since the allocation manager isn't tied closely with the scheduler, + * we use the number of tasks sets that are unschedulable as a heuristic to add more executors. + */ + def pendingUnschedulableTaskSetsPerResourceProfile(rp: Int): Int = { + val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq + attempts.filter(attempt => unschedulableTaskSets.contains(attempt)).size + } + def hasPendingTasks: Boolean = { hasPendingSpeculativeTasks || hasPendingRegularTasks } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 73c95d19387c2..2503ae0856dc7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -332,6 +332,26 @@ private[spark] class DAGScheduler( eventProcessLoop.post(SpeculativeTaskSubmitted(task)) } + /** + * Called by the TaskSetManager when a taskset becomes unschedulable due to blacklisting and + * dynamic allocation is enabled. + */ + def unschedulableTaskSetAdded( + stageId: Int, + stageAttemptId: Int): Unit = { + eventProcessLoop.post(UnschedulableTaskSetAdded(stageId, stageAttemptId)) + } + + /** + * Called by the TaskSetManager when an unschedulable taskset becomes schedulable and dynamic + * allocation is enabled. + */ + def unschedulableTaskSetRemoved( + stageId: Int, + stageAttemptId: Int): Unit = { + eventProcessLoop.post(UnschedulableTaskSetRemoved(stageId, stageAttemptId)) + } + private[scheduler] def getCacheLocs(rdd: RDD[_]): IndexedSeq[Seq[TaskLocation]] = cacheLocs.synchronized { // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times @@ -1035,6 +1055,18 @@ private[spark] class DAGScheduler( listenerBus.post(SparkListenerSpeculativeTaskSubmitted(task.stageId, task.stageAttemptId)) } + private[scheduler] def handleUnschedulableTaskSetAdded( + stageId: Int, + stageAttemptId: Int): Unit = { + listenerBus.post(SparkListenerUnschedulableTaskSetAdded(stageId, stageAttemptId)) + } + + private[scheduler] def handleUnschedulableTaskSetRemoved( + stageId: Int, + stageAttemptId: Int): Unit = { + listenerBus.post(SparkListenerUnschedulableTaskSetRemoved(stageId, stageAttemptId)) + } + private[scheduler] def handleTaskSetFailed( taskSet: TaskSet, reason: String, @@ -2321,6 +2353,12 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case SpeculativeTaskSubmitted(task) => dagScheduler.handleSpeculativeTaskSubmitted(task) + case UnschedulableTaskSetAdded(stageId, stageAttemptId) => + dagScheduler.handleUnschedulableTaskSetAdded(stageId, stageAttemptId) + + case UnschedulableTaskSetRemoved(stageId, stageAttemptId) => + dagScheduler.handleUnschedulableTaskSetRemoved(stageId, stageAttemptId) + case GettingResultEvent(taskInfo) => dagScheduler.handleGetTaskResult(taskInfo) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 78d458338e8fb..d226fe88614d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -97,3 +97,11 @@ private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent private[scheduler] case class SpeculativeTaskSubmitted(task: Task[_]) extends DAGSchedulerEvent +private[scheduler] +case class UnschedulableTaskSetAdded(stageId: Int, stageAttemptId: Int) + extends DAGSchedulerEvent + +private[scheduler] +case class UnschedulableTaskSetRemoved(stageId: Int, stageAttemptId: Int) + extends DAGSchedulerEvent + diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 62d54f3b74a47..8119215b8b74f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -157,6 +157,16 @@ case class SparkListenerNodeBlacklisted( case class SparkListenerNodeUnblacklisted(time: Long, hostId: String) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerUnschedulableTaskSetAdded( + stageId: Int, + stageAttemptId: Int) extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerUnschedulableTaskSetRemoved( + stageId: Int, + stageAttemptId: Int) extends SparkListenerEvent + @DeveloperApi case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends SparkListenerEvent @@ -339,6 +349,20 @@ private[spark] trait SparkListenerInterface { */ def onNodeUnblacklisted(nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit + /** + * Called when a taskset becomes unschedulable due to blacklisting and dynamic allocation + * is enabled. + */ + def onUnschedulableTaskSetAdded( + unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded): Unit + + /** + * Called when an unschedulable taskset becomes schedulable and dynamic allocation + * is enabled. + */ + def onUnschedulableTaskSetRemoved( + unschedulableTaskSetRemoved: SparkListenerUnschedulableTaskSetRemoved): Unit + /** * Called when the driver receives a block update info. */ @@ -425,6 +449,12 @@ abstract class SparkListener extends SparkListenerInterface { override def onNodeUnblacklisted( nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit = { } + override def onUnschedulableTaskSetAdded( + unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded): Unit = { } + + override def onUnschedulableTaskSetRemoved( + unschedulableTaskSetRemoved: SparkListenerUnschedulableTaskSetRemoved): Unit = { } + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { } override def onSpeculativeTaskSubmitted( diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 3d316c948db7e..13e65f4291fd0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -79,6 +79,10 @@ private[spark] trait SparkListenerBus listener.onBlockUpdated(blockUpdated) case speculativeTaskSubmitted: SparkListenerSpeculativeTaskSubmitted => listener.onSpeculativeTaskSubmitted(speculativeTaskSubmitted) + case unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded => + listener.onUnschedulableTaskSetAdded(unschedulableTaskSetAdded) + case unschedulableTaskSetRemoved: SparkListenerUnschedulableTaskSetRemoved => + listener.onUnschedulableTaskSetRemoved(unschedulableTaskSetRemoved) case resourceProfileAdded: SparkListenerResourceProfileAdded => listener.onResourceProfileAdded(resourceProfileAdded) case _ => listener.onOtherEvent(event) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 28e138ea9b79c..510318afcb8df 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -637,10 +637,9 @@ private[spark] class TaskSchedulerImpl( if (!launchedAnyTask) { taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors).foreach { taskIndex => // If the taskSet is unschedulable we try to find an existing idle blacklisted - // executor. If we cannot find one, we abort immediately. Else we kill the idle - // executor and kick off an abortTimer which if it doesn't schedule a task within the - // the timeout will abort the taskSet if we were unable to schedule any task from the - // taskSet. + // executor and kill the idle executor and kick off an abortTimer which if it doesn't + // schedule a task within the the timeout will abort the taskSet if we were unable to + // schedule any task from the taskSet. // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per // task basis. // Note 2: The taskSet can still be aborted when there are more than one idle @@ -648,22 +647,33 @@ private[spark] class TaskSchedulerImpl( // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort // timer to expire and abort the taskSet. + // + // If there are no idle executors and dynamic allocation is enabled, then we would + // notify ExecutorAllocationManager to allocate more executors to schedule the + // unschedulable tasks else we will abort immediately. executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match { case Some ((executorId, _)) => if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId)) - - val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 - unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout - logInfo(s"Waiting for $timeout ms for completely " - + s"blacklisted task to be schedulable again before aborting $taskSet.") - abortTimer.schedule( - createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout) + updateUnschedulableTaskSetTimeoutAndStartAbortTimer(taskSet, taskIndex) + } + case None => + // Notify ExecutorAllocationManager about the unschedulable task set, + // in order to provision more executors to make them schedulable + if (Utils.isDynamicAllocationEnabled(conf)) { + if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { + logInfo(s"Notifying ExecutorAllocationManager to allocate more executors to" + + s" schedule the unschedulable task before aborting $taskSet.") + dagScheduler.unschedulableTaskSetAdded(taskSet.taskSet.stageId, + taskSet.taskSet.stageAttemptId) + updateUnschedulableTaskSetTimeoutAndStartAbortTimer(taskSet, taskIndex) + } + } else { + // Abort Immediately + logInfo("Cannot schedule any task because of complete blacklisting. No idle" + + s" executors can be found to kill. Aborting $taskSet.") + taskSet.abortSinceCompletelyBlacklisted(taskIndex) } - case None => // Abort Immediately - logInfo("Cannot schedule any task because of complete blacklisting. No idle" + - s" executors can be found to kill. Aborting $taskSet." ) - taskSet.abortSinceCompletelyBlacklisted(taskIndex) } } } else { @@ -676,6 +686,10 @@ private[spark] class TaskSchedulerImpl( if (unschedulableTaskSetToExpiryTime.nonEmpty) { logInfo("Clearing the expiry times for all unschedulable taskSets as a task was " + "recently scheduled.") + // Notify ExecutorAllocationManager as well as other subscribers that a task now + // recently becomes schedulable + dagScheduler.unschedulableTaskSetRemoved(taskSet.taskSet.stageId, + taskSet.taskSet.stageAttemptId) unschedulableTaskSetToExpiryTime.clear() } } @@ -722,6 +736,17 @@ private[spark] class TaskSchedulerImpl( return tasks.map(_.toSeq) } + private def updateUnschedulableTaskSetTimeoutAndStartAbortTimer( + taskSet: TaskSetManager, + taskIndex: Int): Unit = { + val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 + unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout + logInfo(s"Waiting for $timeout ms for completely " + + s"blacklisted task to be schedulable again before aborting $taskSet.") + abortTimer.schedule( + createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout) + } + private def createUnschedulableTaskSetAbortTimer( taskSet: TaskSetManager, taskIndex: Int): TimerTask = { diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 8037f4a9447dd..ea6e010ef29a7 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -21,15 +21,15 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import org.mockito.ArgumentMatchers.{any, eq => meq} -import org.mockito.Mockito.{mock, never, times, verify, when} +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito._ import org.scalatest.PrivateMethodTester import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, ResourceProfileManager, TaskResourceRequests} +import org.apache.spark.resource._ import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -501,6 +501,175 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(numExecutorsToAddForDefaultProfile(manager) === 1) } + test("SPARK-31418: one stage being unschedulable") { + val clock = new ManualClock() + val conf = createConf(0, 5, 0).set(config.EXECUTOR_CORES, 2) + val manager = createManager(conf, clock = clock) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(createStageInfo(0, 2))) + + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + + onExecutorAddedDefaultProfile(manager, "0") + val t1 = createTaskInfo(0, 0, executorId = s"0") + val t2 = createTaskInfo(1, 1, executorId = s"0") + post(SparkListenerTaskStart(0, 0, t1)) + post(SparkListenerTaskStart(0, 0, t2)) + + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + + // Stage 0 becomes unschedulable due to blacklisting + post(SparkListenerUnschedulableTaskSetAdded(0, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + // Assert that we are getting additional executor to schedule unschedulable tasks + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) + + // Add a new executor + onExecutorAddedDefaultProfile(manager, "1") + // Now once the task becomes schedulable, clear the unschedulableTaskSets + post(SparkListenerUnschedulableTaskSetRemoved(0, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + } + + test("SPARK-31418: multiple stages being unschedulable") { + val clock = new ManualClock() + val conf = createConf(0, 10, 0).set(config.EXECUTOR_CORES, 2) + val manager = createManager(conf, clock = clock) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(createStageInfo(0, 2))) + post(SparkListenerStageSubmitted(createStageInfo(1, 2))) + post(SparkListenerStageSubmitted(createStageInfo(2, 2))) + + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + + // Add necessary executors + (0 to 2).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) + + // Start all the tasks + (0 to 2).foreach { + i => + val t1Info = createTaskInfo(0, (i * 2) + 1, executorId = s"${i / 2}") + val t2Info = createTaskInfo(1, (i * 2) + 2, executorId = s"${i / 2}") + post(SparkListenerTaskStart(i, 0, t1Info)) + post(SparkListenerTaskStart(i, 0, t2Info)) + } + assert(numExecutorsTarget(manager, defaultProfile.id) === 3) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 3) + + // Complete the stage 0 tasks. + val t1Info = createTaskInfo(0, 0, executorId = s"0") + val t2Info = createTaskInfo(1, 1, executorId = s"0") + post(SparkListenerTaskEnd(0, 0, null, Success, t1Info, new ExecutorMetrics, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, t2Info, new ExecutorMetrics, null)) + post(SparkListenerStageCompleted(createStageInfo(0, 2))) + + // Stage 1 and 2 becomes unschedulable now due to blacklisting + post(SparkListenerUnschedulableTaskSetAdded(1, 0)) + post(SparkListenerUnschedulableTaskSetAdded(2, 0)) + + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + // Assert that we are getting additional executor to schedule unschedulable tasks + assert(numExecutorsTarget(manager, defaultProfile.id) === 4) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 4) + + // Add a new executor + onExecutorAddedDefaultProfile(manager, "3") + + // Now once the task becomes schedulable, clear the unschedulableTaskSets + post(SparkListenerUnschedulableTaskSetRemoved(1, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 4) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) + } + + test("SPARK-31418: remove executors after unschedulable tasks end") { + val clock = new ManualClock() + val stage = createStageInfo(0, 10) + val conf = createConf(0, 6, 0).set(config.EXECUTOR_CORES, 2) + val manager = createManager(conf, clock = clock) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(stage)) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + + (0 to 4).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) + (0 to 9).map { i => createTaskInfo(i, i, executorId = s"${i / 2}") }.foreach { + info => post(SparkListenerTaskStart(0, 0, info)) + } + assert(numExecutorsTarget(manager, defaultProfile.id) === 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) + + // 8 tasks (0 - 7) finished + (0 to 7).map { i => createTaskInfo(i, i, executorId = s"${i / 2}") }.foreach { + info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) + } + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + (0 to 3).foreach { i => assert(removeExecutorDefaultProfile(manager, i.toString)) } + (0 to 3).foreach { i => onExecutorRemoved(manager, i.toString) } + + // Now due to blacklisting, the task becomes unschedulable + post(SparkListenerUnschedulableTaskSetAdded(0, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) + + // New executor got added + onExecutorAddedDefaultProfile(manager, "5") + + // Now once the task becomes schedulable, clear the unschedulableTaskSets + post(SparkListenerUnschedulableTaskSetRemoved(0, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(9, 9, "4"), new ExecutorMetrics, null)) + // Unschedulable task successfully ran on the new executor provisioned + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(8, 8, "5"), new ExecutorMetrics, null)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + post(SparkListenerStageCompleted(stage)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 0) + assert(removeExecutorDefaultProfile(manager, "4")) + onExecutorRemoved(manager, "4") + assert(removeExecutorDefaultProfile(manager, "5")) + onExecutorRemoved(manager, "5") + } + test("SPARK-30511 remove executors when speculative tasks end") { val clock = new ManualClock() val stage = createStageInfo(0, 40) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index e43be60e956be..9ca3ce9d43ca5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -1000,6 +1000,42 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!tsm.isZombie) } + test("SPARK-31418 abort timer should kick in when task is completely blacklisted &" + + "allocation manager could not acquire a new executor before the timeout") { + // set the abort timer to fail immediately + taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0", + config.DYN_ALLOCATION_ENABLED.key -> "true") + + // We have 2 tasks remaining with 1 executor + val taskSet = FakeTask.createTaskSet(numTasks = 2) + taskScheduler.submitTasks(taskSet) + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("executor0", "host0", 2))).flatten + + // Fail the running task + failTask(0, TaskState.FAILED, UnknownReason, tsm) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", 0)).thenReturn(true) + + // If the executor is busy, then dynamic allocation should kick in and try + // to acquire additional executors to schedule the blacklisted task + assert(taskScheduler.isExecutorBusy("executor0")) + + // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // timer to kick in immediately + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten.size === 0) + // Wait for the abort timer to kick in. Even though we configure the timeout to be 0, there is a + // slight delay as the abort timer is launched in a separate thread. + eventually(timeout(500.milliseconds)) { + assert(tsm.isZombie) + } + } + /** * Helper for performance tests. Takes the explicitly blacklisted nodes and executors; verifies * that the blacklists are used efficiently to ensure scheduling is not O(numPendingTasks). From be2eca22e9888d2e2ea61b159a215c1f42f9261f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 23 Jul 2020 16:20:17 -0700 Subject: [PATCH 263/384] [SPARK-32398][TESTS][CORE][STREAMING][SQL][ML] Update to scalatest 3.2.0 for Scala 2.13.3+ ### What changes were proposed in this pull request? Updates to scalatest 3.2.0. Though it looks large, it is 99% changes to the new location of scalatest classes. ### Why are the changes needed? 3.2.0+ has a fix that is required for Scala 2.13.3+ compatibility. ### Does this PR introduce _any_ user-facing change? No, only affects tests. ### How was this patch tested? Existing tests. Closes #29196 from srowen/SPARK-32398. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../spark/util/sketch/BitArraySuite.scala | 4 +-- .../spark/util/sketch/BloomFilterSuite.scala | 4 +-- .../util/sketch/CountMinSketchSuite.scala | 4 +-- .../types/UTF8StringPropertyCheckSuite.scala | 5 +-- .../spark/api/java/JavaSparkContext.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 2 +- .../org/apache/spark/DistributedSuite.scala | 2 +- .../spark/ExternalShuffleServiceSuite.scala | 1 + .../org/apache/spark/FutureActionSuite.scala | 4 ++- .../apache/spark/JobCancellationSuite.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 3 +- .../org/apache/spark/SortShuffleSuite.scala | 1 + .../org/apache/spark/SparkContextSuite.scala | 2 +- .../org/apache/spark/SparkFunSuite.scala | 5 +-- .../org/apache/spark/StatusTrackerSuite.scala | 3 +- .../api/python/PythonBroadcastSuite.scala | 3 +- .../org/apache/spark/deploy/ClientSuite.scala | 3 +- .../spark/deploy/SparkSubmitSuite.scala | 4 ++- .../history/ApplicationCacheSuite.scala | 3 +- .../history/FsHistoryProviderSuite.scala | 3 +- .../deploy/history/HistoryServerSuite.scala | 16 ++++++--- .../RealBrowserUIHistoryServerSuite.scala | 8 +++-- .../spark/deploy/master/MasterSuite.scala | 4 ++- ...HadoopFSDelegationTokenProviderSuite.scala | 3 +- .../deploy/worker/CommandUtilsSuite.scala | 4 ++- .../spark/deploy/worker/WorkerSuite.scala | 4 ++- .../spark/launcher/LauncherBackendSuite.scala | 3 +- .../NettyBlockTransferSecuritySuite.scala | 3 +- .../NettyBlockTransferServiceSuite.scala | 4 ++- .../org/apache/spark/rdd/SortingSuite.scala | 3 +- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 3 +- .../sort/SortShuffleManagerSuite.scala | 2 +- .../shuffle/sort/SortShuffleWriterSuite.scala | 2 +- .../status/ElementTrackingStoreSuite.scala | 2 +- .../status/api/v1/SimpleDateParamSuite.scala | 3 +- .../BlockManagerDecommissionUnitSuite.scala | 2 +- .../BlockManagerReplicationSuite.scala | 3 +- .../spark/storage/BlockManagerSuite.scala | 4 ++- .../storage/BlockReplicationPolicySuite.scala | 3 +- .../spark/storage/TopologyMapperSuite.scala | 3 +- .../spark/ui/RealBrowserUISeleniumSuite.scala | 4 ++- .../org/apache/spark/ui/UISeleniumSuite.scala | 4 ++- .../apache/spark/util/DistributionSuite.scala | 3 +- .../util/MutableURLClassLoaderSuite.scala | 3 +- .../apache/spark/util/NextIteratorSuite.scala | 3 +- .../ExternalAppendOnlyMapSuite.scala | 2 +- .../util/collection/OpenHashMapSuite.scala | 3 +- .../util/collection/OpenHashSetSuite.scala | 3 +- .../PrimitiveKeyOpenHashMapSuite.scala | 3 +- .../util/random/RandomSamplerSuite.scala | 3 +- .../util/random/XORShiftRandomSuite.scala | 3 +- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 4 +-- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 4 +-- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 4 +-- .../kinesis/KinesisReceiverSuite.scala | 4 ++- .../kinesis/KinesisStreamSuite.scala | 2 +- .../spark/graphx/lib/ShortestPathsSuite.scala | 2 +- .../org/apache/spark/ml/SparkMLFunSuite.scala | 5 +-- .../LogisticRegressionSuite.scala | 4 +-- .../optimization/GradientDescentSuite.scala | 2 +- .../spark/mllib/optimization/LBFGSSuite.scala | 2 +- .../regression/IsotonicRegressionSuite.scala | 2 +- pom.xml | 35 ++++++++++++++++++- .../k8s/integrationtest/KubernetesSuite.scala | 3 +- .../integrationtest/SecretsTestsSuite.scala | 2 +- .../mesos/MesosSchedulerUtilsSuite.scala | 3 +- .../deploy/yarn/BaseYarnClusterSuite.scala | 3 +- .../spark/deploy/yarn/ClientSuite.scala | 3 +- .../ContainerPlacementStrategySuite.scala | 4 +-- .../deploy/yarn/FailureTrackerSuite.scala | 3 +- .../yarn/ResourceRequestHelperSuite.scala | 3 +- .../YarnAllocatorBlacklistTrackerSuite.scala | 3 +- .../deploy/yarn/YarnAllocatorSuite.scala | 4 ++- .../spark/deploy/yarn/YarnClusterSuite.scala | 3 +- .../yarn/YarnShuffleIntegrationSuite.scala | 3 +- .../yarn/YarnSparkHadoopUtilSuite.scala | 3 +- .../yarn/YarnShuffleServiceMetricsSuite.scala | 3 +- .../yarn/YarnShuffleServiceSuite.scala | 4 ++- .../scala/org/apache/spark/sql/RowTest.scala | 6 ++-- .../AnalysisExternalCatalogSuite.scala | 3 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../analysis/TableLookupCacheSuite.scala | 2 +- .../expressions/UnsafeRowConverterSuite.scala | 3 +- .../BufferHolderSparkSubmitSuite.scala | 3 +- .../CheckCartesianProductsSuite.scala | 2 +- .../catalyst/util/DateTimeUtilsSuite.scala | 3 +- .../util/DatetimeFormatterSuite.scala | 2 +- .../catalyst/util/RebaseDateTimeSuite.scala | 2 +- .../util/TimestampFormatterSuite.scala | 2 ++ .../catalog/LookupCatalogSuite.scala | 2 +- .../spark/sql/ColumnExpressionSuite.scala | 4 +-- .../spark/sql/DataFrameAggregateSuite.scala | 3 +- .../apache/spark/sql/DataFrameStatSuite.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 3 +- .../sql/DataFrameWindowFunctionsSuite.scala | 2 +- .../UnsafeFixedWidthAggregationMapSuite.scala | 3 +- .../WholeStageCodegenSparkSubmitSuite.scala | 6 ++-- .../streaming/EventTimeWatermarkSuite.scala | 4 ++- .../sql/streaming/ui/UISeleniumSuite.scala | 4 ++- .../spark/sql/streaming/ui/UIUtilsSuite.scala | 2 +- .../spark/sql/test/GenericFlatSpecSuite.scala | 6 ++-- .../spark/sql/test/GenericFunSpecSuite.scala | 4 +-- .../spark/sql/test/GenericWordSpecSuite.scala | 6 ++-- .../hive/thriftserver/UISeleniumSuite.scala | 4 ++- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 3 +- .../execution/ObjectHashAggregateSuite.scala | 4 +-- .../PruneFileSourcePartitionsSuite.scala | 2 +- .../streaming/ReceivedBlockHandlerSuite.scala | 4 ++- .../streaming/ReceivedBlockTrackerSuite.scala | 4 ++- .../streaming/StreamingListenerSuite.scala | 3 +- .../spark/streaming/UISeleniumSuite.scala | 4 ++- .../receiver/BlockGeneratorSuite.scala | 3 +- .../rate/PIDRateEstimatorSuite.scala | 3 +- .../StreamingJobProgressListenerSuite.scala | 3 +- .../spark/streaming/ui/UIUtilsSuite.scala | 5 +-- 116 files changed, 270 insertions(+), 145 deletions(-) diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala index ff728f0ebcb85..4c535a8dd0411 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.util.sketch import scala.util.Random -import org.scalatest.FunSuite // scalastyle:ignore funsuite +import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite -class BitArraySuite extends FunSuite { // scalastyle:ignore funsuite +class BitArraySuite extends AnyFunSuite { // scalastyle:ignore funsuite test("error case when create BitArray") { intercept[IllegalArgumentException](new BitArray(0)) diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala index a0408d2da4dff..8b289fc86af0f 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala @@ -22,9 +22,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.reflect.ClassTag import scala.util.Random -import org.scalatest.FunSuite // scalastyle:ignore funsuite +import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite -class BloomFilterSuite extends FunSuite { // scalastyle:ignore funsuite +class BloomFilterSuite extends AnyFunSuite { // scalastyle:ignore funsuite private final val EPSILON = 0.01 // Serializes and deserializes a given `BloomFilter`, then checks whether the deserialized diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala index 174eb01986c4f..087dae26047ef 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala @@ -22,9 +22,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.reflect.ClassTag import scala.util.Random -import org.scalatest.FunSuite // scalastyle:ignore funsuite +import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite -class CountMinSketchSuite extends FunSuite { // scalastyle:ignore funsuite +class CountMinSketchSuite extends AnyFunSuite { // scalastyle:ignore funsuite private val epsOfTotalCount = 0.01 private val confidence = 0.9 diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala index 72aa682bb95bc..69a082053aa65 100644 --- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala @@ -21,14 +21,15 @@ import org.apache.commons.text.similarity.LevenshteinDistance import org.scalacheck.{Arbitrary, Gen} import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks // scalastyle:off -import org.scalatest.{FunSuite, Matchers} +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.matchers.must.Matchers import org.apache.spark.unsafe.types.UTF8String.{fromString => toUTF8} /** * This TestSuite utilize ScalaCheck to generate randomized inputs for UTF8String testing. */ -class UTF8StringPropertyCheckSuite extends FunSuite with ScalaCheckDrivenPropertyChecks with Matchers { +class UTF8StringPropertyCheckSuite extends AnyFunSuite with ScalaCheckDrivenPropertyChecks with Matchers { // scalastyle:on test("toString") { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 39eb1ee731d50..608158caacd37 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -727,7 +727,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { * @note This does not necessarily mean the caching or computation was successful. */ def getPersistentRDDs: JMap[java.lang.Integer, JavaRDD[_]] = { - sc.getPersistentRDDs.mapValues(s => JavaRDD.fromRDD(s)) + sc.getPersistentRDDs.mapValues(s => JavaRDD.fromRDD(s)).toMap .asJava.asInstanceOf[JMap[java.lang.Integer, JavaRDD[_]]] } diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index d701cb65460af..e4dfa149a7d20 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -25,8 +25,8 @@ import scala.collection.mutable.ArrayBuffer import scala.ref.WeakReference import scala.util.control.NonFatal -import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException +import org.scalatest.matchers.must.Matchers import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 27862806c0840..3478b16325dd6 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark import org.scalatest.Assertions._ -import org.scalatest.Matchers import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} +import org.scalatest.matchers.must.Matchers import org.scalatest.time.{Millis, Span} import org.apache.spark.internal.config diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 65391db405a55..9026447e5a98c 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.apache.spark.internal.config diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala index 70b6309be7d53..737a856ba13f1 100644 --- a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala +++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark import scala.concurrent.duration.Duration -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.util.ThreadUtils diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 94ad8d8880027..082a92ef41d3b 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -25,7 +25,7 @@ import scala.concurrent.Future import scala.concurrent.duration._ import org.scalatest.BeforeAndAfter -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Deploy._ diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 3d6690cb85348..5ea43e2dac7d3 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark import java.util.{Locale, Properties} import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService} -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.internal.config diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index 1a563621a5179..3d853ff4294be 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.BeforeAndAfterAll +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.internal.config import org.apache.spark.rdd.ShuffledRDD diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index d111bb33ce8ff..132e994c37a50 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -31,8 +31,8 @@ import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} import org.json4s.{DefaultFormats, Extraction} -import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.must.Matchers._ import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index ec641f8294b29..581786ce0a70c 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -25,7 +25,8 @@ import org.apache.log4j.spi.LoggingEvent import scala.annotation.tailrec import org.apache.log4j.{Appender, AppenderSkeleton, Level, Logger} -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, Outcome} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, Outcome} +import org.scalatest.funsuite.AnyFunSuite import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.util.{AccumulatorContext, Utils} @@ -57,7 +58,7 @@ import scala.collection.mutable.ArrayBuffer * } */ abstract class SparkFunSuite - extends FunSuite + extends AnyFunSuite with BeforeAndAfterAll with BeforeAndAfterEach with ThreadAudit diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index f527bbe718524..fae6c4af1240c 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -20,8 +20,9 @@ package org.apache.spark import scala.concurrent.duration._ import scala.language.implicitConversions -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.JobExecutionStatus._ diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala index dffdd96cd2dcc..daf0151ad65a7 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -21,7 +21,8 @@ import java.io.{File, PrintWriter} import scala.io.Source -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} import org.apache.spark.internal.config.Kryo._ diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 6a99dbca64f4b..792168834dea2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.deploy -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index fd3d4bcf62f69..35311d372e478 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -29,8 +29,10 @@ import com.google.common.io.ByteStreams import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FSDataInputStream, Path} -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ diff --git a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala index 48bd088d07ff9..566eb9cf3c143 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala @@ -27,7 +27,8 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index ade03a0095c19..21a99a462aa1e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -34,8 +34,9 @@ import org.apache.hadoop.security.AccessControlException import org.json4s.jackson.JsonMethods._ import org.mockito.ArgumentMatchers.{any, argThat} import org.mockito.Mockito.{doThrow, mock, spy, verify, when} -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{JobExecutionStatus, SecurityManager, SPARK_VERSION, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 39b339caea385..d30c1dfea9145 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -35,8 +35,10 @@ import org.json4s.jackson.JsonMethods._ import org.mockito.Mockito._ import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.scalatestplus.selenium.WebBrowser @@ -309,14 +311,18 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val urlsThroughKnox = responseThroughKnox \\ "@href" map (_.toString) val siteRelativeLinksThroughKnox = urlsThroughKnox filter (_.startsWith("/")) - all (siteRelativeLinksThroughKnox) should startWith (knoxBaseUrl) + for (link <- siteRelativeLinksThroughKnox) { + link should startWith (knoxBaseUrl) + } val directRequest = mock[HttpServletRequest] val directResponse = page.render(directRequest) val directUrls = directResponse \\ "@href" map (_.toString) val directSiteRelativeLinks = directUrls filter (_.startsWith("/")) - all (directSiteRelativeLinks) should not startWith (knoxBaseUrl) + for (link <- directSiteRelativeLinks) { + link should not startWith (knoxBaseUrl) + } } test("static relative links are prefixed with uiRoot (spark.ui.proxyBase)") { @@ -331,7 +337,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // then val urls = response \\ "@href" map (_.toString) val siteRelativeLinks = urls filter (_.startsWith("/")) - all (siteRelativeLinks) should startWith (uiRoot) + for (link <- siteRelativeLinks) { + link should startWith (uiRoot) + } } test("/version api endpoint") { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/RealBrowserUIHistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/RealBrowserUIHistoryServerSuite.scala index 8a1e22c694497..4a5c34f86753c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/RealBrowserUIHistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/RealBrowserUIHistoryServerSuite.scala @@ -22,7 +22,9 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.proxy.ProxyServlet import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.openqa.selenium.WebDriver -import org.scalatest._ +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.selenium.WebBrowser import org.apache.spark._ @@ -146,7 +148,9 @@ abstract class RealBrowserUIHistoryServerSuite(val driverProp: String) // there are at least some URL links that were generated via javascript, // and they all contain the spark.ui.proxyBase (uiRoot) links.length should be > 4 - all(links) should startWith(url + uiRoot) + for (link <- links) { + link should startWith(url + uiRoot) + } } finally { contextHandler.stop() quit() diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index d98a6b29be9e8..8898d68664f36 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -31,8 +31,10 @@ import scala.reflect.ClassTag import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, Matchers, PrivateMethodTester} +import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala index 44f38e7043dcd..2cfc60f927e1c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config.{STAGING_DIR, SUBMIT_DEPLOY_MODE} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala index 2d3cc5d3abd65..28e35bc8183ba 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.deploy.worker -import org.scalatest.{Matchers, PrivateMethodTester} +import org.scalatest.PrivateMethodTester +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.Command diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index 2d3d0afe3f80c..5bbd60f99f77e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -29,8 +29,10 @@ import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.TestUtils.{createTempJsonFile, createTempScriptWithExpectedOutput} diff --git a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala index edec968d0745a..473782ee28d1c 100644 --- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala @@ -21,8 +21,9 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration._ -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.internal.config.UI.UI_ENABLED diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index c726329ce8a84..13bb811b840d5 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -28,7 +28,8 @@ import scala.util.{Failure, Success, Try} import com.google.common.io.CharStreams import org.mockito.Mockito._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala index c804102e4ab2c..baa878eb14047 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala @@ -25,7 +25,9 @@ import scala.util.Random import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, times, verify, when} -import org.scalatest._ +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{ExecutorDeadException, SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.network.BlockDataManager diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index d5f7d30a253fe..5b01b54a0a9f4 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.rdd -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SharedSparkContext, SparkFunSuite} import org.apache.spark.internal.Logging diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index c82a5ef1111bb..2b38aa1cde6cd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2281,7 +2281,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(stackTraceString.contains("org.apache.spark.rdd.RDD.count")) // should include the FunSuite setup: - assert(stackTraceString.contains("org.scalatest.FunSuite")) + assert(stackTraceString.contains("org.scalatest.funsuite.AnyFunSuite")) } test("catch errors in event loop") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 270b2c606ad0c..a4a84b0e89809 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -24,7 +24,8 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.mockito.Mockito -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala index e5f3aab6a6a1a..f75db20808bee 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.shuffle.sort import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, Serializer} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala index 4c5694fcf0305..c0f5db1892e7e 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.shuffle.sort import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Mockito._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} import org.apache.spark.memory.MemoryTestingUtils diff --git a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala index 38e88e6a012c4..fec7007279894 100644 --- a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.status import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import org.mockito.Mockito._ -import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config.Status._ diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala index 18baeb1cb9c71..ff2060e278465 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.status.api.v1 import javax.ws.rs.WebApplicationException -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala index 5ff1ff05cc4e0..41b68d5978d16 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -21,8 +21,8 @@ import scala.concurrent.duration._ import org.mockito.{ArgumentMatchers => mc} import org.mockito.Mockito.{mock, times, verify, when} -import org.scalatest._ import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.internal.config diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index d18d84dfaa9e5..0b673c580d71f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -26,8 +26,9 @@ import scala.language.implicitConversions import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{doAnswer, mock, spy, when} -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 62bb4d9f0d1fb..5588669c9f986 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -32,9 +32,11 @@ import scala.reflect.ClassTag import org.apache.commons.lang3.RandomUtils import org.mockito.{ArgumentCaptor, ArgumentMatchers => mc} import org.mockito.Mockito.{doAnswer, mock, never, spy, times, verify, when} -import org.scalatest._ +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager diff --git a/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala index 4000218e71a8b..9ce4acc75ec43 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala @@ -21,7 +21,8 @@ import scala.collection.mutable import scala.language.implicitConversions import scala.util.Random -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers import org.apache.spark.{LocalSparkContext, SparkFunSuite} diff --git a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala index 0bc26adeeb443..03e27bfaf2d45 100644 --- a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.storage import java.io.{File, FileOutputStream} -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.internal.config.STORAGE_REPLICATION_TOPOLOGY_FILE diff --git a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala index 4b018f69b1660..2c135850b5a21 100644 --- a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.ui import org.openqa.selenium.{By, WebDriver} -import org.scalatest._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 4f808f03e5dab..6817fccd10305 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -30,8 +30,10 @@ import org.json4s._ import org.json4s.jackson.JsonMethods import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala index d3a95e399c289..0ffa9842e16f5 100644 --- a/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.util -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala index 597e0b9597c38..3063e79704fff 100644 --- a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala @@ -21,7 +21,8 @@ import java.net.URLClassLoader import scala.collection.JavaConverters._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkContext, SparkException, SparkFunSuite, TestUtils} diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index 1efd399b5db68..4909d5f8325cc 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -21,7 +21,8 @@ import java.util.NoSuchElementException import scala.collection.mutable.Buffer -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 0b4e1494bf300..83595ba22aa57 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -21,8 +21,8 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.ref.WeakReference -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.internal.config._ diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index 68bcc5e5a5092..08fed93364060 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.util.SizeEstimator diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index 44d2118d77945..89a308556d5df 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.util.collection -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.util.SizeEstimator diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala index f5ee428020fd4..3b42300ccdd6a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.util.SizeEstimator diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index fef514e0c4e37..b3f54ff186194 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -22,7 +22,8 @@ import java.util.Random import scala.collection.mutable.ArrayBuffer import org.apache.commons.math3.distribution.PoissonDistribution -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index df3483830ca9c..267ab1020a185 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.util.random import org.apache.commons.math3.stat.inference.ChiSquareTest -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.util.Utils.times diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 344806e447689..0c60231d0bd01 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -157,9 +157,9 @@ 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 netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar -objenesis/2.5.1//objenesis-2.5.1.jar +objenesis/2.6//objenesis-2.6.jar okhttp/3.12.6//okhttp-3.12.6.jar -okio/1.15.0//okio-1.15.0.jar +okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar orc-core/1.5.10/nohive/orc-core-1.5.10-nohive.jar orc-mapreduce/1.5.10/nohive/orc-mapreduce-1.5.10-nohive.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 969249b963e7b..5151bbecd003e 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -172,9 +172,9 @@ 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 netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar -objenesis/2.5.1//objenesis-2.5.1.jar +objenesis/2.6//objenesis-2.6.jar okhttp/3.12.6//okhttp-3.12.6.jar -okio/1.15.0//okio-1.15.0.jar +okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar orc-core/1.5.10//orc-core-1.5.10.jar orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar 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 e98e4676107ed..ed7fe552e181e 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -185,10 +185,10 @@ metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.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 +objenesis/2.6//objenesis-2.6.jar okhttp/2.7.5//okhttp-2.7.5.jar okhttp/3.12.6//okhttp-3.12.6.jar -okio/1.15.0//okio-1.15.0.jar +okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar orc-core/1.5.10//orc-core-1.5.10.jar orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index 470a8cecc8fd9..d008de3b3f1c4 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -26,7 +26,9 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason import com.amazonaws.services.kinesis.model.Record import org.mockito.ArgumentMatchers.{anyList, anyString, eq => meq} import org.mockito.Mockito.{never, times, verify, when} -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.streaming.{Duration, TestSuiteBase} diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index eee62d25e62bb..647f17859d041 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -23,8 +23,8 @@ import scala.util.Random import com.amazonaws.services.kinesis.model.Record import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} -import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.network.util.JavaUtils diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala index 994395bbffa56..f909b703ac6e0 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala @@ -34,7 +34,7 @@ class ShortestPathsSuite extends SparkFunSuite with LocalSparkContext { val graph = Graph.fromEdgeTuples(edges, 1) val landmarks = Seq(1, 4).map(_.toLong) val results = ShortestPaths.run(graph, landmarks).vertices.collect.map { - case (v, spMap) => (v, spMap.mapValues(i => i)) + case (v, spMap) => (v, spMap.mapValues(i => i).toMap) } assert(results.toSet === shortestPaths) } diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala index cb3b56bba87b0..d5ba1f906d389 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml // scalastyle:off -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite /** * Base abstract class for all unit tests in Spark for handling common functionality. */ private[spark] abstract class SparkMLFunSuite - extends FunSuite + extends AnyFunSuite with BeforeAndAfterAll { // scalastyle:on } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index d96a4da46a630..584e7555eb0d9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -21,8 +21,8 @@ import scala.collection.JavaConverters._ import scala.util.Random import scala.util.control.Breaks._ -import org.scalatest.Assertions._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.{Vector, Vectors} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index a5542565c7401..edea67e524aad 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.optimization import scala.collection.JavaConverters._ import scala.util.Random -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index 69c303ee932e0..27e21acc275c3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.optimization import scala.util.Random -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala index 02ea74b87f684..8066900dfa011 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.regression -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext diff --git a/pom.xml b/pom.xml index cfcb55b27fa99..af71850ff6b5f 100644 --- a/pom.xml +++ b/pom.xml @@ -334,6 +334,21 @@ scalatest_${scala.binary.version} test + + org.scalatestplus + scalatestplus-scalacheck_${scala.binary.version} + test + + + org.scalatestplus + scalatestplus-mockito_${scala.binary.version} + test + + + org.scalatestplus + scalatestplus-selenium_${scala.binary.version} + test + junit junit @@ -892,7 +907,25 @@ org.scalatest scalatest_${scala.binary.version} - 3.0.8 + 3.2.0 + test + + + org.scalatestplus + scalatestplus-scalacheck_${scala.binary.version} + 3.1.0.0-RC2 + test + + + org.scalatestplus + scalatestplus-mockito_${scala.binary.version} + 1.0.0-SNAP5 + test + + + org.scalatestplus + scalatestplus-selenium_${scala.binary.version} + 1.0.0-SNAP5 test diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index ebf71e8cb83e4..279386d94b35d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -28,9 +28,10 @@ import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, Tag} -import org.scalatest.Matchers import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.concurrent.PatienceConfiguration.{Interval, Timeout} +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.{Minutes, Seconds, Span} import org.apache.spark.SparkFunSuite diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala index 54a9dbf07816c..39ae94b17c63b 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala @@ -22,8 +22,8 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{Pod, SecretBuilder} import org.apache.commons.codec.binary.Base64 -import org.apache.commons.io.output.ByteArrayOutputStream import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite._ diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala index 0ed6fe66c56eb..f2efe86c3f9ec 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala @@ -25,7 +25,8 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import org.apache.mesos.Protos.{FrameworkInfo, Resource, Value} import org.mockito.Mockito._ -import org.scalatest._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index f8ef0d08d829c..20f5339c46fef 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -28,8 +28,9 @@ import scala.concurrent.duration._ import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.MiniYARNCluster -import org.scalatest.{BeforeAndAfterAll, Matchers} +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.deploy.yarn.config._ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 9c5c376ce5357..ea3acec3bb78b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -43,7 +43,8 @@ import org.apache.hadoop.yarn.util.Records import org.mockito.ArgumentMatchers.{any, anyBoolean, anyShort, eq => meq} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TestUtils} import org.apache.spark.deploy.yarn.ResourceRequestHelper._ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala index d83a0d2efec1f..d64aad9ad470b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.deploy.yarn import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers -import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite import org.apache.spark.resource.ResourceProfile diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/FailureTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/FailureTrackerSuite.scala index 4f77b9c99dd25..bfefdbc903b0c 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/FailureTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/FailureTrackerSuite.scala @@ -16,7 +16,8 @@ */ package org.apache.spark.deploy.yarn -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.util.ManualClock diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala index f5ec531e26e0c..3e54dd0034307 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.deploy.yarn import org.apache.hadoop.yarn.api.records.Resource import org.apache.hadoop.yarn.util.Records -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy.yarn.ResourceRequestHelper._ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala index c07a4ac76b989..97615f5c936b0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala @@ -22,7 +22,8 @@ import java.util.Collections import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.mockito.Mockito._ -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.yarn.config.{YARN_EXCLUDE_NODES, YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 2003d0bb87b2d..63e2b97e0ecab 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -28,7 +28,9 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.ArgumentCaptor import org.mockito.Mockito._ -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index b7c9e83446012..14438bc141056 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -28,8 +28,9 @@ import scala.io.Source import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.ConverterUtils -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.api.python.PythonUtils diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 8c62069a8dd67..547bfca2891f1 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -23,7 +23,8 @@ import java.nio.charset.StandardCharsets import com.google.common.io.Files import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.deploy.yarn.config._ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index c88bb292aa77f..9cd37479a9db6 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -23,7 +23,8 @@ import java.nio.charset.StandardCharsets import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala index 632c66d77b707..63ac1af8a9127 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala @@ -21,7 +21,8 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.metrics2.MetricsRecordBuilder import org.mockito.ArgumentMatchers.{any, anyDouble, anyInt, anyLong} import org.mockito.Mockito.{mock, times, verify, when} -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.network.server.OneForOneStreamManager diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 1a5a099217f55..46e596575533d 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -34,8 +34,10 @@ import org.apache.hadoop.service.ServiceStateException import org.apache.hadoop.yarn.api.records.ApplicationId import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext} -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SecurityManager import org.apache.spark.SparkFunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala index 25699de33d717..385f749736846 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql -import org.scalatest.{FunSpec, Matchers} +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericRow, GenericRowWithSchema} import org.apache.spark.sql.types._ -class RowTest extends FunSpec with Matchers { +class RowTest extends AnyFunSpec with Matchers { val schema = StructType( StructField("col1", StringType) :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala index a85ac3fc4d0b7..3dd38091051d8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala @@ -21,7 +21,7 @@ import java.io.File import java.net.URI import org.mockito.Mockito._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils class AnalysisExternalCatalogSuite extends AnalysisTest with Matchers { private def getAnalyzer(externCatalog: ExternalCatalog, databasePath: File): Analyzer = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 77e427d93daeb..9c9f59d155695 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import org.apache.log4j.Level -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala index eed962cd0f69d..06ea531833a43 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala @@ -22,7 +22,7 @@ import java.io.File import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 0149f0deb8d2b..220728fcaa2f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.catalyst.expressions import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala index d2862c8f41d1b..972a832255155 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.catalyst.expressions.codegen -import org.scalatest.{Assertions, BeforeAndAfterEach, Matchers} +import org.scalatest.{Assertions, BeforeAndAfterEach} +import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkFunSuite, TestUtils} import org.apache.spark.deploy.SparkSubmitSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala index 788fedb3c8e8e..dea2b36ecc844 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.scalatest.Matchers._ +import org.scalatest.matchers.must.Matchers._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index caf4b7e16f285..fe761f672c041 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -23,7 +23,8 @@ import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId} import java.util.Locale import java.util.concurrent.TimeUnit -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala index 3b9a4ae88d586..54a0aec4cf612 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.time.DateTimeException -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkFunSuite, SparkUpgradeException} import org.apache.spark.sql.catalyst.plans.SQLHelper diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index b74a219540bb2..428a0c028a5f6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -21,7 +21,7 @@ import java.sql.{Date, Timestamp} import java.time.{Instant, LocalDate, LocalDateTime, ZoneId} import java.util.TimeZone -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala index 79c06cf8313b8..b3e0d6394940a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.util import java.time.{DateTimeException, Instant, LocalDateTime, LocalTime} import java.util.concurrent.TimeUnit +import org.scalatest.matchers.should.Matchers._ + import org.apache.spark.SparkUpgradeException import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index b36ded3d9d2ac..0db758d5147f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -20,7 +20,7 @@ import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.scalatest.Inside -import org.scalatest.Matchers._ +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 131ab1b94f59e..761632e76b165 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -20,11 +20,9 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} import java.util.Locale -import scala.collection.JavaConverters._ - import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} -import org.scalatest.Matchers._ +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.sql.catalyst.expressions.{InSet, Literal, NamedExpression} import org.apache.spark.sql.execution.ProjectExec diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 54327b38c100b..e954e2bf1c46d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.util.Random -import org.scalatest.Matchers.the +import org.scalatest.matchers.must.Matchers.the import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -31,7 +31,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.DecimalData import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index 19601726fa393..8078165cf8ec9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import java.util.Random -import org.scalatest.Matchers._ +import org.scalatest.matchers.must.Matchers._ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.stat.StatFunctions diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 52ef5895ed9ad..0d0e91e2287e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -26,7 +26,8 @@ import java.util.concurrent.atomic.AtomicLong import scala.reflect.runtime.universe.TypeTag import scala.util.Random -import org.scalatest.Matchers._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 09ce43e4e2b27..bc6adfb857b02 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.scalatest.Matchers.the +import org.scalatest.matchers.must.Matchers.the import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.optimizer.TransposeWindow diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index ef81f1b788496..97e5c1148c244 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -24,7 +24,8 @@ import scala.util.{Random, Try} import scala.util.control.NonFatal import org.mockito.Mockito._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala index c5a01de911962..ee5e0e09fca21 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.sql.execution -import org.scalatest.{Assertions, BeforeAndAfterEach, Matchers} -import org.scalatest.concurrent.TimeLimits +import org.scalatest.{Assertions, BeforeAndAfterEach} +import org.scalatest.matchers.must.Matchers import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkFunSuite, TestUtils} import org.apache.spark.deploy.SparkSubmitSuite import org.apache.spark.internal.Logging -import org.apache.spark.sql.{LocalSparkSession, QueryTest, Row, SparkSession} +import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.functions.{array, col, count, lit} import org.apache.spark.sql.types.IntegerType import org.apache.spark.unsafe.Platform diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 705e980df86a0..67ab72a79145e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -24,7 +24,9 @@ import java.util.{Calendar, Date, Locale} import java.util.concurrent.TimeUnit._ import org.apache.commons.io.FileUtils -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, Dataset} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala index 63b5792ebd515..82aa1453f9ba2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql.streaming.ui import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala index 46f2eadc05835..fbf4dcead4a9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.streaming.ui import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.streaming.StreamingQueryProgress diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala index 2861b80190abe..dfa0348f1227c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala @@ -17,15 +17,13 @@ package org.apache.spark.sql.test -import org.scalatest.FlatSpec - -import org.apache.spark.sql.Dataset +import org.scalatest.flatspec.AnyFlatSpec /** * The purpose of this suite is to make sure that generic FlatSpec-based scala * tests work with a shared spark session */ -class GenericFlatSpecSuite extends FlatSpec with SharedSparkSessionBase { +class GenericFlatSpecSuite extends AnyFlatSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala index efdaac3ae1f0c..1b6724054a3ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.test -import org.scalatest.FunSpec +import org.scalatest.funspec.AnyFunSpec import org.apache.spark.sql.Dataset @@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset * The purpose of this suite is to make sure that generic FunSpec-based scala * tests work with a shared spark session */ -class GenericFunSpecSuite extends FunSpec with SharedSparkSessionBase { +class GenericFunSpecSuite extends AnyFunSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala index 3a3540c1dbdbf..e693dc9c9b134 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala @@ -17,15 +17,13 @@ package org.apache.spark.sql.test -import org.scalatest.WordSpec - -import org.apache.spark.sql.Dataset +import org.scalatest.wordspec.AnyWordSpec /** * The purpose of this suite is to make sure that generic WordSpec-based scala * tests work with a shared spark session */ -class GenericWordSpecSuite extends WordSpec with SharedSparkSessionBase { +class GenericWordSpecSuite extends AnyWordSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index 7f731f3d05e51..d0b829c240327 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -22,8 +22,10 @@ import scala.util.Random import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest.{BeforeAndAfterAll, Matchers} +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 3a7e92ee1c00b..6feaaea3dfb89 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -23,8 +23,9 @@ import scala.util.Properties import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.fs.Path -import org.scalatest.{BeforeAndAfterEach, Matchers} import org.scalatest.Assertions._ +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.internal.Logging diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala index 327e4104d59a8..72aeb4f502213 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala @@ -20,11 +20,9 @@ package org.apache.spark.sql.hive.execution import scala.util.Random import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax -import org.scalatest.Matchers._ +import org.scalatest.matchers.must.Matchers._ import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.analysis.UnresolvedFunction import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 24aecb0274ece..ab37645b1c90c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.scalatest.Matchers._ +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 558e2c99e0442..86c20f5a46b9a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -26,8 +26,10 @@ import scala.concurrent.duration._ import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 368411cc2214b..420f7f20ce186 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -28,8 +28,10 @@ import scala.util.Random import org.apache.hadoop.conf.Configuration import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{doThrow, reset, spy} -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.Logging diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 679c58dbae92b..b54d60aa29c4f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -25,8 +25,9 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future import org.mockito.Mockito.{mock, reset, verifyNoMoreInteractions} -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 7041e46f99b18..f797101992573 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -21,8 +21,10 @@ import scala.collection.mutable.Queue import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala index 4c0dd0d258c53..cd867aa8132bc 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala @@ -23,9 +23,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.scalatest.BeforeAndAfter -import org.scalatest.Matchers._ import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala index 1a0460cd669af..bad5cd2564988 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.streaming.scheduler.rate import scala.util.Random import org.scalatest.Inspectors.forAll -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.streaming.Seconds diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala index 10f92f9386173..01177f1cca70a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.streaming.ui import java.util.Properties -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.scheduler.SparkListenerJobStart import org.apache.spark.streaming.{LocalStreamingContext, _} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala index 576083723f8bd..7f96bbfd7b19a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala @@ -20,12 +20,13 @@ package org.apache.spark.streaming.ui import java.util.TimeZone import java.util.concurrent.TimeUnit -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.ui.{UIUtils => SparkUIUtils} -class UIUtilsSuite extends SparkFunSuite with Matchers{ +class UIUtilsSuite extends SparkFunSuite with Matchers { test("shortTimeUnitString") { assert("ns" === UIUtils.shortTimeUnitString(TimeUnit.NANOSECONDS)) From 658e87471c101065d5dac35a20b5a1b99a4609a1 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 24 Jul 2020 09:54:11 +0900 Subject: [PATCH 264/384] [SPARK-30648][SQL][FOLLOWUP] Refactoring of JsonFilters: move config checking out ### What changes were proposed in this pull request? Refactoring of `JsonFilters`: - Add an assert to the `skipRow` method to check the input `index` - Move checking of the SQL config `spark.sql.json.filterPushdown.enabled` from `JsonFilters` to `JacksonParser`. ### Why are the changes needed? 1. The assert should catch incorrect usage of `JsonFilters` 2. The config checking out of `JsonFilters` makes it consistent with `OrderedFilters` (see https://github.com/apache/spark/pull/29145). 3. `JsonFilters` can be used by other datasource in the future and don't depend from the JSON configs. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By existing tests suites: ``` $ build/sbt "sql/test:testOnly org.apache.spark.sql.execution.datasources.json.*" $ build/sbt "test:testOnly org.apache.spark.sql.catalyst.json.*" ``` Closes #29206 from MaxGekk/json-filters-pushdown-followup. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- .../sql/catalyst/json/JacksonParser.scala | 6 +- .../spark/sql/catalyst/json/JsonFilters.scala | 65 ++++++++++--------- 2 files changed, 38 insertions(+), 33 deletions(-) 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 9ddbacbbc3e6e..da76f8eeaf350 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 @@ -87,7 +87,11 @@ class JacksonParser( private def makeStructRootConverter(st: StructType): JsonParser => Iterable[InternalRow] = { val elementConverter = makeConverter(st) val fieldConverters = st.map(_.dataType).map(makeConverter).toArray - val jsonFilters = new JsonFilters(filters, st) + val jsonFilters = if (SQLConf.get.jsonFilterPushDown) { + new JsonFilters(filters, st) + } else { + new NoopFilters + } (parser: JsonParser) => parseJsonToken[Iterable[InternalRow]](parser, st) { case START_OBJECT => convertObject(parser, st, fieldConverters, jsonFilters) // SPARK-3308: support reading top level JSON arrays and take every element diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala index 09022bfc15403..d6adbe83584e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala @@ -89,39 +89,37 @@ class JsonFilters(pushedFilters: Seq[sources.Filter], schema: StructType) // 1: Array(AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) private val predicates: Array[Array[JsonPredicate]] = { val groupedPredicates = Array.fill(schema.length)(Array.empty[JsonPredicate]) - if (SQLConf.get.jsonFilterPushDown) { - val groupedByRefSet: Map[Set[String], JsonPredicate] = filters - // Group filters that have the same set of references. For example: - // IsNotNull("i") -> Set("i"), AlwaysTrue -> Set(), - // Or(EqualTo("i", 0), StringStartsWith("s", "abc")) -> Set("i", "s") - // By grouping filters we could avoid tracking their state of references in the - // current row separately. - .groupBy(_.references.toSet) - // Combine all filters from the same group by `And` because all filters should - // return `true` to do not skip a row. The result is compiled to a predicate. - .map { case (refSet, refsFilters) => - (refSet, JsonPredicate(toPredicate(refsFilters), refSet.size)) - } - // Apply predicates w/o references like `AlwaysTrue` and `AlwaysFalse` to all fields. - // We cannot set such predicates to a particular position because skipRow() can - // be invoked for any index due to unpredictable order of JSON fields in JSON records. - val withLiterals: Map[Set[String], JsonPredicate] = groupedByRefSet.map { - case (refSet, pred) if refSet.isEmpty => - (schema.fields.map(_.name).toSet, pred.copy(totalRefs = 1)) - case others => others - } - // Build a map where key is only one field and value is seq of predicates refer to the field - // "i" -> Seq(AlwaysTrue, IsNotNull("i"), Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) - // "s" -> Seq(AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) - val groupedByFields: Map[String, Seq[(String, JsonPredicate)]] = withLiterals.toSeq - .flatMap { case (refSet, pred) => refSet.map((_, pred)) } - .groupBy(_._1) - // Build the final array by converting keys of `groupedByFields` to their - // indexes in the provided schema. - groupedByFields.foreach { case (fieldName, fieldPredicates) => - val fieldIndex = schema.fieldIndex(fieldName) - groupedPredicates(fieldIndex) = fieldPredicates.map(_._2).toArray + val groupedByRefSet: Map[Set[String], JsonPredicate] = filters + // Group filters that have the same set of references. For example: + // IsNotNull("i") -> Set("i"), AlwaysTrue -> Set(), + // Or(EqualTo("i", 0), StringStartsWith("s", "abc")) -> Set("i", "s") + // By grouping filters we could avoid tracking their state of references in the + // current row separately. + .groupBy(_.references.toSet) + // Combine all filters from the same group by `And` because all filters should + // return `true` to do not skip a row. The result is compiled to a predicate. + .map { case (refSet, refsFilters) => + (refSet, JsonPredicate(toPredicate(refsFilters), refSet.size)) } + // Apply predicates w/o references like `AlwaysTrue` and `AlwaysFalse` to all fields. + // We cannot set such predicates to a particular position because skipRow() can + // be invoked for any index due to unpredictable order of JSON fields in JSON records. + val withLiterals: Map[Set[String], JsonPredicate] = groupedByRefSet.map { + case (refSet, pred) if refSet.isEmpty => + (schema.fields.map(_.name).toSet, pred.copy(totalRefs = 1)) + case others => others + } + // Build a map where key is only one field and value is seq of predicates refer to the field + // "i" -> Seq(AlwaysTrue, IsNotNull("i"), Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + // "s" -> Seq(AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + val groupedByFields: Map[String, Seq[(String, JsonPredicate)]] = withLiterals.toSeq + .flatMap { case (refSet, pred) => refSet.map((_, pred)) } + .groupBy(_._1) + // Build the final array by converting keys of `groupedByFields` to their + // indexes in the provided schema. + groupedByFields.foreach { case (fieldName, fieldPredicates) => + val fieldIndex = schema.fieldIndex(fieldName) + groupedPredicates(fieldIndex) = fieldPredicates.map(_._2).toArray } groupedPredicates } @@ -139,6 +137,9 @@ class JsonFilters(pushedFilters: Seq[sources.Filter], schema: StructType) * return `false`. It returns `false` if all predicates return `true`. */ def skipRow(row: InternalRow, index: Int): Boolean = { + assert(0 <= index && index < schema.fields.length, + s"The index $index is out of the valid range [0, ${schema.fields.length}). " + + s"It must point out to a field of the schema: ${schema.catalogString}.") var skip = false for (pred <- predicates(index) if !skip) { pred.refCount -= 1 From 19e3ed765afa1434c2e32c77ab150f67c358ace7 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 24 Jul 2020 09:55:36 +0900 Subject: [PATCH 265/384] [SPARK-32415][SQL][TESTS] Enable tests for JSON option: allowNonNumericNumbers ### What changes were proposed in this pull request? Enable two tests from `JsonParsingOptionsSuite`: - `allowNonNumericNumbers off` - `allowNonNumericNumbers on` ### 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 the enabled tests. Closes #29207 from MaxGekk/allowNonNumericNumbers-tests. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- .../json/JsonParsingOptionsSuite.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index 7592809d7c85b..8c1afbfa7ec8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.datasources.json -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.json.JSONOptions import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{DoubleType, StringType, StructType} /** * Test cases for various [[JSONOptions]]. @@ -102,20 +103,20 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession { assert(df.first().getLong(0) == 18) } - // The following two tests are not really working - need to look into Jackson's - // JsonReadFeature.ALLOW_NON_NUMERIC_NUMBERS. - ignore("allowNonNumericNumbers off") { + test("allowNonNumericNumbers off") { val str = """{"age": NaN}""" - val df = spark.read.json(Seq(str).toDS()) + val df = spark.read.option("allowNonNumericNumbers", false).json(Seq(str).toDS()) - assert(df.schema.head.name == "_corrupt_record") + assert(df.schema === new StructType().add("_corrupt_record", StringType)) + checkAnswer(df, Row(str)) } - ignore("allowNonNumericNumbers on") { + test("allowNonNumericNumbers on") { val str = """{"age": NaN}""" - val df = spark.read.option("allowNonNumericNumbers", "true").json(Seq(str).toDS()) + val df = spark.read.option("allowNonNumericNumbers", true).json(Seq(str).toDS()) assert(df.schema.head.name == "age") + assert(df.schema.head.dataType == DoubleType) assert(df.first().getDouble(0).isNaN) } From 84efa04c57a95ee2d289fa699036f1aa44840f72 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 24 Jul 2020 04:33:18 +0000 Subject: [PATCH 266/384] [SPARK-32308][SQL] Move by-name resolution logic of unionByName from API code to analysis phase ### What changes were proposed in this pull request? Currently the by-name resolution logic of `unionByName` is put in API code. This patch moves the logic to analysis phase. See https://github.com/apache/spark/pull/28996#discussion_r453460284. ### Why are the changes needed? Logically we should do resolution in analysis phase. This refactoring cleans up API method and makes consistent resolution. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit tests. Closes #29107 from viirya/move-union-by-name. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 8 +- .../sql/catalyst/analysis/ResolveUnion.scala | 100 ++++++++++++++++++ .../sql/catalyst/analysis/TypeCoercion.scala | 4 +- .../sql/catalyst/optimizer/Optimizer.scala | 31 ++++-- .../optimizer/PropagateEmptyRelation.scala | 4 +- .../plans/logical/basicLogicalOperators.scala | 14 ++- .../analysis/DecimalPrecisionSuite.scala | 2 +- .../catalyst/analysis/ResolveUnionSuite.scala | 75 +++++++++++++ .../optimizer/SetOperationSuite.scala | 17 +++ .../sql/catalyst/trees/TreeNodeSuite.scala | 4 +- .../scala/org/apache/spark/sql/Dataset.scala | 46 +------- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../sql/execution/SparkPlannerSuite.scala | 4 +- 13 files changed, 243 insertions(+), 70 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnion.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 2a849023ea13b..13d98d8ce9b00 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -249,6 +249,7 @@ class Analyzer( ResolveTimeZone(conf) :: ResolveRandomSeed :: ResolveBinaryArithmetic :: + ResolveUnion :: TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*), @@ -1430,10 +1431,11 @@ class Analyzer( i.copy(right = dedupRight(left, right)) case e @ Except(left, right, _) if !e.duplicateResolved => e.copy(right = dedupRight(left, right)) - case u @ Union(children) if !u.duplicateResolved => + // Only after we finish by-name resolution for Union + case u: Union if !u.byName && !u.duplicateResolved => // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing // feature in streaming. - val newChildren = children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => + val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => head +: tail.map { case child if head.outputSet.intersect(child.outputSet).isEmpty => child @@ -3447,7 +3449,7 @@ object EliminateSubqueryAliases extends Rule[LogicalPlan] { */ object EliminateUnions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case Union(children) if children.size == 1 => children.head + case u: Union if u.children.size == 1 => u.children.head } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnion.scala new file mode 100644 index 0000000000000..693a5a4e75443 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnion.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} +import org.apache.spark.sql.catalyst.optimizer.CombineUnions +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.SchemaUtils + +/** + * Resolves different children of Union to a common set of columns. + */ +object ResolveUnion extends Rule[LogicalPlan] { + private def unionTwoSides( + left: LogicalPlan, + right: LogicalPlan, + allowMissingCol: Boolean): LogicalPlan = { + val resolver = SQLConf.get.resolver + val leftOutputAttrs = left.output + val rightOutputAttrs = right.output + + // Builds a project list for `right` based on `left` output names + val rightProjectList = leftOutputAttrs.map { lattr => + rightOutputAttrs.find { rattr => resolver(lattr.name, rattr.name) }.getOrElse { + if (allowMissingCol) { + Alias(Literal(null, lattr.dataType), lattr.name)() + } else { + throw new AnalysisException( + s"""Cannot resolve column name "${lattr.name}" among """ + + s"""(${rightOutputAttrs.map(_.name).mkString(", ")})""") + } + } + } + + // Delegates failure checks to `CheckAnalysis` + val notFoundAttrs = rightOutputAttrs.diff(rightProjectList) + val rightChild = Project(rightProjectList ++ notFoundAttrs, right) + + // Builds a project for `logicalPlan` based on `right` output names, if allowing + // missing columns. + val leftChild = if (allowMissingCol) { + val missingAttrs = notFoundAttrs.map { attr => + Alias(Literal(null, attr.dataType), attr.name)() + } + if (missingAttrs.nonEmpty) { + Project(leftOutputAttrs ++ missingAttrs, left) + } else { + left + } + } else { + left + } + Union(leftChild, rightChild) + } + + // Check column name duplication + private def checkColumnNames(left: LogicalPlan, right: LogicalPlan): Unit = { + val caseSensitiveAnalysis = SQLConf.get.caseSensitiveAnalysis + val leftOutputAttrs = left.output + val rightOutputAttrs = right.output + + SchemaUtils.checkColumnNameDuplication( + leftOutputAttrs.map(_.name), + "in the left attributes", + caseSensitiveAnalysis) + SchemaUtils.checkColumnNameDuplication( + rightOutputAttrs.map(_.name), + "in the right attributes", + caseSensitiveAnalysis) + } + + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { + case e if !e.childrenResolved => e + + case Union(children, byName, allowMissingCol) if byName => + val union = children.reduceLeft { (left, right) => + checkColumnNames(left, right) + unionTwoSides(left, right, allowMissingCol) + } + CombineUnions(union) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 3484108a5503f..604a082be4e55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -341,10 +341,10 @@ object TypeCoercion { assert(newChildren.length == 2) Intersect(newChildren.head, newChildren.last, isAll) - case s: Union if s.childrenResolved && + case s: Union if s.childrenResolved && !s.byName && s.children.forall(_.output.length == s.children.head.output.length) && !s.resolved => val newChildren: Seq[LogicalPlan] = buildNewChildrenWithWiderTypes(s.children) - s.makeCopy(Array(newChildren)) + s.copy(children = newChildren) } /** Build new children with the widest types for each attribute among all the children */ 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 79d00d32c9307..b8da954d938c4 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 @@ -497,8 +497,8 @@ object LimitPushDown extends Rule[LogicalPlan] { // Note: right now Union means UNION ALL, which does not de-duplicate rows, so it is safe to // pushdown Limit through it. Once we add UNION DISTINCT, however, we will not be able to // pushdown Limit. - case LocalLimit(exp, Union(children)) => - LocalLimit(exp, Union(children.map(maybePushLocalLimit(exp, _)))) + case LocalLimit(exp, u: Union) => + LocalLimit(exp, u.copy(children = u.children.map(maybePushLocalLimit(exp, _)))) // Add extra limits below OUTER JOIN. For LEFT OUTER and RIGHT OUTER JOIN we push limits to // the left and right sides, respectively. It's not safe to push limits below FULL OUTER // JOIN in the general case without a more invasive rewrite. @@ -556,15 +556,15 @@ object PushProjectionThroughUnion extends Rule[LogicalPlan] with PredicateHelper def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Push down deterministic projection through UNION ALL - case p @ Project(projectList, Union(children)) => - assert(children.nonEmpty) + case p @ Project(projectList, u: Union) => + assert(u.children.nonEmpty) if (projectList.forall(_.deterministic)) { - val newFirstChild = Project(projectList, children.head) - val newOtherChildren = children.tail.map { child => - val rewrites = buildRewrites(children.head, child) + val newFirstChild = Project(projectList, u.children.head) + val newOtherChildren = u.children.tail.map { child => + val rewrites = buildRewrites(u.children.head, child) Project(projectList.map(pushToRight(_, rewrites)), child) } - Union(newFirstChild +: newOtherChildren) + u.copy(children = newFirstChild +: newOtherChildren) } else { p } @@ -928,19 +928,28 @@ object CombineUnions extends Rule[LogicalPlan] { } private def flattenUnion(union: Union, flattenDistinct: Boolean): Union = { + val topByName = union.byName + val topAllowMissingCol = union.allowMissingCol + val stack = mutable.Stack[LogicalPlan](union) val flattened = mutable.ArrayBuffer.empty[LogicalPlan] + // Note that we should only flatten the unions with same byName and allowMissingCol. + // Although we do `UnionCoercion` at analysis phase, we manually run `CombineUnions` + // in some places like `Dataset.union`. Flattening unions with different resolution + // rules (by position and by name) could cause incorrect results. while (stack.nonEmpty) { stack.pop() match { - case Distinct(Union(children)) if flattenDistinct => + case Distinct(Union(children, byName, allowMissingCol)) + if flattenDistinct && byName == topByName && allowMissingCol == topAllowMissingCol => stack.pushAll(children.reverse) - case Union(children) => + case Union(children, byName, allowMissingCol) + if byName == topByName && allowMissingCol == topAllowMissingCol => stack.pushAll(children.reverse) case child => flattened += child } } - Union(flattened.toSeq) + union.copy(children = flattened) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala index 0299646150ff3..d3cdd71eafdb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala @@ -50,8 +50,8 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper wit override def conf: SQLConf = SQLConf.get def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case p @ Union(children) if children.exists(isEmptyLocalRelation) => - val newChildren = children.filterNot(isEmptyLocalRelation) + case p: Union if p.children.exists(isEmptyLocalRelation) => + val newChildren = p.children.filterNot(isEmptyLocalRelation) if (newChildren.isEmpty) { empty(p) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index f1a363cca752e..c83f6a376aa8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -220,8 +220,18 @@ object Union { /** * Logical plan for unioning two plans, without a distinct. This is UNION ALL in SQL. + * + * @param byName Whether resolves columns in the children by column names. + * @param allowMissingCol Allows missing columns in children query plans. If it is true, + * this function allows different set of column names between two Datasets. + * This can be set to true only if `byName` is true. */ -case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { +case class Union( + children: Seq[LogicalPlan], + byName: Boolean = false, + allowMissingCol: Boolean = false) extends LogicalPlan { + assert(!allowMissingCol || byName, "`allowMissingCol` can be true only if `byName` is true.") + override def maxRows: Option[Long] = { if (children.exists(_.maxRows.isEmpty)) { None @@ -271,7 +281,7 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { child.output.zip(children.head.output).forall { case (l, r) => l.dataType.sameType(r.dataType) }) - children.length > 1 && childrenResolved && allChildrenCompatible + children.length > 1 && !(byName || allowMissingCol) && childrenResolved && allChildrenCompatible } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index c3e18c7f9557f..d5991ff10ce6c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -73,7 +73,7 @@ class DecimalPrecisionSuite extends AnalysisTest with BeforeAndAfter { Union(Project(Seq(Alias(left, "l")()), relation), Project(Seq(Alias(right, "r")()), relation)) val (l, r) = analyzer.execute(plan).collect { - case Union(Seq(child1, child2)) => (child1.output.head, child2.output.head) + case Union(Seq(child1, child2), _, _) => (child1.output.head, child2.output.head) }.head assert(l.dataType === expectedType) assert(r.dataType === expectedType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnionSuite.scala new file mode 100644 index 0000000000000..5c7ad0067a456 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnionSuite.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types._ + +class ResolveUnionSuite extends AnalysisTest { + test("Resolve Union") { + val table1 = LocalRelation( + AttributeReference("i", IntegerType)(), + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("b", ByteType)(), + AttributeReference("d", DoubleType)()) + val table2 = LocalRelation( + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("b", ByteType)(), + AttributeReference("d", DoubleType)(), + AttributeReference("i", IntegerType)()) + val table3 = LocalRelation( + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("d", DoubleType)(), + AttributeReference("i", IntegerType)()) + val table4 = LocalRelation( + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("i", IntegerType)()) + + val rules = Seq(ResolveUnion) + val analyzer = new RuleExecutor[LogicalPlan] { + override val batches = Seq(Batch("Resolution", Once, rules: _*)) + } + + // By name resolution + val union1 = Union(table1 :: table2 :: Nil, true, false) + val analyzed1 = analyzer.execute(union1) + val projected1 = + Project(Seq(table2.output(3), table2.output(0), table2.output(1), table2.output(2)), table2) + val expected1 = Union(table1 :: projected1 :: Nil) + comparePlans(analyzed1, expected1) + + // Allow missing column + val union2 = Union(table1 :: table3 :: Nil, true, true) + val analyzed2 = analyzer.execute(union2) + val nullAttr1 = Alias(Literal(null, ByteType), "b")() + val projected2 = + Project(Seq(table2.output(3), table2.output(0), nullAttr1, table2.output(2)), table3) + val expected2 = Union(table1 :: projected2 :: Nil) + comparePlans(analyzed2, expected2) + + // Allow missing column + Allow missing column + val union3 = Union(union2 :: table4 :: Nil, true, true) + val analyzed3 = analyzer.execute(union3) + val nullAttr2 = Alias(Literal(null, DoubleType), "d")() + val projected3 = + Project(Seq(table2.output(3), table2.output(0), nullAttr1, nullAttr2), table4) + val expected3 = Union(table1 :: projected2 :: projected3 :: Nil) + comparePlans(analyzed3, expected3) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala index ccc30b1d2f8ce..2eea840e21a31 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala @@ -223,4 +223,21 @@ class SetOperationSuite extends PlanTest { val unionCorrectAnswer = unionQuery.analyze comparePlans(unionOptimized, unionCorrectAnswer) } + + test("CombineUnions only flatten the unions with same byName and allowMissingCol") { + val union1 = Union(testRelation :: testRelation :: Nil, true, false) + val union2 = Union(testRelation :: testRelation :: Nil, true, true) + val union3 = Union(testRelation :: testRelation2 :: Nil, false, false) + + val union4 = Union(union1 :: union2 :: union3 :: Nil) + val unionOptimized1 = Optimize.execute(union4) + val unionCorrectAnswer1 = Union(union1 :: union2 :: testRelation :: testRelation2 :: Nil) + comparePlans(unionOptimized1, unionCorrectAnswer1, false) + + val union5 = Union(union1 :: union1 :: Nil, true, false) + val unionOptimized2 = Optimize.execute(union5) + val unionCorrectAnswer2 = + Union(testRelation :: testRelation :: testRelation :: testRelation :: Nil, true, false) + comparePlans(unionOptimized2, unionCorrectAnswer2, false) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index f5259706325eb..ff51bc0071c80 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -583,7 +583,9 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { JObject( "class" -> classOf[Union].getName, "num-children" -> 2, - "children" -> List(0, 1)), + "children" -> List(0, 1), + "byName" -> JBool(false), + "allowMissingCol" -> JBool(false)), JObject( "class" -> classOf[JsonTestTreeNode].getName, "num-children" -> 0, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index d5501326397c9..7b0bae6a82053 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -227,7 +227,7 @@ class Dataset[T] private[sql]( val plan = queryExecution.analyzed match { case c: Command => LocalRelation(c.output, withAction("command", queryExecution)(_.executeCollect())) - case u @ Union(children) if children.forall(_.isInstanceOf[Command]) => + case u @ Union(children, _, _) if children.forall(_.isInstanceOf[Command]) => LocalRelation(u.output, withAction("command", queryExecution)(_.executeCollect())) case _ => queryExecution.analyzed @@ -2071,51 +2071,9 @@ class Dataset[T] private[sql]( * @since 3.1.0 */ def unionByName(other: Dataset[T], allowMissingColumns: Boolean): Dataset[T] = withSetOperator { - // Check column name duplication - val resolver = sparkSession.sessionState.analyzer.resolver - val leftOutputAttrs = logicalPlan.output - val rightOutputAttrs = other.logicalPlan.output - - SchemaUtils.checkColumnNameDuplication( - leftOutputAttrs.map(_.name), - "in the left attributes", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - SchemaUtils.checkColumnNameDuplication( - rightOutputAttrs.map(_.name), - "in the right attributes", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - - // Builds a project list for `other` based on `logicalPlan` output names - val rightProjectList = leftOutputAttrs.map { lattr => - rightOutputAttrs.find { rattr => resolver(lattr.name, rattr.name) }.getOrElse { - if (allowMissingColumns) { - Alias(Literal(null, lattr.dataType), lattr.name)() - } else { - throw new AnalysisException( - s"""Cannot resolve column name "${lattr.name}" among """ + - s"""(${rightOutputAttrs.map(_.name).mkString(", ")})""") - } - } - } - - // Delegates failure checks to `CheckAnalysis` - val notFoundAttrs = rightOutputAttrs.diff(rightProjectList) - val rightChild = Project(rightProjectList ++ notFoundAttrs, other.logicalPlan) - - // Builds a project for `logicalPlan` based on `other` output names, if allowing - // missing columns. - val leftChild = if (allowMissingColumns) { - val missingAttrs = notFoundAttrs.map { attr => - Alias(Literal(null, attr.dataType), attr.name)() - } - Project(leftOutputAttrs ++ missingAttrs, logicalPlan) - } else { - logicalPlan - } - // This breaks caching, but it's usually ok because it addresses a very specific use case: // using union to union many files or partitions. - CombineUnions(Union(leftChild, rightChild)) + CombineUnions(Union(logicalPlan :: other.logicalPlan :: Nil, true, allowMissingColumns)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 7b5d8f15962d0..78aa258387daa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -683,8 +683,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.LocalLimitExec(limit, planLater(child)) :: Nil case logical.GlobalLimit(IntegerLiteral(limit), child) => execution.GlobalLimitExec(limit, planLater(child)) :: Nil - case logical.Union(unionChildren) => - execution.UnionExec(unionChildren.map(planLater)) :: Nil + case union: logical.Union => + execution.UnionExec(union.children.map(planLater)) :: Nil case g @ logical.Generate(generator, _, outer, _, _, child) => execution.GenerateExec( generator, g.requiredChildOutput, outer, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala index 9107f8afa83d7..b4cb7e3fce3cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala @@ -37,9 +37,9 @@ class SparkPlannerSuite extends SharedSparkSession { case ReturnAnswer(child) => planned += 1 planLater(child) :: planLater(NeverPlanned) :: Nil - case Union(children) => + case u: Union => planned += 1 - UnionExec(children.map(planLater)) :: planLater(NeverPlanned) :: Nil + UnionExec(u.children.map(planLater)) :: planLater(NeverPlanned) :: Nil case LocalRelation(output, data, _) => planned += 1 LocalTableScanExec(output, data) :: planLater(NeverPlanned) :: Nil From 8896f4af87f358b9bd4e253783d003d0b4e9ee8e Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 24 Jul 2020 18:14:19 +0800 Subject: [PATCH 267/384] Revert "[SPARK-32253][INFRA] Show errors only for the sbt tests of github actions" ### What changes were proposed in this pull request? This reverts commit 026b0b926dfd40038f2cee932f38b917eb25b77e. ### Why are the changes needed? As HyukjinKwon pointed out in https://github.com/apache/spark/pull/29133#issuecomment-663339240, there is no JUnit test report after https://github.com/apache/spark/pull/29133. Let's revert https://github.com/apache/spark/pull/29133 for now and find a better solution to improve the log output later. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? GitHub Actions build Closes #29219 from gengliangwang/revertErrorOnly. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang --- dev/run-tests.py | 3 --- project/SparkBuild.scala | 7 ------- 2 files changed, 10 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 8e29f89c3a0d0..065a27c0e853b 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -653,9 +653,6 @@ def main(): # If we're running the tests in Github Actions, attempt to detect and test # only the affected modules. if test_env == "github_actions": - # Set the log level of sbt as ERROR to make the output more readable. - if build_tool == "sbt": - extra_profiles.append("--error") if os.environ["GITHUB_BASE_REF"] != "": # Pull requests changed_files = identify_changed_files_from_git_commits( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c05e41cfcc0c2..5ff9d14a5aa17 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1031,13 +1031,6 @@ object TestSettings { }.getOrElse(Nil): _*), // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), - // Show only the failed test cases with full stack traces in github action to make the log more - // readable. - // Check https://www.scalatest.org/user_guide/using_the_runner for the details of options . - testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, - sys.env.get("GITHUB_ACTIONS").map { _ => - Seq("-eNCXEHLOPQMDF") - }.getOrElse(Nil): _*), testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), // Required to detect Junit tests for each project, see also https://github.com/sbt/junit-interface/issues/35 crossPaths := false, From 8bc799f92005c903868ef209f5aec8deb6ccce5a Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 24 Jul 2020 14:12:43 +0000 Subject: [PATCH 268/384] [SPARK-32375][SQL] Basic functionality of table catalog v2 for JDBC ### What changes were proposed in this pull request? This PR implements basic functionalities of the `TableCatalog` interface, so that end-users can use the JDBC as a catalog. ### Why are the changes needed? To have at least one built implementation of Catalog Plugin API available to end users. JDBC is perfectly fit for this. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? By new test suite `JDBCTableCatalogSuite`. Closes #29168 from MaxGekk/jdbc-v2. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../jdbc/JdbcRelationProvider.scala | 4 +- .../datasources/jdbc/JdbcUtils.scala | 50 ++++-- .../datasources/v2/jdbc/JDBCTable.scala | 37 ++++ .../v2/jdbc/JDBCTableCatalog.scala | 158 ++++++++++++++++++ .../apache/spark/sql/jdbc/JdbcDialects.scala | 13 ++ .../v2/jdbc/JDBCTableCatalogSuite.scala | 109 ++++++++++++ .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 5 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 6 +- 8 files changed, 365 insertions(+), 17 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index e7456f9c8ed0b..314012feef4c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -59,7 +59,7 @@ class JdbcRelationProvider extends CreatableRelationProvider } else { // Otherwise, do not truncate the table, instead drop and recreate it dropTable(conn, options.table, options) - createTable(conn, df, options) + createTable(conn, options.table, df.schema, isCaseSensitive, options) saveTable(df, Some(df.schema), isCaseSensitive, options) } @@ -78,7 +78,7 @@ class JdbcRelationProvider extends CreatableRelationProvider // Therefore, it is okay to do nothing here and then just return the relation below. } } else { - createTable(conn, df, options) + createTable(conn, options.table, df.schema, isCaseSensitive, options) saveTable(df, Some(df.schema), isCaseSensitive, options) } } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index db4715ef068b6..3c6649b26ecd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -746,15 +746,16 @@ object JdbcUtils extends Logging { * Compute the schema string for this RDD. */ def schemaString( - df: DataFrame, + schema: StructType, + caseSensitive: Boolean, url: String, createTableColumnTypes: Option[String] = None): String = { val sb = new StringBuilder() val dialect = JdbcDialects.get(url) val userSpecifiedColTypesMap = createTableColumnTypes - .map(parseUserSpecifiedCreateTableColumnTypes(df, _)) + .map(parseUserSpecifiedCreateTableColumnTypes(schema, caseSensitive, _)) .getOrElse(Map.empty[String, String]) - df.schema.fields.foreach { field => + schema.fields.foreach { field => val name = dialect.quoteIdentifier(field.name) val typ = userSpecifiedColTypesMap .getOrElse(field.name, getJdbcType(field.dataType, dialect).databaseTypeDefinition) @@ -770,7 +771,8 @@ object JdbcUtils extends Logging { * use in-place of the default data type. */ private def parseUserSpecifiedCreateTableColumnTypes( - df: DataFrame, + schema: StructType, + caseSensitive: Boolean, createTableColumnTypes: String): Map[String, String] = { def typeName(f: StructField): String = { // char/varchar gets translated to string type. Real data type specified by the user @@ -783,7 +785,11 @@ object JdbcUtils extends Logging { } val userSchema = CatalystSqlParser.parseTableSchema(createTableColumnTypes) - val nameEquality = df.sparkSession.sessionState.conf.resolver + val nameEquality = if (caseSensitive) { + org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution + } else { + org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution + } // checks duplicate columns in the user specified column types. SchemaUtils.checkColumnNameDuplication( @@ -791,16 +797,15 @@ object JdbcUtils extends Logging { // checks if user specified column names exist in the DataFrame schema userSchema.fieldNames.foreach { col => - df.schema.find(f => nameEquality(f.name, col)).getOrElse { + schema.find(f => nameEquality(f.name, col)).getOrElse { throw new AnalysisException( s"createTableColumnTypes option column $col not found in schema " + - df.schema.catalogString) + schema.catalogString) } } val userSchemaMap = userSchema.fields.map(f => f.name -> typeName(f)).toMap - val isCaseSensitive = df.sparkSession.sessionState.conf.caseSensitiveAnalysis - if (isCaseSensitive) userSchemaMap else CaseInsensitiveMap(userSchemaMap) + if (caseSensitive) userSchemaMap else CaseInsensitiveMap(userSchemaMap) } /** @@ -865,17 +870,18 @@ object JdbcUtils extends Logging { */ def createTable( conn: Connection, - df: DataFrame, + tableName: String, + schema: StructType, + caseSensitive: Boolean, options: JdbcOptionsInWrite): Unit = { val strSchema = schemaString( - df, options.url, options.createTableColumnTypes) - val table = options.table + schema, caseSensitive, options.url, options.createTableColumnTypes) val createTableOptions = options.createTableOptions // Create the table if the table does not exist. // To allow certain options to append when create a new table, which can be // table_options or partition_options. // E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" - val sql = s"CREATE TABLE $table ($strSchema) $createTableOptions" + val sql = s"CREATE TABLE $tableName ($strSchema) $createTableOptions" val statement = conn.createStatement try { statement.setQueryTimeout(options.queryTimeout) @@ -884,4 +890,22 @@ object JdbcUtils extends Logging { statement.close() } } + + /** + * Rename a table from the JDBC database. + */ + def renameTable( + conn: Connection, + oldTable: String, + newTable: String, + options: JDBCOptions): Unit = { + val dialect = JdbcDialects.get(options.url) + val statement = conn.createStatement + try { + statement.setQueryTimeout(options.queryTimeout) + statement.executeUpdate(dialect.renameTable(oldTable, newTable)) + } finally { + statement.close() + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala new file mode 100644 index 0000000000000..c91723d1331c5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.jdbc + +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.sql.types.StructType + +// TODO (SPARK-32396): Implement the `SupportsRead` interface +// TODO (SPARK-32410): Implement the `SupportsWrite` interface +case class JDBCTable(ident: Identifier, schema: StructType, jdbcOptions: JDBCOptions) + extends Table { + assert(ident.namespace().length == 1) + + override def name(): String = ident.toString + + override def capabilities(): java.util.Set[TableCapability] = { + val capabilities = new java.util.HashSet[TableCapability] + capabilities.add(BATCH_READ) + capabilities + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala new file mode 100644 index 0000000000000..5d64cf4ca896e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.jdbc + +import java.sql.{Connection, SQLException} + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException} +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog, TableChange} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JDBCRDD, JdbcUtils} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class JDBCTableCatalog extends TableCatalog with Logging { + + private var catalogName: String = null + private var options: JDBCOptions = _ + private var dialect: JdbcDialect = _ + + override def name(): String = { + require(catalogName != null, "The JDBC table catalog is not initialed") + catalogName + } + + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + assert(catalogName == null, "The JDBC table catalog is already initialed") + catalogName = name + + val map = options.asCaseSensitiveMap().asScala.toMap + // The `JDBCOptions` checks the existence of the table option. This is required by JDBC v1, but + // JDBC V2 only knows the table option when loading a table. Here we put a table option with a + // fake value, so that it can pass the check of `JDBCOptions`. + this.options = new JDBCOptions(map + (JDBCOptions.JDBC_TABLE_NAME -> "__invalid_dbtable")) + dialect = JdbcDialects.get(this.options.url) + } + + override def listTables(namespace: Array[String]): Array[Identifier] = { + checkNamespace(namespace) + withConnection { conn => + val schemaPattern = if (namespace.length == 1) namespace.head else null + val rs = conn.getMetaData + .getTables(null, schemaPattern, "%", Array("TABLE")); + new Iterator[Identifier] { + def hasNext = rs.next() + def next = Identifier.of(namespace, rs.getString("TABLE_NAME")) + }.toArray + } + } + + override def tableExists(ident: Identifier): Boolean = { + checkNamespace(ident.namespace()) + val writeOptions = new JdbcOptionsInWrite( + options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident))) + withConnection(JdbcUtils.tableExists(_, writeOptions)) + } + + override def dropTable(ident: Identifier): Boolean = { + checkNamespace(ident.namespace()) + withConnection { conn => + try { + JdbcUtils.dropTable(conn, getTableName(ident), options) + true + } catch { + case _: SQLException => false + } + } + } + + override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { + checkNamespace(oldIdent.namespace()) + withConnection { conn => + JdbcUtils.renameTable(conn, getTableName(oldIdent), getTableName(newIdent), options) + } + } + + override def loadTable(ident: Identifier): Table = { + checkNamespace(ident.namespace()) + val optionsWithTableName = new JDBCOptions( + options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident))) + try { + val schema = JDBCRDD.resolveTable(optionsWithTableName) + JDBCTable(ident, schema, optionsWithTableName) + } catch { + case _: SQLException => throw new NoSuchTableException(ident) + } + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: java.util.Map[String, String]): Table = { + checkNamespace(ident.namespace()) + if (partitions.nonEmpty) { + throw new UnsupportedOperationException("Cannot create JDBC table with partition") + } + // TODO (SPARK-32405): Apply table options while creating tables in JDBC Table Catalog + if (!properties.isEmpty) { + logWarning("Cannot create JDBC table with properties, these properties will be " + + "ignored: " + properties.asScala.map { case (k, v) => s"$k=$v" }.mkString("[", ", ", "]")) + } + + val writeOptions = new JdbcOptionsInWrite( + options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident))) + val caseSensitive = SQLConf.get.caseSensitiveAnalysis + withConnection { conn => + JdbcUtils.createTable(conn, getTableName(ident), schema, caseSensitive, writeOptions) + } + + JDBCTable(ident, schema, writeOptions) + } + + // TODO (SPARK-32402): Implement ALTER TABLE in JDBC Table Catalog + override def alterTable(ident: Identifier, changes: TableChange*): Table = { + // scalastyle:off throwerror + throw new NotImplementedError() + // scalastyle:on throwerror + } + + private def checkNamespace(namespace: Array[String]): Unit = { + // In JDBC there is no nested database/schema + if (namespace.length > 1) { + throw new NoSuchNamespaceException(namespace) + } + } + + private def withConnection[T](f: Connection => T): T = { + val conn = JdbcUtils.createConnectionFactory(options)() + try { + f(conn) + } finally { + conn.close() + } + } + + private def getTableName(ident: Identifier): String = { + (ident.namespace() :+ ident.name()).map(dialect.quoteIdentifier).mkString(".") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index a0c6d20f36451..b0f9aba859d3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -180,6 +180,19 @@ abstract class JdbcDialect extends Serializable { * None: The behavior of TRUNCATE TABLE is unknown (default). */ def isCascadingTruncateTable(): Option[Boolean] = None + + /** + * Rename an existing table. + * + * TODO (SPARK-32382): Override this method in the dialects that don't support such syntax. + * + * @param oldTable The existing table. + * @param newTable New name of the table. + * @return The SQL statement to use for renaming the table. + */ + def renameTable(oldTable: String, newTable: String): String = { + s"ALTER TABLE $oldTable RENAME TO $newTable" + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala new file mode 100644 index 0000000000000..0eb96b7813e6e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.jdbc + +import java.sql.{Connection, DriverManager} +import java.util.Properties + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} +import org.apache.spark.util.Utils + +class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { + + val tempDir = Utils.createTempDir() + val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" + var conn: java.sql.Connection = null + + override def sparkConf: SparkConf = super.sparkConf + .set("spark.sql.catalog.h2", classOf[JDBCTableCatalog].getName) + .set("spark.sql.catalog.h2.url", url) + .set("spark.sql.catalog.h2.driver", "org.h2.Driver") + + private def withConnection[T](f: Connection => T): T = { + val conn = DriverManager.getConnection(url, new Properties()) + try { + f(conn) + } finally { + conn.close() + } + } + + override def beforeAll(): Unit = { + super.beforeAll() + Utils.classForName("org.h2.Driver") + withConnection { conn => + conn.prepareStatement("""CREATE SCHEMA "test"""").executeUpdate() + conn.prepareStatement( + """CREATE TABLE "test"."people" (name TEXT(32) NOT NULL, id INTEGER NOT NULL)""") + .executeUpdate() + } + } + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDir) + super.afterAll() + } + + test("show tables") { + checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) + } + + test("drop a table and test whether the table exists") { + withConnection { conn => + conn.prepareStatement("""CREATE TABLE "test"."to_drop" (id INTEGER)""").executeUpdate() + } + checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "to_drop"), Row("test", "people"))) + sql("DROP TABLE h2.test.to_drop") + checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) + } + + test("rename a table") { + withTable("h2.test.dst_table") { + withConnection { conn => + conn.prepareStatement("""CREATE TABLE "test"."src_table" (id INTEGER)""").executeUpdate() + } + checkAnswer( + sql("SHOW TABLES IN h2.test"), + Seq(Row("test", "src_table"), Row("test", "people"))) + sql("ALTER TABLE h2.test.src_table RENAME TO test.dst_table") + checkAnswer( + sql("SHOW TABLES IN h2.test"), + Seq(Row("test", "dst_table"), Row("test", "people"))) + } + } + + test("load a table") { + val t = spark.table("h2.test.people") + val expectedSchema = new StructType() + .add("NAME", StringType) + .add("ID", IntegerType) + assert(t.schema === expectedSchema) + } + + test("create a table") { + withTable("h2.test.new_table") { + // TODO (SPARK-32427): Omit USING in CREATE TABLE + sql("CREATE TABLE h2.test.new_table(i INT, j STRING) USING _") + checkAnswer( + sql("SHOW TABLES IN h2.test"), + Seq(Row("test", "people"), Row("test", "new_table"))) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 23cf25a4030c9..06dd6615a817b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -1184,7 +1184,10 @@ class JDBCSuite extends QueryTest test("SPARK-16387: Reserved SQL words are not escaped by JDBC writer") { val df = spark.createDataset(Seq("a", "b", "c")).toDF("order") - val schema = JdbcUtils.schemaString(df, "jdbc:mysql://localhost:3306/temp") + val schema = JdbcUtils.schemaString( + df.schema, + df.sqlContext.conf.caseSensitiveAnalysis, + "jdbc:mysql://localhost:3306/temp") assert(schema.contains("`order` TEXT")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 8021ef1a17a18..3f621e04338a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -390,7 +390,11 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { val expectedSchemaStr = colTypes.map { case (col, dataType) => s""""$col" $dataType """ }.mkString(", ") - assert(JdbcUtils.schemaString(df, url1, Option(createTableColTypes)) == expectedSchemaStr) + assert(JdbcUtils.schemaString( + df.schema, + df.sqlContext.conf.caseSensitiveAnalysis, + url1, + Option(createTableColTypes)) == expectedSchemaStr) } testCreateTableColDataTypes(Seq("boolean")) From fa184c3308fcc598c0ff873d606a65d0ad987654 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 24 Jul 2020 08:52:30 -0700 Subject: [PATCH 269/384] [SPARK-32408][BUILD] Enable crossPaths back to prevent side effects ### What changes were proposed in this pull request? This PR proposes to enable `corssPaths` back for now to match with the build as it was. It still indeterministically doesn't run JUnit tests given my observation, and this PR basically reverts the partial fix from https://github.com/apache/spark/pull/29057. See also https://github.com/apache/spark/pull/29205 for the full context. ### Why are the changes needed? To prevent the side effects from crossPaths such as SPARK_PREPEND_CLASSES or tests that run conditionally if the test classes are present in PySpark. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually tested: ```bash build/sbt -Phadoop-2.7 -Phive -Phive-2.3 -Phive-thriftserver -DskipTests clean test:package ./python/run-tests --python-executable=python3 --testname="pyspark.sql.tests.test_dataframe QueryExecutionListenerTests" ``` Closes #29218 from HyukjinKwon/SPARK-32408-1. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- project/SparkBuild.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5ff9d14a5aa17..5a3ac213c2057 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1032,8 +1032,6 @@ object TestSettings { // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), - // Required to detect Junit tests for each project, see also https://github.com/sbt/junit-interface/issues/35 - crossPaths := false, // Enable Junit testing. libraryDependencies += "com.novocode" % "junit-interface" % "0.11" % "test", // `parallelExecutionInTest` controls whether test suites belonging to the same SBT project From d3596c04b0275b19d6edc0126a77f749b4e9ba70 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 24 Jul 2020 09:13:26 -0700 Subject: [PATCH 270/384] [SPARK-32406][SQL] Make RESET syntax support single configuration reset MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR extends the RESET command to support reset SQL configuration one by one. ### Why are the changes needed? Currently, the reset command only supports restore all of the runtime configurations to their defaults. In most cases, users do not want this, but just want to restore one or a small group of settings. The SET command can work as a workaround for this, but you have to keep the defaults in your mind or by temp variables, which turns out not very convenient to use. Hive supports this: https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients#HiveServer2Clients-BeelineExample reset | Resets the value of a particular configuration variable (key) to the default value.Note: If you misspell the variable name, Beeline will not show an error. -- | -- PostgreSQL supports this too https://www.postgresql.org/docs/9.1/sql-reset.html ### Does this PR introduce _any_ user-facing change? yes, reset can restore one configuration now ### How was this patch tested? add new unit tests. Closes #29202 from yaooqinn/SPARK-32406. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 19 +++++++++++++++++-- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 3 ++- .../sql/execution/command/SetCommand.scala | 14 ++++++++++---- .../spark/sql/internal/SQLConfSuite.scala | 15 +++++++++++++++ 5 files changed, 45 insertions(+), 8 deletions(-) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index 4caf57a232f89..68df0913b29d4 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -21,19 +21,34 @@ license: | ### Description -Reset any runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. +The RESET command resets runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. ### Syntax ```sql -RESET +RESET; + +RESET configuration_key; ``` +### Parameters + +* **(none)** + + Reset any runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. + +* **configuration_key** + + Restore the value of the `configuration_key` to the default value. If the default value is undefined, the `configuration_key` will be removed. + ### Examples ```sql -- Reset any runtime configurations specific to the current session which were set via the SET command to their default values. RESET; + +-- If you start your application with --conf spark.foo=bar and set spark.foo=foobar in runtime, the example below will restore it to 'bar'. If spark.foo is not specified during starting, the example bellow will remove this config from the SQLConf. It will ignore nonexistent keys. +RESET spark.abc; ``` ### Related Statements diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index b9c46616caf34..4631165981aa5 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -245,7 +245,7 @@ statement | SET TIME ZONE timezone=(STRING | LOCAL) #setTimeZone | SET TIME ZONE .*? #setTimeZone | SET .*? #setConfiguration - | RESET #resetConfiguration + | RESET .*? #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand ; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 44d5285e56074..012ae0a76043c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -85,11 +85,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Example SQL : * {{{ * RESET; + * RESET spark.sql.session.timeZone; * }}} */ override def visitResetConfiguration( ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { - ResetCommand + ResetCommand(Option(remainder(ctx.RESET().getSymbol).trim).filter(_.nonEmpty)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 3dc1d52697714..24c43ee407caa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -166,15 +166,21 @@ object SetCommand { * via [[SetCommand]] will get reset to default value. Command that runs * {{{ * reset; + * reset spark.sql.session.timeZone; * }}} */ -case object ResetCommand extends RunnableCommand with IgnoreCachedData { +case class ResetCommand(config: Option[String]) extends RunnableCommand with IgnoreCachedData { override def run(sparkSession: SparkSession): Seq[Row] = { val conf = sparkSession.sessionState.conf - conf.clear() - sparkSession.sparkContext.conf.getAll.foreach { case (k, v) => - conf.setConfString(k, v) + val defaults = sparkSession.sparkContext.conf + config match { + case Some(key) => + conf.unsetConf(key) + defaults.getOption(key).foreach(conf.setConfString(key, _)) + case None => + conf.clear() + defaults.getAll.foreach { case (k, v) => conf.setConfString(k, v) } } Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 36cb5d94a027d..0ecc5ee04ce16 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -181,6 +181,21 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } + test("SPARK-32406: reset - single configuration") { + val appId = spark.sparkContext.getConf.getAppId + sql("RESET spark.app.id") + assert(spark.conf.get("spark.app.id") === appId, "Should not change spark core ones") + sql("SET spark.abc=xyz") + assert(spark.conf.get("spark.abc") === "xyz") + sql("RESET spark.abc") + intercept[NoSuchElementException](spark.conf.get("spark.abc")) + sql("RESET spark.abc") // ignore nonexistent keys + val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) + sql(s"SET ${SQLConf.GROUP_BY_ORDINAL.key}=false") + sql(s"RESET ${SQLConf.GROUP_BY_ORDINAL.key}") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === original) + } + test("invalid conf value") { spark.sessionState.conf.clear() val e = intercept[IllegalArgumentException] { From 64a01c0a559396fccd615dc00576a80bc8cc5648 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 24 Jul 2020 11:03:57 -0700 Subject: [PATCH 271/384] [SPARK-32430][SQL] Extend SparkSessionExtensions to inject rules into AQE query stage preparation ### What changes were proposed in this pull request? Provide a generic mechanism for plugins to inject rules into the AQE "query prep" stage that happens before query stage creation. This goes along with https://issues.apache.org/jira/browse/SPARK-32332 where the current AQE implementation doesn't allow for users to properly extend it for columnar processing. ### Why are the changes needed? The issue here is that we create new query stages but we do not have access to the parent plan of the new query stage so certain things can not be determined because you have to know what the parent did. With this change it would allow you to add TAGs to be able to figure out what is going on. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? A new unit test is included in the PR. Closes #29224 from andygrove/insert-aqe-rule. Authored-by: Andy Grove Signed-off-by: Dongjoon Hyun --- .../spark/sql/SparkSessionExtensions.scala | 20 +++++++- .../adaptive/AdaptiveSparkPlanExec.scala | 2 +- .../internal/BaseSessionStateBuilder.scala | 9 +++- .../spark/sql/internal/SessionState.scala | 4 +- .../sql/SparkSessionExtensionSuite.scala | 49 +++++++++++++++++++ 5 files changed, 79 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index ff706b5061f0a..6952f4bfd0566 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.ColumnarRule +import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} /** * :: Experimental :: @@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.ColumnarRule *
            • Customized Parser.
            • *
            • (External) Catalog listeners.
            • *
            • Columnar Rules.
            • + *
            • Adaptive Query Stage Preparation Rules.
            • *
            * * The extensions can be used by calling `withExtensions` on the [[SparkSession.Builder]], for @@ -96,8 +97,10 @@ class SparkSessionExtensions { type ParserBuilder = (SparkSession, ParserInterface) => ParserInterface type FunctionDescription = (FunctionIdentifier, ExpressionInfo, FunctionBuilder) type ColumnarRuleBuilder = SparkSession => ColumnarRule + type QueryStagePrepRuleBuilder = SparkSession => Rule[SparkPlan] private[this] val columnarRuleBuilders = mutable.Buffer.empty[ColumnarRuleBuilder] + private[this] val queryStagePrepRuleBuilders = mutable.Buffer.empty[QueryStagePrepRuleBuilder] /** * Build the override rules for columnar execution. @@ -106,6 +109,13 @@ class SparkSessionExtensions { columnarRuleBuilders.map(_.apply(session)).toSeq } + /** + * Build the override rules for the query stage preparation phase of adaptive query execution. + */ + private[sql] def buildQueryStagePrepRules(session: SparkSession): Seq[Rule[SparkPlan]] = { + queryStagePrepRuleBuilders.map(_.apply(session)).toSeq + } + /** * Inject a rule that can override the columnar execution of an executor. */ @@ -113,6 +123,14 @@ class SparkSessionExtensions { columnarRuleBuilders += builder } + /** + * Inject a rule that can override the the query stage preparation phase of adaptive query + * execution. + */ + def injectQueryStagePrepRule(builder: QueryStagePrepRuleBuilder): Unit = { + queryStagePrepRuleBuilders += builder + } + private[this] val resolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder] /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 112090640040a..34db0a334f67f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -90,7 +90,7 @@ case class AdaptiveSparkPlanExec( // Exchange nodes) after running these rules. private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( ensureRequirements - ) + ) ++ context.session.sessionState.queryStagePrepRules // A list of physical optimizer rules to be applied to a new stage before its execution. These // optimizations should be stage-independent. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 4ae12f8716752..83a7a557305e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.execution.{ColumnarRule, QueryExecution, SparkOptimizer, SparkPlanner, SparkSqlParser} +import org.apache.spark.sql.execution.{ColumnarRule, QueryExecution, SparkOptimizer, SparkPlan, SparkPlanner, SparkSqlParser} import org.apache.spark.sql.execution.aggregate.ResolveEncodersInScalaAgg import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.command.CommandCheck @@ -286,6 +286,10 @@ abstract class BaseSessionStateBuilder( extensions.buildColumnarRules(session) } + protected def queryStagePrepRules: Seq[Rule[SparkPlan]] = { + extensions.buildQueryStagePrepRules(session) + } + /** * Create a query execution object. */ @@ -337,7 +341,8 @@ abstract class BaseSessionStateBuilder( () => resourceLoader, createQueryExecution, createClone, - columnarRules) + columnarRules, + queryStagePrepRules) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index abd1250628539..cd425b04ef311 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution._ import org.apache.spark.sql.streaming.StreamingQueryManager @@ -73,7 +74,8 @@ private[sql] class SessionState( resourceLoaderBuilder: () => SessionResourceLoader, createQueryExecution: LogicalPlan => QueryExecution, createClone: (SparkSession, SessionState) => SessionState, - val columnarRules: Seq[ColumnarRule]) { + val columnarRules: Seq[ColumnarRule], + val queryStagePrepRules: Seq[Rule[SparkPlan]]) { // The following fields are lazy to avoid creating the Hive client when creating SessionState. lazy val catalog: SessionCatalog = catalogBuilder() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index d9c90c7dbd085..44e784de5164f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -26,7 +26,9 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, UnresolvedHint} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.COLUMN_BATCH_SIZE @@ -145,6 +147,28 @@ class SparkSessionExtensionSuite extends SparkFunSuite { } } + test("inject adaptive query prep rule") { + val extensions = create { extensions => + // inject rule that will run during AQE query stage preparation and will add custom tags + // to the plan + extensions.injectQueryStagePrepRule(session => MyQueryStagePrepRule()) + // inject rule that will run during AQE query stage optimization and will verify that the + // custom tags were written in the preparation phase + extensions.injectColumnar(session => + MyColumarRule(MyNewQueryStageRule(), MyNewQueryStageRule())) + } + withSession(extensions) { session => + session.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, true) + assert(session.sessionState.queryStagePrepRules.contains(MyQueryStagePrepRule())) + assert(session.sessionState.columnarRules.contains( + MyColumarRule(MyNewQueryStageRule(), MyNewQueryStageRule()))) + import session.sqlContext.implicits._ + val data = Seq((100L), (200L), (300L)).toDF("vals").repartition(1) + val df = data.selectExpr("vals + 1") + df.collect() + } + } + test("inject columnar") { val extensions = create { extensions => extensions.injectColumnar(session => @@ -731,6 +755,31 @@ class MyExtensions extends (SparkSessionExtensions => Unit) { } } +object QueryPrepRuleHelper { + val myPrepTag: TreeNodeTag[String] = TreeNodeTag[String]("myPrepTag") + val myPrepTagValue: String = "myPrepTagValue" +} + +// this rule will run during AQE query preparation and will write custom tags to each node +case class MyQueryStagePrepRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case plan => + plan.setTagValue(QueryPrepRuleHelper.myPrepTag, QueryPrepRuleHelper.myPrepTagValue) + plan + } +} + +// this rule will run during AQE query stage optimization and will verify custom tags were +// already written during query preparation phase +case class MyNewQueryStageRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case plan if !plan.isInstanceOf[AdaptiveSparkPlanExec] => + assert(plan.getTagValue(QueryPrepRuleHelper.myPrepTag).get == + QueryPrepRuleHelper.myPrepTagValue) + plan + } +} + case class MyRule2(spark: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan } From e6ef27be52dcd14dc94384c2ada85861be44d843 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 24 Jul 2020 11:12:28 -0700 Subject: [PATCH 272/384] [SPARK-32287][TESTS] Flaky Test: ExecutorAllocationManagerSuite.add executors default profile ### What changes were proposed in this pull request? I wasn't able to reproduce the failure but the best I can tell is that the allocation manager timer triggers and call doRequest. The timeout is 10s so try to increase that to 30seconds. ### Why are the changes needed? test failure ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? unit test Closes #29225 from tgravescs/SPARK-32287. Authored-by: Thomas Graves Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/ExecutorAllocationManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index ea6e010ef29a7..5b367d2fb01d4 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -1603,7 +1603,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { .set(config.DYN_ALLOCATION_TESTING, true) // SPARK-22864: effectively disable the allocation schedule by setting the period to a // really long value. - .set(TEST_SCHEDULE_INTERVAL, 10000L) + .set(TEST_SCHEDULE_INTERVAL, 30000L) sparkConf } From b890fdc8df64f1d0b0f78b790d36be883e852b0d Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Fri, 24 Jul 2020 11:41:42 -0700 Subject: [PATCH 273/384] [SPARK-32387][SS] Extract UninterruptibleThread runner logic from KafkaOffsetReader ### What changes were proposed in this pull request? `UninterruptibleThread` running functionality is baked into `KafkaOffsetReader` which can be extracted into a class. The main intention is to simplify `KafkaOffsetReader` in order to make easier to solve SPARK-32032. In this PR I've made this extraction without functionality change. ### Why are the changes needed? `UninterruptibleThread` running functionality is baked into `KafkaOffsetReader`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing + additional unit tests. Closes #29187 from gaborgsomogyi/SPARK-32387. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- .../util/UninterruptibleThreadRunner.scala | 55 ++++++++++++++++ .../UninterruptibleThreadRunnerSuite.scala | 64 +++++++++++++++++++ .../sql/kafka010/KafkaOffsetReader.scala | 46 +++---------- 3 files changed, 129 insertions(+), 36 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/UninterruptibleThreadRunner.scala create mode 100644 core/src/test/scala/org/apache/spark/util/UninterruptibleThreadRunnerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThreadRunner.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThreadRunner.scala new file mode 100644 index 0000000000000..18108aa819db9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThreadRunner.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.concurrent.Executors + +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration + +/** + * [[UninterruptibleThreadRunner]] ensures that all tasks are running in an + * [[UninterruptibleThread]]. A good example is Kafka consumer usage. + */ +private[spark] class UninterruptibleThreadRunner(threadName: String) { + private val thread = Executors.newSingleThreadExecutor((r: Runnable) => { + val t = new UninterruptibleThread(threadName) { + override def run(): Unit = { + r.run() + } + } + t.setDaemon(true) + t + }) + private val execContext = ExecutionContext.fromExecutorService(thread) + + def runUninterruptibly[T](body: => T): T = { + if (!Thread.currentThread.isInstanceOf[UninterruptibleThread]) { + val future = Future { + body + }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + } else { + body + } + } + + def shutdown(): Unit = { + thread.shutdown() + } +} diff --git a/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadRunnerSuite.scala b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadRunnerSuite.scala new file mode 100644 index 0000000000000..40312beacdff6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadRunnerSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import org.apache.spark.SparkFunSuite + +class UninterruptibleThreadRunnerSuite extends SparkFunSuite { + private var runner: UninterruptibleThreadRunner = null + + override def beforeEach(): Unit = { + runner = new UninterruptibleThreadRunner("ThreadName") + } + + override def afterEach(): Unit = { + runner.shutdown() + } + + test("runUninterruptibly should switch to UninterruptibleThread") { + assert(!Thread.currentThread().isInstanceOf[UninterruptibleThread]) + var isUninterruptibleThread = false + runner.runUninterruptibly { + isUninterruptibleThread = Thread.currentThread().isInstanceOf[UninterruptibleThread] + } + assert(isUninterruptibleThread, "The runner task must run in UninterruptibleThread") + } + + test("runUninterruptibly should not add new UninterruptibleThread") { + var isInitialUninterruptibleThread = false + var isRunnerUninterruptibleThread = false + val t = new UninterruptibleThread("test") { + override def run(): Unit = { + runUninterruptibly { + val initialThread = Thread.currentThread() + isInitialUninterruptibleThread = initialThread.isInstanceOf[UninterruptibleThread] + runner.runUninterruptibly { + val runnerThread = Thread.currentThread() + isRunnerUninterruptibleThread = runnerThread.isInstanceOf[UninterruptibleThread] + assert(runnerThread.eq(initialThread)) + } + } + } + } + t.start() + t.join() + assert(isInitialUninterruptibleThread, + "The initiator must already run in UninterruptibleThread") + assert(isRunnerUninterruptibleThread, "The runner task must run in UninterruptibleThread") + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 5ab7862674956..6d30bd2a6d2cd 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -18,12 +18,9 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.util.concurrent.Executors import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.Duration import scala.util.control.NonFatal import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetAndTimestamp} @@ -33,7 +30,7 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} +import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner} /** * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets from Kafka. @@ -51,19 +48,13 @@ private[kafka010] class KafkaOffsetReader( val driverKafkaParams: ju.Map[String, Object], readerOptions: CaseInsensitiveMap[String], driverGroupIdPrefix: String) extends Logging { + /** - * Used to ensure execute fetch operations execute in an UninterruptibleThread + * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an + * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an + * [[UninterruptibleThread]], however for batch mode this is not the case. */ - val kafkaReaderThread = Executors.newSingleThreadExecutor((r: Runnable) => { - val t = new UninterruptibleThread("Kafka Offset Reader") { - override def run(): Unit = { - r.run() - } - } - t.setDaemon(true) - t - }) - val execContext = ExecutionContext.fromExecutorService(kafkaReaderThread) + val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader") /** * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is @@ -126,14 +117,14 @@ private[kafka010] class KafkaOffsetReader( * Closes the connection to Kafka, and cleans up state. */ def close(): Unit = { - if (_consumer != null) runUninterruptibly { stopConsumer() } - kafkaReaderThread.shutdown() + if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() } + uninterruptibleThreadRunner.shutdown() } /** * @return The Set of TopicPartitions for a given topic */ - def fetchTopicPartitions(): Set[TopicPartition] = runUninterruptibly { + def fetchTopicPartitions(): Set[TopicPartition] = uninterruptibleThreadRunner.runUninterruptibly { assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) // Poll to get the latest assigned partitions consumer.poll(0) @@ -531,7 +522,7 @@ private[kafka010] class KafkaOffsetReader( private def partitionsAssignedToConsumer( body: ju.Set[TopicPartition] => Map[TopicPartition, Long], fetchingEarliestOffset: Boolean = false) - : Map[TopicPartition, Long] = runUninterruptibly { + : Map[TopicPartition, Long] = uninterruptibleThreadRunner.runUninterruptibly { withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions @@ -551,23 +542,6 @@ private[kafka010] class KafkaOffsetReader( } } - /** - * This method ensures that the closure is called in an [[UninterruptibleThread]]. - * This is required when communicating with the [[KafkaConsumer]]. In the case - * of streaming queries, we are already running in an [[UninterruptibleThread]], - * however for batch mode this is not the case. - */ - private def runUninterruptibly[T](body: => T): T = { - if (!Thread.currentThread.isInstanceOf[UninterruptibleThread]) { - val future = Future { - body - }(execContext) - ThreadUtils.awaitResult(future, Duration.Inf) - } else { - body - } - } - /** * Helper function that does multiple retries on a body of code that returns offsets. * Retries are needed to handle transient failures. For e.g. race conditions between getting From 8e36a8f33fb9d20c481e9eeb0ea8155aa1569439 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 25 Jul 2020 13:09:23 +0900 Subject: [PATCH 274/384] [SPARK-32419][PYTHON][BUILD] Avoid using subshell for Conda env (de)activation in pip packaging test ### What changes were proposed in this pull request? This PR proposes to avoid using subshell when it activates Conda environment. Looks like it ends up with activating the env within the subshell even if you use `conda` command. ### Why are the changes needed? If you take a close look for GitHub Actions log: ``` Installing dist into virtual env Processing ./python/dist/pyspark-3.1.0.dev0.tar.gz Collecting py4j==0.10.9 Downloading py4j-0.10.9-py2.py3-none-any.whl (198 kB) Using legacy setup.py install for pyspark, since package 'wheel' is not installed. Installing collected packages: py4j, pyspark Running setup.py install for pyspark: started Running setup.py install for pyspark: finished with status 'done' Successfully installed py4j-0.10.9 pyspark-3.1.0.dev0 ... Installing dist into virtual env Obtaining file:///home/runner/work/spark/spark/python Collecting py4j==0.10.9 Downloading py4j-0.10.9-py2.py3-none-any.whl (198 kB) Installing collected packages: py4j, pyspark Attempting uninstall: py4j Found existing installation: py4j 0.10.9 Uninstalling py4j-0.10.9: Successfully uninstalled py4j-0.10.9 Attempting uninstall: pyspark Found existing installation: pyspark 3.1.0.dev0 Uninstalling pyspark-3.1.0.dev0: Successfully uninstalled pyspark-3.1.0.dev0 Running setup.py develop for pyspark Successfully installed py4j-0.10.9 pyspark ``` It looks not properly using Conda as it removes the previously installed one when it reinstalls again. We should ideally test it with Conda environment as it's intended. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? GitHub Actions will test. I also manually tested in my local. Closes #29212 from HyukjinKwon/SPARK-32419. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/run-pip-tests | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/run-pip-tests b/dev/run-pip-tests index be96ed9dda20f..b322d3f61b444 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -85,7 +85,7 @@ for python in "${PYTHON_EXECS[@]}"; do source "$CONDA_PREFIX/etc/profile.d/conda.sh" fi conda create -y -p "$VIRTUALENV_PATH" python=$python numpy pandas pip setuptools - source activate "$VIRTUALENV_PATH" || (echo "Falling back to 'conda activate'" && conda activate "$VIRTUALENV_PATH") + source activate "$VIRTUALENV_PATH" || conda activate "$VIRTUALENV_PATH" else mkdir -p "$VIRTUALENV_PATH" virtualenv --python=$python "$VIRTUALENV_PATH" @@ -128,7 +128,7 @@ for python in "${PYTHON_EXECS[@]}"; do # conda / virtualenv environments need to be deactivated differently if [ -n "$USE_CONDA" ]; then - source deactivate || (echo "Falling back to 'conda deactivate'" && conda deactivate) + source deactivate || conda deactivate else deactivate fi From 277a4063ef4ff6e91d999864b820607fa1a206aa Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 25 Jul 2020 03:06:45 -0700 Subject: [PATCH 275/384] [SPARK-32422][SQL][TESTS] Use python3 executable instead of python3.6 in IntegratedUDFTestUtils ### What changes were proposed in this pull request? This PR uses `python3` instead of `python3.6` executable as a fallback in `IntegratedUDFTestUtils`. ### Why are the changes needed? Currently, GitHub Actions skips pandas UDFs. Python 3.8 is installed explicitly but somehow `python3.6` looks available in GitHub Actions build environment by default. ``` [info] - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python3.6]. !!! IGNORED !!! ... [info] - udf/postgreSQL/udf-select_having.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python3.6]. !!! IGNORED !!! ... ``` It was chosen as `python3.6` for Jenkins to pick one Python explicitly; however, looks we're already using `python3` here and there. It will also reduce the overhead to fix when we deprecate or drop Python versions. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? It should be tested in Jenkins and GitHub Actions environments here. Closes #29217 from HyukjinKwon/SPARK-32422. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index 6391d5664d5c6..80346b350c142 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -197,7 +197,7 @@ object IntegratedUDFTestUtils extends SQLHelper { lazy val pythonExec: String = { val pythonExec = sys.env.getOrElse( - "PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python3.6")) + "PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python3")) if (TestUtils.testCommandAvailable(pythonExec)) { pythonExec } else { From be9f03dc71b87263bb9cc843616bcece8f80f186 Mon Sep 17 00:00:00 2001 From: sychen Date: Sat, 25 Jul 2020 03:30:01 -0700 Subject: [PATCH 276/384] [SPARK-32426][SQL] ui shows sql after variable substitution ### What changes were proposed in this pull request? When submitting sql with variables, the sql displayed by ui is not replaced by variables. ### Why are the changes needed? See the final executed sql in ui ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? manual test Closes #29221 from cxzl25/SPARK-32426. Authored-by: sychen Signed-off-by: Dongjoon Hyun --- .../hive/thriftserver/SparkExecuteStatementOperation.scala | 4 +++- .../apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index f2d53a77d6816..d30951f89cf6b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -35,6 +35,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext} import org.apache.spark.sql.execution.HiveResult.{getTimeFormatters, toHiveString, TimeFormatters} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.VariableSubstitution import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.{Utils => SparkUtils} @@ -276,7 +277,8 @@ private[hive] class SparkExecuteStatementOperation( parentSession.getSessionState.getConf.setClassLoader(executionHiveClassLoader) } - sqlContext.sparkContext.setJobGroup(statementId, statement) + val substitutorStatement = new VariableSubstitution(sqlContext.conf).substitute(statement) + sqlContext.sparkContext.setJobGroup(statementId, substitutorStatement) result = sqlContext.sql(statement) logDebug(result.queryExecution.toString()) HiveThriftServer2.eventManager.onStatementParsed(statementId, diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 12fba0eae6dce..8faeee523d983 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.internal.VariableSubstitution private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlContext) @@ -59,7 +60,8 @@ private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlCont override def run(command: String): CommandProcessorResponse = { // TODO unify the error code try { - context.sparkContext.setJobDescription(command) + val substitutorCommand = new VariableSubstitution(context.conf).substitute(command) + context.sparkContext.setJobDescription(substitutorCommand) val execution = context.sessionState.executePlan(context.sql(command).logicalPlan) hiveResponse = SQLExecution.withNewExecutionId(execution) { hiveResultString(execution.executedPlan) From f642234d8584e2ce1ee0de9102c9942523b6d9a3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 25 Jul 2020 08:07:28 -0700 Subject: [PATCH 277/384] [SPARK-32437][CORE] Improve MapStatus deserialization speed with RoaringBitmap 0.9.0 ### What changes were proposed in this pull request? This PR aims to speed up `MapStatus` deserialization by 5~18% with the latest RoaringBitmap `0.9.0` and new APIs. Note that we focus on `deserialization` time because `serialization` occurs once while `deserialization` occurs many times. ### Why are the changes needed? The current version is too old. We had better upgrade it to get the performance improvement and bug fixes. Although `MapStatusesSerDeserBenchmark` is synthetic, the benchmark result is updated with this patch. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins or GitHub Action. Closes #29233 from dongjoon-hyun/SPARK-ROAR. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- ...tatusesSerDeserBenchmark-jdk11-results.txt | 54 +++++++++---------- .../MapStatusesSerDeserBenchmark-results.txt | 54 +++++++++---------- .../apache/spark/scheduler/MapStatus.scala | 4 +- pom.xml | 2 +- 4 files changed, 57 insertions(+), 57 deletions(-) diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt index db23cf5c12ea7..4b16bd445f39d 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt @@ -1,64 +1,64 @@ -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 170 178 9 1.2 849.7 1.0X -Deserialization 530 535 9 0.4 2651.1 0.3X +------------------------------------------------------------------------------------------------------------------------- +Serialization 175 183 12 1.1 874.1 1.0X +Deserialization 458 462 6 0.4 2288.6 0.4X -Compressed Serialized MapStatus sizes: 411 bytes +Compressed Serialized MapStatus sizes: 410 bytes Compressed Serialized Broadcast MapStatus sizes: 2 MB -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 157 165 7 1.3 785.4 1.0X -Deserialization 495 588 79 0.4 2476.7 0.3X +-------------------------------------------------------------------------------------------------------------------------- +Serialization 160 171 8 1.2 801.1 1.0X +Deserialization 453 484 38 0.4 2263.4 0.4X Compressed Serialized MapStatus sizes: 2 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 344 351 4 0.6 1720.4 1.0X -Deserialization 527 579 99 0.4 2635.9 0.7X +-------------------------------------------------------------------------------------------------------------------------- +Serialization 343 346 2 0.6 1717.1 1.0X +Deserialization 492 540 59 0.4 2459.6 0.7X -Compressed Serialized MapStatus sizes: 427 bytes +Compressed Serialized MapStatus sizes: 426 bytes Compressed Serialized Broadcast MapStatus sizes: 13 MB -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 317 321 4 0.6 1583.8 1.0X -Deserialization 530 540 15 0.4 2648.3 0.6X +--------------------------------------------------------------------------------------------------------------------------- +Serialization 297 299 2 0.7 1486.2 1.0X +Deserialization 489 535 87 0.4 2446.5 0.6X Compressed Serialized MapStatus sizes: 13 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 1738 1849 156 0.1 8692.0 1.0X -Deserialization 946 977 33 0.2 4730.2 1.8X +--------------------------------------------------------------------------------------------------------------------------- +Serialization 1641 1819 252 0.1 8204.1 1.0X +Deserialization 844 882 37 0.2 4219.7 1.9X -Compressed Serialized MapStatus sizes: 556 bytes +Compressed Serialized MapStatus sizes: 553 bytes Compressed Serialized Broadcast MapStatus sizes: 121 MB -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 1379 1432 76 0.1 6892.6 1.0X -Deserialization 929 941 19 0.2 4645.5 1.5X +---------------------------------------------------------------------------------------------------------------------------- +Serialization 1360 1412 73 0.1 6799.3 1.0X +Deserialization 850 859 13 0.2 4249.9 1.6X Compressed Serialized MapStatus sizes: 121 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt index 053f4bf771923..996b79b896dba 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt @@ -1,64 +1,64 @@ -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 178 187 15 1.1 887.5 1.0X -Deserialization 530 558 32 0.4 2647.5 0.3X +------------------------------------------------------------------------------------------------------------------------- +Serialization 193 202 15 1.0 966.2 1.0X +Deserialization 477 521 44 0.4 2387.5 0.4X -Compressed Serialized MapStatus sizes: 411 bytes +Compressed Serialized MapStatus sizes: 410 bytes Compressed Serialized Broadcast MapStatus sizes: 2 MB -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 167 175 7 1.2 835.7 1.0X -Deserialization 523 537 22 0.4 2616.2 0.3X +-------------------------------------------------------------------------------------------------------------------------- +Serialization 181 187 11 1.1 907.1 1.0X +Deserialization 477 492 27 0.4 2383.5 0.4X Compressed Serialized MapStatus sizes: 2 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 351 416 147 0.6 1754.4 1.0X -Deserialization 546 551 8 0.4 2727.6 0.6X +-------------------------------------------------------------------------------------------------------------------------- +Serialization 359 436 162 0.6 1797.5 1.0X +Deserialization 506 514 15 0.4 2530.8 0.7X -Compressed Serialized MapStatus sizes: 427 bytes +Compressed Serialized MapStatus sizes: 426 bytes Compressed Serialized Broadcast MapStatus sizes: 13 MB -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 320 321 1 0.6 1598.0 1.0X -Deserialization 542 549 7 0.4 2709.0 0.6X +--------------------------------------------------------------------------------------------------------------------------- +Serialization 322 323 0 0.6 1610.5 1.0X +Deserialization 508 513 5 0.4 2538.4 0.6X Compressed Serialized MapStatus sizes: 13 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 1671 1877 290 0.1 8357.3 1.0X -Deserialization 943 970 32 0.2 4715.8 1.8X +--------------------------------------------------------------------------------------------------------------------------- +Serialization 1740 1903 231 0.1 8700.0 1.0X +Deserialization 872 888 24 0.2 4360.9 2.0X -Compressed Serialized MapStatus sizes: 556 bytes +Compressed Serialized MapStatus sizes: 553 bytes Compressed Serialized Broadcast MapStatus sizes: 121 MB -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 1373 1436 89 0.1 6865.0 1.0X -Deserialization 940 970 37 0.2 4699.1 1.5X +---------------------------------------------------------------------------------------------------------------------------- +Serialization 1461 1469 11 0.1 7306.1 1.0X +Deserialization 871 889 22 0.2 4353.9 1.7X Compressed Serialized MapStatus sizes: 121 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 0af3a2e171906..757e8d31bf5d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -205,7 +205,7 @@ private[spark] class HighlyCompressedMapStatus private ( override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) - emptyBlocks.writeExternal(out) + emptyBlocks.serialize(out) out.writeLong(avgSize) out.writeInt(hugeBlockSizes.size) hugeBlockSizes.foreach { kv => @@ -218,7 +218,7 @@ private[spark] class HighlyCompressedMapStatus private ( override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) emptyBlocks = new RoaringBitmap() - emptyBlocks.readExternal(in) + emptyBlocks.deserialize(in) avgSize = in.readLong() val count = in.readInt() val hugeBlockSizesImpl = mutable.Map.empty[Int, Byte] diff --git a/pom.xml b/pom.xml index af71850ff6b5f..e1be59c790107 100644 --- a/pom.xml +++ b/pom.xml @@ -712,7 +712,7 @@ org.roaringbitmap RoaringBitmap - 0.7.45 + 0.9.0 commons-net From aab1e09f1c40b9d51554bbb4afca67e8d4ccbac6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 25 Jul 2020 08:19:02 -0700 Subject: [PATCH 278/384] [SPARK-32434][CORE] Support Scala 2.13 in AbstractCommandBuilder and load-spark-env scripts ### What changes were proposed in this pull request? This PR aims to support Scala 2.11 at `AbstractCommandBuilder.java` and `load-spark-env` scripts. ### Why are the changes needed? Currently, Scala 2.12 is only supported and the following fails. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -DwildcardSuites=none -Dtest=org.apache.spark.launcher.SparkLauncherSuite ... [ERROR] Failures: [ERROR] SparkLauncherSuite.testChildProcLauncher:123 expected:<0> but was:<1> [ERROR] SparkLauncherSuite.testSparkLauncherGetError:274 [ERROR] Tests run: 6, Failures: 2, Errors: 0, Skipped: 0 ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This should be tested manually with the above command. ``` [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 3.1.0-SNAPSHOT: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 2.186 s] [INFO] Spark Project Tags ................................. SUCCESS [ 4.400 s] [INFO] Spark Project Local DB ............................. SUCCESS [ 1.744 s] [INFO] Spark Project Networking ........................... SUCCESS [ 2.233 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 1.527 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 5.564 s] [INFO] Spark Project Launcher ............................. SUCCESS [ 1.946 s] [INFO] Spark Project Core ................................. SUCCESS [01:21 min] [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 01:41 min [INFO] Finished at: 2020-07-24T20:04:34-07:00 [INFO] ------------------------------------------------------------------------ ``` Closes #29227 from dongjoon-hyun/SPARK-32434. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- bin/load-spark-env.cmd | 42 +++++++++---------- bin/load-spark-env.sh | 42 +++++++++---------- .../launcher/AbstractCommandBuilder.java | 23 +++++----- 3 files changed, 50 insertions(+), 57 deletions(-) diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index 5f98cc34b6bab..ebbde66153e14 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -37,26 +37,24 @@ if [%SPARK_ENV_LOADED%] == [] ( rem Setting SPARK_SCALA_VERSION if not already set. -rem TODO: revisit for Scala 2.13 support -set SPARK_SCALA_VERSION=2.12 -rem if [%SPARK_SCALA_VERSION%] == [] ( -rem set SCALA_VERSION_1=2.12 -rem set SCALA_VERSION_2=2.11 -rem -rem set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% -rem set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% -rem set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables -rem if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( -rem echo "Presence of build for multiple Scala versions detected (%ASSEMBLY_DIR1% and %ASSEMBLY_DIR2%)." -rem echo "Remove one of them or, set SPARK_SCALA_VERSION=%SCALA_VERSION_1% in %SPARK_ENV_CMD%." -rem echo "Visit %ENV_VARIABLE_DOC% for more details about setting environment variables in spark-env.cmd." -rem echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." -rem exit 1 -rem ) -rem if exist %ASSEMBLY_DIR1% ( -rem set SPARK_SCALA_VERSION=%SCALA_VERSION_1% -rem ) else ( -rem set SPARK_SCALA_VERSION=%SCALA_VERSION_2% -rem ) -rem ) +if [%SPARK_SCALA_VERSION%] == [] ( + set SCALA_VERSION_1=2.13 + set SCALA_VERSION_2=2.12 + + set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% + set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% + set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables + if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( + echo "Presence of build for multiple Scala versions detected (%ASSEMBLY_DIR1% and %ASSEMBLY_DIR2%)." + echo "Remove one of them or, set SPARK_SCALA_VERSION=%SCALA_VERSION_1% in %SPARK_ENV_CMD%." + echo "Visit %ENV_VARIABLE_DOC% for more details about setting environment variables in spark-env.cmd." + echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." + exit 1 + ) + if exist %ASSEMBLY_DIR1% ( + set SPARK_SCALA_VERSION=%SCALA_VERSION_1% + ) else ( + set SPARK_SCALA_VERSION=%SCALA_VERSION_2% + ) +) exit /b 0 diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 107e7991c28bc..04adaeed7ac61 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -43,25 +43,23 @@ fi # Setting SPARK_SCALA_VERSION if not already set. -# TODO: revisit for Scala 2.13 support -export SPARK_SCALA_VERSION=2.12 -#if [ -z "$SPARK_SCALA_VERSION" ]; then -# SCALA_VERSION_1=2.12 -# SCALA_VERSION_2=2.11 -# -# ASSEMBLY_DIR_1="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_1}" -# ASSEMBLY_DIR_2="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_2}" -# ENV_VARIABLE_DOC="https://spark.apache.org/docs/latest/configuration.html#environment-variables" -# if [[ -d "$ASSEMBLY_DIR_1" && -d "$ASSEMBLY_DIR_2" ]]; then -# echo "Presence of build for multiple Scala versions detected ($ASSEMBLY_DIR_1 and $ASSEMBLY_DIR_2)." 1>&2 -# echo "Remove one of them or, export SPARK_SCALA_VERSION=$SCALA_VERSION_1 in ${SPARK_ENV_SH}." 1>&2 -# echo "Visit ${ENV_VARIABLE_DOC} for more details about setting environment variables in spark-env.sh." 1>&2 -# exit 1 -# fi -# -# if [[ -d "$ASSEMBLY_DIR_1" ]]; then -# export SPARK_SCALA_VERSION=${SCALA_VERSION_1} -# else -# export SPARK_SCALA_VERSION=${SCALA_VERSION_2} -# fi -#fi +if [ -z "$SPARK_SCALA_VERSION" ]; then + SCALA_VERSION_1=2.13 + SCALA_VERSION_2=2.12 + + ASSEMBLY_DIR_1="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_1}" + ASSEMBLY_DIR_2="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_2}" + ENV_VARIABLE_DOC="https://spark.apache.org/docs/latest/configuration.html#environment-variables" + if [[ -d "$ASSEMBLY_DIR_1" && -d "$ASSEMBLY_DIR_2" ]]; then + echo "Presence of build for multiple Scala versions detected ($ASSEMBLY_DIR_1 and $ASSEMBLY_DIR_2)." 1>&2 + echo "Remove one of them or, export SPARK_SCALA_VERSION=$SCALA_VERSION_1 in ${SPARK_ENV_SH}." 1>&2 + echo "Visit ${ENV_VARIABLE_DOC} for more details about setting environment variables in spark-env.sh." 1>&2 + exit 1 + fi + + if [[ -d "$ASSEMBLY_DIR_1" ]]; then + export SPARK_SCALA_VERSION=${SCALA_VERSION_1} + else + export SPARK_SCALA_VERSION=${SCALA_VERSION_2} + fi +fi diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 3ae4633c79b04..778fd46b91fa1 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -235,20 +235,17 @@ String getScalaVersion() { return scala; } String sparkHome = getSparkHome(); - // TODO: revisit for Scala 2.13 support File scala212 = new File(sparkHome, "launcher/target/scala-2.12"); - // File scala211 = new File(sparkHome, "launcher/target/scala-2.11"); - // checkState(!scala212.isDirectory() || !scala211.isDirectory(), - // "Presence of build for multiple Scala versions detected.\n" + - // "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); - // if (scala212.isDirectory()) { - // return "2.12"; - // } else { - // checkState(scala211.isDirectory(), "Cannot find any build directories."); - // return "2.11"; - // } - checkState(scala212.isDirectory(), "Cannot find any build directories."); - return "2.12"; + File scala213 = new File(sparkHome, "launcher/target/scala-2.13"); + checkState(!scala212.isDirectory() || !scala213.isDirectory(), + "Presence of build for multiple Scala versions detected.\n" + + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); + if (scala213.isDirectory()) { + return "2.13"; + } else { + checkState(scala212.isDirectory(), "Cannot find any build directories."); + return "2.12"; + } } String getSparkHome() { From f9f18673dca16c78a77c3386a1629136031bf2ba Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 25 Jul 2020 10:16:01 -0700 Subject: [PATCH 279/384] [SPARK-32436][CORE] Initialize numNonEmptyBlocks in HighlyCompressedMapStatus.readExternal ### What changes were proposed in this pull request? This PR aims to initialize `numNonEmptyBlocks` in `HighlyCompressedMapStatus.readExternal`. In Scala 2.12, this is initialized to `-1` via the following. ```scala protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only ``` ### Why are the changes needed? In Scala 2.13, this causes several UT failures because `HighlyCompressedMapStatus.readExternal` doesn't initialize this field. The following is one example. - org.apache.spark.scheduler.MapStatusSuite ``` MapStatusSuite: - compressSize - decompressSize *** RUN ABORTED *** java.lang.NoSuchFieldError: numNonEmptyBlocks at org.apache.spark.scheduler.HighlyCompressedMapStatus.(MapStatus.scala:181) at org.apache.spark.scheduler.HighlyCompressedMapStatus$.apply(MapStatus.scala:281) at org.apache.spark.scheduler.MapStatus$.apply(MapStatus.scala:73) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$8(MapStatusSuite.scala:64) at scala.runtime.java8.JFunction1$mcVD$sp.apply(JFunction1$mcVD$sp.scala:18) at scala.collection.immutable.List.foreach(List.scala:333) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$7(MapStatusSuite.scala:61) at scala.runtime.java8.JFunction1$mcVJ$sp.apply(JFunction1$mcVJ$sp.scala:18) at scala.collection.immutable.List.foreach(List.scala:333) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$6(MapStatusSuite.scala:60) ... ``` ### Does this PR introduce _any_ user-facing change? No. This is a private class. ### How was this patch tested? 1. Pass the GitHub Action or Jenkins with the existing tests. 2. Test with Scala-2.13 with `MapStatusSuite`. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.scheduler.MapStatusSuite ... MapStatusSuite: - compressSize - decompressSize - MapStatus should never report non-empty blocks' sizes as 0 - large tasks should use org.apache.spark.scheduler.HighlyCompressedMapStatus - HighlyCompressedMapStatus: estimated size should be the average non-empty block size - SPARK-22540: ensure HighlyCompressedMapStatus calculates correct avgSize - RoaringBitmap: runOptimize succeeded - RoaringBitmap: runOptimize failed - Blocks which are bigger than SHUFFLE_ACCURATE_BLOCK_THRESHOLD should not be underestimated. - SPARK-21133 HighlyCompressedMapStatus#writeExternal throws NPE Run completed in 7 seconds, 971 milliseconds. Total number of tests run: 10 Suites: completed 2, aborted 0 Tests: succeeded 10, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes #29231 from dongjoon-hyun/SPARK-32436. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 757e8d31bf5d0..cfc2e141290c4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -217,6 +217,7 @@ private[spark] class HighlyCompressedMapStatus private ( override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) + numNonEmptyBlocks = -1 // SPARK-32436 Scala 2.13 doesn't initialize this during deserialization emptyBlocks = new RoaringBitmap() emptyBlocks.deserialize(in) avgSize = in.readLong() From 80e8898158616f4d75cb9bb6b76645e6f6769c78 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 25 Jul 2020 10:52:55 -0700 Subject: [PATCH 280/384] [SPARK-32438][CORE][TESTS] Use HashMap.withDefaultValue in RDDSuite ### What changes were proposed in this pull request? Since Scala 2.13, `HashMap` is changed to become a final in the future and `.withDefault` is recommended. This PR aims to use `HashMap.withDefaultValue` instead of overriding manually in the test case. - https://www.scala-lang.org/api/current/scala/collection/mutable/HashMap.html ```scala deprecatedInheritance(message = "HashMap wil be made final; use .withDefault for the common use case of computing a default value", since = "2.13.0") ``` ### Why are the changes needed? In Scala 2.13, the existing code causes a failure because the default value function doesn't work correctly. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.RDDSuite - aggregate *** FAILED *** org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 61.0 failed 1 times, most recent failure: Lost task 0.0 in stage 61.0 (TID 198, localhost, executor driver): java.util.NoSuchElementException: key not found: a ``` ### Does this PR introduce _any_ user-facing change? No. This is a test case change. ### How was this patch tested? 1. **Scala 2.12:** Pass the Jenkins or GitHub with the existing tests. 2. **Scala 2.13**: Manually do the following. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.RDDSuite ... Tests: succeeded 72, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes #29235 from dongjoon-hyun/SPARK-32438. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 79f9c1396c87b..8962fd6740bf6 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -237,10 +237,8 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { test("aggregate") { val pairs = sc.makeRDD(Seq(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) - type StringMap = HashMap[String, Int] - val emptyMap = new StringMap { - override def default(key: String): Int = 0 - } + type StringMap = scala.collection.mutable.Map[String, Int] + val emptyMap = HashMap[String, Int]().withDefaultValue(0).asInstanceOf[StringMap] val mergeElement: (StringMap, (String, Int)) => StringMap = (map, pair) => { map(pair._1) += pair._2 map From d1301af4ebd20cfc826503c8c4e8a535834740c3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 25 Jul 2020 10:58:25 -0700 Subject: [PATCH 281/384] [SPARK-32437][CORE][FOLLOWUP] Update dependency manifest for RoaringBitmap 0.9.0 --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 4 ++-- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 4 ++-- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 0c60231d0bd01..eb01e60557ccb 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -1,7 +1,7 @@ JLargeArrays/1.5//JLargeArrays-1.5.jar JTransforms/3.1//JTransforms-3.1.jar JavaEWAH/0.3.2//JavaEWAH-0.3.2.jar -RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +RoaringBitmap/0.9.0//RoaringBitmap-0.9.0.jar ST4/4.0.4//ST4-4.0.4.jar activation/1.1.1//activation-1.1.1.jar aircompressor/0.10//aircompressor-0.10.jar @@ -184,7 +184,7 @@ scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar scala-reflect/2.12.10//scala-reflect-2.12.10.jar scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar -shims/0.7.45//shims-0.7.45.jar +shims/0.9.0//shims-0.9.0.jar slf4j-api/1.7.30//slf4j-api-1.7.30.jar slf4j-log4j12/1.7.30//slf4j-log4j12-1.7.30.jar snakeyaml/1.24//snakeyaml-1.24.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 5151bbecd003e..11df0f9299fba 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -1,7 +1,7 @@ HikariCP/2.5.1//HikariCP-2.5.1.jar JLargeArrays/1.5//JLargeArrays-1.5.jar JTransforms/3.1//JTransforms-3.1.jar -RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +RoaringBitmap/0.9.0//RoaringBitmap-0.9.0.jar ST4/4.0.4//ST4-4.0.4.jar activation/1.1.1//activation-1.1.1.jar aircompressor/0.10//aircompressor-0.10.jar @@ -198,7 +198,7 @@ scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar scala-reflect/2.12.10//scala-reflect-2.12.10.jar scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar -shims/0.7.45//shims-0.7.45.jar +shims/0.9.0//shims-0.9.0.jar slf4j-api/1.7.30//slf4j-api-1.7.30.jar slf4j-log4j12/1.7.30//slf4j-log4j12-1.7.30.jar snakeyaml/1.24//snakeyaml-1.24.jar 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 ed7fe552e181e..69d42c759eb81 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -1,7 +1,7 @@ HikariCP/2.5.1//HikariCP-2.5.1.jar JLargeArrays/1.5//JLargeArrays-1.5.jar JTransforms/3.1//JTransforms-3.1.jar -RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +RoaringBitmap/0.9.0//RoaringBitmap-0.9.0.jar ST4/4.0.4//ST4-4.0.4.jar accessors-smart/1.2//accessors-smart-1.2.jar activation/1.1.1//activation-1.1.1.jar @@ -213,7 +213,7 @@ scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar scala-reflect/2.12.10//scala-reflect-2.12.10.jar scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar -shims/0.7.45//shims-0.7.45.jar +shims/0.9.0//shims-0.9.0.jar slf4j-api/1.7.30//slf4j-api-1.7.30.jar slf4j-log4j12/1.7.30//slf4j-log4j12-1.7.30.jar snakeyaml/1.24//snakeyaml-1.24.jar From 147022a5c6e4066db00b82bf9bf29d430a49bd4e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 25 Jul 2020 15:54:21 -0700 Subject: [PATCH 282/384] [SPARK-32440][CORE][TESTS] Make BlockManagerSuite robust from Scala object size difference ### What changes were proposed in this pull request? This PR aims to increase the memory parameter in `BlockManagerSuite`'s worker decommission test cases. ### Why are the changes needed? Scala 2.13 generates different Java objects and this affects Spark's `SizeEstimator/SizeTracker/SizeTrackingVector`. This causes UT failures like the following. If we decrease the values, those test cases fails in Scala 2.12, too. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.storage.BlockManagerSuite ... - test decommission block manager should not be part of peers *** FAILED *** 0 did not equal 2 (BlockManagerSuite.scala:1869) - test decommissionRddCacheBlocks should offload all cached blocks *** FAILED *** 0 did not equal 2 (BlockManagerSuite.scala:1884) ... Tests: succeeded 81, failed 2, canceled 0, ignored 0, pending 0 *** 2 TESTS FAILED *** ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.storage.BlockManagerSuite ... Tests: succeeded 83, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes #29238 from dongjoon-hyun/SPARK-32440. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/storage/BlockManagerSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 5588669c9f986..2a3d2d7f86a7e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1857,9 +1857,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val exec1 = "exec1" val exec2 = "exec2" val exec3 = "exec3" - val store1 = makeBlockManager(800, exec1) - val store2 = makeBlockManager(800, exec2) - val store3 = makeBlockManager(800, exec3) + val store1 = makeBlockManager(1000, exec1) + val store2 = makeBlockManager(1000, exec2) + val store3 = makeBlockManager(1000, exec3) assert(master.getPeers(store3.blockManagerId).map(_.executorId).toSet === Set(exec1, exec2)) @@ -1874,9 +1874,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("test decommissionRddCacheBlocks should offload all cached blocks") { - val store1 = makeBlockManager(800, "exec1") - val store2 = makeBlockManager(800, "exec2") - val store3 = makeBlockManager(800, "exec3") + val store1 = makeBlockManager(1000, "exec1") + val store2 = makeBlockManager(1000, "exec2") + val store3 = makeBlockManager(1000, "exec3") val data = new Array[Byte](4) val blockId = rdd(0, 0) From 83ffef7ffb989e639eb722b83fa2dd5a54fe8457 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 25 Jul 2020 20:34:31 -0700 Subject: [PATCH 283/384] [SPARK-32441][BUILD][CORE] Update json4s to 3.7.0-M5 for Scala 2.13 ### What changes were proposed in this pull request? This PR aims to upgrade `json4s` to from 3.6.6 to 3.7.0-M5 for Scala 2.13 support at Apache Spark 3.1.0 on December. We will upgrade to the latest `json4s` around November. ### Why are the changes needed? `json4s` starts to support Scala 2.13 since v3.7.0-M4. - https://github.com/json4s/json4s/issues/660 - https://github.com/json4s/json4s/commit/b013af8e757ee15c15a6a1f19c672f7e7044a868 Old `json4s` causes many UT failures with `NoSuchMethodException`. ```scala Cause: java.lang.NoSuchMethodException: scala.collection.immutable.Seq$.apply(scala.collection.Seq) at java.lang.Class.getMethod(Class.java:1786) ``` The following is one example. ```scala $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.executor.CoarseGrainedExecutorBackendSuite ... Tests: succeeded 4, failed 9, canceled 0, ignored 0, pending 0 *** 9 TESTS FAILED *** ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? 1. **Scala 2.12**: Pass the Jenkins or GitHub Action with the existing tests. 2. **Scala 2.13**: Do the following manually at least. ```scala $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.executor.CoarseGrainedExecutorBackendSuite ... Tests: succeeded 13, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes #29239 from dongjoon-hyun/SPARK-32441. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 8 ++++---- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 8 ++++---- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 8 ++++---- pom.xml | 2 +- 4 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index eb01e60557ccb..45c346fa5ecc4 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -129,10 +129,10 @@ jline/2.14.6//jline-2.14.6.jar joda-time/2.10.5//joda-time-2.10.5.jar jodd-core/3.5.2//jodd-core-3.5.2.jar jpam/1.1//jpam-1.1.jar -json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar -json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +json4s-ast_2.12/3.7.0-M5//json4s-ast_2.12-3.7.0-M5.jar +json4s-core_2.12/3.7.0-M5//json4s-core_2.12-3.7.0-M5.jar +json4s-jackson_2.12/3.7.0-M5//json4s-jackson_2.12-3.7.0-M5.jar +json4s-scalap_2.12/3.7.0-M5//json4s-scalap_2.12-3.7.0-M5.jar jsp-api/2.1//jsp-api-2.1.jar jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 11df0f9299fba..4dc36d65f8632 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -144,10 +144,10 @@ joda-time/2.10.5//joda-time-2.10.5.jar jodd-core/3.5.2//jodd-core-3.5.2.jar jpam/1.1//jpam-1.1.jar json/1.8//json-1.8.jar -json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar -json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +json4s-ast_2.12/3.7.0-M5//json4s-ast_2.12-3.7.0-M5.jar +json4s-core_2.12/3.7.0-M5//json4s-core_2.12-3.7.0-M5.jar +json4s-jackson_2.12/3.7.0-M5//json4s-jackson_2.12-3.7.0-M5.jar +json4s-scalap_2.12/3.7.0-M5//json4s-scalap_2.12-3.7.0-M5.jar jsp-api/2.1//jsp-api-2.1.jar jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar 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 69d42c759eb81..40feed25d3a6d 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -142,10 +142,10 @@ jodd-core/3.5.2//jodd-core-3.5.2.jar jpam/1.1//jpam-1.1.jar json-smart/2.3//json-smart-2.3.jar json/1.8//json-1.8.jar -json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar -json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +json4s-ast_2.12/3.7.0-M5//json4s-ast_2.12-3.7.0-M5.jar +json4s-core_2.12/3.7.0-M5//json4s-core_2.12-3.7.0-M5.jar +json4s-jackson_2.12/3.7.0-M5//json4s-jackson_2.12-3.7.0-M5.jar +json4s-scalap_2.12/3.7.0-M5//json4s-scalap_2.12-3.7.0-M5.jar jsp-api/2.1//jsp-api-2.1.jar jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar diff --git a/pom.xml b/pom.xml index e1be59c790107..13a1218173764 100644 --- a/pom.xml +++ b/pom.xml @@ -861,7 +861,7 @@ org.json4s json4s-jackson_${scala.binary.version} - 3.6.6 + 3.7.0-M5 com.fasterxml.jackson.core From 86ead044e3789b3291a38ec2142cbb343d1290c1 Mon Sep 17 00:00:00 2001 From: Itsuki Toyota Date: Sun, 26 Jul 2020 09:12:43 -0500 Subject: [PATCH 284/384] =?UTF-8?q?[SPARK-32428][EXAMPLES]=20Make=20Binary?= =?UTF-8?q?ClassificationMetricsExample=20cons=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …istently print the metrics on driver's stdout ### What changes were proposed in this pull request? Call collect on RDD before calling foreach so that it sends the result to the driver node and print it on this node's stdout. ### Why are the changes needed? Some RDDs in this example (e.g., precision, recall) call println without calling collect. If the job is under local mode, it sends the data to the driver node and prints the metrics on the driver's stdout. However if the job is under cluster mode, the job prints the metrics on the executor's stdout. It seems inconsistent compared to the other metrics nothing to do with RDD (e.g., auPRC, auROC) since these metrics always output the result on the driver's stdout. All of the metrics should output its result on the driver's stdout. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? This is example code. It doesn't have any tests. Closes #29222 from titsuki/SPARK-32428. Authored-by: Itsuki Toyota Signed-off-by: Sean Owen --- .../mllib/BinaryClassificationMetricsExample.scala | 8 ++++---- .../spark/examples/mllib/ChiSqSelectorExample.scala | 2 +- .../spark/examples/mllib/ElementwiseProductExample.scala | 4 ++-- .../apache/spark/examples/mllib/NormalizerExample.scala | 4 ++-- .../spark/examples/mllib/StandardScalerExample.scala | 4 ++-- .../org/apache/spark/examples/mllib/TFIDFExample.scala | 4 ++-- 6 files changed, 13 insertions(+), 13 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala index c6312d71cc912..a606cc4218778 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala @@ -59,13 +59,13 @@ object BinaryClassificationMetricsExample { // Precision by threshold val precision = metrics.precisionByThreshold - precision.foreach { case (t, p) => + precision.collect.foreach { case (t, p) => println(s"Threshold: $t, Precision: $p") } // Recall by threshold val recall = metrics.recallByThreshold - recall.foreach { case (t, r) => + recall.collect.foreach { case (t, r) => println(s"Threshold: $t, Recall: $r") } @@ -74,13 +74,13 @@ object BinaryClassificationMetricsExample { // F-measure val f1Score = metrics.fMeasureByThreshold - f1Score.foreach { case (t, f) => + f1Score.collect.foreach { case (t, f) => println(s"Threshold: $t, F-score: $f, Beta = 1") } val beta = 0.5 val fScore = metrics.fMeasureByThreshold(beta) - f1Score.foreach { case (t, f) => + fScore.collect.foreach { case (t, f) => println(s"Threshold: $t, F-score: $f, Beta = 0.5") } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala index 5e400b7d715b4..6ed59a3366d93 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala @@ -53,7 +53,7 @@ object ChiSqSelectorExample { // $example off$ println("filtered data: ") - filteredData.foreach(x => println(x)) + filteredData.collect.foreach(x => println(x)) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala index 1855058bb4f3c..d6ec678b422f9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala @@ -45,10 +45,10 @@ object ElementwiseProductExample { // $example off$ println("transformedData: ") - transformedData.foreach(x => println(x)) + transformedData.collect.foreach(x => println(x)) println("transformedData2: ") - transformedData2.foreach(x => println(x)) + transformedData2.collect.foreach(x => println(x)) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala index b3a9604c2be3e..b1cad7bf8125f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala @@ -46,10 +46,10 @@ object NormalizerExample { // $example off$ println("data1: ") - data1.foreach(x => println(x)) + data1.collect.foreach(x => println(x)) println("data2: ") - data2.foreach(x => println(x)) + data2.collect.foreach(x => println(x)) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala index 769fc17b3dc65..66a608ced0f83 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala @@ -49,10 +49,10 @@ object StandardScalerExample { // $example off$ println("data1: ") - data1.foreach(x => println(x)) + data1.collect.foreach(x => println(x)) println("data2: ") - data2.foreach(x => println(x)) + data2.collect.foreach(x => println(x)) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala index a5bdcd8f2ed32..14b2a2015c241 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala @@ -55,10 +55,10 @@ object TFIDFExample { // $example off$ println("tfidf: ") - tfidf.foreach(x => println(x)) + tfidf.collect.foreach(x => println(x)) println("tfidfIgnore: ") - tfidfIgnore.foreach(x => println(x)) + tfidfIgnore.collect.foreach(x => println(x)) sc.stop() } From 7e0c5b3b539f208ef795d6848c34b52db2076a5c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 26 Jul 2020 07:54:30 -0700 Subject: [PATCH 285/384] [SPARK-32442][CORE][TESTS] Fix TaskSetManagerSuite by hiding `o.a.s.FakeSchedulerBackend` ### What changes were proposed in this pull request? There exists two `FakeSchedulerBackend` classes. ``` $ git grep "class FakeSchedulerBackend" core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala:private class FakeSchedulerBackend( core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala:class FakeSchedulerBackend extends SchedulerBackend { ``` This PR aims to hide the following at `TaskSetManagerSuite`. ```scala import org.apache.spark.{FakeSchedulerBackend => _, _} ``` ### Why are the changes needed? Although `TaskSetManagerSuite` is inside `org.apache.spark.scheduler` package, `import org.apache.spark._` makes Scala 2.13 confused and causes 4 UT failures. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.scheduler.TaskSetManagerSuite ... Tests: succeeded 48, failed 4, canceled 0, ignored 0, pending 0 *** 4 TESTS FAILED *** ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - **Scala 2.12**: Pass the Jenkins or GitHub Action - **Scala 2.13**: Pass the following manually. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.scheduler.TaskSetManagerSuite ... Tests: succeeded 52, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes #29240 from dongjoon-hyun/SPARK-32442. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index ae51b55aa1373..0a98030a56edc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.Assertions._ import org.scalatest.PrivateMethodTester import org.scalatest.concurrent.Eventually -import org.apache.spark._ +import org.apache.spark.{FakeSchedulerBackend => _, _} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.SKIP_VALIDATE_CORES_TESTING From 4f79b9fffd592a2f24e5e759d726232134bda409 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 26 Jul 2020 15:55:48 -0700 Subject: [PATCH 286/384] [SPARK-32447][CORE] Use python3 by default in pyspark and find-spark-home scripts ### What changes were proposed in this pull request? This PR aims to use `python3` instead of `python` inside `bin/pyspark`, `bin/find-spark-home` and `bin/find-spark-home.cmd` script. ``` $ git diff master --stat bin/find-spark-home | 4 ++-- bin/find-spark-home.cmd | 4 ++-- bin/pyspark | 4 ++-- ``` ### Why are the changes needed? According to [PEP 394](https://www.python.org/dev/peps/pep-0394/), we have four different cases for `python` while `python3` will be there always. ``` - Distributors may choose to set the behavior of the python command as follows: python2, python3, not provide python command, allow python to be configurable by an end user or a system administrator. ``` Moreover, these scripts already depend on `find_spark_home.py` which is using `#!/usr/bin/env python3`. ``` FIND_SPARK_HOME_PYTHON_SCRIPT="$(cd "$(dirname "$0")"; pwd)/find_spark_home.py" ``` ### Does this PR introduce _any_ user-facing change? No. Apache Spark 3.1 already drops Python 2.7 via SPARK-32138 . ### How was this patch tested? Pass the Jenkins or GitHub Action. Closes #29246 from dongjoon-hyun/SPARK-FIND-SPARK-HOME. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- bin/find-spark-home | 4 ++-- bin/find-spark-home.cmd | 4 ++-- bin/pyspark | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/bin/find-spark-home b/bin/find-spark-home index 617dbaa4fff86..462b538b00a04 100755 --- a/bin/find-spark-home +++ b/bin/find-spark-home @@ -33,9 +33,9 @@ elif [ ! -f "$FIND_SPARK_HOME_PYTHON_SCRIPT" ]; then export SPARK_HOME="$(cd "$(dirname "$0")"/..; pwd)" else # We are pip installed, use the Python script to resolve a reasonable SPARK_HOME - # Default to standard python interpreter unless told otherwise + # Default to standard python3 interpreter unless told otherwise if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then - PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"python"}" + PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"python3"}" fi export SPARK_HOME=$($PYSPARK_DRIVER_PYTHON "$FIND_SPARK_HOME_PYTHON_SCRIPT") fi diff --git a/bin/find-spark-home.cmd b/bin/find-spark-home.cmd index 6025f67c38de4..f795d146d49c7 100644 --- a/bin/find-spark-home.cmd +++ b/bin/find-spark-home.cmd @@ -20,8 +20,8 @@ rem rem Path to Python script finding SPARK_HOME set FIND_SPARK_HOME_PYTHON_SCRIPT=%~dp0find_spark_home.py -rem Default to standard python interpreter unless told otherwise -set PYTHON_RUNNER=python +rem Default to standard python3 interpreter unless told otherwise +set PYTHON_RUNNER=python3 rem If PYSPARK_DRIVER_PYTHON is set, it overwrites the python version if not "x%PYSPARK_DRIVER_PYTHON%"=="x" ( set PYTHON_RUNNER=%PYSPARK_DRIVER_PYTHON% diff --git a/bin/pyspark b/bin/pyspark index ad4132fb59eb0..463a2dcfc7e6c 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -37,9 +37,9 @@ if [[ -n "$IPYTHON" || -n "$IPYTHON_OPTS" ]]; then exit 1 fi -# Default to standard python interpreter unless told otherwise +# Default to standard python3 interpreter unless told otherwise if [[ -z "$PYSPARK_PYTHON" ]]; then - PYSPARK_PYTHON=python + PYSPARK_PYTHON=python3 fi if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then PYSPARK_DRIVER_PYTHON=$PYSPARK_PYTHON From 70ac594bb3aa74a97f942796115ec86261ef7828 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 27 Jul 2020 10:43:32 +0900 Subject: [PATCH 287/384] [SPARK-32450][PYTHON] Upgrade pycodestyle to v2.6.0 ### What changes were proposed in this pull request? This patch upgrades pycodestyle from v2.4.0 to v2.6.0. The changes at each release: 2.5.0: https://pycodestyle.pycqa.org/en/latest/developer.html#id3 2.6.0a1: https://pycodestyle.pycqa.org/en/latest/developer.html#a1-2020-04-23 2.6.0: https://pycodestyle.pycqa.org/en/latest/developer.html#id2 Changes: Dropped Python 2.6 and 3.3 support, added Python 3.7 and 3.8 support... ### Why are the changes needed? Including bug fixes and newer Python version support. ### Does this PR introduce _any_ user-facing change? No, dev only. ### How was this patch tested? Ran `dev/lint-python` locally. Closes #29249 from viirya/upgrade-pycodestyle. Authored-by: Liang-Chi Hsieh Signed-off-by: HyukjinKwon --- dev/lint-python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/lint-python b/dev/lint-python index 1fddbfa64b32c..9dfd296383a81 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -20,7 +20,7 @@ FLAKE8_BUILD="flake8" MINIMUM_FLAKE8="3.5.0" PYCODESTYLE_BUILD="pycodestyle" -MINIMUM_PYCODESTYLE="2.4.0" +MINIMUM_PYCODESTYLE="2.6.0" SPHINX_BUILD="sphinx-build" From 8153f56286b02f3d290340289d033e81b505fe4a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 26 Jul 2020 18:51:25 -0700 Subject: [PATCH 288/384] [SPARK-32451][R] Support Apache Arrow 1.0.0 ### What changes were proposed in this pull request? Currently, `GitHub Action` is broken due to `SparkR UT failure` from new Apache Arrow 1.0.0. ![Screen Shot 2020-07-26 at 5 12 08 PM](https://user-images.githubusercontent.com/9700541/88492923-3409f080-cf63-11ea-8fea-6051298c2dd0.png) This PR aims to update R code according to Apache Arrow 1.0.0 recommendation to pass R unit tests. An alternative is pinning Apache Arrow version at 0.17.1 and I also created a PR to compare with this. - https://github.com/apache/spark/pull/29251 ### Why are the changes needed? - Apache Spark 3.1 supports Apache Arrow 0.15.1+. - Apache Arrow released 1.0.0 a few days ago and this causes GitHub Action SparkR test failures due to warnings. - https://github.com/apache/spark/commits/master ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - [x] Pass the Jenkins (https://github.com/apache/spark/pull/29252#issuecomment-664067492) - [x] Pass the GitHub (https://github.com/apache/spark/runs/912656867) Closes #29252 from dongjoon-hyun/SPARK-ARROW. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- R/pkg/R/DataFrame.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 15b3ce2935427..4d38f979c4103 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1233,7 +1233,7 @@ setMethod("collect", port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout) output <- tryCatch({ doServerAuth(conn, authSecret) - arrowTable <- arrow::read_arrow(readRaw(conn)) + arrowTable <- arrow::read_ipc_stream(readRaw(conn)) # Arrow drops `as_tibble` since 0.14.0, see ARROW-5190. if (exists("as_tibble", envir = asNamespace("arrow"))) { as.data.frame(arrow::as_tibble(arrowTable), stringsAsFactors = stringsAsFactors) From 13c64c298016eb3882ed20a6f6c60f1ea3988b3b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 26 Jul 2020 19:25:41 -0700 Subject: [PATCH 289/384] [SPARK-32448][K8S][TESTS] Use single version for exec-maven-plugin/scalatest-maven-plugin ### What changes were proposed in this pull request? Two different versions are used for the same artifacts, `exec-maven-plugin` and `scalatest-maven-plugin`. This PR aims to use the same versions for `exec-maven-plugin` and `scalatest-maven-plugin`. In addition, this PR removes `scala-maven-plugin.version` from `K8s` integration suite because it's unused. ### Why are the changes needed? This will prevent the mistake which upgrades only one place and forgets the others. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins K8S IT. Closes #29248 from dongjoon-hyun/SPARK-32448. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 6 ++++-- resource-managers/kubernetes/integration-tests/pom.xml | 3 --- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 13a1218173764..065b8f2a9abc5 100644 --- a/pom.xml +++ b/pom.xml @@ -116,6 +116,7 @@ ${java.version} ${java.version} 3.6.3 + 1.6.0 spark 1.7.30 1.2.17 @@ -162,6 +163,7 @@ 3.2.2 2.12.10 2.12 + 2.0.0 --test true @@ -2540,7 +2542,7 @@ org.scalatest scalatest-maven-plugin - 2.0.0 + ${scalatest-maven-plugin.version} ${project.build.directory}/surefire-reports @@ -2682,7 +2684,7 @@ org.codehaus.mojo exec-maven-plugin - 1.6.0 + ${exec-maven-plugin.version} org.apache.maven.plugins diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index d1e00cc0b5b10..4a55ead38aae2 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -27,11 +27,8 @@ spark-kubernetes-integration-tests_2.12 1.3.0 - 1.4.0 4.9.2 - 3.2.2 - 1.0 kubernetes-integration-tests From 01cf8a4ce8a986364526588954ffc85da6c9e390 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Mon, 27 Jul 2020 04:51:32 +0000 Subject: [PATCH 290/384] [SPARK-32383][SQL] Preserve hash join (BHJ and SHJ) stream side ordering ### What changes were proposed in this pull request? Currently `BroadcastHashJoinExec` and `ShuffledHashJoinExec` do not preserve children output ordering information (inherit from `SparkPlan.outputOrdering`, which is Nil). This can add unnecessary sort in complex queries involved multiple joins. Example: ``` withSQLConf( SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50") { val df1 = spark.range(100).select($"id".as("k1")) val df2 = spark.range(100).select($"id".as("k2")) val df3 = spark.range(3).select($"id".as("k3")) val df4 = spark.range(100).select($"id".as("k4")) val plan = df1.join(df2, $"k1" === $"k2") .join(df3, $"k1" === $"k3") .join(df4, $"k1" === $"k4") .queryExecution .executedPlan } ``` Current physical plan (extra sort on `k1` before top sort merge join): ``` *(9) SortMergeJoin [k1#220L], [k4#232L], Inner :- *(6) Sort [k1#220L ASC NULLS FIRST], false, 0 : +- *(6) BroadcastHashJoin [k1#220L], [k3#228L], Inner, BuildRight : :- *(6) SortMergeJoin [k1#220L], [k2#224L], Inner : : :- *(2) Sort [k1#220L ASC NULLS FIRST], false, 0 : : : +- Exchange hashpartitioning(k1#220L, 5), true, [id=#128] : : : +- *(1) Project [id#218L AS k1#220L] : : : +- *(1) Range (0, 100, step=1, splits=2) : : +- *(4) Sort [k2#224L ASC NULLS FIRST], false, 0 : : +- Exchange hashpartitioning(k2#224L, 5), true, [id=#134] : : +- *(3) Project [id#222L AS k2#224L] : : +- *(3) Range (0, 100, step=1, splits=2) : +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])), [id=#141] : +- *(5) Project [id#226L AS k3#228L] : +- *(5) Range (0, 3, step=1, splits=2) +- *(8) Sort [k4#232L ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(k4#232L, 5), true, [id=#148] +- *(7) Project [id#230L AS k4#232L] +- *(7) Range (0, 100, step=1, splits=2) ``` Ideal physical plan (no extra sort on `k1` before top sort merge join): ``` *(9) SortMergeJoin [k1#220L], [k4#232L], Inner :- *(6) BroadcastHashJoin [k1#220L], [k3#228L], Inner, BuildRight : :- *(6) SortMergeJoin [k1#220L], [k2#224L], Inner : : :- *(2) Sort [k1#220L ASC NULLS FIRST], false, 0 : : : +- Exchange hashpartitioning(k1#220L, 5), true, [id=#127] : : : +- *(1) Project [id#218L AS k1#220L] : : : +- *(1) Range (0, 100, step=1, splits=2) : : +- *(4) Sort [k2#224L ASC NULLS FIRST], false, 0 : : +- Exchange hashpartitioning(k2#224L, 5), true, [id=#133] : : +- *(3) Project [id#222L AS k2#224L] : : +- *(3) Range (0, 100, step=1, splits=2) : +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])), [id=#140] : +- *(5) Project [id#226L AS k3#228L] : +- *(5) Range (0, 3, step=1, splits=2) +- *(8) Sort [k4#232L ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(k4#232L, 5), true, [id=#146] +- *(7) Project [id#230L AS k4#232L] +- *(7) Range (0, 100, step=1, splits=2) ``` ### Why are the changes needed? To avoid unnecessary sort in query, and it has most impact when users read sorted bucketed table. Though the unnecessary sort is operating on already sorted data, it would have obvious negative impact on IO and query run time if the data is large and external sorting happens. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit test in `JoinSuite`. Closes #29181 from c21/ordering. Authored-by: Cheng Su Signed-off-by: Wenchen Fan --- .../spark/sql/execution/joins/HashJoin.scala | 36 +++++++++++++++- .../org/apache/spark/sql/JoinSuite.scala | 43 +++++++++++++++++++ 2 files changed, 78 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 8d9ba54f6568d..fa3a55aa5ad94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -52,7 +52,41 @@ trait HashJoin extends BaseJoinExec { } } - override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning + override def outputPartitioning: Partitioning = buildSide match { + case BuildLeft => + joinType match { + case _: InnerLike | RightOuter => right.outputPartitioning + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building left side") + } + case BuildRight => + joinType match { + case _: InnerLike | LeftOuter | LeftSemi | LeftAnti | _: ExistenceJoin => + left.outputPartitioning + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building right side") + } + } + + override def outputOrdering: Seq[SortOrder] = buildSide match { + case BuildLeft => + joinType match { + case _: InnerLike | RightOuter => right.outputOrdering + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building left side") + } + case BuildRight => + joinType match { + case _: InnerLike | LeftOuter | LeftSemi | LeftAnti | _: ExistenceJoin => + left.outputOrdering + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building right side") + } + } protected lazy val (buildPlan, streamedPlan) = buildSide match { case BuildLeft => (left, right) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index b4f626270cfc9..c42d4c6f74a93 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -1104,4 +1104,47 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan }) } } + + test("SPARK-32383: Preserve hash join (BHJ and SHJ) stream side ordering") { + val df1 = spark.range(100).select($"id".as("k1")) + val df2 = spark.range(100).select($"id".as("k2")) + val df3 = spark.range(3).select($"id".as("k3")) + val df4 = spark.range(100).select($"id".as("k4")) + + // Test broadcast hash join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50") { + Seq("inner", "left_outer").foreach(joinType => { + val plan = df1.join(df2, $"k1" === $"k2", joinType) + .join(df3, $"k1" === $"k3", joinType) + .join(df4, $"k1" === $"k4", joinType) + .queryExecution + .executedPlan + assert(plan.collect { case _: SortMergeJoinExec => true }.size === 2) + assert(plan.collect { case _: BroadcastHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join + assert(plan.collect { case _: SortExec => true }.size === 3) + }) + } + + // Test shuffled hash join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { + val df3 = spark.range(10).select($"id".as("k3")) + + Seq("inner", "left_outer").foreach(joinType => { + val plan = df1.join(df2, $"k1" === $"k2", joinType) + .join(df3, $"k1" === $"k3", joinType) + .join(df4, $"k1" === $"k4", joinType) + .queryExecution + .executedPlan + assert(plan.collect { case _: SortMergeJoinExec => true }.size === 2) + assert(plan.collect { case _: ShuffledHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join + assert(plan.collect { case _: SortExec => true }.size === 3) + }) + } + } } From bfa5d57bbd9b190724c584c43075c49c2a7e507c Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 27 Jul 2020 14:21:15 +0900 Subject: [PATCH 291/384] [SPARK-32452][R][SQL] Bump up the minimum Arrow version as 1.0.0 in SparkR ### What changes were proposed in this pull request? This PR proposes to set the minimum Arrow version as 1.0.0 to minimise the maintenance overhead and keep the minimal version up to date. Other required changes to support 1.0.0 were already made in SPARK-32451. ### Why are the changes needed? R side, people rather aggressively encourage people to use the latest version, and SparkR vectorization is very experimental that was added from Spark 3.0. Also, we're technically not testing old Arrow versions in SparkR for now. ### Does this PR introduce _any_ user-facing change? Yes, users wouldn't be able to use SparkR with old Arrow. ### How was this patch tested? GitHub Actions and AppVeyor are already testing them. Closes #29253 from HyukjinKwon/SPARK-32452. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/pkg/DESCRIPTION | 2 +- docs/sparkr.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index c5c08bd7a0636..2047f0d75ca18 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -23,7 +23,7 @@ Suggests: testthat, e1071, survival, - arrow (>= 0.15.1) + arrow (>= 1.0.0) Collate: 'schema.R' 'generics.R' diff --git a/docs/sparkr.md b/docs/sparkr.md index d86fa86c89853..fa1bb1b851815 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -674,7 +674,7 @@ Rscript -e 'install.packages("arrow", repos="https://cloud.r-project.org/")' Please refer [the official documentation of Apache Arrow](https://arrow.apache.org/docs/r/) for more detials. Note that you must ensure that Arrow R package is installed and available on all cluster nodes. -The current supported minimum version is 0.15.1; however, this might change between the minor releases since Arrow optimization in SparkR is experimental. +The current supported minimum version is 1.0.0; however, this might change between the minor releases since Arrow optimization in SparkR is experimental. ## Enabling for Conversion to/from R DataFrame, `dapply` and `gapply` From 99f33ec30f04bb0f7b09c3c2abfc5d5b6af50599 Mon Sep 17 00:00:00 2001 From: SaurabhChawla Date: Mon, 27 Jul 2020 08:14:02 +0000 Subject: [PATCH 292/384] [SPARK-32234][FOLLOWUP][SQL] Update the description of utility method ### What changes were proposed in this pull request? As the part of this PR https://github.com/apache/spark/pull/29045 added the helper method. This PR is the FOLLOWUP PR to update the description of helper method. ### Why are the changes needed? For better readability and understanding of the code ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Since its only change of updating the description , So ran the Spark shell Closes #29232 from SaurabhChawla100/SPARK-32234-Desc. Authored-by: SaurabhChawla Signed-off-by: Wenchen Fan --- .../sql/execution/datasources/orc/OrcUtils.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index e10253989788b..072e670081d1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -207,10 +207,16 @@ object OrcUtils extends Logging { } /** - * @return Returns the result schema string based on the canPruneCols flag. - * resultSchemaString will be created using resultsSchema in case of - * canPruneCols is true and for canPruneCols as false value - * resultSchemaString will be created using the actual dataSchema. + * Returns the result schema to read from ORC file. In addition, It sets + * the schema string to 'orc.mapred.input.schema' so ORC reader can use later. + * + * @param canPruneCols Flag to decide whether pruned cols schema is send to resultSchema + * or to send the entire dataSchema to resultSchema. + * @param dataSchema Schema of the orc files. + * @param resultSchema Result data schema created after pruning cols. + * @param partitionSchema Schema of partitions. + * @param conf Hadoop Configuration. + * @return Returns the result schema as string. */ def orcResultSchemaString( canPruneCols: Boolean, From 6ab29b37cf1270b139bf93b821e7a6c2f6e27c6f Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 27 Jul 2020 17:49:21 +0900 Subject: [PATCH 293/384] [SPARK-32179][SPARK-32188][PYTHON][DOCS] Replace and redesign the documentation base ### What changes were proposed in this pull request? This PR proposes to redesign the PySpark documentation. I made a demo site to make it easier to review: https://hyukjin-spark.readthedocs.io/en/stable/reference/index.html. Here is the initial draft for the final PySpark docs shape: https://hyukjin-spark.readthedocs.io/en/latest/index.html. In more details, this PR proposes: 1. Use [pydata_sphinx_theme](https://github.com/pandas-dev/pydata-sphinx-theme) theme - [pandas](https://pandas.pydata.org/docs/) and [Koalas](https://koalas.readthedocs.io/en/latest/) use this theme. The CSS overwrite is ported from Koalas. The colours in the CSS were actually chosen by designers to use in Spark. 2. Use the Sphinx option to separate `source` and `build` directories as the documentation pages will likely grow. 3. Port current API documentation into the new style. It mimics Koalas and pandas to use the theme most effectively. One disadvantage of this approach is that you should list up APIs or classes; however, I think this isn't a big issue in PySpark since we're being conservative on adding APIs. I also intentionally listed classes only instead of functions in ML and MLlib to make it relatively easier to manage. ### Why are the changes needed? Often I hear the complaints, from the users, that current PySpark documentation is pretty messy to read - https://spark.apache.org/docs/latest/api/python/index.html compared other projects such as [pandas](https://pandas.pydata.org/docs/) and [Koalas](https://koalas.readthedocs.io/en/latest/). It would be nicer if we can make it more organised instead of just listing all classes, methods and attributes to make it easier to navigate. Also, the documentation has been there from almost the very first version of PySpark. Maybe it's time to update it. ### Does this PR introduce _any_ user-facing change? Yes, PySpark API documentation will be redesigned. ### How was this patch tested? Manually tested, and the demo site was made to show. Closes #29188 from HyukjinKwon/SPARK-32179. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .github/workflows/master.yml | 8 +- .gitignore | 1 + LICENSE | 2 +- dev/create-release/spark-rm/Dockerfile | 5 +- dev/lint-python | 18 +- dev/requirements.txt | 1 + dev/tox.ini | 2 +- docs/README.md | 7 +- docs/_plugins/copy_api_dirs.rb | 4 +- docs/img/spark-logo-reverse.png | Bin 0 -> 21547 bytes python/docs/Makefile | 4 +- python/docs/_static/pyspark.css | 90 --- python/docs/_static/pyspark.js | 99 ---- python/docs/_templates/layout.html | 6 - python/docs/index.rst | 53 -- python/docs/make2.bat | 4 +- python/docs/pyspark.ml.rst | 122 ---- python/docs/pyspark.mllib.rst | 99 ---- python/docs/pyspark.resource.rst | 11 - python/docs/pyspark.rst | 21 - python/docs/pyspark.sql.rst | 37 -- python/docs/pyspark.streaming.rst | 17 - .../docs/{ => source}/_static/copybutton.js | 0 python/docs/source/_static/css/pyspark.css | 94 +++ .../source/_templates/class_with_docs.rst | 79 +++ python/docs/{ => source}/conf.py | 51 +- python/docs/source/development/index.rst | 21 + python/docs/source/getting_started/index.rst | 22 + python/docs/source/index.rst | 32 ++ python/docs/source/migration_guide/index.rst | 22 + python/docs/source/reference/index.rst | 34 ++ python/docs/source/reference/pyspark.ml.rst | 363 ++++++++++++ .../docs/source/reference/pyspark.mllib.rst | 253 ++++++++ .../source/reference/pyspark.resource.rst | 38 ++ python/docs/source/reference/pyspark.rst | 275 +++++++++ python/docs/source/reference/pyspark.sql.rst | 542 ++++++++++++++++++ python/docs/source/reference/pyspark.ss.rst | 90 +++ .../source/reference/pyspark.streaming.rst | 130 +++++ python/docs/source/user_guide/index.rst | 22 + python/pyspark/sql/functions.py | 4 + python/pyspark/sql/readwriter.py | 17 + python/pyspark/sql/streaming.py | 13 + 42 files changed, 2129 insertions(+), 584 deletions(-) create mode 100644 docs/img/spark-logo-reverse.png delete mode 100644 python/docs/_static/pyspark.css delete mode 100644 python/docs/_static/pyspark.js delete mode 100644 python/docs/_templates/layout.html delete mode 100644 python/docs/index.rst delete mode 100644 python/docs/pyspark.ml.rst delete mode 100644 python/docs/pyspark.mllib.rst delete mode 100644 python/docs/pyspark.resource.rst delete mode 100644 python/docs/pyspark.rst delete mode 100644 python/docs/pyspark.sql.rst delete mode 100644 python/docs/pyspark.streaming.rst rename python/docs/{ => source}/_static/copybutton.js (100%) create mode 100644 python/docs/source/_static/css/pyspark.css create mode 100644 python/docs/source/_templates/class_with_docs.rst rename python/docs/{ => source}/conf.py (89%) create mode 100644 python/docs/source/development/index.rst create mode 100644 python/docs/source/getting_started/index.rst create mode 100644 python/docs/source/index.rst create mode 100644 python/docs/source/migration_guide/index.rst create mode 100644 python/docs/source/reference/index.rst create mode 100644 python/docs/source/reference/pyspark.ml.rst create mode 100644 python/docs/source/reference/pyspark.mllib.rst create mode 100644 python/docs/source/reference/pyspark.resource.rst create mode 100644 python/docs/source/reference/pyspark.rst create mode 100644 python/docs/source/reference/pyspark.sql.rst create mode 100644 python/docs/source/reference/pyspark.ss.rst create mode 100644 python/docs/source/reference/pyspark.streaming.rst create mode 100644 python/docs/source/user_guide/index.rst diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index fe01b92036377..29f98ed36e2ec 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -200,7 +200,9 @@ jobs: architecture: x64 - name: Install Python linter dependencies run: | - pip3 install flake8 sphinx numpy + # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. + # See also https://github.com/sphinx-doc/sphinx/issues/7551. + pip3 install flake8 'sphinx<3.1.0' numpy pydata_sphinx_theme - name: Install R 3.6 uses: r-lib/actions/setup-r@v1 with: @@ -218,7 +220,9 @@ jobs: - name: Install dependencies for documentation generation run: | sudo apt-get install -y libcurl4-openssl-dev pandoc - pip install sphinx mkdocs numpy + # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. + # See also https://github.com/sphinx-doc/sphinx/issues/7551. + pip install 'sphinx<3.1.0' mkdocs numpy pydata_sphinx_theme gem install jekyll jekyll-redirect-from rouge sudo Rscript -e "install.packages(c('devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2'), repos='https://cloud.r-project.org/')" - name: Scala linter diff --git a/.gitignore b/.gitignore index 198fdee39be95..0d8addeb10e21 100644 --- a/.gitignore +++ b/.gitignore @@ -64,6 +64,7 @@ python/lib/pyspark.zip python/.eggs/ python/deps python/docs/_site/ +python/docs/source/reference/api/ python/test_coverage/coverage_data python/test_coverage/htmlcov python/pyspark/python diff --git a/LICENSE b/LICENSE index af2cdd275d28d..8cec4f5ea5379 100644 --- a/LICENSE +++ b/LICENSE @@ -223,7 +223,7 @@ Python Software Foundation License ---------------------------------- pyspark/heapq3.py -python/docs/_static/copybutton.js +python/docs/source/_static/copybutton.js BSD 3-Clause ------------ diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 44d602415b262..a02a6b7bccf27 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -33,7 +33,10 @@ ENV DEBCONF_NONINTERACTIVE_SEEN true # These arguments are just for reuse and not really meant to be customized. ARG APT_INSTALL="apt-get install --no-install-recommends -y" -ARG PIP_PKGS="sphinx==2.3.1 mkdocs==1.0.4 numpy==1.18.1" +# TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. +# See also https://github.com/sphinx-doc/sphinx/issues/7551. +# We should use the latest Sphinx version once this is fixed. +ARG PIP_PKGS="sphinx==3.0.4 mkdocs==1.0.4 numpy==1.18.1 pydata_sphinx_theme==0.3.1" ARG GEM_PKGS="jekyll:4.0.0 jekyll-redirect-from:0.16.0 rouge:3.15.0" # Install extra needed repos and refresh. diff --git a/dev/lint-python b/dev/lint-python index 9dfd296383a81..41da41bfda5dd 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -173,7 +173,6 @@ function sphinx_test { return fi - # TODO(SPARK-32279): Install Sphinx in Python 3 of Jenkins machines PYTHON_HAS_SPHINX=$("$PYTHON_EXECUTABLE" -c 'import importlib.util; print(importlib.util.find_spec("sphinx") is not None)') if [[ "$PYTHON_HAS_SPHINX" == "False" ]]; then echo "$PYTHON_EXECUTABLE does not have Sphinx installed. Skipping Sphinx build for now." @@ -181,6 +180,23 @@ function sphinx_test { return fi + # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. + # See also https://github.com/sphinx-doc/sphinx/issues/7551. + PYTHON_HAS_SPHINX_3_0=$("$PYTHON_EXECUTABLE" -c 'from distutils.version import LooseVersion; import sphinx; print(LooseVersion(sphinx.__version__) < LooseVersion("3.1.0"))') + if [[ "$PYTHON_HAS_SPHINX_3_0" == "False" ]]; then + echo "$PYTHON_EXECUTABLE has Sphinx 3.1+ installed but it requires lower then 3.1. Skipping Sphinx build for now." + echo + return + fi + + # TODO(SPARK-32391): Install pydata_sphinx_theme in Jenkins machines + PYTHON_HAS_THEME=$("$PYTHON_EXECUTABLE" -c 'import importlib.util; print(importlib.util.find_spec("pydata_sphinx_theme") is not None)') + if [[ "$PYTHON_HAS_THEME" == "False" ]]; then + echo "$PYTHON_EXECUTABLE does not have pydata_sphinx_theme installed. Skipping Sphinx build for now." + echo + return + fi + echo "starting $SPHINX_BUILD tests..." pushd python/docs &> /dev/null make clean &> /dev/null diff --git a/dev/requirements.txt b/dev/requirements.txt index baea9213dbc97..a862a6e986791 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -3,3 +3,4 @@ jira==1.0.3 PyGithub==1.26.0 Unidecode==0.04.19 sphinx +pydata_sphinx_theme diff --git a/dev/tox.ini b/dev/tox.ini index ba5df084daad7..e25595aa6c9a6 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -16,4 +16,4 @@ [pycodestyle] ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 -exclude=python/pyspark/cloudpickle/*.py,heapq3.py,shared.py,python/docs/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* +exclude=python/pyspark/cloudpickle/*.py,heapq3.py,shared.py,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* diff --git a/docs/README.md b/docs/README.md index 22039871cf63d..e2002a66b0433 100644 --- a/docs/README.md +++ b/docs/README.md @@ -57,8 +57,13 @@ Note: Other versions of roxygen2 might work in SparkR documentation generation b To generate API docs for any language, you'll need to install these libraries: + + ```sh -$ sudo pip install sphinx mkdocs numpy +$ sudo pip install 'sphinx<3.1.0' mkdocs numpy pydata_sphinx_theme ``` ## Generating the Documentation HTML diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 8e2a06e4bc9a8..17da22bf8a433 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -126,8 +126,8 @@ puts "Making directory api/python" mkdir_p "api/python" - puts "cp -r ../python/docs/_build/html/. api/python" - cp_r("../python/docs/_build/html/.", "api/python") + puts "cp -r ../python/docs/build/html/. api/python" + cp_r("../python/docs/build/html/.", "api/python") end if not (ENV['SKIP_RDOC'] == '1') diff --git a/docs/img/spark-logo-reverse.png b/docs/img/spark-logo-reverse.png new file mode 100644 index 0000000000000000000000000000000000000000..a3e4ed4bb3d080dedfe7ee0eb05d00c716d77db8 GIT binary patch literal 21547 zcma%h1y>wR*KHG$;2H?-?(XgqT!IC+0D}g12ry`ZySoK_Q zXTuMWb7Es@6?p(~rvw1s004M|bom|tfDgWa8yxH1;kx~n@`cqirJD#S4Y&@#MRMAgPTW^%ub74 zRzqOKYal=U zK*d@{Ttv-v=_C!#Wt(>T&kt5bn@=c}Y1|KQLRUQw$I{Z$O6tOXchz&PM=uI4!|3QK z$<$?K5FUno=DF(YUG(+!>+89=-gevY0AQf&rF&Zhc6mi_eZP$k0HjBzujYw=SD--w z2|Ygb`U*BA!GQ0V&D{s9*yVy17gzwGX=r;&-ulQQ4hY|@2q}baqLeeELIH7C?AEx1 zZUbHg0Du|;qtEgD=sxCE4mtS^W6Q#SK=KYENE=S2ySyCd8Wm8Y^0R~$cn_$_R^YxK z7@m531OT^J1P!qrRRF-;(lvzsH#qtdo*SgE&((N_4*=p9OJS@7?JM<=tnf#jqNX~n z&_@B3P43(LJPH39P+}$J0Glg8R8eDwAki@V*NQXO z;t{vP1BA0y5K2g@XaHYPLwvXmU=afljvDFNMg7+Z34R(9YT5Iq1^`&8HH}wMIv7aj z#Q0aj%~OgyV~9u8{&?5g8{Y%~VGG!KgA5VQ4}cPO_EVe=amqV@I$VpKNBqC8%9-;c zgnuGsp0N~>C$aGWCG<*BFwE<&H#Y#B?{jA3w`Ipc1DW_+Y6P$TAP5`SdPc?n8RihI z9`J(j&qS5rBpDuGV@W6=5S1;)0f?yl*W!S*{)t8tAIrse9eN1|fDMrz0UD}Ec_<)O z_*iZT3ZD%gpa%YaiU$k-GY6>rG>pV;h5=+|U|0tCEdam^ximdd;;y#>03;HN7&Ssc z1o}FtkjJC^b3{oYw8sJZ7tROZfKujvgO<@nB>#E}DU~5Te;}C<8nW_K+f9IL3=LdI z(^jtdLc9=W`c3m6m%e&tGXK}mj{eUrpq8H^Ud#lNSF&!>^x%Pf0Emrd|KC*yDv zNhzsO5DyA>I5fv>8sv%oxkJ4p_-&N1p*Hj%9z*otM%WT=WpasK!A z-YH@ztnf>~@X|$ql9>*(z+!9y*t{GxRV^C!86bw!8g#u0<*lN*d&5V*B>qi%iUXj+ z`A6MZQLKVB4jgMJ?qc*`ut)>b;s8L2EFgA3;3WgO(2_8{%py;;9#V;g@pcJ!r_Z(s zqxmiT{L{k$r|AyuEPyb1S48geRXYQ7io@GDlkGc!_Zydq_G;lt@-klV0wkJ{YTR&u zP523J_Zb<<=cn=E{Rf8eNHtu>(!$Y9<80iYY)F&?Ipdz62Wb;WFy%5~S|`m?_w0 zY#EPUIFc{tY_jq03f1?J_}aU6nrNwtri}o{mH*rpk}~k3MvdlJG(-6TP9__XKK&ek z@aM75t)H~3XjmoY+IGAl)1_gTM?3P&3e7;i5+Cb?$dwpCOSUpeFi>PRmS4fk8Y%LZ z1c*apgcjoo@La_3mikCbT?>`Tm>y6=r^rcRT`}Y7eeCCsp6h1IBuMu97vg)R@6f{P-@qllPexHyG#0$9!pKk!c3qgD5sBFG z{M@4f)UkFtKv*IeG|4As*SZXmRVpzzow-Q@sG|`^;vrpJG*Q>A?p_n%h`NowoB}{J z>TPuR=N{&2nrh=@N=2z4C6)9Lcp&(f_}M&i3utJ5b@XO5XQ)Mn$g%rlCT-WO)sO_e z`0#t-161E))W0uFeo68)FsVp@uJXM&doDftji-l$$AV=s)pv zV{w^10+J;$_yP|&k<32T4ojG*MoL3i>~U{*ZU_Luhay82hMzOI>pjdM4CKr8BLaW~ z$^28T*vldYHW*Or^J+l8zbP#fNO8E^;y?s^jUta%AYKg;pZD{C7=~yvSD^UtG9(y0e~3KiX;rmBKO;8| zpdJ4F{qTeEs5pcz(|9R({R|>)nC6vg2VUw*s7%W)3BN?`(Mb75kCxMPdnX}dT@(gL zB$65+76df!(i4_v1lV@A!*Bzc!moiP?`{gv%~TcKJ|XDgelQ<_R^r-(XCZt0nhY8` z(qHTGui$I`@W=@$@oe7sh7UDsfoNKecI4?@WlUoi042Un_}+zhG_Dj@40_N-&*kiy zo+lAHphUgd)gez3yEc;W9bJv+wb0J8>NQ{Z$9e@Fy zbsNykAx50Cg+&h%qNFnkLB|6a{j?gm?R2);s%h3byFRz%XhV41c-Me(W+rw7$#B9x z1h~(2L_hk_4GKws*{f?gmSpKx{pN|8zqdifUoa-$peZ;3wXtcW%8QJSvDI_X&~cFR zQv^h`Z;a3QHDw|CB`Ud6`8W#wTWW%uWC#*)o(MHGJ8Jy;<`0O zO^8+^;TkVu*i=+}_yt*P9Z3qJo}|;7P6r_Q6NIG&eha1g7i!kTqE84#O)(@)xAAHw z0`LA|QyY9-F>IIzR!LvmSxUa}O))9#a(WJJ9Xd6wUB0JgXR%o_RS-*xVP-VE>VCeP1~$E^JR3s>GF z$u0~JNDryzvOD~qUr{{dAuI)pL&(cRXNp)UKx_`tGWrRTDRHSU%J57}UKyUF3^ec? zL7c~!-1;Du1K4TVrD#u6cRF-UQUl-6X=E)w4!&#xC`=F-Y zfvA%a4#Yeqs*onEkJ&s|p%5K~QOCQ1j2JpfBj_ol+Nyw^1^+7btmta1zT1DLllcpa zU^fsM<7tQ+_k(Y-(h(U^qw2BJ)^`lRa!bVcbm#Zt&#jJ4pB5?AXA|HXArn{eA%Zm_ zhZRY4&mP6y3p6nHTu|0g@#ph+Q*C6*e_7~$uUMJjc~{D9;Od_jBuxvQ83fZ>v@@SV zib$mw5^)&831RJRCUmnDk>MZlPUqRP^69!*F?t;kLe)A1#>+y2F4-Nh^i*&liBY(r zZEO+o%I=VW1z}M5Vuh~s4-IL=3PCT!j_V6U7;b1 z#SY^?pQe=esIg)tPaQ_xL68!TBr@yx65kINtwsIEm7x2~d0AmJE}mYZ3&|r^{^gqze^EqtKkok-3@t+j#>|VN zkhc6k$TH&8kuq#!sX*qk8MiRvur3H} zh{wvwuKsgAmlfjt#7|GQl;$A_TJImcmY4zTk`PDL3yI&%&kFoCJz-KUn|ns9zko1b zm&3igmvSfbzlj>{va)-B_yTyl_(=QDeBK^ztPK@8amb`EV+zEg{Y{MR|58}9ISV2! zLiA*7aU*S|iBKekTrh+4|~hCj+F*M|&f%Q8}Js&Nn7B8at2 z+%ET?ZpV~_cpOF3z5ZmXBWm_b@p}8_s&91Sqj;9q{V$1D*HyYtx^jN z?V(&AGH?+;*_?$aXX~JhRDRQwyQ=JTTWpK(eU_*w;9Z?=)O_$-!h%*CwD=_1KcA>C z_)NsF&6p>jM@d@3hugACdT{h^_IycRQOAd?X4scniOH!$j^?iKiuVq%1y6hp$#9TSc@rq&VTJzi8ePGXM|V|gv|I+g3`_G58x z1RuzCdG8U6eaJ6EwHVR+be$+kN4lGEMn1w|<1q-fnlVbCfWx~1=#nrE=COO-k`nuz ztVx31$8;;XNFJj4nfwAMr=(!|tBNsSOLfFYBld2qZH^x_SVS!j!ko3eTdPNu0w=~k zNJ!F6j_2COUH+{XKZXeoS8xJ&QXP#RLV0pRW{gR@_)D=8PDC2#rp&iE-zS61&b3^%>dSE-htz%hoG4cH+gnZ- z0+7Vqug--jC`qkYI_EG(>77}xN;<8c#KjEq`%H;Hz>u`gPH<_TZ~6#9$s}-kTgh!a zMtAe`EA^A8ruOb23!q?b%Xur+{dSdT#O-P^HMilY(e~W@+Wq3b`XHv5hqMs1@*>df zeb3p6=ZGF>gu+bsX7cEvKnuSQeb8&|uB0-ot7>tzY-!qjST6&c(l&OWRS1u`AM28G zVvFiv__ct)dEa58bxMf(qk=Fh9A zd^NCrFw_^^sXFE4@e|z2ylJX?7kMS`d&7C>Q*1-tDWooe#G`=PT0TmhmlDj3Vr5e5 zgt6&BmLF}s5tztEmNFwZaDo)bIS;LoSl((|;W0SLD{X1C*x1KB7$eW4S{E|Fiefcb zq0z62Bz4c(TG_?BOvm+_K){0hGd$0)BderyY|e!Pb|DBq((!M{#G5^9)3B5*Cz^Tq zejf=GFHXc1*mfH4WT7fw;iy-Wm7APm8U?Y1Prl<%0UKTgQlr(jyN(;wsVi|~)iA>l z2bxwyn*m{QPjq40+X`W>6|GhrZ9F#4minI*#D)C%`PWhDok=$8q^3JMYn^Q9x!Cw@ zwNFvTI^Fy;!MlN{O>Zs`Bb8MLRXDail$W_{Y?9qw{NnS%PBHu0)y%>OtgEDljP2^|rH`7vi=4h3miI!P za{G<^-TmB~6s(w9z;I%CWNwHDUMb?Tq1Ce7@ldAQX+SxwKWu5tSpLKl$L@BfkyWks zChcGoN|curd;qQU$vMUhrX^a@araXJ0O_kxv{(^$Q zil#pS!|B^w=_3;MhXozV6Ar{wlk|Ot$HPS24#aWo^jP}bubfF^v%e9zaGz$x>ihjn zqfXLtkST+vS6rM(6D5A{N39kg*V5_S(P+V-SIk%XlZCQ%)|RfN+y;3HF@}fg-92v0 z*xa6f0=U@8=n|!TB$sIPt1&SxBw@4 z^Oz)oXE`6WTmB5Nk%f^iRUhY~=>DLh@4DT58%cMYHQbIw-z2~LjSYSrD{9q;L0js3 z&Kx!LW+d@;9d+&Ced=JWgA&aYM>|_M?Sd<96j^NKzZ9|p*O_D z?S0@vvpmxn`>xQ#(L_)z&*nwMUqTCx8LCidm7`~SXqRR( z=r4OF2;VX*tc!M z?XBY$TdfjDPHpX!fTOSAJz|FiCO?ck3rF9WO{(P{(EO449K0jSsW`ka+zD(9n{ zq4#U13&m`1JUY6jrM5m1WVPR0WGA1T{K1YQJvtg+4dX5z1R4j7->KXWny-=Zh#1Oi zK4Y1bJap3xNqK~JoDiBFx;q=Fl1pewoz2s{e)%Nc@}sty^~^0}Jq&>z<_>)5jcKybm*n$rj zTTTVr2YG-`-_(dSV^}R0M0dVlb9N%Pm)j_K9g&X^L2nPr960=xb3!8VgNxY{R1{wBh`$NyR6aA!iQ=@^}8m zOhfPs=pOB6ilvfpq`dOoZ9?%+4^o?nUa^PV!U{;iINd4t~thcsNYD7z*#h*(CI>o zk5WJB;dn|c@(zTW$e8(j?i|gUak;eN6>2Zvtz4M!kG!9?^HfZK>2eTy?CD_c%fi26 z7L#lY%Oq3@ShYpJME-s%%#&v0p0ef(^D4Mu#3eut+t%dl(@&?ixrVL$WH$hx2*m#8^CnQ2cr+#ysXs#H*+YrIe5ZV^VDS%*DNCGYG5E_zQ~u z_C#6cHQI;u>@PxoaNjwn)3&o)_1T&uYlyS=)A^rN2OcuLXF!Bzj1`pzXS7GXW2woe ze{}VpI|q$ZPIRul^Bi0I zWZNg&to&(M7L<=AM3C4>pE>>)E9Xj0*#p0r9fiYGdF&Y5FdNveFn4vs;C6iT%lhSB z&g`(mG>zPUH7VBR`|B&2G8vp+h$%w_nE>9 z!7+;xwL+#0Yvd2?iMM0(3hmnMqQg6_`UKdA22vY_{3Ym9Wrn^B{lAAhSW%ofA61Hj z!p~*O{W=S?8sT2llh(>b`m(Ig9+@yz1q(AC;#(jt>yomdizBW{fO4jhcv8u z8Cj6st%NM%>}huPwO@%{R0R2~SCLZ~%s9ELGy+VHq7}!}x0F7rxl9?l*gSb%bK>x9 zsp~-kzBh9}JfV|EmH3`LoKPmUyo_MbWIQfIU8kTl_cl4+?o8nX+}K!RyFzz|j4a1= ze7vUg;Gd85gm~RlUO11-m+9+L$|4%pS{}+dR!ImVV3No5Z!}I$hJH>IC`j-HOE1T0 zl;VGNk^}OveyD^}x6^*Oa*oElIZ=__#uq1tlsZcz(Ljd#V@I((?VDc+Lg>csB_X6vq#wWPt7of zc(?nWH!s0Q+_qo75JjkOkV!%ER??fQdDnE+dsRSi;Jz6DN=YJzNz{{z2$snI(!dcq zQUHC=QNXjI7iLjSLqL0Dns1N5et5^kFm4l;>5+5R8%c!XE;sa3rn0&`VQ z9mVUZ<`+;r?*lujS2AxW@ONyFlHxM|9&@&Nm_3Y|rhp|Hht~Nx1U8}015{_}lAvv@ zdukb%=`zw+Cjvu^3CAv5ve--;ht7iW{kJymhobW^PG0<0g>Xh-gwS91{GU4FcWG@m zsDEB=)Tn=^=Q~!z(!s$6bGHhrM$_%y`Y{ZuuVoEopZ2eB}8tv$m&N&UcMbwNN12Ls+bUdHR@ zc8raVl83>doa?Q{C)|{9&+3|@iSNv-dMTOAwq#VBlNt_!McD7gY6{eI@Sv^b4|Y*D zih|sv7)93-aW@;i(qJV9hbPy+{Y~;7M$@+YP7`j>b703IkBJGSY7svU8bEQ5%PxZ zkeG*3D!yZPLZL3wKh$?m3=F{Z<1@c&7s9WoY65+-GG37g$-D8}wH?1wyo<(?<{rv) zZdR_+iA%|wHX1v=# z84u!7O>J%sbWjfB5SXRk`IyK>@yGN_7(EZa0TxBoNcT=e+ra9O51dUcTB;EQMN4dz zHX1x>EYV~J5~$&$h@XLp*D)3Q&W}+Dwu==(vRm@7SHvWUG4i%_i)_#Ci5?L%i$f|x z(upMOQvRqOSvh8Qt;9Wixl^NC@o*BWIYFb+A`WkV7WtM?$z}nD_La8ViL6)kc@CC4 zT7ZgYnBP}|Y|Q5Rbi+b%KT-Sigf>}>%?o}r)*vhl0G+gc{zR#GXIuS?8^^5gv7crg zZCoQ?L+l|g7ou#o^YR2)ctp)#58b`q7MA|TQo%!Kc+a~qT;*AmZu``7+ z_WjLlUTJIrlZPR`xnvHS9*83ncCy^u1)&Q4dGti#s#u=_9rsAAeiDwyrO|f#=O9{= zsxD?p#R?i(Y6U<>N(K?9-SOg#AaP*iZf-xw_xs&yZ0&}oz<>}r!oL%M+?bbTtc_oH zfdbr^(C}SPZ=9MmkN@!&P&xT#7Kpr95Z}-Z&RF>Zor(C1Oma2vuo11#@eOmorGPru zGHeu{;1Y?ue;NL7B(yM&2{m7;2d|F^IBbu2GHEy!Gi=?Y`{D^=bD-UpN{#-tdiQy^ zglbG7M#}W4KP^ z;t>Sxe$}_+86{CE-P2=r@{4aql){;jvE)Hi7PCUhY@2bwwN1JT4cMtTV(eD_7#H0% zMp+n={`XbKsc5&H>7-oxfcV(ip&3=m&2VJDIFKb_EyLMH&PopX4JOEWOuvcnB)zdjgfesZ|syb{@_#%a+U8tHtI1}-t4A!5>G5obgIwW8Ugv|%J=hY7Q=VHXM0n| zHCjl2ZE#RHie=9BK9JPk-^f(jCQ*P(2|?D$amTuOIF+6{UZ`!`p&XN?oUtLTPxopM z{?naF7Fq3>Z30bCXPEi%s_#+U9H>pl2o7^Fc!$F3kqQj=PmN2?k`ke-jMAF!$wCsW zC>Xh+vF>%&cISOGt<-H!@HS}dP+{!^ZTvnXQ?1SL46P*j;AXk+wWHxgp1mB3M)U+Y zalYrM;gV?I``j)F=Rh-Js{<-KLXT)1847yno)c+{H*iUX9G>JP@rV1`kCA`47uT_prMm*n+Aca zag9#B0+CXp_8b$~VNwi9e7Y;BcJ1$3kKNw+HRAJ3LvmYX3-or2O>vdj;pJ@WkgClE zoeHvBe|s_M>5Ne3k4HW&?@%cukn>VDtS!+9cF~g?Hx@EB#ia(QX?~Ew+Y23ouuABl zwdyQsh%?{N)vTVj&z!R|)dWF8Ibb(5gULZ(U`*Zm08Mt7BxCn$J0XQ8SiNBw!Ov|Y ztm|aN3ix=5Ps;1{NT^1a?&KDa^9loDK)|0(tEbj(un;a=&>yE zHU#P;eC{vEZxHqP4%8RBXYh<{=>=Hbd(*0XCOkdo!eazSMn;K^$B|VyyC`3fZ}Q^b zdf(8+XEGNYhgkoltVo)}vLs*=4(zgol?C@buw5L1;#*wI&pZbqb&q~58})$kEY_Ml zKpiA>jB)!i_U|YkB&Aou>wRk{B_Op?NR`8wd%NI3 zN9(+1G^~Qwgamw6qTSYdbc^NuyVQ-J8#~Rs)YXoZiB$9@cw(cKeqp74KR?rGz>?V@ z@0U~?wzX%SztPtN&W#Ee64G95%BRV3P8c{oAYd3_?=f`DT z+B*k|V+iCo(BbUX7p~e#B?^TD_KOE0Z zV{(9^7=0#8%O==C!FVFU@sciYCaEQyXOR!ePq-{h^E|72@AtfcQ^cdh_QG(9&wY>84(bAMoh_P)4@a^bgR{Uav&>?PB-3Y9 zzwNdjQ{In)ybWu^@r6wv3{z>@EYQvN$lSx5V8{=Dv`aiJMwvzz1+_FLm%UPoY|pbz z$z~~`kf|qy#FPTcF8DVn_5$H4@@!-45vtAu*T?s49P%%tN3GZ9k(UU{8s3_|`uAV5 zWp7pDVJ>WGE)zCh4ad~eHeL73FmReFL|){1H6B9i8$Gdg-3uGrD90z_s`2+ux~PI8}kXL^5SWfxS8>P zC9DzQ!m}urc8azCv>{w!V>&&VABH+`+_*p2X zkz7g7O?*L?pP}sSp@f{o={;gASMy5w6Ib#g1%2u%wq5tKmOJldlu((68{&nw^WYp_6Sk*7m z%|llChWxYgkoAzdm^UwlbHu10_etx=G1^^gNg3_ii{E-aY#y?%8M<4?i$ld=iW0-v z4=n~9CQmLS{G~)iOCA?`CYPk#+ApKJLtaBwWXI$=grENevEV{u{s~%9q>)+a@d`_= ze2Z++%#Yqk*mz9YXP3%p2(EEAo9^zKRccES+q4|Oi^M2%fUF97_QGvQ=p;8>NxjT# zpurli#ufS@oW-*`R}K>->14BnQ`@wd{>ri;`Z!iJs#S9D#l7Z~6(d*7QT~w}Fs7XM zqORJMJi{<bd{J{tJ3Qp?VF8_RP8y725;jR_78*SA1#%{Ce8Fszv_n}NoOk@ zoPtoy@A+gn_RM0REf>zOnF~W)>bYce+#=AenXE2dA_uU8e?{R&nqpc?J12ZPma?#0 z_nT2aQtBpCe0y>E@#BZb#>2i3|8qfygbkXa_*6QSN(jw0gA2W(g#aiOb#+%ft$o#{ zA5Psd6vqutRNb;L{rliT&0?wLjM9*9nIuxk;;ves;d7PP8lt@Or8q&)@@$d{N({zI zzGyCX^L=uTko4@|BILnPJ@an51SQviD4VS`@l++0BbLB6Q^8Gm%U`d=Y6Vh%6(9eb?WW(N+{I78!9%kjeS)Oxb4mtUr!y4UWXZFqwHBO7ER>tBytRmaXhlILR|`-o2y?xD>+Wx$fYK7~3O`_L>`HW!g&i`OeYb@=rc zr4>DE6&%Ws&6tn$A)``y!sDhLjvp$upA?j zBjVpyP=oFh-YSvaBICCxNmuzX)>E*G1Etmiy(n050+dU)~TyHPky(0;UgjJ!`c z-&y|?%rP+>cN{S>DrTs9(;cZFQAqs(@?yS@x-o%aY}$wQO<*QcW=234?M6wh@9ur~ zx$S$?50j|2Vl!h@<;YTd(Y90_{1|Xpr5YLe{YEx#U~2t*!?u3U6Fn;u%OY)df!IGlM*{Pu;MNx8^17E~|Ub zeFs^ebYMRdoS1OF4KI$o-gWE7yK-JIrQ-SG?onCtGAg8yu%P%b&Hj=b)1WAdS{Oh3 zsxn#7IP>KarQmWdpjH|IX-;;ORX#I_ReZF+cS5XaupO(Vu`$+5QYJrmb^-KW>;j<( z+yCN4WUd_6sb^>|0^5>W!qA<1mo^MX6lu`h=cphsSF=y>QH;9l2v1>m;VT~v7s=Mx zyghooZdp>{AGn?yr9GSVh-)uv(=OA^K@1!#E$Z(JeFNda`>zA9bpQYRDj^ zO;{EnNWXofx}$G&mjog?uh|qv$_q(G`!plw$2zFnQ!XWr^0P+9sM)AycT-jp<)y1= z=S;+b0oC=TQ@r@J^?P$K z{SMbV2g)#kxmia2qp!dSR_gvP&L9*!ux>BVohXCRxqSWGtP&x2YJ{>)`HW;~CK`H@ zN?zBTEC~bVv;9q^&`lzdPL*}q3Ule5)EI}P%Hrqa$Yd~_RP2h&Bus?ol;zpp}X5g^h?}rlQZxI;8GHw;_dFRx<+;P;3c_EuiIu2GL5HmQVrH7Q0Wsnbx35ennE6Z z*fgM`8o;wt>LCTwMfFas3|tu*8*XMr)(Cs%92QKU=07gV$#(JL)p*E@s|n<`OT^nJ zdh1XzC0X)&`*GY&d1&)*d$TSow6uWoLU%}vwuO;m(3~Q2+N2=B9R4tl7lClC_!pX` zI^s@*)VhR_9Fk@QO1ggrif`Xyj3vZV9E@C=3YgKDPY$uBxN5WHt@jKM)s;&-oeIn_ z(?Do;kiuUko$l|(5l8tE&vvtQTVdaUKqQCpQK=n;>@m2M3K2j;o<^TNEa0Zbw;psX zM(*cNRbco*+CswcS2xtH(VRave6{B#&?RPJL3VZa?iq|?nUx3 zrDGq{@KioIJvmf}vw%oqY!PvsuvC!+yDvx3xj_blz4v+QGmE)&d?RW^l8kFInJcZ zofTpZeQDfi*lg~mTN0*N!-Y_Hj0+X1_)%mL1n&b;u&&oxw4@@-v_H*CJEo1gALE)U zmg6)t->Am@(KO`O)jGRi4QbS6%6*I!3}cOpO*L=@=gjAMY(AiW8M(l|qf(0{$hSU= zEAl*!4k-dd4dX3-9;(3lgRts_+e$CsY9LGS`{FFnGAN7@*+LOhLMr9DXGUz>4;Dz84r_ zGBBdJ=b(2T{9@OTEk3KHExC$FkaQQz3vQdoW>fVQ3Jp zmH*h#pV@d(aa&Z2Bin{yS=cGSx`PXXSU}~@%p#;}j*n+;*I$r8ci5 z^DU59b=6+IIJ1x$8Rwrw^m+WnTRj>@%oi~*ild?UbhpeTX>$-Qh@^>OKVIyxgt-x% z?Pntv@huj6l%}>-15a&vhxn_3$rOLC_9t!2+al=t__oZ@kts~FO=2H;Ek!oBv{AHw zpgH7Rnybh|T+6~L3*xnQRpcVUr05MOfrPC#^4Xu%EyqfWVXgF}e>Y>@q*DcC(P^$> z?1o63=4jvuC)B&WWQw5~@mz8QvG8J^;`GR)jL5dl@SQg2qqw+!M>f5`iW@P7;A zqY5}lnXnDLC^)GRq1tAPbR12^&^6PO9%L=+P%03_ctmOs43XFiv-hfm{N8w#F3jVT zye!fz2$co-&&JR<)9=(5G^~?f_$Pmr)#52N5bjB6*iyvQWh2}|5m>Am7S%)1C8D$+ zRF5Z9u}0U^n-=XpfnBg3A~M4ucj`3#jY^|1$zv0k1OSUyG zgip~&&n_rHEVtw!j}Q5P-CeEUz^6d?-QL`d2g8fnW|8ELR?4|}i!>p?Zu1BZtj6tqD!IohrDPq0FnB38NHj4G_ay5Iwm~}j`I$%UtFx;( zF?sossXn^@F6U3bRCs54Rk7plXg$b7|HYrCdG0OP;UG0`C(aYsj1Dwt$~%NMwfDo; zBNI0v30#JXXqc7(PTXVqVg!`RJgMWE+iJ>VatHjl>${@0@{BuE!#|(KL*}Qhd6b|_ zyEVv?w<+(a9Ww{B*icHh=^3`<*&W+7pj>cfIU7;=1VP{b34dMf|K}u;z<^g@>aOfnCE4LkSMM37LXci(25)Q6>|HE=F zu3qW%7G?{*F){WZq zIbsePC~0=xHs{#QY~tU@k*Dv~TfwDX8scca?j>K`nmJ&R{SY@s zta)+LdP_EsdGI-p!p_lx8IR~zACJs2y9=B2tea?(@$flPfa8}$gef?+v_Dag;lqk4 z=8w(GOgyn)<^GlImej&|zxxMHX)hBGE|LQDsqMvYpt3A9VBPz%w+|Ci#vt#= zT_yQ*rH12*DpiRm>4xEl2;90WF1y61Klsa|E~K&z!?^L?-{|{uG&{&)YUkn2Hc29hCe+~ zU!5V8)Lme8+Zq-bFzB9j%|DS9arCH(jVB|d3#HHKCZ8QIQhjrlUPZDfdvB=qjzoyu zh*W$rGC$gMK`yFPU?(k7;2g5A$!vO_=s{a1bIAr5hEnfk9DA#MB%sa?GH)HoZHTl? zk*HWCKH$F4o9URRnVcykS+pEb7t`ko!itwZV6=u2ti&BgHcq4EJwK#a*}eNxSa(tM zr!p$6VySX#`1$2?&V7b`Rh0{oTjCkH`IM+c|r^>vKLkpZDjn z^Yjs+x)2JiD06Np6n`yA#(q>EYIQYF0o1GdcKi+>IinjD`dXb4orsFWun)Y+eAgKI zka4tmF!|`KixMS@%LFg;^ar$9qu;E$b8)bxxg;v-GGLX#H!O)c<}2ngF504pIfQk3 zJDd2{cHb+XA~~I+rOsdH=7%^bZs`&SW^bbk$dkrCiWJ&GB`cofjwkK=%*`4xurA z5^M@XW3*f8#uluZ~9CUk}+GsHz>__n2ImwWSNEo5rghV`{9G#_1 z+^%o~T@jw8Dn!gWnyN1oa_w^mN0 z>Iv?XzjSUXeb83iN{hxTI8UVwmT<4>gEp_zXT@J3GIA2?b$m%JFP=4#6;HWQq&@jL zXVLmm`8oJ|Yt(u%IxO7IS0yXEKa+B$Bd-pVk^)!qs}Z7_c;qrC`HbzsmE9g2R*_nb zS?ypQGe4XSeZ}t192(C${r70cYe3M>&!<)}Xnmb+0%xK6ZB9S!VX_U2Lm;x+iAhGH zXulxbWz)EvApeMbJ)ER_{wMKzAB}M&_#l2a_*VddU%$FnOg)q-iM*!5iHSk@kS^;k ziNR8vJn;e~yL%_aGee=+h%C9eQPQVx@~}(@)6O-ln+nG8t>cz& ztQ@6RTZt1>i0Lx;QHVq)<2|n{_B>`yt)LkAq*#C3eY+e?E1`3;PaR_y%irEccMr^g zN*W6$p*6bwIjkvuEVz-lHoJdju{uAcropj~S{e3$lgLf^X`fu>k6yDA+ZSoSOb3?y zG@9imp4fXR51Lh%EvkANNqUdqw*~2t@BH9pzE!(`8gdOyVtMl+fYsq{podcQXuO|G zqp+r_$_iBZN|Qub_DTgv5l~8Ip8O5uj$}Zu4FuNZo#-p zp_9O(3-Rq^LzjxR-q&&&f$KP&vWAV5 zdYgq+rHiWanKM=`7-N=&s>0|V^-y8E-0_defmN$HMjQV7QfIf^6#D76*WDE*pY3>G zsTZ_lRyB5}?_N#n-xOW03Wy<9mx1VeU~MDZ+otnUd@!eU|$ltdTG+hyS=^# zJcuzj(zrT3=~nf{^1Ed=wzm|ahr*eb88S(p_lYZe#x}wMm(w&DJTDm#l+CfC)3?KX zBLc=LN_E5I&MUOxK{+zOW_io0!Ufs(_?RJph-75oo%YZ9E`nU&&_GM(UVx%<0kC{)@MUj?$0&i?9FDFV#;UFfNvOQ8>> zHB%%3L5AbaZgN@ST{X4(k~uRc;fc*~H2AS(3J)Do=Ewn^(}}M_mwNXcweN~;lD^8y zz@L8M7lYigJ;#O-8=pi;t+?hSxgcqF8=0CZ-a+Btzxd*z&R^2HqF-m;xbmz*?zY-+ zXph%&Cg(r$c}o-QO{Sw#h8(w-cW? z>exN1DqJk7-7U!UXYHI8k#sk*$i<^=11#KH{&hj8I0!7N!k4!o6)p%-ayeT2xo3jL z{U>S$2^m^`!lk!V&L~u7|Kvod&(KpNG8l&Ih9h*oThQlkgl-@Uy82@$90auMw`u_^ zIr0O@`>qULZ3#}T9b&~(>o4kPyWWj+>D?3vIS=(XS{6)einv)6F?q&P}*$O$ zF{7-RL9*k^p+ZBAo9~(GRZ^~lSNipx0*K7=FvNkwVFsFXZthBr$daRBd8N}KsiYk{ zHO9~_MlRF1dc9#&!EmVt?$8#*pI7^Ek29x?K#~^mYYKBdy;}J={C7-B`$LFHz-Dds zTNz=-Z9bN)l`XUAma*44rsNZjn1d3BgiJprlI>?;Y)r62W}K>UJI3r zcHac0zPpQzF8VZf`|GEuCCJfkgqK^EN8y3%bzjU?D3hl)U(9$^IHXV@QrZv1w8mc? zU`kcNmF@=<4xaKx z#rwg{hvc(jZIhq@d3N3ytf%zk?e3wwBmXskha2*5gr~pu;9vJZfFlH2j3+N)dxg<(ykWx`s=)JGNiQI3p?o zQKQ$#65H>fD!%Ho_%pfz0&rcdWTti>2lqY+B`{}4z-o!`bRNRWOIbWT`0Kd1D(vcC z(L!dYX(*!5e}MOEZsl?PX?7%8Y!1wLiNAQbU^W~%IAq%r>PEWwoZ`UKGPvanbg6 zZV?*RF`@c~pSX0uVe@xAFjI_NX0IYbzRq0LgqfVUAFPTfU}M@wR3%O3k5|R_ps#|q zPqny)WFs}0?<8A0Hhv2Aq-e2G6JF%9LID1XNbJ{F9+%!$MasaQp3{rLH1cPU!xK`?dl=Q#Hl5cIO9i<-(i%th0C=hLvySPHM%Qi;A z12-MM(^30Pvdq7S*RY-F+ifH=TJ&nvl+bDT3QcIMzJaIQu+<#}|C0yK!S{hamdw!& zY?AQY-lIdId~$95>UIaRnfdP8bk{WxcaOeqVoA%e>JJT-9qoh5ZjCoqWi;-|eC??( z-B|;|kLoi3r?wuD>U6;~(-&C5#Ih@qACLz8aVuXH|Pg zS1YSDk2{SOF^anp4aw_%Zy zm#yq=gb507x#$`a6W{NOzB9C(anJ1e^?aw6S1wAJ|4O6j5WkkWHRcRzX!{ICd6_71}Q5c|}A$~e` z0F*Jl-qK@SkYYJ;%K#pLhD&lq1L(yFB|flPE*a>(y!zm}fgkX7Uhtw%Qdt8I$UtrE z+#JwR)Ghe|Nb9Rg4=JKN<569$b73KrT1Aa)CE zp!bleRZu_gfmMMFiwur~nOkNh1o za)J;5MoAme0Mb?U|4761Rd7J>aY_+D26!XjoA-yEAkg~|8vuX@11Y{1(11rn0C3ss zZ@@>8!^d(Y2sEWw*^s%@ zKr`#5#-28op0*HcxGnGk5)}~@5fqUU6n$VIA_Wl>hlq*ri-N1j8&ho Rsv9^D(ooS+u2!-L{VxjBLgD}b literal 0 HcmV?d00001 diff --git a/python/docs/Makefile b/python/docs/Makefile index 4272b7488d9a0..763f493a0eb58 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -3,8 +3,8 @@ # You can set these variables from the command line. SPHINXOPTS ?= SPHINXBUILD ?= sphinx-build -SOURCEDIR ?= . -BUILDDIR ?= _build +SOURCEDIR ?= source +BUILDDIR ?= build export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.9-src.zip) diff --git a/python/docs/_static/pyspark.css b/python/docs/_static/pyspark.css deleted file mode 100644 index 41106f2f6e26d..0000000000000 --- a/python/docs/_static/pyspark.css +++ /dev/null @@ -1,90 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -body { - background-color: #ffffff; -} - -div.sphinxsidebar { - width: 274px; -} - -div.bodywrapper { - margin: 0 0 0 274px; -} - -div.sphinxsidebar ul { - margin-right: 10px; -} - -div.sphinxsidebar li a { - word-break: break-all; -} - -span.pys-tag { - font-size: 11px; - font-weight: bold; - margin: 0 0 0 2px; - padding: 1px 3px 1px 3px; - -moz-border-radius: 3px; - -webkit-border-radius: 3px; - border-radius: 3px; - text-align: center; - text-decoration: none; -} - -span.pys-tag-experimental { - background-color: rgb(37, 112, 128); - color: rgb(255, 255, 255); -} - -span.pys-tag-deprecated { - background-color: rgb(238, 238, 238); - color: rgb(62, 67, 73); -} - -div.pys-note-experimental { - background-color: rgb(88, 151, 165); - border-color: rgb(59, 115, 127); - color: rgb(255, 255, 255); -} - -div.pys-note-deprecated { -} - -.hasTooltip { - position:relative; -} -.hasTooltip span { - display:none; -} - -.hasTooltip:hover span.tooltip { - display: inline-block; - -moz-border-radius: 2px; - -webkit-border-radius: 2px; - border-radius: 2px; - background-color: rgb(250, 250, 250); - color: rgb(68, 68, 68); - font-weight: normal; - box-shadow: 1px 1px 3px rgb(127, 127, 127); - position: absolute; - padding: 0 3px 0 3px; - top: 1.3em; - left: 14px; - z-index: 9999 -} diff --git a/python/docs/_static/pyspark.js b/python/docs/_static/pyspark.js deleted file mode 100644 index 75e4c42492a48..0000000000000 --- a/python/docs/_static/pyspark.js +++ /dev/null @@ -1,99 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -$(function (){ - - function startsWith(s, prefix) { - return s && s.indexOf(prefix) === 0; - } - - function buildSidebarLinkMap() { - var linkMap = {}; - $('div.sphinxsidebar a.reference.internal').each(function (i,a) { - var href = $(a).attr('href'); - if (startsWith(href, '#module-')) { - var id = href.substr(8); - linkMap[id] = [$(a), null]; - } - }) - return linkMap; - }; - - function getAdNoteDivs(dd) { - var noteDivs = {}; - dd.find('> div.admonition.note > p.last').each(function (i, p) { - var text = $(p).text(); - if (!noteDivs.experimental && startsWith(text, 'Experimental')) { - noteDivs.experimental = $(p).parent(); - } - if (!noteDivs.deprecated && startsWith(text, 'Deprecated')) { - noteDivs.deprecated = $(p).parent(); - } - }); - return noteDivs; - } - - function getParentId(name) { - var last_idx = name.lastIndexOf('.'); - return last_idx == -1? '': name.substr(0, last_idx); - } - - function buildTag(text, cls, tooltip) { - return '' + text + '' - + tooltip + '' - } - - - var sidebarLinkMap = buildSidebarLinkMap(); - - $('dl.class, dl.function').each(function (i,dl) { - - dl = $(dl); - dt = dl.children('dt').eq(0); - dd = dl.children('dd').eq(0); - var id = dt.attr('id'); - var desc = dt.find('> .descname').text(); - var adNoteDivs = getAdNoteDivs(dd); - - if (id) { - var parent_id = getParentId(id); - - var r = sidebarLinkMap[parent_id]; - if (r) { - if (r[1] === null) { - r[1] = $(' * * @since 2.0.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 1d7e4d3059c68..2b0db4381c6e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -308,6 +308,14 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * It does not change the behavior of partition discovery.
              '); - r[0].parent().append(r[1]); - } - var tags = ''; - if (adNoteDivs.experimental) { - tags += buildTag('E', 'pys-tag-experimental', 'Experimental'); - adNoteDivs.experimental.addClass('pys-note pys-note-experimental'); - } - if (adNoteDivs.deprecated) { - tags += buildTag('D', 'pys-tag-deprecated', 'Deprecated'); - adNoteDivs.deprecated.addClass('pys-note pys-note-deprecated'); - } - var li = $('
            • '); - var a = $('' + desc + ''); - li.append(a); - li.append(tags); - r[1].append(li); - sidebarLinkMap[id] = [a, null]; - } - } - }); -}); diff --git a/python/docs/_templates/layout.html b/python/docs/_templates/layout.html deleted file mode 100644 index ab36ebababf88..0000000000000 --- a/python/docs/_templates/layout.html +++ /dev/null @@ -1,6 +0,0 @@ -{% extends "!layout.html" %} -{% set script_files = script_files + ["_static/pyspark.js"] %} -{% set css_files = css_files + ['_static/pyspark.css'] %} -{% block rootrellink %} - {{ super() }} -{% endblock %} diff --git a/python/docs/index.rst b/python/docs/index.rst deleted file mode 100644 index 6e059264e6bbb..0000000000000 --- a/python/docs/index.rst +++ /dev/null @@ -1,53 +0,0 @@ -.. pyspark documentation master file, created by - sphinx-quickstart on Thu Aug 28 15:17:47 2014. - You can adapt this file completely to your liking, but it should at least - contain the root `toctree` directive. - -Welcome to Spark Python API Docs! -=================================== - -Contents: - -.. toctree:: - :maxdepth: 2 - - pyspark - pyspark.sql - pyspark.streaming - pyspark.ml - pyspark.mllib - pyspark.resource - - -Core classes: ---------------- - - :class:`pyspark.SparkContext` - - Main entry point for Spark functionality. - - :class:`pyspark.RDD` - - A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. - - :class:`pyspark.streaming.StreamingContext` - - Main entry point for Spark Streaming functionality. - - :class:`pyspark.streaming.DStream` - - A Discretized Stream (DStream), the basic abstraction in Spark Streaming. - - :class:`pyspark.sql.SparkSession` - - Main entry point for DataFrame and SQL functionality. - - :class:`pyspark.sql.DataFrame` - - A distributed collection of data grouped into named columns. - - -Indices and tables -================== - -* :ref:`search` diff --git a/python/docs/make2.bat b/python/docs/make2.bat index 7955a83051b8e..2f87032820f42 100644 --- a/python/docs/make2.bat +++ b/python/docs/make2.bat @@ -5,8 +5,8 @@ REM Command file for Sphinx documentation if "%SPHINXBUILD%" == "" ( set SPHINXBUILD=sphinx-build ) -set SOURCEDIR=. -set BUILDDIR=_build +set SOURCEDIR=source +set BUILDDIR=build set PYTHONPATH=..;..\lib\py4j-0.10.9-src.zip diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst deleted file mode 100644 index e31dfddd5988e..0000000000000 --- a/python/docs/pyspark.ml.rst +++ /dev/null @@ -1,122 +0,0 @@ -pyspark.ml package -================== - -ML Pipeline APIs ----------------- - -.. automodule:: pyspark.ml - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.param module ------------------------ - -.. automodule:: pyspark.ml.param - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.feature module -------------------------- - -.. automodule:: pyspark.ml.feature - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.classification module --------------------------------- - -.. automodule:: pyspark.ml.classification - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.clustering module ----------------------------- - -.. automodule:: pyspark.ml.clustering - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.functions module ----------------------------- - -.. automodule:: pyspark.ml.functions - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.linalg module ----------------------------- - -.. automodule:: pyspark.ml.linalg - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.recommendation module --------------------------------- - -.. automodule:: pyspark.ml.recommendation - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.regression module ----------------------------- - -.. automodule:: pyspark.ml.regression - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.stat module ----------------------- - -.. automodule:: pyspark.ml.stat - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.tuning module ------------------------- - -.. automodule:: pyspark.ml.tuning - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.evaluation module ----------------------------- - -.. automodule:: pyspark.ml.evaluation - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.fpm module ----------------------------- - -.. automodule:: pyspark.ml.fpm - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.image module ----------------------------- - -.. automodule:: pyspark.ml.image - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.util module ----------------------------- - -.. automodule:: pyspark.ml.util - :members: - :undoc-members: - :inherited-members: diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst deleted file mode 100644 index 2d54ab118b94b..0000000000000 --- a/python/docs/pyspark.mllib.rst +++ /dev/null @@ -1,99 +0,0 @@ -pyspark.mllib package -===================== - -pyspark.mllib.classification module ------------------------------------ - -.. automodule:: pyspark.mllib.classification - :members: - :undoc-members: - :inherited-members: - -pyspark.mllib.clustering module -------------------------------- - -.. automodule:: pyspark.mllib.clustering - :members: - :undoc-members: - -pyspark.mllib.evaluation module -------------------------------- - -.. automodule:: pyspark.mllib.evaluation - :members: - :undoc-members: - -pyspark.mllib.feature module -------------------------------- - -.. automodule:: pyspark.mllib.feature - :members: - :undoc-members: - :show-inheritance: - -pyspark.mllib.fpm module ------------------------- - -.. automodule:: pyspark.mllib.fpm - :members: - :undoc-members: - -pyspark.mllib.linalg module ---------------------------- - -.. automodule:: pyspark.mllib.linalg - :members: - :undoc-members: - :show-inheritance: - -pyspark.mllib.linalg.distributed module ---------------------------------------- - -.. automodule:: pyspark.mllib.linalg.distributed - :members: - :undoc-members: - :show-inheritance: - -pyspark.mllib.random module ---------------------------- - -.. automodule:: pyspark.mllib.random - :members: - :undoc-members: - -pyspark.mllib.recommendation module ------------------------------------ - -.. automodule:: pyspark.mllib.recommendation - :members: - :undoc-members: - -pyspark.mllib.regression module -------------------------------- - -.. automodule:: pyspark.mllib.regression - :members: - :undoc-members: - :inherited-members: - -pyspark.mllib.stat module -------------------------- - -.. automodule:: pyspark.mllib.stat - :members: - :undoc-members: - -pyspark.mllib.tree module -------------------------- - -.. automodule:: pyspark.mllib.tree - :members: - :undoc-members: - :inherited-members: - -pyspark.mllib.util module -------------------------- - -.. automodule:: pyspark.mllib.util - :members: - :undoc-members: diff --git a/python/docs/pyspark.resource.rst b/python/docs/pyspark.resource.rst deleted file mode 100644 index 7f3a79b9e5b52..0000000000000 --- a/python/docs/pyspark.resource.rst +++ /dev/null @@ -1,11 +0,0 @@ -pyspark.resource module -======================= - -Module Contents ---------------- - -.. automodule:: pyspark.resource - :members: - :undoc-members: - :inherited-members: - diff --git a/python/docs/pyspark.rst b/python/docs/pyspark.rst deleted file mode 100644 index 402d6ce9eb016..0000000000000 --- a/python/docs/pyspark.rst +++ /dev/null @@ -1,21 +0,0 @@ -pyspark package -=============== - -Subpackages ------------ - -.. toctree:: - :maxdepth: 1 - - pyspark.sql - pyspark.streaming - pyspark.ml - pyspark.mllib - pyspark.resource - -Contents --------- - -.. automodule:: pyspark - :members: - :undoc-members: diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst deleted file mode 100644 index 406ada701941a..0000000000000 --- a/python/docs/pyspark.sql.rst +++ /dev/null @@ -1,37 +0,0 @@ -pyspark.sql module -================== - -Module Contents ---------------- - -.. automodule:: pyspark.sql - :members: - :undoc-members: - :inherited-members: - :exclude-members: builder -.. We need `exclude-members` to prevent default description generations - as a workaround for old Sphinx (< 1.6.6). - -pyspark.sql.types module ------------------------- -.. automodule:: pyspark.sql.types - :members: - :undoc-members: - -pyspark.sql.functions module ----------------------------- -.. automodule:: pyspark.sql.functions - :members: - :undoc-members: - -pyspark.sql.avro.functions module ---------------------------------- -.. automodule:: pyspark.sql.avro.functions - :members: - :undoc-members: - -pyspark.sql.streaming module ----------------------------- -.. automodule:: pyspark.sql.streaming - :members: - :undoc-members: diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst deleted file mode 100644 index f7df6438b9169..0000000000000 --- a/python/docs/pyspark.streaming.rst +++ /dev/null @@ -1,17 +0,0 @@ -pyspark.streaming module -======================== - -Module contents ---------------- - -.. automodule:: pyspark.streaming - :members: - :undoc-members: - :show-inheritance: - -pyspark.streaming.kinesis module --------------------------------- -.. automodule:: pyspark.streaming.kinesis - :members: - :undoc-members: - :show-inheritance: diff --git a/python/docs/_static/copybutton.js b/python/docs/source/_static/copybutton.js similarity index 100% rename from python/docs/_static/copybutton.js rename to python/docs/source/_static/copybutton.js diff --git a/python/docs/source/_static/css/pyspark.css b/python/docs/source/_static/css/pyspark.css new file mode 100644 index 0000000000000..2fd8720e2fa0d --- /dev/null +++ b/python/docs/source/_static/css/pyspark.css @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +/* PySpark style CSS overwrite */ + +/* Lato font (from its parent) does not make any difference when it's bold. Defaults to 'Source Sans Pro' */ +body { + font-family:"Source Sans Pro",sans-serif!important; +} + +h1,h2 { + color:#1B5162!important; +} + +h3 { + color: #555555 +} + +/* Top menu */ +#navbar-main { + background: #1B5162!important; + box-shadow: 0px 2px 4px rgba(0, 0, 0, 0.11); +} + +#navbar-main-elements li.nav-item a { + color: rgba(255, 255, 255, 0.8); +} + +#navbar-main-elements li.active a { + font-weight: 600; + color: #FFFFFF!important; +} + +.col-9 { + flex: 0 0 80%; + max-width: 80%; +} + +/* Left pannel size */ +@media (min-width: 768px) { + .col-md-3 { + flex: 0 0 20%; + max-width: 20%; + } +} + +/* Top menu right button */ +.navbar-toggler { + color:rgba(255,255,255,.5)!important; + border-color:rgba(255,255,255,.5)!important; +} + +.navbar-toggler-icon { + background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30'%3E%3Cpath stroke='rgba(255,255,255,.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3E%3C/svg%3E")!important; +} + +/* Left bar list colors */ +.bd-sidebar .nav>.active>a { + color: #1B5162!important; +} + +.bd-sidebar .nav>li>a:hover { + color: #1B5162!important; +} + +.bd-sidebar .nav>.active:hover>a,.bd-sidebar .nav>.active>a { + color: #1B5162!important; +} + +u.bd-sidebar .nav>li>ul>.active:hover>a,.bd-sidebar .nav>li>ul>.active>a { + color: #1B5162!important; +} + +/* Right bar list colors */ +.toc-entry>.nav-link.active { + color: #1B5162!important; + border-left: 2px solid #1B5162!important; +} + diff --git a/python/docs/source/_templates/class_with_docs.rst b/python/docs/source/_templates/class_with_docs.rst new file mode 100644 index 0000000000000..7c37b83c0e90e --- /dev/null +++ b/python/docs/source/_templates/class_with_docs.rst @@ -0,0 +1,79 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +{{ objname }} +{{ underline }} + +.. currentmodule:: {{ module }} + +.. autoclass:: {{ objname }} + + {% if '__init__' in methods %} + {% set caught_result = methods.remove('__init__') %} + {% endif %} + + {% block methods_summary %} + {% if methods %} + + .. rubric:: Methods + + .. autosummary:: + {% for item in methods %} + ~{{ name }}.{{ item }} + {%- endfor %} + + {% endif %} + {% endblock %} + + {% block attributes_summary %} + {% if attributes %} + + .. rubric:: Attributes + + .. autosummary:: + {% for item in attributes %} + ~{{ name }}.{{ item }} + {%- endfor %} + + {% endif %} + {% endblock %} + + {% block methods_documentation %} + {% if methods %} + + .. rubric:: Methods Documentation + + {% for item in methods %} + .. automethod:: {{ item }} + {%- endfor %} + + {% endif %} + {% endblock %} + + {% block attributes_documentation %} + {% if attributes %} + + .. rubric:: Attributes Documentation + + {% for item in attributes %} + .. autoattribute:: {{ item }} + {%- endfor %} + + {% endif %} + {% endblock %} + diff --git a/python/docs/conf.py b/python/docs/source/conf.py similarity index 89% rename from python/docs/conf.py rename to python/docs/source/conf.py index 9e7afb7c07298..7b1939d976080 100644 --- a/python/docs/conf.py +++ b/python/docs/source/conf.py @@ -14,12 +14,24 @@ import sys import os +import shutil +import errno # If extensions (or modules to document with autodoc) are in another directory, # add these directories to sys.path here. If the directory is relative to the # documentation root, use os.path.abspath to make it absolute, like shown here. sys.path.insert(0, os.path.abspath('.')) +# Remove previously generated rst files. Ignore errors just in case it stops +# generating whole docs. +shutil.rmtree( + "%s/reference/api" % os.path.dirname(os.path.abspath(__file__)), ignore_errors=True) +try: + os.mkdir("%s/reference/api" % os.path.dirname(os.path.abspath(__file__))) +except OSError as e: + if e.errno != errno.EEXIST: + raise + # -- General configuration ------------------------------------------------ # If your documentation needs a minimal Sphinx version, state it here. @@ -32,6 +44,7 @@ 'sphinx.ext.autodoc', 'sphinx.ext.viewcode', 'sphinx.ext.mathjax', + 'sphinx.ext.autosummary', ] # Add any paths that contain templates here, relative to this directory. @@ -47,8 +60,8 @@ master_doc = 'index' # General information about the project. -project = u'PySpark' -copyright = u'' +project = 'PySpark' +copyright = '' # The version info for the project you're documenting, acts as replacement for # |version| and |release|, also used in various other places throughout the @@ -101,12 +114,13 @@ # Look at the first line of the docstring for function and method signatures. autodoc_docstring_signature = True +autosummary_generate = True # -- Options for HTML output ---------------------------------------------- # The theme to use for HTML and HTML Help pages. See the documentation for # a list of builtin themes. -html_theme = 'nature' +html_theme = 'pydata_sphinx_theme' # Theme options are theme-specific and customize the look and feel of a theme # further. For a list of options available for each theme, see the @@ -125,7 +139,7 @@ # The name of an image file (relative to this directory) to place at the top # of the sidebar. -html_logo = "../../docs/img/spark-logo-hd.png" +html_logo = "../../../docs/img/spark-logo-reverse.png" # The name of an image file (within the static path) to use as favicon of the # docs. This file should be a Windows icon file (.ico) being 16x16 or 32x32 @@ -137,6 +151,10 @@ # so a file named "default.css" will overwrite the builtin "default.css". html_static_path = ['_static'] +html_css_files = [ + 'css/pyspark.css', +] + # Add any extra paths that contain custom files (such as robots.txt or # .htaccess) here, relative to this directory. These files are copied # directly to the root of the documentation. @@ -204,8 +222,8 @@ # (source start file, target name, title, # author, documentclass [howto, manual, or own class]). latex_documents = [ - ('index', 'pyspark.tex', u'pyspark Documentation', - u'Author', 'manual'), + ('index', 'pyspark.tex', 'pyspark Documentation', + 'Author', 'manual'), ] # The name of an image file (relative to this directory) to place at the top of @@ -234,8 +252,8 @@ # One entry per manual page. List of tuples # (source start file, name, description, authors, manual section). man_pages = [ - ('index', 'pyspark', u'pyspark Documentation', - [u'Author'], 1) + ('index', 'pyspark', 'pyspark Documentation', + ['Author'], 1) ] # If true, show URL addresses after external links. @@ -248,8 +266,8 @@ # (source start file, target name, title, author, # dir menu entry, description, category) texinfo_documents = [ - ('index', 'pyspark', u'pyspark Documentation', - u'Author', 'pyspark', 'One line description of project.', + ('index', 'pyspark', 'pyspark Documentation', + 'Author', 'pyspark', 'One line description of project.', 'Miscellaneous'), ] @@ -269,13 +287,13 @@ # -- Options for Epub output ---------------------------------------------- # Bibliographic Dublin Core info. -epub_title = u'pyspark' -epub_author = u'Author' -epub_publisher = u'Author' -epub_copyright = u'2014, Author' +epub_title = 'pyspark' +epub_author = 'Author' +epub_publisher = 'Author' +epub_copyright = '2014, Author' # The basename for the epub file. It defaults to the project name. -#epub_basename = u'pyspark' +#epub_basename = 'pyspark' # The HTML theme for the epub output. Since the default themes are not optimized # for small screen space, using the same theme for HTML and epub output is @@ -335,7 +353,8 @@ # If false, no index is generated. #epub_use_index = True def setup(app): - app.add_javascript('copybutton.js') + # The app.add_javascript() is deprecated. + getattr(app, "add_js_file", getattr(app, "add_javascript"))('copybutton.js') # Skip sample endpoint link (not expected to resolve) linkcheck_ignore = [r'https://kinesis.us-east-1.amazonaws.com'] diff --git a/python/docs/source/development/index.rst b/python/docs/source/development/index.rst new file mode 100644 index 0000000000000..db9f667332635 --- /dev/null +++ b/python/docs/source/development/index.rst @@ -0,0 +1,21 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +=========== +Development +=========== + diff --git a/python/docs/source/getting_started/index.rst b/python/docs/source/getting_started/index.rst new file mode 100644 index 0000000000000..457368c8194cb --- /dev/null +++ b/python/docs/source/getting_started/index.rst @@ -0,0 +1,22 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +=============== +Getting Started +=============== + diff --git a/python/docs/source/index.rst b/python/docs/source/index.rst new file mode 100644 index 0000000000000..34011ec7c5573 --- /dev/null +++ b/python/docs/source/index.rst @@ -0,0 +1,32 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. PySpark documentation master file + +===================== +PySpark Documentation +===================== + +.. toctree:: + :maxdepth: 2 + + getting_started/index + user_guide/index + reference/index + development/index + migration_guide/index + diff --git a/python/docs/source/migration_guide/index.rst b/python/docs/source/migration_guide/index.rst new file mode 100644 index 0000000000000..fc12668f81a58 --- /dev/null +++ b/python/docs/source/migration_guide/index.rst @@ -0,0 +1,22 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +=============== +Migration Guide +=============== + diff --git a/python/docs/source/reference/index.rst b/python/docs/source/reference/index.rst new file mode 100644 index 0000000000000..77f17da720dd5 --- /dev/null +++ b/python/docs/source/reference/index.rst @@ -0,0 +1,34 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +============= +API Reference +============= + +This page lists an overview of all public PySpark modules, classes, functions and methods. + +.. toctree:: + :maxdepth: 2 + + pyspark.sql + pyspark.ss + pyspark.ml + pyspark.streaming + pyspark.mllib + pyspark + pyspark.resource diff --git a/python/docs/source/reference/pyspark.ml.rst b/python/docs/source/reference/pyspark.ml.rst new file mode 100644 index 0000000000000..b6e7d10276603 --- /dev/null +++ b/python/docs/source/reference/pyspark.ml.rst @@ -0,0 +1,363 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +ML +== + +ML Pipeline APIs +---------------- + +.. currentmodule:: pyspark.ml + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Transformer + UnaryTransformer + Estimator + Model + Predictor + PredictionModel + Pipeline + PipelineModel + + +Parameters +---------- + +.. currentmodule:: pyspark.ml.param + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Param + Params + TypeConverters + + +Feature +------- + +.. currentmodule:: pyspark.ml.feature + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ANOVASelector + ANOVASelectorModel + Binarizer + BucketedRandomProjectionLSH + BucketedRandomProjectionLSHModel + Bucketizer + ChiSqSelector + ChiSqSelectorModel + CountVectorizer + CountVectorizerModel + DCT + ElementwiseProduct + FeatureHasher + FValueSelector + FValueSelectorModel + HashingTF + IDF + IDFModel + Imputer + ImputerModel + IndexToString + Interaction + MaxAbsScaler + MaxAbsScalerModel + MinHashLSH + MinHashLSHModel + MinMaxScaler + MinMaxScalerModel + NGram + Normalizer + OneHotEncoder + OneHotEncoderModel + PCA + PCAModel + PolynomialExpansion + QuantileDiscretizer + RobustScaler + RobustScalerModel + RegexTokenizer + RFormula + RFormulaModel + SQLTransformer + StandardScaler + StandardScalerModel + StopWordsRemover + StringIndexer + StringIndexerModel + Tokenizer + VarianceThresholdSelector + VarianceThresholdSelectorModel + VectorAssembler + VectorIndexer + VectorIndexerModel + VectorSizeHint + VectorSlicer + Word2Vec + Word2VecModel + + +Classification +-------------- + +.. currentmodule:: pyspark.ml.classification + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + LinearSVC + LinearSVCModel + LinearSVCSummary + LinearSVCTrainingSummary + LogisticRegression + LogisticRegressionModel + LogisticRegressionSummary + LogisticRegressionTrainingSummary + BinaryLogisticRegressionSummary + BinaryLogisticRegressionTrainingSummary + DecisionTreeClassifier + DecisionTreeClassificationModel + GBTClassifier + GBTClassificationModel + RandomForestClassifier + RandomForestClassificationModel + RandomForestClassificationSummary + RandomForestClassificationTrainingSummary + BinaryRandomForestClassificationSummary + BinaryRandomForestClassificationTrainingSummary + NaiveBayes + NaiveBayesModel + MultilayerPerceptronClassifier + MultilayerPerceptronClassificationModel + OneVsRest + OneVsRestModel + FMClassifier + FMClassificationModel + FMClassificationSummary + FMClassificationTrainingSummary + + +Clustering +---------- + +.. currentmodule:: pyspark.ml.clustering + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + BisectingKMeans + BisectingKMeansModel + BisectingKMeansSummary + KMeans + KMeansModel + GaussianMixture + GaussianMixtureModel + GaussianMixtureSummary + LDA + LDAModel + LocalLDAModel + DistributedLDAModel + PowerIterationClustering + + +ML Functions +---------------------------- + +.. currentmodule:: pyspark.ml.functions + +.. autosummary:: + :toctree: api/ + + vector_to_array + + +Vector and Matrix +----------------- + +.. currentmodule:: pyspark.ml.linalg + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Vector + DenseVector + SparseVector + Vectors + Matrix + DenseMatrix + SparseMatrix + Matrices + + +Recommendation +-------------- + +.. currentmodule:: pyspark.ml.recommendation + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ALS + ALSModel + + +Regression +---------- + +.. currentmodule:: pyspark.ml.regression + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + AFTSurvivalRegression + AFTSurvivalRegressionModel + DecisionTreeRegressor + DecisionTreeRegressionModel + GBTRegressor + GBTRegressionModel + GeneralizedLinearRegression + GeneralizedLinearRegressionModel + GeneralizedLinearRegressionSummary + GeneralizedLinearRegressionTrainingSummary + IsotonicRegression + IsotonicRegressionModel + LinearRegression + LinearRegressionModel + LinearRegressionSummary + LinearRegressionTrainingSummary + RandomForestRegressor + RandomForestRegressionModel + FMRegressor + FMRegressionModel + + +Statistics +---------- + +.. currentmodule:: pyspark.ml.stat + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ANOVATest + ChiSquareTest + Correlation + FValueTest + KolmogorovSmirnovTest + MultivariateGaussian + Summarizer + SummaryBuilder + + +Tuning +------ + +.. currentmodule:: pyspark.ml.tuning + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ParamGridBuilder + CrossValidator + CrossValidatorModel + TrainValidationSplit + TrainValidationSplitModel + + +Evaluation +---------- + +.. currentmodule:: pyspark.ml.evaluation + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Evaluator + BinaryClassificationEvaluator + RegressionEvaluator + MulticlassClassificationEvaluator + MultilabelClassificationEvaluator + ClusteringEvaluator + RankingEvaluator + + +Frequency Pattern Mining +---------------------------- + +.. currentmodule:: pyspark.ml.fpm + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + FPGrowth + FPGrowthModel + PrefixSpan + + +Image +----- + +.. currentmodule:: pyspark.ml.image + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ImageSchema + _ImageSchema + + +Utilities +--------- + +.. currentmodule:: pyspark.ml.util + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + BaseReadWrite + DefaultParamsReadable + DefaultParamsReader + DefaultParamsWritable + DefaultParamsWriter + GeneralMLWriter + HasTrainingSummary + Identifiable + MLReadable + MLReader + MLWritable + MLWriter + diff --git a/python/docs/source/reference/pyspark.mllib.rst b/python/docs/source/reference/pyspark.mllib.rst new file mode 100644 index 0000000000000..1251b1df752c7 --- /dev/null +++ b/python/docs/source/reference/pyspark.mllib.rst @@ -0,0 +1,253 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +MLlib +===== + +Classification +-------------- + +.. currentmodule:: pyspark.mllib.classification + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + LogisticRegressionModel + LogisticRegressionWithSGD + LogisticRegressionWithLBFGS + SVMModel + SVMWithSGD + NaiveBayesModel + NaiveBayes + StreamingLogisticRegressionWithSGD + + +Clustering +---------- + +.. currentmodule:: pyspark.mllib.clustering + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + + BisectingKMeansModel + BisectingKMeans + KMeansModel + KMeans + GaussianMixtureModel + GaussianMixture + PowerIterationClusteringModel + PowerIterationClustering + StreamingKMeans + StreamingKMeansModel + LDA + LDAModel + + +Evaluation +---------- + +.. currentmodule:: pyspark.mllib.evaluation + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + BinaryClassificationMetrics + RegressionMetrics + MulticlassMetrics + RankingMetrics + + +Feature +------- + +.. currentmodule:: pyspark.mllib.feature + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Normalizer + StandardScalerModel + StandardScaler + HashingTF + IDFModel + IDF + Word2Vec + Word2VecModel + ChiSqSelector + ChiSqSelectorModel + ElementwiseProduct + + +Frequency Pattern Mining +------------------------ + +.. currentmodule:: pyspark.mllib.fpm + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + FPGrowth + FPGrowthModel + PrefixSpan + PrefixSpanModel + + +Vector and Matrix +----------------- + +.. currentmodule:: pyspark.mllib.linalg + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Vector + DenseVector + SparseVector + Vectors + Matrix + DenseMatrix + SparseMatrix + Matrices + QRDecomposition + + +Distributed Representation +~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. currentmodule:: pyspark.mllib.linalg.distributed + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + BlockMatrix + CoordinateMatrix + DistributedMatrix + IndexedRow + IndexedRowMatrix + MatrixEntry + RowMatrix + SingularValueDecomposition + + +Random +------ + +.. currentmodule:: pyspark.mllib.random + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + RandomRDDs + + +Recommendation +-------------- + +.. currentmodule:: pyspark.mllib.recommendation + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + MatrixFactorizationModel + ALS + Rating + + +Regression +---------- + +.. currentmodule:: pyspark.mllib.regression + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + LabeledPoint + LinearModel + LinearRegressionModel + LinearRegressionWithSGD + RidgeRegressionModel + RidgeRegressionWithSGD + LassoModel + LassoWithSGD + IsotonicRegressionModel + IsotonicRegression + StreamingLinearAlgorithm + StreamingLinearRegressionWithSGD + + +Statistics +---------- + +.. currentmodule:: pyspark.mllib.stat + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Statistics + MultivariateStatisticalSummary + ChiSqTestResult + MultivariateGaussian + KernelDensity + + +Tree +---- + +.. currentmodule:: pyspark.mllib.tree + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + DecisionTreeModel + DecisionTree + RandomForestModel + RandomForest + GradientBoostedTreesModel + GradientBoostedTrees + + +Utilities +--------- + +.. currentmodule:: pyspark.mllib.util + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + JavaLoader + JavaSaveable + LinearDataGenerator + Loader + MLUtils + Saveable + diff --git a/python/docs/source/reference/pyspark.resource.rst b/python/docs/source/reference/pyspark.resource.rst new file mode 100644 index 0000000000000..a1d885c44c480 --- /dev/null +++ b/python/docs/source/reference/pyspark.resource.rst @@ -0,0 +1,38 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +=================== +Resource Management +=================== + +Core Classes +------------ + +.. currentmodule:: pyspark.resource + +.. autosummary:: + :toctree: api/ + + ResourceInformation + ResourceProfile + ResourceProfileBuilder + ExecutorResourceRequest + ExecutorResourceRequests + TaskResourceRequest + TaskResourceRequests + diff --git a/python/docs/source/reference/pyspark.rst b/python/docs/source/reference/pyspark.rst new file mode 100644 index 0000000000000..b50ae37b99690 --- /dev/null +++ b/python/docs/source/reference/pyspark.rst @@ -0,0 +1,275 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +========== +Spark Core +========== + +Public Classes +-------------- + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + SparkContext + RDD + Broadcast + Accumulator + SparkConf + SparkFiles + StorageLevel + TaskContext + RDDBarrier + BarrierTaskContext + BarrierTaskInfo + + +Spark Context APIs +------------------ + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + SparkContext.PACKAGE_EXTENSIONS + SparkContext.accumulator + SparkContext.addFile + SparkContext.addPyFile + SparkContext.applicationId + SparkContext.binaryFiles + SparkContext.binaryRecords + SparkContext.broadcast + SparkContext.cancelAllJobs + SparkContext.cancelJobGroup + SparkContext.defaultMinPartitions + SparkContext.defaultParallelism + SparkContext.dump_profiles + SparkContext.emptyRDD + SparkContext.getConf + SparkContext.getLocalProperty + SparkContext.getOrCreate + SparkContext.hadoopFile + SparkContext.hadoopRDD + SparkContext.newAPIHadoopFile + SparkContext.newAPIHadoopRDD + SparkContext.parallelize + SparkContext.pickleFile + SparkContext.range + SparkContext.resources + SparkContext.runJob + SparkContext.sequenceFile + SparkContext.setCheckpointDir + SparkContext.setJobDescription + SparkContext.setJobGroup + SparkContext.setLocalProperty + SparkContext.setLogLevel + SparkContext.setSystemProperty + SparkContext.show_profiles + SparkContext.sparkUser + SparkContext.startTime + SparkContext.statusTracker + SparkContext.stop + SparkContext.textFile + SparkContext.uiWebUrl + SparkContext.union + SparkContext.version + SparkContext.wholeTextFiles + + +RDD APIs +-------- + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + RDD.aggregate + RDD.aggregateByKey + RDD.barrier + RDD.cache + RDD.cartesian + RDD.checkpoint + RDD.coalesce + RDD.cogroup + RDD.collect + RDD.collectAsMap + RDD.collectWithJobGroup + RDD.combineByKey + RDD.context + RDD.count + RDD.countApprox + RDD.countApproxDistinct + RDD.countByKey + RDD.countByValue + RDD.distinct + RDD.filter + RDD.first + RDD.flatMap + RDD.flatMapValues + RDD.fold + RDD.foldByKey + RDD.foreach + RDD.foreachPartition + RDD.fullOuterJoin + RDD.getCheckpointFile + RDD.getNumPartitions + RDD.getResourceProfile + RDD.getStorageLevel + RDD.glom + RDD.groupBy + RDD.groupByKey + RDD.groupWith + RDD.histogram + RDD.id + RDD.intersection + RDD.isCheckpointed + RDD.isEmpty + RDD.isLocallyCheckpointed + RDD.join + RDD.keyBy + RDD.keys + RDD.leftOuterJoin + RDD.localCheckpoint + RDD.lookup + RDD.map + RDD.mapPartitions + RDD.mapPartitionsWithIndex + RDD.mapPartitionsWithSplit + RDD.mapValues + RDD.max + RDD.mean + RDD.meanApprox + RDD.min + RDD.name + RDD.partitionBy + RDD.persist + RDD.pipe + RDD.randomSplit + RDD.reduce + RDD.reduceByKey + RDD.reduceByKeyLocally + RDD.repartition + RDD.repartitionAndSortWithinPartitions + RDD.rightOuterJoin + RDD.sample + RDD.sampleByKey + RDD.sampleStdev + RDD.sampleVariance + RDD.saveAsHadoopDataset + RDD.saveAsHadoopFile + RDD.saveAsNewAPIHadoopDataset + RDD.saveAsNewAPIHadoopFile + RDD.saveAsPickleFile + RDD.saveAsSequenceFile + RDD.saveAsTextFile + RDD.setName + RDD.sortBy + RDD.sortByKey + RDD.stats + RDD.stdev + RDD.subtract + RDD.subtractByKey + RDD.sum + RDD.sumApprox + RDD.take + RDD.takeOrdered + RDD.takeSample + RDD.toDebugString + RDD.toLocalIterator + RDD.top + RDD.treeAggregate + RDD.treeReduce + RDD.union + RDD.unpersist + RDD.values + RDD.variance + RDD.withResources + RDD.zip + RDD.zipWithIndex + RDD.zipWithUniqueId + + +Broadcast and Accumulator +------------------------- + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + Broadcast.destroy + Broadcast.dump + Broadcast.load + Broadcast.load_from_path + Broadcast.unpersist + Broadcast.value + Accumulator.add + Accumulator.value + + +Management +---------- + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + SparkConf.contains + SparkConf.get + SparkConf.getAll + SparkConf.set + SparkConf.setAll + SparkConf.setAppName + SparkConf.setExecutorEnv + SparkConf.setIfMissing + SparkConf.setMaster + SparkConf.setSparkHome + SparkConf.toDebugString + SparkFiles.get + SparkFiles.getRootDirectory + StorageLevel.DISK_ONLY + StorageLevel.DISK_ONLY_2 + StorageLevel.MEMORY_AND_DISK + StorageLevel.MEMORY_AND_DISK_2 + StorageLevel.MEMORY_ONLY + StorageLevel.MEMORY_ONLY_2 + StorageLevel.OFF_HEAP + TaskContext.attemptNumber + TaskContext.get + TaskContext.getLocalProperty + TaskContext.partitionId + TaskContext.resources + TaskContext.stageId + TaskContext.taskAttemptId + RDDBarrier.mapPartitions + RDDBarrier.mapPartitionsWithIndex + BarrierTaskContext.allGather + BarrierTaskContext.attemptNumber + BarrierTaskContext.barrier + BarrierTaskContext.get + BarrierTaskContext.getLocalProperty + BarrierTaskContext.getTaskInfos + BarrierTaskContext.partitionId + BarrierTaskContext.resources + BarrierTaskContext.stageId + BarrierTaskContext.taskAttemptId diff --git a/python/docs/source/reference/pyspark.sql.rst b/python/docs/source/reference/pyspark.sql.rst new file mode 100644 index 0000000000000..7e0357cf9d858 --- /dev/null +++ b/python/docs/source/reference/pyspark.sql.rst @@ -0,0 +1,542 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +========= +Spark SQL +========= + +Core Classes +------------ + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + SparkSession + DataFrame + Column + Row + GroupedData + DataFrameNaFunctions + DataFrameStatFunctions + Window + + +Spark Session APIs +------------------ + +.. currentmodule:: pyspark.sql + +The entry point to programming Spark with the Dataset and DataFrame API. +To create a Spark session, you should use ``SparkSession.builder`` attribute. +See also :class:`SparkSession`. + +.. autosummary:: + :toctree: api/ + + SparkSession.builder.appName + SparkSession.builder.config + SparkSession.builder.enableHiveSupport + SparkSession.builder.getOrCreate + SparkSession.builder.master + SparkSession.catalog + SparkSession.conf + SparkSession.createDataFrame + SparkSession.getActiveSession + SparkSession.newSession + SparkSession.range + SparkSession.read + SparkSession.readStream + SparkSession.sparkContext + SparkSession.sql + SparkSession.stop + SparkSession.streams + SparkSession.table + SparkSession.udf + SparkSession.version + + +Input and Output +---------------- + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + DataFrameReader.csv + DataFrameReader.format + DataFrameReader.jdbc + DataFrameReader.json + DataFrameReader.load + DataFrameReader.option + DataFrameReader.options + DataFrameReader.orc + DataFrameReader.parquet + DataFrameReader.schema + DataFrameReader.table + DataFrameWriter.bucketBy + DataFrameWriter.csv + DataFrameWriter.format + DataFrameWriter.insertInto + DataFrameWriter.jdbc + DataFrameWriter.json + DataFrameWriter.mode + DataFrameWriter.option + DataFrameWriter.options + DataFrameWriter.orc + DataFrameWriter.parquet + DataFrameWriter.partitionBy + DataFrameWriter.save + DataFrameWriter.saveAsTable + DataFrameWriter.sortBy + DataFrameWriter.text + + +DataFrame APIs +-------------- + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + DataFrame.agg + DataFrame.alias + DataFrame.approxQuantile + DataFrame.cache + DataFrame.checkpoint + DataFrame.coalesce + DataFrame.colRegex + DataFrame.collect + DataFrame.columns + DataFrame.corr + DataFrame.count + DataFrame.cov + DataFrame.createGlobalTempView + DataFrame.createOrReplaceGlobalTempView + DataFrame.createOrReplaceTempView + DataFrame.createTempView + DataFrame.crossJoin + DataFrame.crosstab + DataFrame.cube + DataFrame.describe + DataFrame.distinct + DataFrame.drop + DataFrame.dropDuplicates + DataFrame.drop_duplicates + DataFrame.dropna + DataFrame.dtypes + DataFrame.exceptAll + DataFrame.explain + DataFrame.fillna + DataFrame.filter + DataFrame.first + DataFrame.foreach + DataFrame.foreachPartition + DataFrame.freqItems + DataFrame.groupBy + DataFrame.head + DataFrame.hint + DataFrame.inputFiles + DataFrame.intersect + DataFrame.intersectAll + DataFrame.isLocal + DataFrame.isStreaming + DataFrame.join + DataFrame.limit + DataFrame.localCheckpoint + DataFrame.mapInPandas + DataFrame.na + DataFrame.orderBy + DataFrame.persist + DataFrame.printSchema + DataFrame.randomSplit + DataFrame.rdd + DataFrame.registerTempTable + DataFrame.repartition + DataFrame.repartitionByRange + DataFrame.replace + DataFrame.rollup + DataFrame.sameSemantics + DataFrame.sample + DataFrame.sampleBy + DataFrame.schema + DataFrame.select + DataFrame.selectExpr + DataFrame.semanticHash + DataFrame.show + DataFrame.sort + DataFrame.sortWithinPartitions + DataFrame.stat + DataFrame.storageLevel + DataFrame.subtract + DataFrame.summary + DataFrame.tail + DataFrame.take + DataFrame.toDF + DataFrame.toJSON + DataFrame.toLocalIterator + DataFrame.toPandas + DataFrame.transform + DataFrame.union + DataFrame.unionAll + DataFrame.unionByName + DataFrame.unpersist + DataFrame.where + DataFrame.withColumn + DataFrame.withColumnRenamed + DataFrame.withWatermark + DataFrame.write + DataFrame.writeStream + DataFrame.writeTo + DataFrameNaFunctions.drop + DataFrameNaFunctions.fill + DataFrameNaFunctions.replace + DataFrameStatFunctions.approxQuantile + DataFrameStatFunctions.corr + DataFrameStatFunctions.cov + DataFrameStatFunctions.crosstab + DataFrameStatFunctions.freqItems + DataFrameStatFunctions.sampleBy + + +Data Types +---------- + +.. currentmodule:: pyspark.sql.types + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ArrayType + BinaryType + BooleanType + ByteType + DataType + DateType + DecimalType + DoubleType + FloatType + IntegerType + LongType + MapType + NullType + ShortType + StringType + StructField + StructType + TimestampType + + +Row +--- + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + Row.asDict + + +Functions +--------- + +.. currentmodule:: pyspark.sql.functions + +.. autosummary:: + :toctree: api/ + + abs + acos + add_months + aggregate + approxCountDistinct + approx_count_distinct + array + array_contains + array_distinct + array_except + array_intersect + array_join + array_max + array_min + array_position + array_remove + array_repeat + array_sort + array_union + arrays_overlap + arrays_zip + asc + asc_nulls_first + asc_nulls_last + ascii + asin + atan + atan2 + avg + base64 + bin + bitwiseNOT + broadcast + bround + bucket + cbrt + ceil + coalesce + col + collect_list + collect_set + column + concat + concat_ws + conv + corr + cos + cosh + count + countDistinct + covar_pop + covar_samp + crc32 + create_map + cume_dist + current_date + current_timestamp + date_add + date_format + date_sub + date_trunc + datediff + dayofmonth + dayofweek + dayofyear + days + decode + degrees + dense_rank + desc + desc_nulls_first + desc_nulls_last + element_at + encode + exists + exp + explode + explode_outer + expm1 + expr + factorial + filter + first + flatten + floor + forall + format_number + format_string + from_csv + from_json + from_unixtime + from_utc_timestamp + get_json_object + greatest + grouping + grouping_id + hash + hex + hour + hours + hypot + initcap + input_file_name + instr + isnan + isnull + json_tuple + kurtosis + lag + last + last_day + lead + least + length + levenshtein + lit + locate + log + log10 + log1p + log2 + lower + lpad + ltrim + map_concat + map_entries + map_filter + map_from_arrays + map_from_entries + map_keys + map_values + map_zip_with + max + md5 + mean + min + minute + monotonically_increasing_id + month + months + months_between + nanvl + next_day + ntile + overlay + pandas_udf + percent_rank + percentile_approx + posexplode + posexplode_outer + pow + quarter + radians + rand + randn + rank + regexp_extract + regexp_replace + repeat + reverse + rint + round + row_number + rpad + rtrim + schema_of_csv + schema_of_json + second + sequence + sha1 + sha2 + shiftLeft + shiftRight + shiftRightUnsigned + shuffle + signum + sin + sinh + size + skewness + slice + sort_array + soundex + spark_partition_id + split + sqrt + stddev + stddev_pop + stddev_samp + struct + substring + substring_index + sum + sumDistinct + tan + tanh + timestamp_seconds + toDegrees + toRadians + to_csv + to_date + to_json + to_timestamp + to_utc_timestamp + transform + transform_keys + transform_values + translate + trim + trunc + udf + unbase64 + unhex + unix_timestamp + upper + var_pop + var_samp + variance + weekofyear + when + window + xxhash64 + year + years + zip_with + + +.. currentmodule:: pyspark.sql.avro.functions + +.. autosummary:: + :toctree: api/ + + from_avro + to_avro + +Window +------ + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + Window.currentRow + Window.orderBy + Window.partitionBy + Window.rangeBetween + Window.rowsBetween + Window.unboundedFollowing + Window.unboundedPreceding + WindowSpec.orderBy + WindowSpec.partitionBy + WindowSpec.rangeBetween + WindowSpec.rowsBetween + +Grouping +-------- + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + GroupedData.agg + GroupedData.apply + GroupedData.applyInPandas + GroupedData.avg + GroupedData.cogroup + GroupedData.count + GroupedData.max + GroupedData.mean + GroupedData.min + GroupedData.pivot + GroupedData.sum + diff --git a/python/docs/source/reference/pyspark.ss.rst b/python/docs/source/reference/pyspark.ss.rst new file mode 100644 index 0000000000000..a7936a4f2a59c --- /dev/null +++ b/python/docs/source/reference/pyspark.ss.rst @@ -0,0 +1,90 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +==================== +Structured Streaming +==================== + +Core Classes +------------ + +.. currentmodule:: pyspark.sql.streaming + +.. autosummary:: + :toctree: api/ + + DataStreamReader + DataStreamWriter + ForeachBatchFunction + StreamingQuery + StreamingQueryException + StreamingQueryManager + +Input and Output +---------------- + +.. currentmodule:: pyspark.sql.streaming + +.. autosummary:: + :toctree: api/ + + DataStreamReader.csv + DataStreamReader.format + DataStreamReader.json + DataStreamReader.load + DataStreamReader.option + DataStreamReader.options + DataStreamReader.orc + DataStreamReader.parquet + DataStreamReader.schema + DataStreamReader.text + DataStreamWriter.foreach + DataStreamWriter.foreachBatch + DataStreamWriter.format + DataStreamWriter.option + DataStreamWriter.options + DataStreamWriter.outputMode + DataStreamWriter.partitionBy + DataStreamWriter.queryName + DataStreamWriter.start + DataStreamWriter.trigger + +Query Management +---------------- + +.. currentmodule:: pyspark.sql.streaming + +.. autosummary:: + :toctree: api/ + + StreamingQuery.awaitTermination + StreamingQuery.exception + StreamingQuery.explain + StreamingQuery.id + StreamingQuery.isActive + StreamingQuery.lastProgress + StreamingQuery.name + StreamingQuery.processAllAvailable + StreamingQuery.recentProgress + StreamingQuery.runId + StreamingQuery.status + StreamingQuery.stop + StreamingQueryManager.active + StreamingQueryManager.awaitAnyTermination + StreamingQueryManager.get + StreamingQueryManager.resetTerminated diff --git a/python/docs/source/reference/pyspark.streaming.rst b/python/docs/source/reference/pyspark.streaming.rst new file mode 100644 index 0000000000000..57cbd00b67e4c --- /dev/null +++ b/python/docs/source/reference/pyspark.streaming.rst @@ -0,0 +1,130 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +=============== +Spark Streaming +=============== + +Core Classes +------------ + +.. currentmodule:: pyspark.streaming + +.. autosummary:: + :toctree: api/ + + StreamingContext + DStream + + +Streaming Management +-------------------- + +.. currentmodule:: pyspark.streaming + +.. autosummary:: + :toctree: api/ + + StreamingContext.addStreamingListener + StreamingContext.awaitTermination + StreamingContext.awaitTerminationOrTimeout + StreamingContext.checkpoint + StreamingContext.getActive + StreamingContext.getActiveOrCreate + StreamingContext.getOrCreate + StreamingContext.remember + StreamingContext.sparkContext + StreamingContext.start + StreamingContext.stop + StreamingContext.transform + StreamingContext.union + + +Input and Output +---------------- + +.. autosummary:: + :toctree: api/ + + StreamingContext.binaryRecordsStream + StreamingContext.queueStream + StreamingContext.socketTextStream + StreamingContext.textFileStream + DStream.pprint + DStream.saveAsTextFiles + + +Transformations and Actions +--------------------------- + +.. currentmodule:: pyspark.streaming + +.. autosummary:: + :toctree: api/ + + DStream.cache + DStream.checkpoint + DStream.cogroup + DStream.combineByKey + DStream.context + DStream.count + DStream.countByValue + DStream.countByValueAndWindow + DStream.countByWindow + DStream.filter + DStream.flatMap + DStream.flatMapValues + DStream.foreachRDD + DStream.fullOuterJoin + DStream.glom + DStream.groupByKey + DStream.groupByKeyAndWindow + DStream.join + DStream.leftOuterJoin + DStream.map + DStream.mapPartitions + DStream.mapPartitionsWithIndex + DStream.mapValues + DStream.partitionBy + DStream.persist + DStream.reduce + DStream.reduceByKey + DStream.reduceByKeyAndWindow + DStream.reduceByWindow + DStream.repartition + DStream.rightOuterJoin + DStream.slice + DStream.transform + DStream.transformWith + DStream.union + DStream.updateStateByKey + DStream.window + + +Kinesis +------- + +.. currentmodule:: pyspark.streaming.kinesis + +.. autosummary:: + :toctree: api/ + + KinesisUtils.createStream + InitialPositionInStream.LATEST + InitialPositionInStream.TRIM_HORIZON + diff --git a/python/docs/source/user_guide/index.rst b/python/docs/source/user_guide/index.rst new file mode 100644 index 0000000000000..e8a8d905f46d1 --- /dev/null +++ b/python/docs/source/user_guide/index.rst @@ -0,0 +1,22 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +========== +User Guide +========== + diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index d7ac6d89c045a..02a6f0022e7ab 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1196,6 +1196,8 @@ def to_date(col, format=None): By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format is omitted. Equivalent to ``col.cast("date")``. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_date(df.t).alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] @@ -1219,6 +1221,8 @@ def to_timestamp(col, format=None): By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format is omitted. Equivalent to ``col.cast("timestamp")``. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_timestamp(df.t).alias('dt')).collect() [Row(dt=datetime.datetime(1997, 2, 28, 10, 30))] diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 6925adf567fb6..e5553a8bb162b 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -335,6 +335,9 @@ def parquet(self, *paths, **options): :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> df = spark.read.parquet('python/test_support/sql/parquet_partitioned') >>> df.dtypes [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')] @@ -367,6 +370,9 @@ def text(self, paths, wholetext=False, lineSep=None, pathGlobFilter=None, :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> df = spark.read.text('python/test_support/sql/text-test.txt') >>> df.collect() [Row(value='hello'), Row(value='this')] @@ -502,6 +508,10 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df = spark.read.csv('python/test_support/sql/ages.csv') >>> df.dtypes [('_c0', 'string'), ('_c1', 'string')] @@ -561,6 +571,9 @@ def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=N :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> df = spark.read.orc('python/test_support/sql/orc_partitioned') >>> df.dtypes [('a', 'bigint'), ('b', 'int'), ('c', 'int')] @@ -893,6 +906,8 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm :param ignoreNullFields: Whether to ignore null fields when generating JSON objects. If None is set, it uses the default value, ``true``. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) @@ -1007,6 +1022,8 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No :param lineSep: defines the line separator that should be used for writing. If None is set, it uses the default value, ``\\n``. Maximum length is 1 character. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 5c528c1d54df7..4ec47305aa13e 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -541,6 +541,9 @@ def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=N :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp()) >>> orc_sdf.isStreaming True @@ -571,6 +574,9 @@ def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLook :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> parquet_sdf = spark.readStream.schema(sdf_schema).parquet(tempfile.mkdtemp()) >>> parquet_sdf.isStreaming True @@ -607,6 +613,9 @@ def text(self, path, wholetext=False, lineSep=None, pathGlobFilter=None, :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> text_sdf = spark.readStream.text(tempfile.mkdtemp()) >>> text_sdf.isStreaming True @@ -737,6 +746,10 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema) >>> csv_sdf.isStreaming True From a82aee044127825ffefa0ed09b0ae5b987b9dd21 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 27 Jul 2020 20:10:13 +0900 Subject: [PATCH 294/384] [SPARK-32435][PYTHON] Remove heapq3 port from Python 3 ### What changes were proposed in this pull request? This PR removes the manual port of `heapq3.py` introduced from SPARK-3073. The main reason of this was to support Python 2.6 and 2.7 because Python 2's `heapq.merge()` doesn't not support `key` and `reverse`. See - https://docs.python.org/2/library/heapq.html#heapq.merge in Python 2 - https://docs.python.org/3.8/library/heapq.html#heapq.merge in Python 3 Since we dropped the Python 2 at SPARK-32138, we can remove this away. ### Why are the changes needed? To remove unnecessary codes. Also, we can leverage bug fixes made in Python 3.x at `heapq`. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Existing tests should cover. I locally ran and verified: ```bash ./python/run-tests --python-executable=python3 --testname="pyspark.tests.test_shuffle" ./python/run-tests --python-executable=python3 --testname="pyspark.shuffle ExternalSorter" ./python/run-tests --python-executable=python3 --testname="pyspark.tests.test_rdd RDDTests.test_external_group_by_key" ``` Closes #29229 from HyukjinKwon/SPARK-32435. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- LICENSE | 1 - LICENSE-binary | 6 - dev/.rat-excludes | 1 - dev/tox.ini | 2 +- licenses-binary/LICENSE-heapq.txt | 280 ---------- licenses/LICENSE-heapq.txt | 49 -- python/pylintrc | 2 +- python/pyspark/heapq3.py | 890 ------------------------------ python/pyspark/shuffle.py | 6 +- 9 files changed, 5 insertions(+), 1232 deletions(-) delete mode 100644 licenses-binary/LICENSE-heapq.txt delete mode 100644 licenses/LICENSE-heapq.txt delete mode 100644 python/pyspark/heapq3.py diff --git a/LICENSE b/LICENSE index 8cec4f5ea5379..df6bed16f4471 100644 --- a/LICENSE +++ b/LICENSE @@ -222,7 +222,6 @@ external/spark-ganglia-lgpl/src/main/java/com/codahale/metrics/ganglia/GangliaRe Python Software Foundation License ---------------------------------- -pyspark/heapq3.py python/docs/source/_static/copybutton.js BSD 3-Clause diff --git a/LICENSE-binary b/LICENSE-binary index b50da6be4e697..d363661b1cc7e 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -557,12 +557,6 @@ jakarta.ws.rs:jakarta.ws.rs-api https://github.com/eclipse-ee4j/jaxrs-api org.glassfish.hk2.external:jakarta.inject -Python Software Foundation License ----------------------------------- - -pyspark/heapq3.py - - Public Domain ------------- diff --git a/dev/.rat-excludes b/dev/.rat-excludes index db6a4ce644edd..3889dc9ba3d3a 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -49,7 +49,6 @@ jsonFormatter.min.js .*log pyspark-coverage-site/* cloudpickle/* -heapq3.py join.py SparkExprTyper.scala SparkILoop.scala diff --git a/dev/tox.ini b/dev/tox.ini index e25595aa6c9a6..5bf27d1abde9b 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -16,4 +16,4 @@ [pycodestyle] ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 -exclude=python/pyspark/cloudpickle/*.py,heapq3.py,shared.py,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* +exclude=python/pyspark/cloudpickle/*.py,shared.py,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* diff --git a/licenses-binary/LICENSE-heapq.txt b/licenses-binary/LICENSE-heapq.txt deleted file mode 100644 index 0c4c4b954bea4..0000000000000 --- a/licenses-binary/LICENSE-heapq.txt +++ /dev/null @@ -1,280 +0,0 @@ - -# A. HISTORY OF THE SOFTWARE -# ========================== -# -# Python was created in the early 1990s by Guido van Rossum at Stichting -# Mathematisch Centrum (CWI, see http://www.cwi.nl) in the Netherlands -# as a successor of a language called ABC. Guido remains Python's -# principal author, although it includes many contributions from others. -# -# In 1995, Guido continued his work on Python at the Corporation for -# National Research Initiatives (CNRI, see http://www.cnri.reston.va.us) -# in Reston, Virginia where he released several versions of the -# software. -# -# In May 2000, Guido and the Python core development team moved to -# BeOpen.com to form the BeOpen PythonLabs team. In October of the same -# year, the PythonLabs team moved to Digital Creations (now Zope -# Corporation, see http://www.zope.com). In 2001, the Python Software -# Foundation (PSF, see http://www.python.org/psf/) was formed, a -# non-profit organization created specifically to own Python-related -# Intellectual Property. Zope Corporation is a sponsoring member of -# the PSF. -# -# All Python releases are Open Source (see http://www.opensource.org for -# the Open Source Definition). Historically, most, but not all, Python -# releases have also been GPL-compatible; the table below summarizes -# the various releases. -# -# Release Derived Year Owner GPL- -# from compatible? (1) -# -# 0.9.0 thru 1.2 1991-1995 CWI yes -# 1.3 thru 1.5.2 1.2 1995-1999 CNRI yes -# 1.6 1.5.2 2000 CNRI no -# 2.0 1.6 2000 BeOpen.com no -# 1.6.1 1.6 2001 CNRI yes (2) -# 2.1 2.0+1.6.1 2001 PSF no -# 2.0.1 2.0+1.6.1 2001 PSF yes -# 2.1.1 2.1+2.0.1 2001 PSF yes -# 2.2 2.1.1 2001 PSF yes -# 2.1.2 2.1.1 2002 PSF yes -# 2.1.3 2.1.2 2002 PSF yes -# 2.2.1 2.2 2002 PSF yes -# 2.2.2 2.2.1 2002 PSF yes -# 2.2.3 2.2.2 2003 PSF yes -# 2.3 2.2.2 2002-2003 PSF yes -# 2.3.1 2.3 2002-2003 PSF yes -# 2.3.2 2.3.1 2002-2003 PSF yes -# 2.3.3 2.3.2 2002-2003 PSF yes -# 2.3.4 2.3.3 2004 PSF yes -# 2.3.5 2.3.4 2005 PSF yes -# 2.4 2.3 2004 PSF yes -# 2.4.1 2.4 2005 PSF yes -# 2.4.2 2.4.1 2005 PSF yes -# 2.4.3 2.4.2 2006 PSF yes -# 2.4.4 2.4.3 2006 PSF yes -# 2.5 2.4 2006 PSF yes -# 2.5.1 2.5 2007 PSF yes -# 2.5.2 2.5.1 2008 PSF yes -# 2.5.3 2.5.2 2008 PSF yes -# 2.6 2.5 2008 PSF yes -# 2.6.1 2.6 2008 PSF yes -# 2.6.2 2.6.1 2009 PSF yes -# 2.6.3 2.6.2 2009 PSF yes -# 2.6.4 2.6.3 2009 PSF yes -# 2.6.5 2.6.4 2010 PSF yes -# 2.7 2.6 2010 PSF yes -# -# Footnotes: -# -# (1) GPL-compatible doesn't mean that we're distributing Python under -# the GPL. All Python licenses, unlike the GPL, let you distribute -# a modified version without making your changes open source. The -# GPL-compatible licenses make it possible to combine Python with -# other software that is released under the GPL; the others don't. -# -# (2) According to Richard Stallman, 1.6.1 is not GPL-compatible, -# because its license has a choice of law clause. According to -# CNRI, however, Stallman's lawyer has told CNRI's lawyer that 1.6.1 -# is "not incompatible" with the GPL. -# -# Thanks to the many outside volunteers who have worked under Guido's -# direction to make these releases possible. -# -# -# B. TERMS AND CONDITIONS FOR ACCESSING OR OTHERWISE USING PYTHON -# =============================================================== -# -# PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 -# -------------------------------------------- -# -# 1. This LICENSE AGREEMENT is between the Python Software Foundation -# ("PSF"), and the Individual or Organization ("Licensee") accessing and -# otherwise using this software ("Python") in source or binary form and -# its associated documentation. -# -# 2. Subject to the terms and conditions of this License Agreement, PSF hereby -# grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, -# analyze, test, perform and/or display publicly, prepare derivative works, -# distribute, and otherwise use Python alone or in any derivative version, -# provided, however, that PSF's License Agreement and PSF's notice of copyright, -# i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, -# 2011, 2012, 2013 Python Software Foundation; All Rights Reserved" are retained -# in Python alone or in any derivative version prepared by Licensee. -# -# 3. In the event Licensee prepares a derivative work that is based on -# or incorporates Python or any part thereof, and wants to make -# the derivative work available to others as provided herein, then -# Licensee hereby agrees to include in any such work a brief summary of -# the changes made to Python. -# -# 4. PSF is making Python available to Licensee on an "AS IS" -# basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -# FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, -# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 6. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 7. Nothing in this License Agreement shall be deemed to create any -# relationship of agency, partnership, or joint venture between PSF and -# Licensee. This License Agreement does not grant permission to use PSF -# trademarks or trade name in a trademark sense to endorse or promote -# products or services of Licensee, or any third party. -# -# 8. By copying, installing or otherwise using Python, Licensee -# agrees to be bound by the terms and conditions of this License -# Agreement. -# -# -# BEOPEN.COM LICENSE AGREEMENT FOR PYTHON 2.0 -# ------------------------------------------- -# -# BEOPEN PYTHON OPEN SOURCE LICENSE AGREEMENT VERSION 1 -# -# 1. This LICENSE AGREEMENT is between BeOpen.com ("BeOpen"), having an -# office at 160 Saratoga Avenue, Santa Clara, CA 95051, and the -# Individual or Organization ("Licensee") accessing and otherwise using -# this software in source or binary form and its associated -# documentation ("the Software"). -# -# 2. Subject to the terms and conditions of this BeOpen Python License -# Agreement, BeOpen hereby grants Licensee a non-exclusive, -# royalty-free, world-wide license to reproduce, analyze, test, perform -# and/or display publicly, prepare derivative works, distribute, and -# otherwise use the Software alone or in any derivative version, -# provided, however, that the BeOpen Python License is retained in the -# Software, alone or in any derivative version prepared by Licensee. -# -# 3. BeOpen is making the Software available to Licensee on an "AS IS" -# basis. BEOPEN MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, BEOPEN MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF THE SOFTWARE WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 4. BEOPEN SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF THE -# SOFTWARE FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS -# AS A RESULT OF USING, MODIFYING OR DISTRIBUTING THE SOFTWARE, OR ANY -# DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 5. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 6. This License Agreement shall be governed by and interpreted in all -# respects by the law of the State of California, excluding conflict of -# law provisions. Nothing in this License Agreement shall be deemed to -# create any relationship of agency, partnership, or joint venture -# between BeOpen and Licensee. This License Agreement does not grant -# permission to use BeOpen trademarks or trade names in a trademark -# sense to endorse or promote products or services of Licensee, or any -# third party. As an exception, the "BeOpen Python" logos available at -# http://www.pythonlabs.com/logos.html may be used according to the -# permissions granted on that web page. -# -# 7. By copying, installing or otherwise using the software, Licensee -# agrees to be bound by the terms and conditions of this License -# Agreement. -# -# -# CNRI LICENSE AGREEMENT FOR PYTHON 1.6.1 -# --------------------------------------- -# -# 1. This LICENSE AGREEMENT is between the Corporation for National -# Research Initiatives, having an office at 1895 Preston White Drive, -# Reston, VA 20191 ("CNRI"), and the Individual or Organization -# ("Licensee") accessing and otherwise using Python 1.6.1 software in -# source or binary form and its associated documentation. -# -# 2. Subject to the terms and conditions of this License Agreement, CNRI -# hereby grants Licensee a nonexclusive, royalty-free, world-wide -# license to reproduce, analyze, test, perform and/or display publicly, -# prepare derivative works, distribute, and otherwise use Python 1.6.1 -# alone or in any derivative version, provided, however, that CNRI's -# License Agreement and CNRI's notice of copyright, i.e., "Copyright (c) -# 1995-2001 Corporation for National Research Initiatives; All Rights -# Reserved" are retained in Python 1.6.1 alone or in any derivative -# version prepared by Licensee. Alternately, in lieu of CNRI's License -# Agreement, Licensee may substitute the following text (omitting the -# quotes): "Python 1.6.1 is made available subject to the terms and -# conditions in CNRI's License Agreement. This Agreement together with -# Python 1.6.1 may be located on the Internet using the following -# unique, persistent identifier (known as a handle): 1895.22/1013. This -# Agreement may also be obtained from a proxy server on the Internet -# using the following URL: http://hdl.handle.net/1895.22/1013". -# -# 3. In the event Licensee prepares a derivative work that is based on -# or incorporates Python 1.6.1 or any part thereof, and wants to make -# the derivative work available to others as provided herein, then -# Licensee hereby agrees to include in any such work a brief summary of -# the changes made to Python 1.6.1. -# -# 4. CNRI is making Python 1.6.1 available to Licensee on an "AS IS" -# basis. CNRI MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, CNRI MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 1.6.1 WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 5. CNRI SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -# 1.6.1 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 1.6.1, -# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 6. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 7. This License Agreement shall be governed by the federal -# intellectual property law of the United States, including without -# limitation the federal copyright law, and, to the extent such -# U.S. federal law does not apply, by the law of the Commonwealth of -# Virginia, excluding Virginia's conflict of law provisions. -# Notwithstanding the foregoing, with regard to derivative works based -# on Python 1.6.1 that incorporate non-separable material that was -# previously distributed under the GNU General Public License (GPL), the -# law of the Commonwealth of Virginia shall govern this License -# Agreement only as to issues arising under or with respect to -# Paragraphs 4, 5, and 7 of this License Agreement. Nothing in this -# License Agreement shall be deemed to create any relationship of -# agency, partnership, or joint venture between CNRI and Licensee. This -# License Agreement does not grant permission to use CNRI trademarks or -# trade name in a trademark sense to endorse or promote products or -# services of Licensee, or any third party. -# -# 8. By clicking on the "ACCEPT" button where indicated, or by copying, -# installing or otherwise using Python 1.6.1, Licensee agrees to be -# bound by the terms and conditions of this License Agreement. -# -# ACCEPT -# -# -# CWI LICENSE AGREEMENT FOR PYTHON 0.9.0 THROUGH 1.2 -# -------------------------------------------------- -# -# Copyright (c) 1991 - 1995, Stichting Mathematisch Centrum Amsterdam, -# The Netherlands. All rights reserved. -# -# Permission to use, copy, modify, and distribute this software and its -# documentation for any purpose and without fee is hereby granted, -# provided that the above copyright notice appear in all copies and that -# both that copyright notice and this permission notice appear in -# supporting documentation, and that the name of Stichting Mathematisch -# Centrum or CWI not be used in advertising or publicity pertaining to -# distribution of the software without specific, written prior -# permission. -# -# STICHTING MATHEMATISCH CENTRUM DISCLAIMS ALL WARRANTIES WITH REGARD TO -# THIS SOFTWARE, INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND -# FITNESS, IN NO EVENT SHALL STICHTING MATHEMATISCH CENTRUM BE LIABLE -# FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES -# WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN -# ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT -# OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. \ No newline at end of file diff --git a/licenses/LICENSE-heapq.txt b/licenses/LICENSE-heapq.txt deleted file mode 100644 index 45be6b83a53be..0000000000000 --- a/licenses/LICENSE-heapq.txt +++ /dev/null @@ -1,49 +0,0 @@ -PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 --------------------------------------------- - -1. This LICENSE AGREEMENT is between the Python Software Foundation -("PSF"), and the Individual or Organization ("Licensee") accessing and -otherwise using this software ("Python") in source or binary form and -its associated documentation. - -2. Subject to the terms and conditions of this License Agreement, PSF hereby -grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, -analyze, test, perform and/or display publicly, prepare derivative works, -distribute, and otherwise use Python alone or in any derivative version, -provided, however, that PSF's License Agreement and PSF's notice of copyright, -i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, -2011, 2012, 2013, 2014, 2015, 2016, 2017, 2018, 2019 Python Software Foundation; -All Rights Reserved" are retained in Python alone or in any derivative version -prepared by Licensee. - -3. In the event Licensee prepares a derivative work that is based on -or incorporates Python or any part thereof, and wants to make -the derivative work available to others as provided herein, then -Licensee hereby agrees to include in any such work a brief summary of -the changes made to Python. - -4. PSF is making Python available to Licensee on an "AS IS" -basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND -DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT -INFRINGE ANY THIRD PARTY RIGHTS. - -5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, -OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. - -6. This License Agreement will automatically terminate upon a material -breach of its terms and conditions. - -7. Nothing in this License Agreement shall be deemed to create any -relationship of agency, partnership, or joint venture between PSF and -Licensee. This License Agreement does not grant permission to use PSF -trademarks or trade name in a trademark sense to endorse or promote -products or services of Licensee, or any third party. - -8. By copying, installing or otherwise using Python, Licensee -agrees to be bound by the terms and conditions of this License -Agreement. - diff --git a/python/pylintrc b/python/pylintrc index 26d2741d3b56f..548377416ca3d 100644 --- a/python/pylintrc +++ b/python/pylintrc @@ -29,7 +29,7 @@ profile=no # Add files or directories to the ignoreList. They should be base names, not # paths. -ignore=pyspark.heapq3 +#ignore= # Pickle collected data for later comparisons. persistent=yes diff --git a/python/pyspark/heapq3.py b/python/pyspark/heapq3.py deleted file mode 100644 index 37a2914ebac05..0000000000000 --- a/python/pyspark/heapq3.py +++ /dev/null @@ -1,890 +0,0 @@ -# -*- encoding: utf-8 -*- -# back ported from CPython 3 -# A. HISTORY OF THE SOFTWARE -# ========================== -# -# Python was created in the early 1990s by Guido van Rossum at Stichting -# Mathematisch Centrum (CWI, see http://www.cwi.nl) in the Netherlands -# as a successor of a language called ABC. Guido remains Python's -# principal author, although it includes many contributions from others. -# -# In 1995, Guido continued his work on Python at the Corporation for -# National Research Initiatives (CNRI, see http://www.cnri.reston.va.us) -# in Reston, Virginia where he released several versions of the -# software. -# -# In May 2000, Guido and the Python core development team moved to -# BeOpen.com to form the BeOpen PythonLabs team. In October of the same -# year, the PythonLabs team moved to Digital Creations (now Zope -# Corporation, see http://www.zope.com). In 2001, the Python Software -# Foundation (PSF, see http://www.python.org/psf/) was formed, a -# non-profit organization created specifically to own Python-related -# Intellectual Property. Zope Corporation is a sponsoring member of -# the PSF. -# -# All Python releases are Open Source (see http://www.opensource.org for -# the Open Source Definition). Historically, most, but not all, Python -# releases have also been GPL-compatible; the table below summarizes -# the various releases. -# -# Release Derived Year Owner GPL- -# from compatible? (1) -# -# 0.9.0 thru 1.2 1991-1995 CWI yes -# 1.3 thru 1.5.2 1.2 1995-1999 CNRI yes -# 1.6 1.5.2 2000 CNRI no -# 2.0 1.6 2000 BeOpen.com no -# 1.6.1 1.6 2001 CNRI yes (2) -# 2.1 2.0+1.6.1 2001 PSF no -# 2.0.1 2.0+1.6.1 2001 PSF yes -# 2.1.1 2.1+2.0.1 2001 PSF yes -# 2.2 2.1.1 2001 PSF yes -# 2.1.2 2.1.1 2002 PSF yes -# 2.1.3 2.1.2 2002 PSF yes -# 2.2.1 2.2 2002 PSF yes -# 2.2.2 2.2.1 2002 PSF yes -# 2.2.3 2.2.2 2003 PSF yes -# 2.3 2.2.2 2002-2003 PSF yes -# 2.3.1 2.3 2002-2003 PSF yes -# 2.3.2 2.3.1 2002-2003 PSF yes -# 2.3.3 2.3.2 2002-2003 PSF yes -# 2.3.4 2.3.3 2004 PSF yes -# 2.3.5 2.3.4 2005 PSF yes -# 2.4 2.3 2004 PSF yes -# 2.4.1 2.4 2005 PSF yes -# 2.4.2 2.4.1 2005 PSF yes -# 2.4.3 2.4.2 2006 PSF yes -# 2.4.4 2.4.3 2006 PSF yes -# 2.5 2.4 2006 PSF yes -# 2.5.1 2.5 2007 PSF yes -# 2.5.2 2.5.1 2008 PSF yes -# 2.5.3 2.5.2 2008 PSF yes -# 2.6 2.5 2008 PSF yes -# 2.6.1 2.6 2008 PSF yes -# 2.6.2 2.6.1 2009 PSF yes -# 2.6.3 2.6.2 2009 PSF yes -# 2.6.4 2.6.3 2009 PSF yes -# 2.6.5 2.6.4 2010 PSF yes -# 2.7 2.6 2010 PSF yes -# -# Footnotes: -# -# (1) GPL-compatible doesn't mean that we're distributing Python under -# the GPL. All Python licenses, unlike the GPL, let you distribute -# a modified version without making your changes open source. The -# GPL-compatible licenses make it possible to combine Python with -# other software that is released under the GPL; the others don't. -# -# (2) According to Richard Stallman, 1.6.1 is not GPL-compatible, -# because its license has a choice of law clause. According to -# CNRI, however, Stallman's lawyer has told CNRI's lawyer that 1.6.1 -# is "not incompatible" with the GPL. -# -# Thanks to the many outside volunteers who have worked under Guido's -# direction to make these releases possible. -# -# -# B. TERMS AND CONDITIONS FOR ACCESSING OR OTHERWISE USING PYTHON -# =============================================================== -# -# PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 -# -------------------------------------------- -# -# 1. This LICENSE AGREEMENT is between the Python Software Foundation -# ("PSF"), and the Individual or Organization ("Licensee") accessing and -# otherwise using this software ("Python") in source or binary form and -# its associated documentation. -# -# 2. Subject to the terms and conditions of this License Agreement, PSF hereby -# grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, -# analyze, test, perform and/or display publicly, prepare derivative works, -# distribute, and otherwise use Python alone or in any derivative version, -# provided, however, that PSF's License Agreement and PSF's notice of copyright, -# i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, -# 2011, 2012, 2013 Python Software Foundation; All Rights Reserved" are retained -# in Python alone or in any derivative version prepared by Licensee. -# -# 3. In the event Licensee prepares a derivative work that is based on -# or incorporates Python or any part thereof, and wants to make -# the derivative work available to others as provided herein, then -# Licensee hereby agrees to include in any such work a brief summary of -# the changes made to Python. -# -# 4. PSF is making Python available to Licensee on an "AS IS" -# basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -# FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, -# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 6. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 7. Nothing in this License Agreement shall be deemed to create any -# relationship of agency, partnership, or joint venture between PSF and -# Licensee. This License Agreement does not grant permission to use PSF -# trademarks or trade name in a trademark sense to endorse or promote -# products or services of Licensee, or any third party. -# -# 8. By copying, installing or otherwise using Python, Licensee -# agrees to be bound by the terms and conditions of this License -# Agreement. -# -# -# BEOPEN.COM LICENSE AGREEMENT FOR PYTHON 2.0 -# ------------------------------------------- -# -# BEOPEN PYTHON OPEN SOURCE LICENSE AGREEMENT VERSION 1 -# -# 1. This LICENSE AGREEMENT is between BeOpen.com ("BeOpen"), having an -# office at 160 Saratoga Avenue, Santa Clara, CA 95051, and the -# Individual or Organization ("Licensee") accessing and otherwise using -# this software in source or binary form and its associated -# documentation ("the Software"). -# -# 2. Subject to the terms and conditions of this BeOpen Python License -# Agreement, BeOpen hereby grants Licensee a non-exclusive, -# royalty-free, world-wide license to reproduce, analyze, test, perform -# and/or display publicly, prepare derivative works, distribute, and -# otherwise use the Software alone or in any derivative version, -# provided, however, that the BeOpen Python License is retained in the -# Software, alone or in any derivative version prepared by Licensee. -# -# 3. BeOpen is making the Software available to Licensee on an "AS IS" -# basis. BEOPEN MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, BEOPEN MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF THE SOFTWARE WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 4. BEOPEN SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF THE -# SOFTWARE FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS -# AS A RESULT OF USING, MODIFYING OR DISTRIBUTING THE SOFTWARE, OR ANY -# DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 5. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 6. This License Agreement shall be governed by and interpreted in all -# respects by the law of the State of California, excluding conflict of -# law provisions. Nothing in this License Agreement shall be deemed to -# create any relationship of agency, partnership, or joint venture -# between BeOpen and Licensee. This License Agreement does not grant -# permission to use BeOpen trademarks or trade names in a trademark -# sense to endorse or promote products or services of Licensee, or any -# third party. As an exception, the "BeOpen Python" logos available at -# http://www.pythonlabs.com/logos.html may be used according to the -# permissions granted on that web page. -# -# 7. By copying, installing or otherwise using the software, Licensee -# agrees to be bound by the terms and conditions of this License -# Agreement. -# -# -# CNRI LICENSE AGREEMENT FOR PYTHON 1.6.1 -# --------------------------------------- -# -# 1. This LICENSE AGREEMENT is between the Corporation for National -# Research Initiatives, having an office at 1895 Preston White Drive, -# Reston, VA 20191 ("CNRI"), and the Individual or Organization -# ("Licensee") accessing and otherwise using Python 1.6.1 software in -# source or binary form and its associated documentation. -# -# 2. Subject to the terms and conditions of this License Agreement, CNRI -# hereby grants Licensee a nonexclusive, royalty-free, world-wide -# license to reproduce, analyze, test, perform and/or display publicly, -# prepare derivative works, distribute, and otherwise use Python 1.6.1 -# alone or in any derivative version, provided, however, that CNRI's -# License Agreement and CNRI's notice of copyright, i.e., "Copyright (c) -# 1995-2001 Corporation for National Research Initiatives; All Rights -# Reserved" are retained in Python 1.6.1 alone or in any derivative -# version prepared by Licensee. Alternately, in lieu of CNRI's License -# Agreement, Licensee may substitute the following text (omitting the -# quotes): "Python 1.6.1 is made available subject to the terms and -# conditions in CNRI's License Agreement. This Agreement together with -# Python 1.6.1 may be located on the Internet using the following -# unique, persistent identifier (known as a handle): 1895.22/1013. This -# Agreement may also be obtained from a proxy server on the Internet -# using the following URL: http://hdl.handle.net/1895.22/1013". -# -# 3. In the event Licensee prepares a derivative work that is based on -# or incorporates Python 1.6.1 or any part thereof, and wants to make -# the derivative work available to others as provided herein, then -# Licensee hereby agrees to include in any such work a brief summary of -# the changes made to Python 1.6.1. -# -# 4. CNRI is making Python 1.6.1 available to Licensee on an "AS IS" -# basis. CNRI MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, CNRI MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 1.6.1 WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 5. CNRI SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -# 1.6.1 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 1.6.1, -# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 6. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 7. This License Agreement shall be governed by the federal -# intellectual property law of the United States, including without -# limitation the federal copyright law, and, to the extent such -# U.S. federal law does not apply, by the law of the Commonwealth of -# Virginia, excluding Virginia's conflict of law provisions. -# Notwithstanding the foregoing, with regard to derivative works based -# on Python 1.6.1 that incorporate non-separable material that was -# previously distributed under the GNU General Public License (GPL), the -# law of the Commonwealth of Virginia shall govern this License -# Agreement only as to issues arising under or with respect to -# Paragraphs 4, 5, and 7 of this License Agreement. Nothing in this -# License Agreement shall be deemed to create any relationship of -# agency, partnership, or joint venture between CNRI and Licensee. This -# License Agreement does not grant permission to use CNRI trademarks or -# trade name in a trademark sense to endorse or promote products or -# services of Licensee, or any third party. -# -# 8. By clicking on the "ACCEPT" button where indicated, or by copying, -# installing or otherwise using Python 1.6.1, Licensee agrees to be -# bound by the terms and conditions of this License Agreement. -# -# ACCEPT -# -# -# CWI LICENSE AGREEMENT FOR PYTHON 0.9.0 THROUGH 1.2 -# -------------------------------------------------- -# -# Copyright (c) 1991 - 1995, Stichting Mathematisch Centrum Amsterdam, -# The Netherlands. All rights reserved. -# -# Permission to use, copy, modify, and distribute this software and its -# documentation for any purpose and without fee is hereby granted, -# provided that the above copyright notice appear in all copies and that -# both that copyright notice and this permission notice appear in -# supporting documentation, and that the name of Stichting Mathematisch -# Centrum or CWI not be used in advertising or publicity pertaining to -# distribution of the software without specific, written prior -# permission. -# -# STICHTING MATHEMATISCH CENTRUM DISCLAIMS ALL WARRANTIES WITH REGARD TO -# THIS SOFTWARE, INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND -# FITNESS, IN NO EVENT SHALL STICHTING MATHEMATISCH CENTRUM BE LIABLE -# FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES -# WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN -# ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT -# OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. -"""Heap queue algorithm (a.k.a. priority queue). - -Heaps are arrays for which a[k] <= a[2*k+1] and a[k] <= a[2*k+2] for -all k, counting elements from 0. For the sake of comparison, -non-existing elements are considered to be infinite. The interesting -property of a heap is that a[0] is always its smallest element. - -Usage: - -heap = [] # creates an empty heap -heappush(heap, item) # pushes a new item on the heap -item = heappop(heap) # pops the smallest item from the heap -item = heap[0] # smallest item on the heap without popping it -heapify(x) # transforms list into a heap, in-place, in linear time -item = heapreplace(heap, item) # pops and returns smallest item, and adds - # new item; the heap size is unchanged - -Our API differs from textbook heap algorithms as follows: - -- We use 0-based indexing. This makes the relationship between the - index for a node and the indexes for its children slightly less - obvious, but is more suitable since Python uses 0-based indexing. - -- Our heappop() method returns the smallest item, not the largest. - -These two make it possible to view the heap as a regular Python list -without surprises: heap[0] is the smallest item, and heap.sort() -maintains the heap invariant! -""" - -# Original code by Kevin O'Connor, augmented by Tim Peters and Raymond Hettinger - -__about__ = """Heap queues - -[explanation by François Pinard] - -Heaps are arrays for which a[k] <= a[2*k+1] and a[k] <= a[2*k+2] for -all k, counting elements from 0. For the sake of comparison, -non-existing elements are considered to be infinite. The interesting -property of a heap is that a[0] is always its smallest element. - -The strange invariant above is meant to be an efficient memory -representation for a tournament. The numbers below are `k', not a[k]: - - 0 - - 1 2 - - 3 4 5 6 - - 7 8 9 10 11 12 13 14 - - 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 - - -In the tree above, each cell `k' is topping `2*k+1' and `2*k+2'. In -an usual binary tournament we see in sports, each cell is the winner -over the two cells it tops, and we can trace the winner down the tree -to see all opponents s/he had. However, in many computer applications -of such tournaments, we do not need to trace the history of a winner. -To be more memory efficient, when a winner is promoted, we try to -replace it by something else at a lower level, and the rule becomes -that a cell and the two cells it tops contain three different items, -but the top cell "wins" over the two topped cells. - -If this heap invariant is protected at all time, index 0 is clearly -the overall winner. The simplest algorithmic way to remove it and -find the "next" winner is to move some loser (let's say cell 30 in the -diagram above) into the 0 position, and then percolate this new 0 down -the tree, exchanging values, until the invariant is re-established. -This is clearly logarithmic on the total number of items in the tree. -By iterating over all items, you get an O(n ln n) sort. - -A nice feature of this sort is that you can efficiently insert new -items while the sort is going on, provided that the inserted items are -not "better" than the last 0'th element you extracted. This is -especially useful in simulation contexts, where the tree holds all -incoming events, and the "win" condition means the smallest scheduled -time. When an event schedule other events for execution, they are -scheduled into the future, so they can easily go into the heap. So, a -heap is a good structure for implementing schedulers (this is what I -used for my MIDI sequencer :-). - -Various structures for implementing schedulers have been extensively -studied, and heaps are good for this, as they are reasonably speedy, -the speed is almost constant, and the worst case is not much different -than the average case. However, there are other representations which -are more efficient overall, yet the worst cases might be terrible. - -Heaps are also very useful in big disk sorts. You most probably all -know that a big sort implies producing "runs" (which are pre-sorted -sequences, which size is usually related to the amount of CPU memory), -followed by a merging passes for these runs, which merging is often -very cleverly organised[1]. It is very important that the initial -sort produces the longest runs possible. Tournaments are a good way -to that. If, using all the memory available to hold a tournament, you -replace and percolate items that happen to fit the current run, you'll -produce runs which are twice the size of the memory for random input, -and much better for input fuzzily ordered. - -Moreover, if you output the 0'th item on disk and get an input which -may not fit in the current tournament (because the value "wins" over -the last output value), it cannot fit in the heap, so the size of the -heap decreases. The freed memory could be cleverly reused immediately -for progressively building a second heap, which grows at exactly the -same rate the first heap is melting. When the first heap completely -vanishes, you switch heaps and start a new run. Clever and quite -effective! - -In a word, heaps are useful memory structures to know. I use them in -a few applications, and I think it is good to keep a `heap' module -around. :-) - --------------------- -[1] The disk balancing algorithms which are current, nowadays, are -more annoying than clever, and this is a consequence of the seeking -capabilities of the disks. On devices which cannot seek, like big -tape drives, the story was quite different, and one had to be very -clever to ensure (far in advance) that each tape movement will be the -most effective possible (that is, will best participate at -"progressing" the merge). Some tapes were even able to read -backwards, and this was also used to avoid the rewinding time. -Believe me, real good tape sorts were quite spectacular to watch! -From all times, sorting has always been a Great Art! :-) -""" - -__all__ = ['heappush', 'heappop', 'heapify', 'heapreplace', 'merge', - 'nlargest', 'nsmallest', 'heappushpop'] - -def heappush(heap, item): - """Push item onto heap, maintaining the heap invariant.""" - heap.append(item) - _siftdown(heap, 0, len(heap)-1) - -def heappop(heap): - """Pop the smallest item off the heap, maintaining the heap invariant.""" - lastelt = heap.pop() # raises appropriate IndexError if heap is empty - if heap: - returnitem = heap[0] - heap[0] = lastelt - _siftup(heap, 0) - return returnitem - return lastelt - -def heapreplace(heap, item): - """Pop and return the current smallest value, and add the new item. - - This is more efficient than heappop() followed by heappush(), and can be - more appropriate when using a fixed-size heap. Note that the value - returned may be larger than item! That constrains reasonable uses of - this routine unless written as part of a conditional replacement: - - if item > heap[0]: - item = heapreplace(heap, item) - """ - returnitem = heap[0] # raises appropriate IndexError if heap is empty - heap[0] = item - _siftup(heap, 0) - return returnitem - -def heappushpop(heap, item): - """Fast version of a heappush followed by a heappop.""" - if heap and heap[0] < item: - item, heap[0] = heap[0], item - _siftup(heap, 0) - return item - -def heapify(x): - """Transform list into a heap, in-place, in O(len(x)) time.""" - n = len(x) - # Transform bottom-up. The largest index there's any point to looking at - # is the largest with a child index in-range, so must have 2*i + 1 < n, - # or i < (n-1)/2. If n is even = 2*j, this is (2*j-1)/2 = j-1/2 so - # j-1 is the largest, which is n//2 - 1. If n is odd = 2*j+1, this is - # (2*j+1-1)/2 = j so j-1 is the largest, and that's again n//2-1. - for i in reversed(range(n//2)): - _siftup(x, i) - -def _heappop_max(heap): - """Maxheap version of a heappop.""" - lastelt = heap.pop() # raises appropriate IndexError if heap is empty - if heap: - returnitem = heap[0] - heap[0] = lastelt - _siftup_max(heap, 0) - return returnitem - return lastelt - -def _heapreplace_max(heap, item): - """Maxheap version of a heappop followed by a heappush.""" - returnitem = heap[0] # raises appropriate IndexError if heap is empty - heap[0] = item - _siftup_max(heap, 0) - return returnitem - -def _heapify_max(x): - """Transform list into a maxheap, in-place, in O(len(x)) time.""" - n = len(x) - for i in reversed(range(n//2)): - _siftup_max(x, i) - -# 'heap' is a heap at all indices >= startpos, except possibly for pos. pos -# is the index of a leaf with a possibly out-of-order value. Restore the -# heap invariant. -def _siftdown(heap, startpos, pos): - newitem = heap[pos] - # Follow the path to the root, moving parents down until finding a place - # newitem fits. - while pos > startpos: - parentpos = (pos - 1) >> 1 - parent = heap[parentpos] - if newitem < parent: - heap[pos] = parent - pos = parentpos - continue - break - heap[pos] = newitem - -# The child indices of heap index pos are already heaps, and we want to make -# a heap at index pos too. We do this by bubbling the smaller child of -# pos up (and so on with that child's children, etc) until hitting a leaf, -# then using _siftdown to move the oddball originally at index pos into place. -# -# We *could* break out of the loop as soon as we find a pos where newitem <= -# both its children, but turns out that's not a good idea, and despite that -# many books write the algorithm that way. During a heap pop, the last array -# element is sifted in, and that tends to be large, so that comparing it -# against values starting from the root usually doesn't pay (= usually doesn't -# get us out of the loop early). See Knuth, Volume 3, where this is -# explained and quantified in an exercise. -# -# Cutting the # of comparisons is important, since these routines have no -# way to extract "the priority" from an array element, so that intelligence -# is likely to be hiding in custom comparison methods, or in array elements -# storing (priority, record) tuples. Comparisons are thus potentially -# expensive. -# -# On random arrays of length 1000, making this change cut the number of -# comparisons made by heapify() a little, and those made by exhaustive -# heappop() a lot, in accord with theory. Here are typical results from 3 -# runs (3 just to demonstrate how small the variance is): -# -# Compares needed by heapify Compares needed by 1000 heappops -# -------------------------- -------------------------------- -# 1837 cut to 1663 14996 cut to 8680 -# 1855 cut to 1659 14966 cut to 8678 -# 1847 cut to 1660 15024 cut to 8703 -# -# Building the heap by using heappush() 1000 times instead required -# 2198, 2148, and 2219 compares: heapify() is more efficient, when -# you can use it. -# -# The total compares needed by list.sort() on the same lists were 8627, -# 8627, and 8632 (this should be compared to the sum of heapify() and -# heappop() compares): list.sort() is (unsurprisingly!) more efficient -# for sorting. - -def _siftup(heap, pos): - endpos = len(heap) - startpos = pos - newitem = heap[pos] - # Bubble up the smaller child until hitting a leaf. - childpos = 2*pos + 1 # leftmost child position - while childpos < endpos: - # Set childpos to index of smaller child. - rightpos = childpos + 1 - if rightpos < endpos and not heap[childpos] < heap[rightpos]: - childpos = rightpos - # Move the smaller child up. - heap[pos] = heap[childpos] - pos = childpos - childpos = 2*pos + 1 - # The leaf at pos is empty now. Put newitem there, and bubble it up - # to its final resting place (by sifting its parents down). - heap[pos] = newitem - _siftdown(heap, startpos, pos) - -def _siftdown_max(heap, startpos, pos): - 'Maxheap variant of _siftdown' - newitem = heap[pos] - # Follow the path to the root, moving parents down until finding a place - # newitem fits. - while pos > startpos: - parentpos = (pos - 1) >> 1 - parent = heap[parentpos] - if parent < newitem: - heap[pos] = parent - pos = parentpos - continue - break - heap[pos] = newitem - -def _siftup_max(heap, pos): - 'Maxheap variant of _siftup' - endpos = len(heap) - startpos = pos - newitem = heap[pos] - # Bubble up the larger child until hitting a leaf. - childpos = 2*pos + 1 # leftmost child position - while childpos < endpos: - # Set childpos to index of larger child. - rightpos = childpos + 1 - if rightpos < endpos and not heap[rightpos] < heap[childpos]: - childpos = rightpos - # Move the larger child up. - heap[pos] = heap[childpos] - pos = childpos - childpos = 2*pos + 1 - # The leaf at pos is empty now. Put newitem there, and bubble it up - # to its final resting place (by sifting its parents down). - heap[pos] = newitem - _siftdown_max(heap, startpos, pos) - -def merge(iterables, key=None, reverse=False): - '''Merge multiple sorted inputs into a single sorted output. - - Similar to sorted(itertools.chain(*iterables)) but returns a generator, - does not pull the data into memory all at once, and assumes that each of - the input streams is already sorted (smallest to largest). - - >>> list(merge([1,3,5,7], [0,2,4,8], [5,10,15,20], [], [25])) - [0, 1, 2, 3, 4, 5, 5, 7, 8, 10, 15, 20, 25] - - If *key* is not None, applies a key function to each element to determine - its sort order. - - >>> list(merge(['dog', 'horse'], ['cat', 'fish', 'kangaroo'], key=len)) - ['dog', 'cat', 'fish', 'horse', 'kangaroo'] - - ''' - - h = [] - h_append = h.append - - if reverse: - _heapify = _heapify_max - _heappop = _heappop_max - _heapreplace = _heapreplace_max - direction = -1 - else: - _heapify = heapify - _heappop = heappop - _heapreplace = heapreplace - direction = 1 - - if key is None: - for order, it in enumerate(map(iter, iterables)): - try: - h_append([next(it), order * direction, it]) - except StopIteration: - pass - _heapify(h) - while len(h) > 1: - try: - while True: - value, order, it = s = h[0] - yield value - s[0] = next(it) # raises StopIteration when exhausted - _heapreplace(h, s) # restore heap condition - except StopIteration: - _heappop(h) # remove empty iterator - if h: - # fast case when only a single iterator remains - value, order, it = h[0] - yield value - for value in it: - yield value - return - - for order, it in enumerate(map(iter, iterables)): - try: - value = next(it) - h_append([key(value), order * direction, value, it]) - except StopIteration: - pass - _heapify(h) - while len(h) > 1: - try: - while True: - key_value, order, value, it = s = h[0] - yield value - value = next(it) - s[0] = key(value) - s[2] = value - _heapreplace(h, s) - except StopIteration: - _heappop(h) - if h: - key_value, order, value, it = h[0] - yield value - for value in it: - yield value - - -# Algorithm notes for nlargest() and nsmallest() -# ============================================== -# -# Make a single pass over the data while keeping the k most extreme values -# in a heap. Memory consumption is limited to keeping k values in a list. -# -# Measured performance for random inputs: -# -# number of comparisons -# n inputs k-extreme values (average of 5 trials) % more than min() -# ------------- ---------------- --------------------- ----------------- -# 1,000 100 3,317 231.7% -# 10,000 100 14,046 40.5% -# 100,000 100 105,749 5.7% -# 1,000,000 100 1,007,751 0.8% -# 10,000,000 100 10,009,401 0.1% -# -# Theoretical number of comparisons for k smallest of n random inputs: -# -# Step Comparisons Action -# ---- -------------------------- --------------------------- -# 1 1.66 * k heapify the first k-inputs -# 2 n - k compare remaining elements to top of heap -# 3 k * (1 + lg2(k)) * ln(n/k) replace the topmost value on the heap -# 4 k * lg2(k) - (k/2) final sort of the k most extreme values -# -# Combining and simplifying for a rough estimate gives: -# -# comparisons = n + k * (log(k, 2) * log(n/k) + log(k, 2) + log(n/k)) -# -# Computing the number of comparisons for step 3: -# ----------------------------------------------- -# * For the i-th new value from the iterable, the probability of being in the -# k most extreme values is k/i. For example, the probability of the 101st -# value seen being in the 100 most extreme values is 100/101. -# * If the value is a new extreme value, the cost of inserting it into the -# heap is 1 + log(k, 2). -# * The probability times the cost gives: -# (k/i) * (1 + log(k, 2)) -# * Summing across the remaining n-k elements gives: -# sum((k/i) * (1 + log(k, 2)) for i in range(k+1, n+1)) -# * This reduces to: -# (H(n) - H(k)) * k * (1 + log(k, 2)) -# * Where H(n) is the n-th harmonic number estimated by: -# gamma = 0.5772156649 -# H(n) = log(n, e) + gamma + 1 / (2 * n) -# http://en.wikipedia.org/wiki/Harmonic_series_(mathematics)#Rate_of_divergence -# * Substituting the H(n) formula: -# comparisons = k * (1 + log(k, 2)) * (log(n/k, e) + (1/n - 1/k) / 2) -# -# Worst-case for step 3: -# ---------------------- -# In the worst case, the input data is reversed sorted so that every new element -# must be inserted in the heap: -# -# comparisons = 1.66 * k + log(k, 2) * (n - k) -# -# Alternative Algorithms -# ---------------------- -# Other algorithms were not used because they: -# 1) Took much more auxiliary memory, -# 2) Made multiple passes over the data. -# 3) Made more comparisons in common cases (small k, large n, semi-random input). -# See the more detailed comparison of approach at: -# http://code.activestate.com/recipes/577573-compare-algorithms-for-heapqsmallest - -def nsmallest(n, iterable, key=None): - """Find the n smallest elements in a dataset. - - Equivalent to: sorted(iterable, key=key)[:n] - """ - - # Short-cut for n==1 is to use min() - if n == 1: - it = iter(iterable) - sentinel = object() - if key is None: - result = min(it, default=sentinel) - else: - result = min(it, default=sentinel, key=key) - return [] if result is sentinel else [result] - - # When n>=size, it's faster to use sorted() - try: - size = len(iterable) - except (TypeError, AttributeError): - pass - else: - if n >= size: - return sorted(iterable, key=key)[:n] - - # When key is none, use simpler decoration - if key is None: - it = iter(iterable) - # put the range(n) first so that zip() doesn't - # consume one too many elements from the iterator - result = [(elem, i) for i, elem in zip(range(n), it)] - if not result: - return result - _heapify_max(result) - top = result[0][0] - order = n - _heapreplace = _heapreplace_max - for elem in it: - if elem < top: - _heapreplace(result, (elem, order)) - top = result[0][0] - order += 1 - result.sort() - return [r[0] for r in result] - - # General case, slowest method - it = iter(iterable) - result = [(key(elem), i, elem) for i, elem in zip(range(n), it)] - if not result: - return result - _heapify_max(result) - top = result[0][0] - order = n - _heapreplace = _heapreplace_max - for elem in it: - k = key(elem) - if k < top: - _heapreplace(result, (k, order, elem)) - top = result[0][0] - order += 1 - result.sort() - return [r[2] for r in result] - -def nlargest(n, iterable, key=None): - """Find the n largest elements in a dataset. - - Equivalent to: sorted(iterable, key=key, reverse=True)[:n] - """ - - # Short-cut for n==1 is to use max() - if n == 1: - it = iter(iterable) - sentinel = object() - if key is None: - result = max(it, default=sentinel) - else: - result = max(it, default=sentinel, key=key) - return [] if result is sentinel else [result] - - # When n>=size, it's faster to use sorted() - try: - size = len(iterable) - except (TypeError, AttributeError): - pass - else: - if n >= size: - return sorted(iterable, key=key, reverse=True)[:n] - - # When key is none, use simpler decoration - if key is None: - it = iter(iterable) - result = [(elem, i) for i, elem in zip(range(0, -n, -1), it)] - if not result: - return result - heapify(result) - top = result[0][0] - order = -n - _heapreplace = heapreplace - for elem in it: - if top < elem: - _heapreplace(result, (elem, order)) - top = result[0][0] - order -= 1 - result.sort(reverse=True) - return [r[0] for r in result] - - # General case, slowest method - it = iter(iterable) - result = [(key(elem), i, elem) for i, elem in zip(range(0, -n, -1), it)] - if not result: - return result - heapify(result) - top = result[0][0] - order = -n - _heapreplace = heapreplace - for elem in it: - k = key(elem) - if top < k: - _heapreplace(result, (k, order, elem)) - top = result[0][0] - order -= 1 - result.sort(reverse=True) - return [r[2] for r in result] - -# If available, use C implementation -try: - from _heapq import * -except ImportError: - pass -try: - from _heapq import _heapreplace_max -except ImportError: - pass -try: - from _heapq import _heapify_max -except ImportError: - pass -try: - from _heapq import _heappop_max -except ImportError: - pass - - -if __name__ == "__main__": - import doctest - import sys - (failure_count, test_count) = doctest.testmod() - if failure_count: - sys.exit(-1) diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 5d2d63850e9b2..308305e5f46c6 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -25,7 +25,7 @@ import random import sys -import pyspark.heapq3 as heapq +import heapq from pyspark.serializers import BatchedSerializer, PickleSerializer, FlattenedValuesSerializer, \ CompressedSerializer, AutoBatchedSerializer from pyspark.util import fail_on_stopiteration @@ -498,7 +498,7 @@ def load(f): if current_chunk: chunks.append(iter(current_chunk)) - return heapq.merge(chunks, key=key, reverse=reverse) + return heapq.merge(*chunks, key=key, reverse=reverse) class ExternalList(object): @@ -796,7 +796,7 @@ def load_partition(j): if self._sorted: # all the partitions are already sorted - sorted_items = heapq.merge(disk_items, key=operator.itemgetter(0)) + sorted_items = heapq.merge(*disk_items, key=operator.itemgetter(0)) else: # Flatten the combined values, so it will not consume huge From 998086c9a179692b2687bc9a104dbbb35f5a44e2 Mon Sep 17 00:00:00 2001 From: Warren Zhu Date: Mon, 27 Jul 2020 08:16:13 -0500 Subject: [PATCH 295/384] [SPARK-30794][CORE] Stage Level scheduling: Add ability to set off heap memory ### What changes were proposed in this pull request? Support set off heap memory in `ExecutorResourceRequests` ### Why are the changes needed? Support stage level scheduling ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added UT in `ResourceProfileSuite` and `DAGSchedulerSuite` Closes #28972 from warrenzhu25/30794. Authored-by: Warren Zhu Signed-off-by: Thomas Graves --- .../resource/ExecutorResourceRequests.scala | 14 ++++++ .../spark/resource/ResourceProfile.scala | 8 ++- .../spark/resource/ResourceProfileSuite.scala | 50 ++++++++++++++++--- .../spark/scheduler/DAGSchedulerSuite.scala | 5 +- python/pyspark/resource/requests.py | 9 ++++ .../pyspark/resource/tests/test_resources.py | 5 +- .../spark/deploy/yarn/YarnAllocator.scala | 3 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 23 +++++++-- 8 files changed, 102 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index 9da6ffb1d2577..654afa01c7113 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -54,6 +54,20 @@ class ExecutorResourceRequests() extends Serializable { this } + /** + * Specify off heap memory. The value specified will be converted to MiB. + * This value only take effect when MEMORY_OFFHEAP_ENABLED is true. + * + * @param amount Amount of memory. In the same format as JVM memory strings (e.g. 512m, 2g). + * Default unit is MiB if not specified. + */ + def offHeapMemory(amount: String): this.type = { + val amountMiB = JavaUtils.byteStringAsMb(amount) + val req = new ExecutorResourceRequest(OFFHEAP_MEM, amountMiB) + _executorResources.put(OFFHEAP_MEM, req) + this + } + /** * Specify overhead memory. The value specified will be converted to MiB. * diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index f56ea69f6cec5..8a37670c31b9a 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -243,13 +243,15 @@ object ResourceProfile extends Logging { // task resources val CPUS = "cpus" // Executor resources + // Make sure add new executor resource in below allSupportedExecutorResources val CORES = "cores" val MEMORY = "memory" + val OFFHEAP_MEM = "offHeap" val OVERHEAD_MEM = "memoryOverhead" val PYSPARK_MEM = "pyspark.memory" // all supported spark executor resources (minus the custom resources like GPUs/FPGAs) - val allSupportedExecutorResources = Seq(CORES, MEMORY, OVERHEAD_MEM, PYSPARK_MEM) + val allSupportedExecutorResources = Seq(CORES, MEMORY, OVERHEAD_MEM, PYSPARK_MEM, OFFHEAP_MEM) val UNKNOWN_RESOURCE_PROFILE_ID = -1 val DEFAULT_RESOURCE_PROFILE_ID = 0 @@ -295,6 +297,10 @@ object ResourceProfile extends Logging { ereqs.memory(conf.get(EXECUTOR_MEMORY).toString) conf.get(EXECUTOR_MEMORY_OVERHEAD).map(mem => ereqs.memoryOverhead(mem.toString)) conf.get(PYSPARK_EXECUTOR_MEMORY).map(mem => ereqs.pysparkMemory(mem.toString)) + if (conf.get(MEMORY_OFFHEAP_ENABLED)) { + // Explicitly add suffix b as default unit of offHeapMemory is Mib + ereqs.offHeapMemory(conf.get(MEMORY_OFFHEAP_SIZE).toString + "b") + } val execReq = ResourceUtils.parseAllResourceRequests(conf, SPARK_EXECUTOR_PREFIX) execReq.foreach { req => val name = req.id.resourceName diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index 29d3ef130ce57..d0479ca7db40c 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.resource import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY import org.apache.spark.resource.TestResourceIDs._ @@ -55,6 +55,8 @@ class ResourceProfileSuite extends SparkFunSuite { "pyspark memory empty if not specified") assert(rprof.executorResources.get(ResourceProfile.OVERHEAD_MEM) == None, "overhead memory empty if not specified") + assert(rprof.executorResources.get(ResourceProfile.OFFHEAP_MEM) == None, + "offHeap memory empty if not specified") assert(rprof.taskResources.size === 1, "Task resources should just contain cpus by default") assert(rprof.taskResources(ResourceProfile.CPUS).amount === 1, @@ -69,14 +71,16 @@ class ResourceProfileSuite extends SparkFunSuite { conf.set(EXECUTOR_MEMORY_OVERHEAD.key, "1g") conf.set(EXECUTOR_MEMORY.key, "4g") conf.set(EXECUTOR_CORES.key, "4") + conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") + conf.set(MEMORY_OFFHEAP_SIZE.key, "3m") conf.set(TASK_GPU_ID.amountConf, "1") conf.set(EXECUTOR_GPU_ID.amountConf, "1") conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, "nameOfScript") val rprof = ResourceProfile.getOrCreateDefaultProfile(conf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val execResources = rprof.executorResources - assert(execResources.size === 5, s"Executor resources should contain cores, pyspark " + - s"memory, memory overhead, memory, and gpu $execResources") + assert(execResources.size === 6, s"Executor resources should contain cores, pyspark " + + s"memory, memory overhead, memory, offHeap memory and gpu $execResources") assert(execResources.contains("gpu"), "Executor resources should have gpu") assert(rprof.executorResources(ResourceProfile.CORES).amount === 4, "Executor resources should have 4 core") @@ -88,6 +92,8 @@ class ResourceProfileSuite extends SparkFunSuite { "pyspark memory empty if not specified") assert(rprof.executorResources(ResourceProfile.OVERHEAD_MEM).amount == 1024, "overhead memory empty if not specified") + assert(rprof.executorResources(ResourceProfile.OFFHEAP_MEM).amount == 3, + "Executor resources should have 3 offHeap memory") assert(rprof.taskResources.size === 2, "Task resources should just contain cpus and gpu") assert(rprof.taskResources.contains("gpu"), "Task resources should have gpu") @@ -172,14 +178,14 @@ class ResourceProfileSuite extends SparkFunSuite { val ereqs = new ExecutorResourceRequests() ereqs.cores(2).memory("4096") - ereqs.memoryOverhead("2048").pysparkMemory("1024") + ereqs.memoryOverhead("2048").pysparkMemory("1024").offHeapMemory("3072") val treqs = new TaskResourceRequests() treqs.cpus(1) rprof.require(treqs) rprof.require(ereqs) - assert(rprof.executorResources.size === 5) + assert(rprof.executorResources.size === 6) assert(rprof.executorResources(ResourceProfile.CORES).amount === 2, "Executor resources should have 2 cores") assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, @@ -188,6 +194,8 @@ class ResourceProfileSuite extends SparkFunSuite { "Executor resources should have 2048 overhead memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount === 1024, "Executor resources should have 1024 pyspark memory") + assert(rprof.executorResources(ResourceProfile.OFFHEAP_MEM).amount === 3072, + "Executor resources should have 3072 offHeap memory") assert(rprof.taskResources.size === 2) assert(rprof.taskResources("cpus").amount === 1, "Task resources should have cpu") @@ -217,7 +225,7 @@ class ResourceProfileSuite extends SparkFunSuite { val rprof = new ResourceProfileBuilder() val ereqs = new ExecutorResourceRequests() ereqs.memory("4g") - ereqs.memoryOverhead("2000m").pysparkMemory("512000k") + ereqs.memoryOverhead("2000m").pysparkMemory("512000k").offHeapMemory("1g") rprof.require(ereqs) assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, @@ -226,6 +234,8 @@ class ResourceProfileSuite extends SparkFunSuite { "Executor resources should have 2000 overhead memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount === 500, "Executor resources should have 512 pyspark memory") + assert(rprof.executorResources(ResourceProfile.OFFHEAP_MEM).amount === 1024, + "Executor resources should have 1024 offHeap memory") } test("Test TaskResourceRequest fractional") { @@ -256,4 +266,32 @@ class ResourceProfileSuite extends SparkFunSuite { }.getMessage() assert(taskError.contains("The resource amount 0.7 must be either <= 0.5, or a whole number.")) } + + test("ResourceProfile has correct custom executor resources") { + val rprof = new ResourceProfileBuilder() + val eReq = new ExecutorResourceRequests() + .cores(2).memory("4096") + .memoryOverhead("2048").pysparkMemory("1024").offHeapMemory("3072") + .resource("gpu", 2) + rprof.require(eReq) + + // Update this if new resource type added + assert(ResourceProfile.allSupportedExecutorResources.size === 5, + "Executor resources should have 5 supported resources") + assert(ResourceProfile.getCustomExecutorResources(rprof.build).size === 1, + "Executor resources should have 1 custom resource") + } + + test("ResourceProfile has correct custom task resources") { + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests() + .resource("gpu", 1) + val eReq = new ExecutorResourceRequests() + .cores(2).memory("4096") + .memoryOverhead("2048").pysparkMemory("1024").offHeapMemory("3072") + rprof.require(taskReq).require(eReq) + + assert(ResourceProfile.getCustomTaskResources(rprof.build).size === 1, + "Task resources should have 1 custom resource") + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 2b38aa1cde6cd..45af0d086890f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3286,7 +3286,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mergedRp.taskResources.get(GPU).get.amount == 1) val ereqs5 = new ExecutorResourceRequests().cores(1).memory("3g") - .memoryOverhead("1g").pysparkMemory("2g").resource(GPU, 1, "disc") + .memoryOverhead("1g").pysparkMemory("2g").offHeapMemory("4g").resource(GPU, 1, "disc") val treqs5 = new TaskResourceRequests().cpus(1).resource(GPU, 1) val rp5 = new ResourceProfile(ereqs5.requests, treqs5.requests) val ereqs6 = new ExecutorResourceRequests().cores(8).resource(FPGA, 2, "fdisc") @@ -3296,7 +3296,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mergedRp.getTaskCpus.get == 2) assert(mergedRp.getExecutorCores.get == 8) - assert(mergedRp.executorResources.size == 6) + assert(mergedRp.executorResources.size == 7) assert(mergedRp.taskResources.size == 3) assert(mergedRp.executorResources.get(GPU).get.amount == 1) assert(mergedRp.executorResources.get(GPU).get.discoveryScript == "disc") @@ -3307,6 +3307,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mergedRp.executorResources.get(ResourceProfile.MEMORY).get.amount == 3072) assert(mergedRp.executorResources.get(ResourceProfile.PYSPARK_MEM).get.amount == 2048) assert(mergedRp.executorResources.get(ResourceProfile.OVERHEAD_MEM).get.amount == 1024) + assert(mergedRp.executorResources.get(ResourceProfile.OFFHEAP_MEM).get.amount == 4096) val ereqs7 = new ExecutorResourceRequests().cores(1).memory("3g") .resource(GPU, 4, "disc") diff --git a/python/pyspark/resource/requests.py b/python/pyspark/resource/requests.py index 56ad6e8be9bcb..61491080a8c43 100644 --- a/python/pyspark/resource/requests.py +++ b/python/pyspark/resource/requests.py @@ -91,6 +91,7 @@ class ExecutorResourceRequests(object): _MEMORY = "memory" _OVERHEAD_MEM = "memoryOverhead" _PYSPARK_MEM = "pyspark.memory" + _OFFHEAP_MEM = "offHeap" def __init__(self, _jvm=None, _requests=None): from pyspark import SparkContext @@ -139,6 +140,14 @@ def pysparkMemory(self, amount): ExecutorResourceRequest(self._PYSPARK_MEM, _parse_memory(amount)) return self + def offheapMemory(self, amount): + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.offHeapMemory(amount) + else: + self._executor_resources[self._OFFHEAP_MEM] = \ + ExecutorResourceRequest(self._OFFHEAP_MEM, _parse_memory(amount)) + return self + def cores(self, amount): if self._java_executor_resource_requests is not None: self._java_executor_resource_requests.cores(amount) diff --git a/python/pyspark/resource/tests/test_resources.py b/python/pyspark/resource/tests/test_resources.py index 9eb5a35e58df5..09c0d3ca20703 100644 --- a/python/pyspark/resource/tests/test_resources.py +++ b/python/pyspark/resource/tests/test_resources.py @@ -25,15 +25,16 @@ class ResourceProfileTests(unittest.TestCase): def test_profile_before_sc(self): rpb = ResourceProfileBuilder() ereqs = ExecutorResourceRequests().cores(2).memory("6g").memoryOverhead("1g") - ereqs.pysparkMemory("2g").resource("gpu", 2, "testGpus", "nvidia.com") + ereqs.pysparkMemory("2g").offheapMemory("3g").resource("gpu", 2, "testGpus", "nvidia.com") treqs = TaskResourceRequests().cpus(2).resource("gpu", 2) def assert_request_contents(exec_reqs, task_reqs): - self.assertEqual(len(exec_reqs), 5) + self.assertEqual(len(exec_reqs), 6) self.assertEqual(exec_reqs["cores"].amount, 2) self.assertEqual(exec_reqs["memory"].amount, 6144) self.assertEqual(exec_reqs["memoryOverhead"].amount, 1024) self.assertEqual(exec_reqs["pyspark.memory"].amount, 2048) + self.assertEqual(exec_reqs["offHeap"].amount, 3072) self.assertEqual(exec_reqs["gpu"].amount, 2) self.assertEqual(exec_reqs["gpu"].discoveryScript, "testGpus") self.assertEqual(exec_reqs["gpu"].resourceName, "gpu") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index dc093235288a9..adbbbc01a0bd5 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -308,7 +308,6 @@ private[yarn] class YarnAllocator( if (!rpIdToYarnResource.contains(rp.id)) { // Start with the application or default settings var heapMem = executorMemory.toLong - // Note we currently don't support off heap memory in ResourceProfile - SPARK-30794 var offHeapMem = executorOffHeapMemory.toLong var overheadMem = memoryOverhead.toLong var pysparkMem = pysparkWorkerMemory.toLong @@ -326,6 +325,8 @@ private[yarn] class YarnAllocator( overheadMem = execReq.amount case ResourceProfile.PYSPARK_MEM => pysparkMem = execReq.amount + case ResourceProfile.OFFHEAP_MEM => + offHeapMem = YarnSparkHadoopUtil.executorOffHeapMemorySizeAsMb(sparkConf, execReq) case ResourceProfile.CORES => cores = execReq.amount.toInt case "gpu" => diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 9d6b776a69d85..fe8990be7ee6f 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.config._ import org.apache.spark.launcher.YarnCommandBuilderUtils +import org.apache.spark.resource.ExecutorResourceRequest import org.apache.spark.util.Utils object YarnSparkHadoopUtil { @@ -187,11 +188,27 @@ object YarnSparkHadoopUtil { * Convert MEMORY_OFFHEAP_SIZE to MB Unit, return 0 if MEMORY_OFFHEAP_ENABLED is false. */ def executorOffHeapMemorySizeAsMb(sparkConf: SparkConf): Int = { + val sizeInMB = Utils.memoryStringToMb(sparkConf.get(MEMORY_OFFHEAP_SIZE).toString) + checkOffHeapEnabled(sparkConf, sizeInMB).toInt + } + + /** + * Get offHeap memory size from [[ExecutorResourceRequest]] + * return 0 if MEMORY_OFFHEAP_ENABLED is false. + */ + def executorOffHeapMemorySizeAsMb(sparkConf: SparkConf, + execRequest: ExecutorResourceRequest): Long = { + checkOffHeapEnabled(sparkConf, execRequest.amount) + } + + /** + * return 0 if MEMORY_OFFHEAP_ENABLED is false. + */ + def checkOffHeapEnabled(sparkConf: SparkConf, offHeapSize: Long): Long = { if (sparkConf.get(MEMORY_OFFHEAP_ENABLED)) { - val sizeInMB = Utils.memoryStringToMb(sparkConf.get(MEMORY_OFFHEAP_SIZE).toString) - require(sizeInMB > 0, + require(offHeapSize > 0, s"${MEMORY_OFFHEAP_SIZE.key} must be > 0 when ${MEMORY_OFFHEAP_ENABLED.key} == true") - sizeInMB + offHeapSize } else { 0 } From ea58e5282304a99f1c12489f722d6b3fb18c4085 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 27 Jul 2020 22:37:08 +0900 Subject: [PATCH 296/384] [SPARK-32434][CORE][FOLLOW-UP] Fix load-spark-env.cmd to be able to run in Windows properly ### What changes were proposed in this pull request? This PR is basically a followup of SPARK-26132 and SPARK-32434. You can't define an environment variable within an-if to use it within the block. See also https://superuser.com/questions/78496/variables-in-batch-file-not-being-set-when-inside-if ### Why are the changes needed? For Windows users to use Spark and fix the build in AppVeyor. ### Does this PR introduce _any_ user-facing change? No, it's only in unreleased branches. ### How was this patch tested? Manually tested on a local Windows machine, and AppVeyor build at https://github.com/HyukjinKwon/spark/pull/13. See https://ci.appveyor.com/project/HyukjinKwon/spark/builds/34316409 Closes #29254 from HyukjinKwon/SPARK-32434. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- bin/load-spark-env.cmd | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index ebbde66153e14..6cb8bf7fc7a21 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -22,7 +22,7 @@ rem spark-env.cmd is loaded from SPARK_CONF_DIR if set, or within the current di rem conf\ subdirectory. set SPARK_ENV_CMD=spark-env.cmd -if [%SPARK_ENV_LOADED%] == [] ( +if not defined SPARK_ENV_LOADED ( set SPARK_ENV_LOADED=1 if [%SPARK_CONF_DIR%] == [] ( @@ -37,18 +37,19 @@ if [%SPARK_ENV_LOADED%] == [] ( rem Setting SPARK_SCALA_VERSION if not already set. -if [%SPARK_SCALA_VERSION%] == [] ( - set SCALA_VERSION_1=2.13 - set SCALA_VERSION_2=2.12 +set SCALA_VERSION_1=2.13 +set SCALA_VERSION_2=2.12 - set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% - set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% - set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables +set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% +set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% +set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables + +if not defined SPARK_SCALA_VERSION ( if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( - echo "Presence of build for multiple Scala versions detected (%ASSEMBLY_DIR1% and %ASSEMBLY_DIR2%)." - echo "Remove one of them or, set SPARK_SCALA_VERSION=%SCALA_VERSION_1% in %SPARK_ENV_CMD%." - echo "Visit %ENV_VARIABLE_DOC% for more details about setting environment variables in spark-env.cmd." - echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." + echo Presence of build for multiple Scala versions detected ^(%ASSEMBLY_DIR1% and %ASSEMBLY_DIR2%^). + echo Remove one of them or, set SPARK_SCALA_VERSION=%SCALA_VERSION_1% in spark-env.cmd. + echo Visit %ENV_VARIABLE_DOC% for more details about setting environment variables in spark-env.cmd. + echo Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd. exit 1 ) if exist %ASSEMBLY_DIR1% ( From 548b7db345667fe2c7f98a8af2b0870c56804f10 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Mon, 27 Jul 2020 17:01:03 +0000 Subject: [PATCH 297/384] [SPARK-32420][SQL] Add handling for unique key in non-codegen hash join ### What changes were proposed in this pull request? `HashRelation` has two separate code paths for unique key look up and non-unique key look up E.g. in its subclass [`UnsafeHashedRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala#L144-L177), unique key look up is more efficient as it does not have e.g. extra `Iterator[UnsafeRow].hasNext()/next()` overhead per row. `BroadcastHashJoinExec` has handled unique key vs non-unique key separately in [code-gen path](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala#L289-L321). But the non-codegen path for broadcast hash join and shuffled hash join do not separate it yet, so adding the support here. ### Why are the changes needed? Shuffled hash join and non-codegen broadcast hash join still rely on this code path for execution. So this PR will help save CPU for executing this two type of join. Adding codegen for shuffled hash join would be a different topic and I will add it in https://issues.apache.org/jira/browse/SPARK-32421 . Ran the same query as [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153-L167), with enabling and disabling this feature. Verified 20% wall clock time improvement (switch control and test group order as well to verify the improvement to not be the noise). ``` Running benchmark: shuffle hash join Running case: shuffle hash join unique key SHJ off Stopped after 5 iterations, 4039 ms Running case: shuffle hash join unique key SHJ on Stopped after 5 iterations, 2898 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4 Intel(R) Core(TM) i9-9980HK CPU 2.40GHz shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ shuffle hash join unique key SHJ off 707 808 81 5.9 168.6 1.0X shuffle hash join unique key SHJ on 547 580 50 7.7 130.4 1.3X ``` ``` Running benchmark: shuffle hash join Running case: shuffle hash join unique key SHJ on Stopped after 5 iterations, 3333 ms Running case: shuffle hash join unique key SHJ off Stopped after 5 iterations, 4268 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4 Intel(R) Core(TM) i9-9980HK CPU 2.40GHz shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ shuffle hash join unique key SHJ on 565 667 60 7.4 134.8 1.0X shuffle hash join unique key SHJ off 774 854 85 5.4 184.4 0.7X ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? * Added test in `OuterJoinSuite` to cover left outer and right outer join. * Added test in `ExistenceJoinSuite` to cover left semi join, and existence join. * [Existing `joinSuite` already covered inner join.](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala#L182) * [Existing `ExistenceJoinSuite` already covered left anti join, and existence join.](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala#L228) Closes #29216 from c21/unique-key. Authored-by: Cheng Su Signed-off-by: Wenchen Fan --- .../spark/sql/execution/joins/HashJoin.scala | 154 ++++++++++++------ .../execution/joins/ExistenceJoinSuite.scala | 10 +- .../sql/execution/joins/OuterJoinSuite.scala | 63 ++++++- 3 files changed, 177 insertions(+), 50 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index fa3a55aa5ad94..4f22007b65845 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -142,47 +142,73 @@ trait HashJoin extends BaseJoinExec { hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinRow = new JoinedRow val joinKeys = streamSideKeyGenerator() - streamIter.flatMap { srow => - joinRow.withLeft(srow) - val matches = hashedRelation.get(joinKeys(srow)) - if (matches != null) { - matches.map(joinRow.withRight(_)).filter(boundCondition) - } else { - Seq.empty + + if (hashedRelation.keyIsUnique) { + streamIter.flatMap { srow => + joinRow.withLeft(srow) + val matched = hashedRelation.getValue(joinKeys(srow)) + if (matched != null) { + Some(joinRow.withRight(matched)).filter(boundCondition) + } else { + None + } + } + } else { + streamIter.flatMap { srow => + joinRow.withLeft(srow) + val matches = hashedRelation.get(joinKeys(srow)) + if (matches != null) { + matches.map(joinRow.withRight).filter(boundCondition) + } else { + Seq.empty + } } } } private def outerJoin( streamedIter: Iterator[InternalRow], - hashedRelation: HashedRelation): Iterator[InternalRow] = { + hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinedRow = new JoinedRow() val keyGenerator = streamSideKeyGenerator() val nullRow = new GenericInternalRow(buildPlan.output.length) - streamedIter.flatMap { currentRow => - val rowKey = keyGenerator(currentRow) - joinedRow.withLeft(currentRow) - val buildIter = hashedRelation.get(rowKey) - new RowIterator { - private var found = false - override def advanceNext(): Boolean = { - while (buildIter != null && buildIter.hasNext) { - val nextBuildRow = buildIter.next() - if (boundCondition(joinedRow.withRight(nextBuildRow))) { + if (hashedRelation.keyIsUnique) { + streamedIter.map { currentRow => + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + val matched = hashedRelation.getValue(rowKey) + if (matched != null && boundCondition(joinedRow.withRight(matched))) { + joinedRow + } else { + joinedRow.withRight(nullRow) + } + } + } else { + streamedIter.flatMap { currentRow => + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + val buildIter = hashedRelation.get(rowKey) + new RowIterator { + private var found = false + override def advanceNext(): Boolean = { + while (buildIter != null && buildIter.hasNext) { + val nextBuildRow = buildIter.next() + if (boundCondition(joinedRow.withRight(nextBuildRow))) { + found = true + return true + } + } + if (!found) { + joinedRow.withRight(nullRow) found = true return true } + false } - if (!found) { - joinedRow.withRight(nullRow) - found = true - return true - } - false - } - override def getRow: InternalRow = joinedRow - }.toScala + override def getRow: InternalRow = joinedRow + }.toScala + } } } @@ -191,12 +217,22 @@ trait HashJoin extends BaseJoinExec { hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinKeys = streamSideKeyGenerator() val joinedRow = new JoinedRow - streamIter.filter { current => - val key = joinKeys(current) - lazy val buildIter = hashedRelation.get(key) - !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { - (row: InternalRow) => boundCondition(joinedRow(current, row)) - }) + + if (hashedRelation.keyIsUnique) { + streamIter.filter { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + !key.anyNull && matched != null && + (condition.isEmpty || boundCondition(joinedRow(current, matched))) + } + } else { + streamIter.filter { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { + (row: InternalRow) => boundCondition(joinedRow(current, row)) + }) + } } } @@ -206,14 +242,26 @@ trait HashJoin extends BaseJoinExec { val joinKeys = streamSideKeyGenerator() val result = new GenericInternalRow(Array[Any](null)) val joinedRow = new JoinedRow - streamIter.map { current => - val key = joinKeys(current) - lazy val buildIter = hashedRelation.get(key) - val exists = !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { - (row: InternalRow) => boundCondition(joinedRow(current, row)) - }) - result.setBoolean(0, exists) - joinedRow(current, result) + + if (hashedRelation.keyIsUnique) { + streamIter.map { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + val exists = !key.anyNull && matched != null && + (condition.isEmpty || boundCondition(joinedRow(current, matched))) + result.setBoolean(0, exists) + joinedRow(current, result) + } + } else { + streamIter.map { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + val exists = !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { + (row: InternalRow) => boundCondition(joinedRow(current, row)) + }) + result.setBoolean(0, exists) + joinedRow(current, result) + } } } @@ -222,12 +270,22 @@ trait HashJoin extends BaseJoinExec { hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinKeys = streamSideKeyGenerator() val joinedRow = new JoinedRow - streamIter.filter { current => - val key = joinKeys(current) - lazy val buildIter = hashedRelation.get(key) - key.anyNull || buildIter == null || (condition.isDefined && !buildIter.exists { - row => boundCondition(joinedRow(current, row)) - }) + + if (hashedRelation.keyIsUnique) { + streamIter.filter { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + key.anyNull || matched == null || + (condition.isDefined && !boundCondition(joinedRow(current, matched))) + } + } else { + streamIter.filter { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + key.anyNull || buildIter == null || (condition.isDefined && !buildIter.exists { + row => boundCondition(joinedRow(current, row)) + }) + } } } @@ -245,7 +303,7 @@ trait HashJoin extends BaseJoinExec { semiJoin(streamedIter, hashed) case LeftAnti => antiJoin(streamedIter, hashed) - case j: ExistenceJoin => + case _: ExistenceJoin => existenceJoin(streamedIter, hashed) case x => throw new IllegalArgumentException( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index 554990413c28c..e8ac09fdb634e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -200,6 +200,14 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSparkSession { singleConditionEQ, Seq(Row(2, 1.0), Row(2, 1.0), Row(3, 3.0), Row(6, null))) + testExistenceJoin( + "test single unique condition (equal) for left semi join", + LeftSemi, + left, + right.select(right.col("c")).distinct(), /* Trigger BHJs and SHJs unique key code path! */ + singleConditionEQ, + Seq(Row(2, 1.0), Row(2, 1.0), Row(3, 3.0), Row(6, null))) + testExistenceJoin( "test composed condition (equal & non-equal) for left semi join", LeftSemi, @@ -229,7 +237,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSparkSession { "test single unique condition (equal) for left Anti join", LeftAnti, left, - right.select(right.col("c")).distinct(), /* Trigger BHJs unique key code path! */ + right.select(right.col("c")).distinct(), /* Trigger BHJs and SHJs unique key code path! */ singleConditionEQ, Seq(Row(1, 2.0), Row(1, 2.0), Row(null, null), Row(null, 5.0))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 879f282e4d05d..a466e05816ad8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -47,7 +47,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { sparkContext.parallelize(Seq( Row(0, 0.0), Row(2, 3.0), // This row is duplicated to ensure that we will have multiple buffered matches - Row(2, -1.0), + Row(2, -1.0), // This row is duplicated to ensure that we will have multiple buffered matches Row(2, -1.0), Row(2, 3.0), Row(3, 2.0), @@ -62,6 +62,32 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { LessThan(left.col("b").expr, right.col("d").expr)) } + private lazy val uniqueLeft = spark.createDataFrame( + sparkContext.parallelize(Seq( + Row(1, 2.0), + Row(2, 1.0), + Row(3, 3.0), + Row(5, 1.0), + Row(6, 6.0), + Row(null, null) + )), new StructType().add("a", IntegerType).add("b", DoubleType)) + + private lazy val uniqueRight = spark.createDataFrame( + sparkContext.parallelize(Seq( + Row(0, 0.0), + Row(2, 3.0), + Row(3, 2.0), + Row(4, 1.0), + Row(5, 3.0), + Row(7, 7.0), + Row(null, null) + )), new StructType().add("c", IntegerType).add("d", DoubleType)) + + private lazy val uniqueCondition = { + And((uniqueLeft.col("a") === uniqueRight.col("c")).expr, + LessThan(uniqueLeft.col("b").expr, uniqueRight.col("d").expr)) + } + // Note: the input dataframes and expression must be evaluated lazily because // the SQLContext should be used only within a test to keep SQL tests stable private def testOuterJoin( @@ -243,4 +269,39 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { condition, Seq.empty ) + + // --- Join keys are unique --------------------------------------------------------------------- + + testOuterJoin( + "left outer join with unique keys", + uniqueLeft, + uniqueRight, + LeftOuter, + uniqueCondition, + Seq( + (null, null, null, null), + (1, 2.0, null, null), + (2, 1.0, 2, 3.0), + (3, 3.0, null, null), + (5, 1.0, 5, 3.0), + (6, 6.0, null, null) + ) + ) + + testOuterJoin( + "right outer join with unique keys", + uniqueLeft, + uniqueRight, + RightOuter, + uniqueCondition, + Seq( + (null, null, null, null), + (null, null, 0, 0.0), + (2, 1.0, 2, 3.0), + (null, null, 3, 2.0), + (null, null, 4, 1.0), + (5, 1.0, 5, 3.0), + (null, null, 7, 7.0) + ) + ) } From d315ebf3a739a05a68d0f0ab319920765bf65b0f Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 27 Jul 2020 17:03:14 +0000 Subject: [PATCH 298/384] [SPARK-32424][SQL] Fix silent data change for timestamp parsing if overflow happens MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? When using `Seconds.toMicros` API to convert epoch seconds to microseconds, ```scala /** * Equivalent to * {link #convert(long, TimeUnit) MICROSECONDS.convert(duration, this)}. * param duration the duration * return the converted duration, * or {code Long.MIN_VALUE} if conversion would negatively * overflow, or {code Long.MAX_VALUE} if it would positively overflow. */ ``` This PR change it to `Math.multiplyExact(epochSeconds, MICROS_PER_SECOND)` ### Why are the changes needed? fix silent data change between 3.x and 2.x ``` ~/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200722  bin/spark-sql -S -e "select to_timestamp('300000', 'y');" +294247-01-10 12:00:54.775807 ``` ``` kentyaohulk  ~/Downloads/spark/spark-2.4.5-bin-hadoop2.7  bin/spark-sql -S -e "select to_timestamp('300000', 'y');" 284550-10-19 15:58:1010.448384 ``` ### Does this PR introduce _any_ user-facing change? Yes, we will raise `ArithmeticException` instead of giving the wrong answer if overflow. ### How was this patch tested? add unit test Closes #29220 from yaooqinn/SPARK-32424. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- docs/sql-ref-datetime-pattern.md | 2 +- .../util/DateTimeFormatterHelper.scala | 4 +- .../catalyst/util/TimestampFormatter.scala | 3 +- .../util/TimestampFormatterSuite.scala | 10 ++ .../inputs/datetime-formatting-invalid.sql | 4 +- .../sql-tests/inputs/datetime-formatting.sql | 2 +- .../inputs/datetime-parsing-invalid.sql | 9 ++ .../sql-tests/inputs/datetime-parsing.sql | 28 ++++++ .../datetime-formatting-invalid.sql.out | 4 +- .../datetime-formatting-legacy.sql.out | 18 ++-- .../results/datetime-formatting.sql.out | 18 ++-- .../results/datetime-parsing-invalid.sql.out | 55 ++++++++++- .../results/datetime-parsing-legacy.sql.out | 98 ++++++++++++++++++- .../results/datetime-parsing.sql.out | 98 ++++++++++++++++++- 14 files changed, 323 insertions(+), 30 deletions(-) diff --git a/docs/sql-ref-datetime-pattern.md b/docs/sql-ref-datetime-pattern.md index 3c0bc754f940b..d0299e5a99dc2 100644 --- a/docs/sql-ref-datetime-pattern.md +++ b/docs/sql-ref-datetime-pattern.md @@ -70,7 +70,7 @@ The count of pattern letters determines the format. For formatting, the fraction length would be padded to the number of contiguous 'S' with zeros. Spark supports datetime of micro-of-second precision, which has up to 6 significant digits, but can parse nano-of-second with exceeded part truncated. -- 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. 11 or more letters will fail. +- 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. 7 or more letters will fail. - Month: It follows the rule of Number/Text. The text form is depend on letters - 'M' denotes the 'standard' form, and 'L' is for 'stand-alone' form. These two forms are different only in some certain languages. For example, in Russian, 'Июль' is the stand-alone form of July, and 'Июля' is the standard form. Here are examples for all supported pattern letters: - `'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. 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 5de06af6af12a..f02b2d08c0935 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 @@ -296,7 +296,9 @@ private object DateTimeFormatterHelper { // unchecked `ArrayIndexOutOfBoundsException` by the `NumberPrinterParser` for formatting. It // makes the call side difficult to handle exceptions and easily leads to silent data change // because of the exceptions being suppressed. - Seq("y").map(_ * 11) + // SPARK-32424: The max year that we can actually handle is 6 digits, otherwise, it will + // overflow + Seq("y").map(_ * 7) }.toSet /** 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 8db95044359c3..6832d1a9954fb 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 @@ -24,7 +24,6 @@ import java.time.format.{DateTimeFormatter, DateTimeParseException} import java.time.temporal.ChronoField.MICRO_OF_SECOND import java.time.temporal.TemporalQueries import java.util.{Calendar, GregorianCalendar, Locale, TimeZone} -import java.util.concurrent.TimeUnit.SECONDS import org.apache.commons.lang3.time.FastDateFormat @@ -83,7 +82,7 @@ class Iso8601TimestampFormatter( val epochSeconds = zonedDateTime.toEpochSecond val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND) - Math.addExact(SECONDS.toMicros(epochSeconds), microsOfSecond) + Math.addExact(Math.multiplyExact(epochSeconds, MICROS_PER_SECOND), microsOfSecond) } catch checkParsedDiff(s, legacyFormatter.parse) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala index b3e0d6394940a..103b7a2eded28 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala @@ -437,4 +437,14 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite { assert(formatter.format(date(1970, 4, 10)) == "100") } } + + test("SPARK-32424: avoid silent data change when timestamp overflows") { + val formatter = TimestampFormatter("y", UTC, isParsing = true) + assert(formatter.parse("294247") === date(294247)) + assert(formatter.parse("-290307") === date(-290307)) + val e1 = intercept[ArithmeticException](formatter.parse("294248")) + assert(e1.getMessage === "long overflow") + val e2 = intercept[ArithmeticException](formatter.parse("-290308")) + assert(e2.getMessage === "long overflow") + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-invalid.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-invalid.sql index 9072aa107f252..11bba00e91abf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-invalid.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-invalid.sql @@ -3,8 +3,8 @@ -- separating this from datetime-formatting.sql, because the text form -- for patterns with 5 letters in SimpleDateFormat varies from different JDKs select date_format('2018-11-17 13:33:33.333', 'GGGGG'); --- pattern letter count can not be greater than 10 -select date_format('2018-11-17 13:33:33.333', 'yyyyyyyyyyy'); +-- pattern letter count can not be greater than 6 +select date_format('2018-11-17 13:33:33.333', 'yyyyyyy'); -- q/L in JDK 8 will fail when the count is more than 2 select date_format('2018-11-17 13:33:33.333', 'qqqqq'); select date_format('2018-11-17 13:33:33.333', 'QQQQQ'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting.sql index 3b23a7785f6cd..2d70326f4f3c0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting.sql @@ -11,7 +11,7 @@ create temporary view v as select col from values select col, date_format(col, 'G GG GGG GGGG') from v; -select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy yyyyyyy yyyyyyyy yyyyyyyyy yyyyyyyyyy') from v; +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy') from v; select col, date_format(col, 'q qq') from v; diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql index 0b313e5a0b9f4..a1c02eaa3b0a0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql @@ -1,5 +1,14 @@ --- TESTS FOR DATETIME PARSING FUNCTIONS WITH INVALID VALUES --- +-- parsing invalid value with pattern 'y' +select to_timestamp('294248', 'y'); -- out of year value range [0, 294247] +select to_timestamp('1', 'yy'); -- the number of digits must be 2 for 'yy'. +select to_timestamp('-12', 'yy'); -- out of year value range [0, 99] for reduced two digit form +select to_timestamp('123', 'yy'); -- the number of digits must be 2 for 'yy'. +select to_timestamp('1', 'yyy'); -- the number of digits must be in [3, 6] for 'yyy' + +select to_timestamp('1234567', 'yyyyyyy'); -- the length of 'y' pattern must be less than 7 + -- parsing invalid values with pattern 'D' select to_timestamp('366', 'D'); select to_timestamp('9', 'DD'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql index 74866d9c6ffa1..e058bd675c375 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql @@ -1,5 +1,33 @@ --- TESTS FOR DATETIME PARSING FUNCTIONS --- +-- parsing with pattern 'y'. +-- the range of valid year is [-290307, 294247], +-- but particularly, some thrift client use java.sql.Timestamp to parse timestamp, which allows +-- only positive year values less or equal than 9999. So the cases bellow only use [1, 9999] to pass +-- ThriftServerQueryTestSuite +select to_timestamp('1', 'y'); +select to_timestamp('009999', 'y'); + +-- reduced two digit form is used, the range of valid year is 20-[01, 99] +select to_timestamp('00', 'yy'); +select to_timestamp('99', 'yy'); + +-- the range of valid year is [-290307, 294247], the number of digits must be in [3, 6] for 'yyy' +select to_timestamp('001', 'yyy'); +select to_timestamp('009999', 'yyy'); + +-- the range of valid year is [-9999, 9999], the number of digits must be 4 for 'yyyy'. +select to_timestamp('0001', 'yyyy'); +select to_timestamp('9999', 'yyyy'); + +-- the range of valid year is [-99999, 99999], the number of digits must be 5 for 'yyyyy'. +select to_timestamp('00001', 'yyyyy'); +select to_timestamp('09999', 'yyyyy'); + +-- the range of valid year is [-290307, 294247], the number of digits must be 6 for 'yyyyyy'. +select to_timestamp('000001', 'yyyyyy'); +select to_timestamp('009999', 'yyyyyy'); + -- parsing with pattern 'D' select to_timestamp('9', 'D'); select to_timestamp('300', 'D'); diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out index 248157efacde0..18d1a10068794 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out @@ -12,12 +12,12 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn -- !query -select date_format('2018-11-17 13:33:33.333', 'yyyyyyyyyyy') +select date_format('2018-11-17 13:33:33.333', 'yyyyyyy') -- !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 'yyyyyyyyyyy' 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 +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyyyyy' 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/resources/sql-tests/results/datetime-formatting-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out index b7bc448a952a7..b37922b20807d 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out @@ -32,17 +32,17 @@ struct -- !query -select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy yyyyyyy yyyyyyyy yyyyyyyyy yyyyyyyyyy') from v +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy') from v -- !query schema -struct +struct -- !query output -1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 0001582 00001582 000001582 0000001582 -1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 0001969 00001969 000001969 0000001969 -1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 0001970 00001970 000001970 0000001970 -1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 0001996 00001996 000001996 0000001996 -2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 0002018 00002018 000002018 0000002018 -2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 0002019 00002019 000002019 0000002019 -2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 0002100 00002100 000002100 0000002100 +1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 +1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 +1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 +1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 +2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 +2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 +2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out index f724658d354df..5bed88e168f1e 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out @@ -32,17 +32,17 @@ struct -- !query -select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy yyyyyyy yyyyyyyy yyyyyyyyy yyyyyyyyyy') from v +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy') from v -- !query schema -struct +struct -- !query output -1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 0001582 00001582 000001582 0000001582 -1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 0001969 00001969 000001969 0000001969 -1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 0001970 00001970 000001970 0000001970 -1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 0001996 00001996 000001996 0000001996 -2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 0002018 00002018 000002018 0000002018 -2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 0002019 00002019 000002019 0000002019 -2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 0002100 00002100 000002100 0000002100 +1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 +1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 +1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 +1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 +2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 +2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 +2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out index dcee0d4c270a3..c11f5f39bebd0 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out @@ -1,5 +1,58 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 13 +-- Number of queries: 19 + + +-- !query +select to_timestamp('294248', 'y') +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select to_timestamp('1', 'yy') +-- !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 parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('-12', 'yy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123', 'yy') +-- !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 parse '123' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('1', 'yyy') +-- !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 parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('1234567', 'yyyyyyy') +-- !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 'yyyyyyy' 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/resources/sql-tests/results/datetime-parsing-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out index 742e0839d919a..bb7ce74a29ef5 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out @@ -1,5 +1,101 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 13 +-- Number of queries: 25 + + +-- !query +select to_timestamp('1', 'y') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'y') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00', 'yy') +-- !query schema +struct +-- !query output +2000-01-01 00:00:00 + + +-- !query +select to_timestamp('99', 'yy') +-- !query schema +struct +-- !query output +1999-01-01 00:00:00 + + +-- !query +select to_timestamp('001', 'yyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('0001', 'yyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('9999', 'yyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00001', 'yyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('09999', 'yyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('000001', 'yyyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out index 742e0839d919a..98146a189a005 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out @@ -1,5 +1,101 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 13 +-- Number of queries: 25 + + +-- !query +select to_timestamp('1', 'y') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'y') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00', 'yy') +-- !query schema +struct +-- !query output +2000-01-01 00:00:00 + + +-- !query +select to_timestamp('99', 'yy') +-- !query schema +struct +-- !query output +2099-01-01 00:00:00 + + +-- !query +select to_timestamp('001', 'yyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('0001', 'yyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('9999', 'yyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00001', 'yyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('09999', 'yyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('000001', 'yyyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 -- !query From c1140661bf2cc174a380237f0f494ca56d4768ed Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 27 Jul 2020 12:02:43 -0700 Subject: [PATCH 299/384] [SPARK-32443][CORE] Use POSIX-compatible `command -v` in testCommandAvailable ### What changes were proposed in this pull request? This PR aims to use `command -v` in non-Window operating systems instead of executing the given command. ### Why are the changes needed? 1. `command` is POSIX-compatible - **POSIX.1-2017**: https://pubs.opengroup.org/onlinepubs/9699919799/utilities/command.html 2. `command` is faster and safer than the direct execution - `command` doesn't invoke another process. ```scala scala> sys.process.Process("ls").run().exitValue() LICENSE NOTICE bin doc lib man res1: Int = 0 ``` 3. The existing way behaves inconsistently. - `rm` cannot be checked. **AS-IS** ```scala scala> sys.process.Process("rm").run().exitValue() usage: rm [-f | -i] [-dPRrvW] file ... unlink file res0: Int = 64 ``` **TO-BE** ``` Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262). Type in expressions for evaluation. Or try :help. scala> sys.process.Process(Seq("sh", "-c", s"command -v ls")).run().exitValue() /bin/ls val res1: Int = 0 ``` 4. The existing logic is already broken in Scala 2.13 environment because it hangs like the following. ```scala $ bin/scala Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262). Type in expressions for evaluation. Or try :help. scala> sys.process.Process("cat").run().exitValue() // hang here. ``` ### Does this PR introduce _any_ user-facing change? No. Although this is inside `main` source directory, this is used for testing purpose. ``` $ git grep testCommandAvailable | grep -v 'def testCommandAvailable' core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat")) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("wc")) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat")) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat")) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat")) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable(envCommand)) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(!TestUtils.testCommandAvailable("some_nonexistent_command")) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat")) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat")) core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable(envCommand)) sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala: private lazy val isPythonAvailable: Boolean = TestUtils.testCommandAvailable(pythonExec) sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala: if (TestUtils.testCommandAvailable(pythonExec)) { sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("python")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("echo | sed")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash")) ``` ### How was this patch tested? - **Scala 2.12**: Pass the Jenkins with the existing tests and one modified test. - **Scala 2.13**: Do the following manually. It should pass instead of `hang`. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.PipedRDDSuite ... Tests: succeeded 12, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes #29241 from dongjoon-hyun/SPARK-32443. Lead-authored-by: HyukjinKwon Co-authored-by: Dongjoon Hyun Co-authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/TestUtils.scala | 8 +++++++- .../test/scala/org/apache/spark/rdd/PipedRDDSuite.scala | 6 ++++-- .../apache/spark/sql/hive/execution/SQLQuerySuite.scala | 3 ++- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index d459627930f4c..259cc43cdfdbb 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -236,7 +236,13 @@ private[spark] object TestUtils { * Test if a command is available. */ def testCommandAvailable(command: String): Boolean = { - val attempt = Try(Process(command).run(ProcessLogger(_ => ())).exitValue()) + val attempt = if (Utils.isWindows) { + Try(Process(Seq( + "cmd.exe", "/C", s"where $command")).run(ProcessLogger(_ => ())).exitValue()) + } else { + Try(Process(Seq( + "sh", "-c", s"command -v $command")).run(ProcessLogger(_ => ())).exitValue()) + } attempt.isSuccess && attempt.get == 0 } diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 2da2854dfbcb9..5000011b3c5ee 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -176,7 +176,8 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext with Eventuall } test("pipe with env variable") { - assume(TestUtils.testCommandAvailable(envCommand)) + val executable = envCommand.split("\\s+", 2)(0) + assume(TestUtils.testCommandAvailable(executable)) val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val piped = nums.pipe(s"$envCommand MY_TEST_ENV", Map("MY_TEST_ENV" -> "LALALA")) val c = piped.collect() @@ -238,7 +239,8 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext with Eventuall } def testExportInputFile(varName: String): Unit = { - assume(TestUtils.testCommandAvailable(envCommand)) + val executable = envCommand.split("\\s+", 2)(0) + assume(TestUtils.testCommandAvailable(executable)) val nums = new HadoopRDD(sc, new JobConf(), classOf[TextInputFormat], classOf[LongWritable], classOf[Text], 2) { override def getPartitions: Array[Partition] = Array(generateFakeHadoopPartition()) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 920f6385f8e19..62a411a56159b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -92,7 +92,8 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi test("script") { withTempView("script_table") { assume(TestUtils.testCommandAvailable("/bin/bash")) - assume(TestUtils.testCommandAvailable("echo | sed")) + assume(TestUtils.testCommandAvailable("echo")) + assume(TestUtils.testCommandAvailable("sed")) val scriptFilePath = getTestResourcePath("test_script.sh") val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") df.createOrReplaceTempView("script_table") From f7542d3b61faa29eee3a496935413653c714b6b2 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Mon, 27 Jul 2020 12:05:29 -0700 Subject: [PATCH 300/384] [SPARK-32457][ML] logParam thresholds in DT/GBT/FM/LR/MLP ### What changes were proposed in this pull request? logParam `thresholds` in DT/GBT/FM/LR/MLP ### Why are the changes needed? param `thresholds` is logged in NB/RF, but not in other ProbabilisticClassifier ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? existing testsuites Closes #29257 from zhengruifeng/instr.logParams_add_thresholds. Authored-by: zhengruifeng Signed-off-by: Huaxin Gao --- .../spark/ml/classification/DecisionTreeClassifier.scala | 2 +- .../org/apache/spark/ml/classification/FMClassifier.scala | 6 ++---- .../org/apache/spark/ml/classification/GBTClassifier.scala | 2 +- .../apache/spark/ml/classification/LogisticRegression.scala | 4 ++-- .../ml/classification/MultilayerPerceptronClassifier.scala | 2 +- 5 files changed, 7 insertions(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 530943c910d7d..b2c63cbcb464b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -129,7 +129,7 @@ class DecisionTreeClassifier @Since("1.4.0") ( instr.logNumClasses(numClasses) instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, probabilityCol, leafCol, maxDepth, maxBins, minInstancesPerNode, minInfoGain, - maxMemoryInMB, cacheNodeIds, checkpointInterval, impurity, seed) + maxMemoryInMB, cacheNodeIds, checkpointInterval, impurity, seed, thresholds) val trees = RandomForest.run(instances, strategy, numTrees = 1, featureSubsetStrategy = "all", seed = $(seed), instr = Some(instr), parentUID = Some(uid)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala index 88e6e5c090a21..4188f6893ea37 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala @@ -177,9 +177,7 @@ class FMClassifier @Since("3.0.0") ( def setSeed(value: Long): this.type = set(seed, value) override protected def train( - dataset: Dataset[_] - ): FMClassificationModel = instrumented { instr => - + dataset: Dataset[_]): FMClassificationModel = instrumented { instr => val numClasses = 2 if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + @@ -190,7 +188,7 @@ class FMClassifier @Since("3.0.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, factorSize, fitIntercept, fitLinear, regParam, - miniBatchFraction, initStd, maxIter, stepSize, tol, solver) + miniBatchFraction, initStd, maxIter, stepSize, tol, solver, thresholds) instr.logNumClasses(numClasses) val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 46810bccc8e69..ca43ef863003a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -197,7 +197,7 @@ class GBTClassifier @Since("1.4.0") ( instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, leafCol, impurity, lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, minWeightFractionPerNode, seed, stepSize, subsamplingRate, cacheNodeIds, - checkpointInterval, featureSubsetStrategy, validationIndicatorCol, validationTol) + checkpointInterval, featureSubsetStrategy, validationIndicatorCol, validationTol, thresholds) instr.logNumClasses(numClasses) val categoricalFeatures = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 0e3708bfe9a68..e7dfaec0e37e2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -506,8 +506,8 @@ class LogisticRegression @Since("1.2.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, - probabilityCol, regParam, elasticNetParam, standardization, threshold, maxIter, tol, - fitIntercept, blockSize) + probabilityCol, regParam, elasticNetParam, standardization, threshold, thresholds, maxIter, + tol, fitIntercept, blockSize) val instances = extractInstances(dataset) .setName("training instances") diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 6bffc372b68fe..d23fafb4d79e5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -185,7 +185,7 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, layers, maxIter, - tol, blockSize, solver, stepSize, seed) + tol, blockSize, solver, stepSize, seed, thresholds) val myLayers = $(layers) val labels = myLayers.last From 8de43338be879f0cfeebca328dbbcfd1e5bd70da Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 28 Jul 2020 09:41:53 +0900 Subject: [PATCH 301/384] [SPARK-31753][SQL][DOCS] Add missing keywords in the SQL docs ### What changes were proposed in this pull request? update sql-ref docs, the following key words will be added in this PR. CASE/ELSE WHEN/THEN MAP KEYS TERMINATED BY NULL DEFINED AS LINES TERMINATED BY ESCAPED BY COLLECTION ITEMS TERMINATED BY PIVOT LATERAL VIEW OUTER? ROW FORMAT SERDE ROW FORMAT DELIMITED FIELDS TERMINATED BY IGNORE NULLS FIRST LAST ### Why are the changes needed? let more users know the sql key words usage ### Does this PR introduce _any_ user-facing change? ![image](https://user-images.githubusercontent.com/46367746/88148830-c6dc1f80-cc31-11ea-81ea-13bc9dc34550.png) ![image](https://user-images.githubusercontent.com/46367746/88148968-fb4fdb80-cc31-11ea-8649-e8297cf5813e.png) ![image](https://user-images.githubusercontent.com/46367746/88149000-073b9d80-cc32-11ea-9aa4-f914ecd72663.png) ![image](https://user-images.githubusercontent.com/46367746/88149021-0f93d880-cc32-11ea-86ed-7db8672b5aac.png) ### How was this patch tested? No Closes #29056 from GuoPhilipse/add-missing-keywords. Lead-authored-by: GuoPhilipse Co-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Signed-off-by: Takeshi Yamamuro --- docs/_data/menu-sql.yaml | 6 + ...-ref-syntax-ddl-create-table-hiveformat.md | 94 ++++++++++++- docs/sql-ref-syntax-qry-select-case.md | 109 +++++++++++++++ docs/sql-ref-syntax-qry-select-clusterby.md | 3 + ...sql-ref-syntax-qry-select-distribute-by.md | 3 + docs/sql-ref-syntax-qry-select-groupby.md | 27 ++++ docs/sql-ref-syntax-qry-select-having.md | 3 + .../sql-ref-syntax-qry-select-lateral-view.md | 125 ++++++++++++++++++ docs/sql-ref-syntax-qry-select-limit.md | 3 + docs/sql-ref-syntax-qry-select-orderby.md | 3 + docs/sql-ref-syntax-qry-select-pivot.md | 101 ++++++++++++++ docs/sql-ref-syntax-qry-select-sortby.md | 3 + docs/sql-ref-syntax-qry-select-where.md | 3 + docs/sql-ref-syntax-qry-select.md | 56 +++++--- docs/sql-ref-syntax-qry.md | 3 + docs/sql-ref-syntax.md | 3 + 16 files changed, 520 insertions(+), 25 deletions(-) create mode 100644 docs/sql-ref-syntax-qry-select-case.md create mode 100644 docs/sql-ref-syntax-qry-select-lateral-view.md create mode 100644 docs/sql-ref-syntax-qry-select-pivot.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index ef5a53a69eda6..63f6b4a0a204b 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -187,6 +187,12 @@ url: sql-ref-syntax-qry-select-tvf.html - text: Window Function url: sql-ref-syntax-qry-select-window.html + - text: CASE Clause + url: sql-ref-syntax-qry-select-case.html + - text: LATERAL VIEW Clause + url: sql-ref-syntax-qry-select-lateral-view.html + - text: PIVOT Clause + url: sql-ref-syntax-qry-select-pivot.html - text: EXPLAIN url: sql-ref-syntax-qry-explain.html - text: Auxiliary Statements diff --git a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md index 38f8856a24e3d..7bf847df98150 100644 --- a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md +++ b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md @@ -36,6 +36,14 @@ CREATE [ EXTERNAL ] TABLE [ IF NOT EXISTS ] table_identifier [ LOCATION path ] [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] [ AS select_statement ] + +row_format: + : SERDE serde_class [ WITH SERDEPROPERTIES (k1=v1, k2=v2, ... ) ] + | DELIMITED [ FIELDS TERMINATED BY fields_termiated_char [ ESCAPED BY escaped_char ] ] + [ COLLECTION ITEMS TERMINATED BY collection_items_termiated_char ] + [ MAP KEYS TERMINATED BY map_key_termiated_char ] + [ LINES TERMINATED BY row_termiated_char ] + [ NULL DEFINED AS null_char ] ``` Note that, the clauses between the columns definition clause and the AS SELECT clause can come in @@ -51,15 +59,55 @@ as any order. For example, you can write COMMENT table_comment after TBLPROPERTI * **EXTERNAL** - Table is defined using the path provided as LOCATION, does not use default location for this table. + Table is defined using the path provided as `LOCATION`, does not use default location for this table. * **PARTITIONED BY** Partitions are created on the table, based on the columns specified. + +* **row_format** + + Use the `SERDE` clause to specify a custom SerDe for one table. Otherwise, use the `DELIMITED` clause to use the native SerDe and specify the delimiter, escape character, null character and so on. + +* **SERDE** + + Specifies a custom SerDe for one table. + +* **serde_class** + + Specifies a fully-qualified class name of a custom SerDe. + +* **SERDEPROPERTIES** + + A list of key-value pairs that is used to tag the SerDe definition. + +* **DELIMITED** + + The `DELIMITED` clause can be used to specify the native SerDe and state the delimiter, escape character, null character and so on. + +* **FIELDS TERMINATED BY** -* **ROW FORMAT** + Used to define a column separator. + +* **COLLECTION ITEMS TERMINATED BY** - SERDE is used to specify a custom SerDe or the DELIMITED clause in order to use the native SerDe. + Used to define a collection item separator. + +* **MAP KEYS TERMINATED BY** + + Used to define a map key separator. + +* **LINES TERMINATED BY** + + Used to define a row separator. + +* **NULL DEFINED AS** + + Used to define the specific value for NULL. + +* **ESCAPED BY** + + Used for escape mechanism. * **STORED AS** @@ -114,9 +162,47 @@ CREATE TABLE student (id INT, name STRING) PARTITIONED BY (age INT); --Use Row Format and file format -CREATE TABLE student (id INT,name STRING) +CREATE TABLE student (id INT, name STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE; + +--Use complex datatype +CREATE EXTERNAL TABLE family( + name STRING, + friends ARRAY, + children MAP, + address STRUCT + ) + ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' ESCAPED BY '\\' + COLLECTION ITEMS TERMINATED BY '_' + MAP KEYS TERMINATED BY ':' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'foonull' + STORED AS TEXTFILE + LOCATION '/tmp/family/'; + +--Use predefined custom SerDe +CREATE TABLE avroExample + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + TBLPROPERTIES ('avro.schema.literal'='{ "namespace": "org.apache.hive", + "name": "first_schema", + "type": "record", + "fields": [ + { "name":"string1", "type":"string" }, + { "name":"string2", "type":"string" } + ] }'); + +--Use personalized custom SerDe(we may need to `ADD JAR xxx.jar` first to ensure we can find the serde_class, +--or you may run into `CLASSNOTFOUND` exception) +ADD JAR /tmp/hive_serde_example.jar; + +CREATE EXTERNAL TABLE family (id INT, name STRING) + ROW FORMAT SERDE 'com.ly.spark.serde.SerDeExample' + STORED AS INPUTFORMAT 'com.ly.spark.example.serde.io.SerDeExampleInputFormat' + OUTPUTFORMAT 'com.ly.spark.example.serde.io.SerDeExampleOutputFormat' + LOCATION '/tmp/family/'; ``` ### Related Statements diff --git a/docs/sql-ref-syntax-qry-select-case.md b/docs/sql-ref-syntax-qry-select-case.md new file mode 100644 index 0000000000000..6136b161d234b --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-case.md @@ -0,0 +1,109 @@ +--- +layout: global +title: CASE Clause +displayTitle: CASE Clause +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +### Description + +`CASE` clause uses a rule to return a specific result based on the specified condition, similar to if/else statements in other programming languages. + +### Syntax + +```sql +CASE [ expression ] { WHEN boolean_expression THEN then_expression } [ ... ] + [ ELSE else_expression ] +END +``` + +### Parameters + +* **boolean_expression** + + Specifies any expression that evaluates to a result type `boolean`. Two or + more expressions may be combined together using the logical + operators ( `AND`, `OR` ). + +* **then_expression** + + Specifies the then expression based on the `boolean_expression` condition; `then_expression` and `else_expression` should all be same type or coercible to a common type. + +* **else_expression** + + Specifies the default expression; `then_expression` and `else_expression` should all be same type or coercible to a common type. + +### Examples + +```sql +CREATE TABLE person (id INT, name STRING, age INT); +INSERT INTO person VALUES + (100, 'John', 30), + (200, 'Mary', NULL), + (300, 'Mike', 80), + (400, 'Dan', 50); + +SELECT id, CASE WHEN id > 200 THEN 'bigger' ELSE 'small' END FROM person; ++------+--------------------------------------------------+ +| id | CASE WHEN (id > 200) THEN bigger ELSE small END | ++------+--------------------------------------------------+ +| 100 | small | +| 200 | small | +| 300 | bigger | +| 400 | bigger | ++------+--------------------------------------------------+ + +SELECT id, CASE id WHEN 100 then 'bigger' WHEN id > 300 THEN '300' ELSE 'small' END FROM person; ++------+-----------------------------------------------------------------------------------------------+ +| id | CASE WHEN (id = 100) THEN bigger WHEN (id = CAST((id > 300) AS INT)) THEN 300 ELSE small END | ++------+-----------------------------------------------------------------------------------------------+ +| 100 | bigger | +| 200 | small | +| 300 | small | +| 400 | small | ++------+-----------------------------------------------------------------------------------------------+ + +SELECT * FROM person + WHERE + CASE 1 = 1 + WHEN 100 THEN 'big' + WHEN 200 THEN 'bigger' + WHEN 300 THEN 'biggest' + ELSE 'small' + END = 'small'; ++------+-------+-------+ +| id | name | age | ++------+-------+-------+ +| 100 | John | 30 | +| 200 | Mary | NULL | +| 300 | Mike | 80 | +| 400 | Dan | 50 | ++------+-------+-------+ +``` + +### Related Statements + +* [SELECT Main](sql-ref-syntax-qry-select.html) +* [WHERE Clause](sql-ref-syntax-qry-select-where.html) +* [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +* [HAVING Clause](sql-ref-syntax-qry-select-having.html) +* [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +* [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +* [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-clusterby.md b/docs/sql-ref-syntax-qry-select-clusterby.md index e3bd2ed926ecc..9bcfac5b3b5c6 100644 --- a/docs/sql-ref-syntax-qry-select-clusterby.md +++ b/docs/sql-ref-syntax-qry-select-clusterby.md @@ -99,3 +99,6 @@ SELECT age, name FROM person CLUSTER BY age; * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-distribute-by.md b/docs/sql-ref-syntax-qry-select-distribute-by.md index 1fdfb91dad286..fbf662d3359f4 100644 --- a/docs/sql-ref-syntax-qry-select-distribute-by.md +++ b/docs/sql-ref-syntax-qry-select-distribute-by.md @@ -94,3 +94,6 @@ SELECT age, name FROM person DISTRIBUTE BY age; * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select-groupby.md b/docs/sql-ref-syntax-qry-select-groupby.md index bd9377ef78df6..6137c0d80f313 100644 --- a/docs/sql-ref-syntax-qry-select-groupby.md +++ b/docs/sql-ref-syntax-qry-select-groupby.md @@ -260,6 +260,30 @@ SELECT city, car_model, sum(quantity) AS sum FROM dealer | San Jose| HondaAccord| 8| | San Jose| HondaCivic| 5| +---------+------------+---+ + +--Prepare data for ignore nulls example +CREATE TABLE person (id INT, name STRING, age INT); +INSERT INTO person VALUES + (100, 'Mary', NULL), + (200, 'John', 30), + (300, 'Mike', 80), + (400, 'Dan', 50); + +--Select the first row in cloumn age +SELECT FIRST(age) FROM person; ++--------------------+ +| first(age, false) | ++--------------------+ +| NULL | ++--------------------+ + +--Get the first row in cloumn `age` ignore nulls,last row in column `id` and sum of cloumn `id`. +SELECT FIRST(age IGNORE NULLS), LAST(id), SUM(id) FROM person; ++-------------------+------------------+----------+ +| first(age, true) | last(id, false) | sum(id) | ++-------------------+------------------+----------+ +| 30 | 400 | 1000 | ++-------------------+------------------+----------+ ``` ### Related Statements @@ -272,3 +296,6 @@ SELECT city, car_model, sum(quantity) AS sum FROM dealer * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-having.md b/docs/sql-ref-syntax-qry-select-having.md index 935782c551e1f..59a8c680945e6 100644 --- a/docs/sql-ref-syntax-qry-select-having.md +++ b/docs/sql-ref-syntax-qry-select-having.md @@ -125,3 +125,6 @@ SELECT sum(quantity) AS sum FROM dealer HAVING sum(quantity) > 10; * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-lateral-view.md b/docs/sql-ref-syntax-qry-select-lateral-view.md new file mode 100644 index 0000000000000..f742c8fa57043 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-lateral-view.md @@ -0,0 +1,125 @@ +--- +layout: global +title: LATERAL VIEW Clause +displayTitle: LATERAL VIEW Clause +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +### Description + +The `LATERAL VIEW` clause is used in conjunction with generator functions such as `EXPLODE`, which will generate a virtual table containing one or more rows. `LATERAL VIEW` will apply the rows to each original output row. + +### Syntax + +```sql +LATERAL VIEW [ OUTER ] generator_function ( expression [ , ... ] ) [ table_alias ] AS column_alias [ , ... ] +``` + +### Parameters + +* **OUTER** + + If `OUTER` specified, returns null if an input array/map is empty or null. + +* **generator_function** + + Specifies a generator function (EXPLODE, INLINE, etc.). + +* **table_alias** + + The alias for `generator_function`, which is optional. + +* **column_alias** + + Lists the column aliases of `generator_function`, which may be used in output rows. We may have multiple aliases if `generator_function` have multiple output columns. + +### Examples + +```sql +CREATE TABLE person (id INT, name STRING, age INT, class INT, address STRING); +INSERT INTO person VALUES + (100, 'John', 30, 1, 'Street 1'), + (200, 'Mary', NULL, 1, 'Street 2'), + (300, 'Mike', 80, 3, 'Street 3'), + (400, 'Dan', 50, 4, 'Street 4'); + +SELECT * FROM person + LATERAL VIEW EXPLODE(ARRAY(30, 60)) tabelName AS c_age + LATERAL VIEW EXPLODE(ARRAY(40, 80)) AS d_age; ++------+-------+-------+--------+-----------+--------+--------+ +| id | name | age | class | address | c_age | d_age | ++------+-------+-------+--------+-----------+--------+--------+ +| 100 | John | 30 | 1 | Street 1 | 30 | 40 | +| 100 | John | 30 | 1 | Street 1 | 30 | 80 | +| 100 | John | 30 | 1 | Street 1 | 60 | 40 | +| 100 | John | 30 | 1 | Street 1 | 60 | 80 | +| 200 | Mary | NULL | 1 | Street 2 | 30 | 40 | +| 200 | Mary | NULL | 1 | Street 2 | 30 | 80 | +| 200 | Mary | NULL | 1 | Street 2 | 60 | 40 | +| 200 | Mary | NULL | 1 | Street 2 | 60 | 80 | +| 300 | Mike | 80 | 3 | Street 3 | 30 | 40 | +| 300 | Mike | 80 | 3 | Street 3 | 30 | 80 | +| 300 | Mike | 80 | 3 | Street 3 | 60 | 40 | +| 300 | Mike | 80 | 3 | Street 3 | 60 | 80 | +| 400 | Dan | 50 | 4 | Street 4 | 30 | 40 | +| 400 | Dan | 50 | 4 | Street 4 | 30 | 80 | +| 400 | Dan | 50 | 4 | Street 4 | 60 | 40 | +| 400 | Dan | 50 | 4 | Street 4 | 60 | 80 | ++------+-------+-------+--------+-----------+--------+--------+ + +SELECT c_age, COUNT(1) FROM person + LATERAL VIEW EXPLODE(ARRAY(30, 60)) AS c_age + LATERAL VIEW EXPLODE(ARRAY(40, 80)) AS d_age +GROUP BY c_age; ++--------+-----------+ +| c_age | count(1) | ++--------+-----------+ +| 60 | 8 | +| 30 | 8 | ++--------+-----------+ + +SELECT * FROM person + LATERAL VIEW EXPLODE(ARRAY()) tabelName AS c_age; ++-----+-------+------+--------+----------+--------+ +| id | name | age | class | address | c_age | ++-----+-------+------+--------+----------+--------+ ++-----+-------+------+--------+----------+--------+ + +SELECT * FROM person + LATERAL VIEW OUTER EXPLODE(ARRAY()) tabelName AS c_age; ++------+-------+-------+--------+-----------+--------+ +| id | name | age | class | address | c_age | ++------+-------+-------+--------+-----------+--------+ +| 100 | John | 30 | 1 | Street 1 | NULL | +| 200 | Mary | NULL | 1 | Street 2 | NULL | +| 300 | Mike | 80 | 3 | Street 3 | NULL | +| 400 | Dan | 50 | 4 | Street 4 | NULL | ++------+-------+-------+--------+-----------+--------+ +``` + +### Related Statements + +* [SELECT Main](sql-ref-syntax-qry-select.html) +* [WHERE Clause](sql-ref-syntax-qry-select-where.html) +* [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +* [HAVING Clause](sql-ref-syntax-qry-select-having.html) +* [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +* [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +* [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) diff --git a/docs/sql-ref-syntax-qry-select-limit.md b/docs/sql-ref-syntax-qry-select-limit.md index 03c4df3cbc442..bd64ba890901b 100644 --- a/docs/sql-ref-syntax-qry-select-limit.md +++ b/docs/sql-ref-syntax-qry-select-limit.md @@ -104,3 +104,6 @@ org.apache.spark.sql.AnalysisException: The limit expression must evaluate to a * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-orderby.md b/docs/sql-ref-syntax-qry-select-orderby.md index 85bbe514cdc95..13f0ae40cb828 100644 --- a/docs/sql-ref-syntax-qry-select-orderby.md +++ b/docs/sql-ref-syntax-qry-select-orderby.md @@ -143,3 +143,6 @@ SELECT * FROM person ORDER BY name ASC, age DESC; * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-pivot.md b/docs/sql-ref-syntax-qry-select-pivot.md new file mode 100644 index 0000000000000..649c2518b28eb --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-pivot.md @@ -0,0 +1,101 @@ +--- +layout: global +title: PIVOT Clause +displayTitle: PIVOT Clause +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +### Description + +The `PIVOT` clause is used for data perspective. We can get the aggregated values based on specific column values, which will be turned to multiple columns used in `SELECT` clause. The `PIVOT` clause can be specified after the table name or subquery. + +### Syntax + +```sql +PIVOT ( { aggregate_expression [ AS aggregate_expression_alias ] } [ , ... ] + FOR column_list IN ( expression_list ) ) +``` + +### Parameters + +* **aggregate_expression** + + Specifies an aggregate expression (SUM(a), COUNT(DISTINCT b), etc.). + +* **aggregate_expression_alias** + + Specifies an alias for the aggregate expression. + +* **column_list** + + Contains columns in the `FROM` clause, which specifies the columns we want to replace with new columns. We can use brackets to surround the columns, such as `(c1, c2)`. + +* **expression_list** + + Specifies new columns, which are used to match values in `column_list` as the aggregating condition. We can also add aliases for them. + +### Examples + +```sql +CREATE TABLE person (id INT, name STRING, age INT, class INT, address STRING); +INSERT INTO person VALUES + (100, 'John', 30, 1, 'Street 1'), + (200, 'Mary', NULL, 1, 'Street 2'), + (300, 'Mike', 80, 3, 'Street 3'), + (400, 'Dan', 50, 4, 'Street 4'); + +SELECT * FROM person + PIVOT ( + SUM(age) AS a, AVG(class) AS c + FOR name IN ('John' AS john, 'Mike' AS mike) + ); ++------+-----------+---------+---------+---------+---------+ +| id | address | john_a | john_c | mike_a | mike_c | ++------+-----------+---------+---------+---------+---------+ +| 200 | Street 2 | NULL | NULL | NULL | NULL | +| 100 | Street 1 | 30 | 1.0 | NULL | NULL | +| 300 | Street 3 | NULL | NULL | 80 | 3.0 | +| 400 | Street 4 | NULL | NULL | NULL | NULL | ++------+-----------+---------+---------+---------+---------+ + +SELECT * FROM person + PIVOT ( + SUM(age) AS a, AVG(class) AS c + FOR (name, age) IN (('John', 30) AS c1, ('Mike', 40) AS c2) + ); ++------+-----------+-------+-------+-------+-------+ +| id | address | c1_a | c1_c | c2_a | c2_c | ++------+-----------+-------+-------+-------+-------+ +| 200 | Street 2 | NULL | NULL | NULL | NULL | +| 100 | Street 1 | 30 | 1.0 | NULL | NULL | +| 300 | Street 3 | NULL | NULL | NULL | NULL | +| 400 | Street 4 | NULL | NULL | NULL | NULL | ++------+-----------+-------+-------+-------+-------+ +``` + +### Related Statements + +* [SELECT Main](sql-ref-syntax-qry-select.html) +* [WHERE Clause](sql-ref-syntax-qry-select-where.html) +* [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +* [HAVING Clause](sql-ref-syntax-qry-select-having.html) +* [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +* [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +* [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-sortby.md b/docs/sql-ref-syntax-qry-select-sortby.md index 554bdb569d005..09e559adcd6ab 100644 --- a/docs/sql-ref-syntax-qry-select-sortby.md +++ b/docs/sql-ref-syntax-qry-select-sortby.md @@ -176,3 +176,6 @@ SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-where.md b/docs/sql-ref-syntax-qry-select-where.md index ca3f5ec7866c6..9ff7993d40c58 100644 --- a/docs/sql-ref-syntax-qry-select-where.md +++ b/docs/sql-ref-syntax-qry-select-where.md @@ -125,3 +125,6 @@ SELECT * FROM person AS parent * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md index 987e6479ab20a..453737a2f36b8 100644 --- a/docs/sql-ref-syntax-qry-select.md +++ b/docs/sql-ref-syntax-qry-select.md @@ -31,10 +31,10 @@ of a query along with examples. ```sql [ WITH with_query [ , ... ] ] select_statement [ { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select_statement, ... ] - [ ORDER BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] - [ SORT BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] - [ CLUSTER BY { expression [ , ...] } ] - [ DISTRIBUTE BY { expression [, ...] } ] + [ ORDER BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ... ] } ] + [ SORT BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ... ] } ] + [ CLUSTER BY { expression [ , ... ] } ] + [ DISTRIBUTE BY { expression [, ... ] } ] [ WINDOW { named_window [ , WINDOW named_window, ... ] } ] [ LIMIT { ALL | expression } ] ``` @@ -42,9 +42,11 @@ select_statement [ { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select_stat While `select_statement` is defined as ```sql SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } - FROM { from_item [ , ...] } + FROM { from_item [ , ... ] } + [ PIVOT clause ] + [ LATERAL VIEW clause ] [ ... ] [ WHERE boolean_expression ] - [ GROUP BY expression [ , ...] ] + [ GROUP BY expression [ , ... ] ] [ HAVING boolean_expression ] ``` @@ -75,7 +77,7 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } **Syntax:** `expression [AS] [alias]` - * **from_item** +* **from_item** Specifies a source of input for the query. It can be one of the following: * Table relation @@ -83,62 +85,71 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } * [Table-value function](sql-ref-syntax-qry-select-tvf.html) * [Inline table](sql-ref-syntax-qry-select-inline-table.html) * Subquery + +* **PIVOT** + The `PIVOT` clause is used for data perspective; We can get the aggregated values based on specific column value. - * **WHERE** +* **LATERAL VIEW** + + The `LATERAL VIEW` clause is used in conjunction with generator functions such as `EXPLODE`, which will generate a virtual table containing one or more rows. `LATERAL VIEW` will apply the rows to each original output row. + +* **WHERE** Filters the result of the FROM clause based on the supplied predicates. - * **GROUP BY** +* **GROUP BY** Specifies the expressions that are used to group the rows. This is used in conjunction with aggregate functions (MIN, MAX, COUNT, SUM, AVG, etc.) to group rows based on the grouping expressions and aggregate values in each group. When a FILTER clause is attached to an aggregate function, only the matching rows are passed to that function. - * **HAVING** +* **HAVING** Specifies the predicates by which the rows produced by GROUP BY are filtered. The HAVING clause is used to filter rows after the grouping is performed. If HAVING is specified without GROUP BY, it indicates a GROUP BY without grouping expressions (global aggregate). - * **ORDER BY** +* **ORDER BY** Specifies an ordering of the rows of the complete result set of the query. The output rows are ordered across the partitions. This parameter is mutually exclusive with `SORT BY`, `CLUSTER BY` and `DISTRIBUTE BY` and can not be specified together. - * **SORT BY** +* **SORT BY** Specifies an ordering by which the rows are ordered within each partition. This parameter is mutually exclusive with `ORDER BY` and `CLUSTER BY` and can not be specified together. - * **CLUSTER BY** +* **CLUSTER BY** Specifies a set of expressions that is used to repartition and sort the rows. Using this clause has the same effect of using `DISTRIBUTE BY` and `SORT BY` together. - * **DISTRIBUTE BY** +* **DISTRIBUTE BY** Specifies a set of expressions by which the result rows are repartitioned. This parameter is mutually exclusive with `ORDER BY` and `CLUSTER BY` and can not be specified together. - * **LIMIT** +* **LIMIT** Specifies the maximum number of rows that can be returned by a statement or subquery. This clause is mostly used in the conjunction with `ORDER BY` to produce a deterministic result. - * **boolean_expression** +* **boolean_expression** - Specifies an expression with a return type of boolean. + Specifies any expression that evaluates to a result type `boolean`. Two or + more expressions may be combined together using the logical + operators ( `AND`, `OR` ). - * **expression** +* **expression** Specifies a combination of one or more values, operators, and SQL functions that evaluates to a value. - * **named_window** +* **named_window** - Specifies aliases for one or more source window specifications. The source window specifications can - be referenced in the widow definitions in the query. + Specifies aliases for one or more source window specifications. The source window specifications can + be referenced in the widow definitions in the query. ### Related Statements @@ -159,3 +170,6 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } * [TABLESAMPLE](sql-ref-syntax-qry-select-sampling.html) * [Table-valued Function](sql-ref-syntax-qry-select-tvf.html) * [Window Function](sql-ref-syntax-qry-select-window.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry.md b/docs/sql-ref-syntax-qry.md index 167c394d0fe49..d55ea43d15036 100644 --- a/docs/sql-ref-syntax-qry.md +++ b/docs/sql-ref-syntax-qry.md @@ -45,4 +45,7 @@ ability to generate logical and physical plan for a given query using * [TABLESAMPLE](sql-ref-syntax-qry-select-sampling.html) * [Table-valued Function](sql-ref-syntax-qry-select-tvf.html) * [Window Function](sql-ref-syntax-qry-select-window.html) + * [CASE Clause](sql-ref-syntax-qry-select-case.html) + * [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) + * [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) * [EXPLAIN Statement](sql-ref-syntax-qry-explain.html) diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index 290523a2b1ee1..4e58abb2a8596 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -66,6 +66,9 @@ Spark SQL is Apache Spark's module for working with structured data. The SQL Syn * [Table-valued Function](sql-ref-syntax-qry-select-tvf.html) * [WHERE Clause](sql-ref-syntax-qry-select-where.html) * [Window Function](sql-ref-syntax-qry-select-window.html) + * [CASE Clause](sql-ref-syntax-qry-select-case.html) + * [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) + * [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) * [EXPLAIN](sql-ref-syntax-qry-explain.html) ### Auxiliary Statements From 8323c8eb566cdb49ccaf19537897cc6dcf74d174 Mon Sep 17 00:00:00 2001 From: Frank Yin Date: Tue, 28 Jul 2020 10:00:21 +0900 Subject: [PATCH 302/384] [SPARK-32059][SQL] Allow nested schema pruning thru window/sort plans ### What changes were proposed in this pull request? This PR is intended to solve schema pruning not working with window functions, as described in SPARK-32059. It also solved schema pruning not working with `Sort`. It also generalizes with `Project->Filter->[any node can be pruned]`. ### Why are the changes needed? This is needed because of performance issues with nested structures with querying using window functions as well as sorting. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Introduced two tests: 1) optimizer planning level 2) end-to-end tests with SQL queries. Closes #28898 from frankyin-factual/master. Authored-by: Frank Yin Signed-off-by: Takeshi Yamamuro --- .../optimizer/NestedColumnAliasing.scala | 16 ++ .../optimizer/NestedColumnAliasingSuite.scala | 140 +++++++++++++++++- .../datasources/SchemaPruningSuite.scala | 63 ++++++++ 3 files changed, 217 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index 0c8666b72cace..b053bf6d61e6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -31,6 +31,20 @@ import org.apache.spark.sql.types._ object NestedColumnAliasing { def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { + /** + * This pattern is needed to support [[Filter]] plan cases like + * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]). + * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that + * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule. + */ + case Project(projectList, Filter(condition, child)) + if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) => + val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions + getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map { + case (nestedFieldToAlias, attrToAliases) => + NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases) + } + case Project(projectList, child) if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) => val exprCandidatesToPrune = projectList ++ child.expressions @@ -113,6 +127,8 @@ object NestedColumnAliasing { case _: Sample => true case _: RepartitionByExpression => true case _: Join => true + case _: Window => true + case _: Sort => true case _ => false } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index 0425846637f30..c83ab375ee15a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -145,9 +145,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val ops = Seq( (input: LogicalPlan) => input.distribute('name)(1), (input: LogicalPlan) => input.orderBy('name.asc), - (input: LogicalPlan) => input.orderBy($"name.middle".asc), (input: LogicalPlan) => input.sortBy('name.asc), - (input: LogicalPlan) => input.sortBy($"name.middle".asc), (input: LogicalPlan) => input.union(input) ) @@ -493,6 +491,144 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { comparePlans(optimized3, expected3) } + test("Nested field pruning for Window") { + val spec = windowSpec($"address" :: Nil, $"id".asc :: Nil, UnspecifiedFrame) + val winExpr = windowExpr(RowNumber(), spec) + val query = contact + .select($"name.first", winExpr.as('window)) + .orderBy($"name.last".asc) + .analyze + val optimized = Optimize.execute(query) + val aliases = collectGeneratedAliases(optimized) + val expected = contact + .select($"name.first", $"address", $"id", $"name.last".as(aliases(1))) + .window(Seq(winExpr.as("window")), Seq($"address"), Seq($"id".asc)) + .select($"first", $"window", $"${aliases(1)}".as(aliases(0))) + .orderBy($"${aliases(0)}".asc) + .select($"first", $"window") + .analyze + comparePlans(optimized, expected) + } + + test("Nested field pruning for Filter with other supported operators") { + val spec = windowSpec($"address" :: Nil, $"id".asc :: Nil, UnspecifiedFrame) + val winExpr = windowExpr(RowNumber(), spec) + val query1 = contact.select($"name.first", winExpr.as('window)) + .where($"window" === 1 && $"name.first" === "a") + .analyze + val optimized1 = Optimize.execute(query1) + val aliases1 = collectGeneratedAliases(optimized1) + val expected1 = contact + .select($"name.first", $"address", $"id", $"name.first".as(aliases1(1))) + .window(Seq(winExpr.as("window")), Seq($"address"), Seq($"id".asc)) + .select($"first", $"${aliases1(1)}".as(aliases1(0)), $"window") + .where($"window" === 1 && $"${aliases1(0)}" === "a") + .select($"first", $"window") + .analyze + comparePlans(optimized1, expected1) + + val query2 = contact.sortBy($"name.first".asc) + .where($"name.first" === "a") + .select($"name.first") + .analyze + val optimized2 = Optimize.execute(query2) + val aliases2 = collectGeneratedAliases(optimized2) + val expected2 = contact + .select($"name.first".as(aliases2(1))) + .sortBy($"${aliases2(1)}".asc) + .select($"${aliases2(1)}".as(aliases2(0))) + .where($"${aliases2(0)}" === "a") + .select($"${aliases2(0)}".as("first")) + .analyze + comparePlans(optimized2, expected2) + + val query3 = contact.distribute($"name.first")(100) + .where($"name.first" === "a") + .select($"name.first") + .analyze + val optimized3 = Optimize.execute(query3) + val aliases3 = collectGeneratedAliases(optimized3) + val expected3 = contact + .select($"name.first".as(aliases3(1))) + .distribute($"${aliases3(1)}")(100) + .select($"${aliases3(1)}".as(aliases3(0))) + .where($"${aliases3(0)}" === "a") + .select($"${aliases3(0)}".as("first")) + .analyze + comparePlans(optimized3, expected3) + + val department = LocalRelation( + 'depID.int, + 'personID.string) + val query4 = contact.join(department, condition = Some($"id" === $"depID")) + .where($"name.first" === "a") + .select($"name.first") + .analyze + val optimized4 = Optimize.execute(query4) + val aliases4 = collectGeneratedAliases(optimized4) + val expected4 = contact + .select($"id", $"name.first".as(aliases4(1))) + .join(department.select('depID), condition = Some($"id" === $"depID")) + .select($"${aliases4(1)}".as(aliases4(0))) + .where($"${aliases4(0)}" === "a") + .select($"${aliases4(0)}".as("first")) + .analyze + comparePlans(optimized4, expected4) + + def runTest(basePlan: LogicalPlan => LogicalPlan): Unit = { + val query = basePlan(contact) + .where($"name.first" === "a") + .select($"name.first") + .analyze + val optimized = Optimize.execute(query) + val aliases = collectGeneratedAliases(optimized) + val expected = basePlan(contact + .select($"name.first".as(aliases(0)))) + .where($"${aliases(0)}" === "a") + .select($"${aliases(0)}".as("first")) + .analyze + comparePlans(optimized, expected) + } + Seq( + (plan: LogicalPlan) => plan.limit(100), + (plan: LogicalPlan) => plan.repartition(100), + (plan: LogicalPlan) => Sample(0.0, 0.6, false, 11L, plan)).foreach { base => + runTest(base) + } + } + + test("Nested field pruning for Sort") { + val query1 = contact.select($"name.first", $"name.last") + .sortBy($"name.first".asc, $"name.last".asc) + .analyze + val optimized1 = Optimize.execute(query1) + val aliases1 = collectGeneratedAliases(optimized1) + val expected1 = contact + .select($"name.first", + $"name.last", + $"name.first".as(aliases1(0)), + $"name.last".as(aliases1(1))) + .sortBy($"${aliases1(0)}".asc, $"${aliases1(1)}".asc) + .select($"first", $"last") + .analyze + comparePlans(optimized1, expected1) + + val query2 = contact.select($"name.first", $"name.last") + .orderBy($"name.first".asc, $"name.last".asc) + .analyze + val optimized2 = Optimize.execute(query2) + val aliases2 = collectGeneratedAliases(optimized2) + val expected2 = contact + .select($"name.first", + $"name.last", + $"name.first".as(aliases2(0)), + $"name.last".as(aliases2(1))) + .orderBy($"${aliases2(0)}".asc, $"${aliases2(1)}".asc) + .select($"first", $"last") + .analyze + comparePlans(optimized2, expected2) + } + test("Nested field pruning for Expand") { def runTest(basePlan: LogicalPlan => LogicalPlan): Unit = { val query1 = Expand( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index d51eafa5a8aed..2b5cb27d59ad9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -460,6 +460,69 @@ abstract class SchemaPruningSuite checkAnswer(query4, Row(2, null) :: Row(2, 4) :: Nil) } + testSchemaPruning("select nested field in window function") { + val windowSql = + """ + |with contact_rank as ( + | select row_number() over (partition by address order by id desc) as rank, + | contacts.* + | from contacts + |) + |select name.first, rank from contact_rank + |where name.first = 'Jane' AND rank = 1 + |""".stripMargin + val query = sql(windowSql) + checkScan(query, "struct,address:string>") + checkAnswer(query, Row("Jane", 1) :: Nil) + } + + testSchemaPruning("select nested field in window function and then order by") { + val windowSql = + """ + |with contact_rank as ( + | select row_number() over (partition by address order by id desc) as rank, + | contacts.* + | from contacts + | order by name.last, name.first + |) + |select name.first, rank from contact_rank + |""".stripMargin + val query = sql(windowSql) + checkScan(query, "struct,address:string>") + checkAnswer(query, + Row("Jane", 1) :: + Row("John", 1) :: + Row("Janet", 1) :: + Row("Jim", 1) :: Nil) + } + + testSchemaPruning("select nested field in Sort") { + val query1 = sql("select name.first, name.last from contacts order by name.first, name.last") + checkScan(query1, "struct>") + checkAnswer(query1, + Row("Jane", "Doe") :: + Row("Janet", "Jones") :: + Row("Jim", "Jones") :: + Row("John", "Doe") :: Nil) + + withTempView("tmp_contacts") { + // Create a repartitioned view because `SORT BY` is a local sort + sql("select * from contacts").repartition(1).createOrReplaceTempView("tmp_contacts") + val sortBySql = + """ + |select name.first, name.last from tmp_contacts + |sort by name.first, name.last + |""".stripMargin + val query2 = sql(sortBySql) + checkScan(query2, "struct>") + checkAnswer(query2, + Row("Jane", "Doe") :: + Row("Janet", "Jones") :: + Row("Jim", "Jones") :: + Row("John", "Doe") :: Nil) + } + } + testSchemaPruning("select nested field in Expand") { import org.apache.spark.sql.catalyst.dsl.expressions._ From 77f2ca6cced1c723d1c2e6082a1534f6436c6d2a Mon Sep 17 00:00:00 2001 From: Shantanu <12621235+hauntsaninja@users.noreply.github.com> Date: Tue, 28 Jul 2020 11:22:18 +0900 Subject: [PATCH 303/384] [MINOR][PYTHON] Fix spacing in error message ### What changes were proposed in this pull request? Fixes spacing in an error message ### Why are the changes needed? Makes error messages easier to read ### Does this PR introduce _any_ user-facing change? Yes, it changes the error message ### How was this patch tested? This patch doesn't affect any logic, so existing tests should cover it Closes #29264 from hauntsaninja/patch-1. Authored-by: Shantanu <12621235+hauntsaninja@users.noreply.github.com> Signed-off-by: HyukjinKwon --- python/pyspark/worker.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 9b54affb137f5..84e5cca5d3c00 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -468,7 +468,7 @@ def main(infile, outfile): version = utf8_deserializer.loads(infile) if version != "%d.%d" % sys.version_info[:2]: raise Exception(("Python in worker has different version %s than that in " + - "driver %s, PySpark cannot run with different minor versions." + + "driver %s, PySpark cannot run with different minor versions. " + "Please check environment variables PYSPARK_PYTHON and " + "PYSPARK_DRIVER_PYTHON are correctly set.") % ("%d.%d" % sys.version_info[:2], version)) From 44a5258ac28f843dc7385868b9a86db77eeff266 Mon Sep 17 00:00:00 2001 From: Tianshi Zhu Date: Tue, 28 Jul 2020 12:32:19 +0900 Subject: [PATCH 304/384] [SPARK-31525][SQL] Return an empty list for df.head() when df is empty ### What changes were proposed in this pull request? return an empty list instead of None when calling `df.head()` ### Why are the changes needed? `df.head()` and `df.head(1)` are inconsistent when df is empty. ### Does this PR introduce _any_ user-facing change? Yes. If a user relies on `df.head()` to return None, things like `if df.head() is None:` will be broken. ### How was this patch tested? Closes #29214 from tianshizz/SPARK-31525. Authored-by: Tianshi Zhu Signed-off-by: HyukjinKwon --- docs/pyspark-migration-guide.md | 3 +++ python/pyspark/sql/dataframe.py | 5 +++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md index 2c9ea410f217e..efc2175a0459d 100644 --- a/docs/pyspark-migration-guide.md +++ b/docs/pyspark-migration-guide.md @@ -26,6 +26,9 @@ Note that this migration guide describes the items specific to PySpark. Many items of SQL migration can be applied when migrating PySpark to higher versions. Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). +## Upgrading from PySpark 3.0 to 3.1 +- In Spark 3.1, PySpark `DataFrame.head()` will return `[]` if the PySpark DataFrame is empty. In Spark 3.0 or prior, it will return `None`. The bahavior remains the same for non-empty PySpark DataFrame. + ## Upgrading from PySpark 2.4 to 3.0 - In Spark 3.0, PySpark requires a pandas version of 0.23.2 or higher to use pandas related functionality, such as `toPandas`, `createDataFrame` from pandas DataFrame, and so on. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 1027918adbe15..f216f0b4dcd25 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1323,7 +1323,8 @@ def head(self, n=None): :param n: int, default 1. Number of rows to return. :return: If n is greater than 1, return a list of :class:`Row`. - If n is 1, return a single Row. + If n is 1, return a single Row if it exists. Otherwise, we will return an + empty list to match the behavior of `head(1)` when the dataframe is empty. >>> df.head() Row(age=2, name='Alice') @@ -1332,7 +1333,7 @@ def head(self, n=None): """ if n is None: rs = self.head(1) - return rs[0] if rs else None + return rs[0] if rs else [] return self.take(n) @since(1.3) From 12b9787a7f07718eaef1215331034a17302ae93d Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Tue, 28 Jul 2020 04:42:15 +0000 Subject: [PATCH 305/384] [SPARK-32290][SQL] SingleColumn Null Aware Anti Join Optimize ### What changes were proposed in this pull request? Normally, a Null aware anti join will be planed into BroadcastNestedLoopJoin which is very time consuming, for instance, in TPCH Query 16. ``` select p_brand, p_type, p_size, count(distinct ps_suppkey) as supplier_cnt from partsupp, part where p_partkey = ps_partkey and p_brand <> 'Brand#45' and p_type not like 'MEDIUM POLISHED%' and p_size in (49, 14, 23, 45, 19, 3, 36, 9) and ps_suppkey not in ( select s_suppkey from supplier where s_comment like '%Customer%Complaints%' ) group by p_brand, p_type, p_size order by supplier_cnt desc, p_brand, p_type, p_size ``` In above query, will planed into LeftAnti condition Or((ps_suppkey=s_suppkey), IsNull(ps_suppkey=s_suppkey)) Inside BroadcastNestedLoopJoinExec will perform O(M\*N), BUT if there is only single column in NAAJ, we can always change buildSide into a HashSet, and streamedSide just need to lookup in the HashSet, then the calculation will be optimized into O(M). But this optimize is only targeting on null aware anti join with single column case, because multi-column support is much more complicated, we might be able to support multi-column in future. After apply this patch, the TPCH Query 16 performance decrease from 41mins to 30s The semantic of null-aware anti join is: ![image](https://user-images.githubusercontent.com/17242071/88077041-66a39a00-cbad-11ea-8fb6-c235c4d219b4.png) ### Why are the changes needed? TPCH is a common benchmark for distributed compute engine, all other 21 Query works fine on Spark, except for Query 16, apply this patch will make Spark more competitive among all these popular engine. BTW, this patch has restricted rules and only apply on NAAJ Single Column case, which is safe enough. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? 1. SQLQueryTestSuite with NOT IN keyword SQL, add CONFIG_DIM with spark.sql.optimizeNullAwareAntiJoin on and off 2. added case in org.apache.spark.sql.JoinSuite. 3. added case in org.apache.spark.sql.SubquerySuite. 3. Compare performance before and after applying this patch against TPCH Query 16. 4. config combination against e2e test with following ``` Map( "spark.sql.optimizeNullAwareAntiJoin" -> "true", "spark.sql.adaptive.enabled" -> "false", "spark.sql.codegen.wholeStage" -> "false" ), Map( "sspark.sql.optimizeNullAwareAntiJoin" -> "true", "spark.sql.adaptive.enabled" -> "false", "spark.sql.codegen.wholeStage" -> "true" ), Map( "spark.sql.optimizeNullAwareAntiJoin" -> "true", "spark.sql.adaptive.enabled" -> "true", "spark.sql.codegen.wholeStage" -> "false" ), Map( "spark.sql.optimizeNullAwareAntiJoin" -> "true", "spark.sql.adaptive.enabled" -> "true", "spark.sql.codegen.wholeStage" -> "true" ) ``` Closes #29104 from leanken/leanken-SPARK-32290. Authored-by: xuewei.linxuewei Signed-off-by: Wenchen Fan --- .../sql/catalyst/planning/patterns.scala | 36 ++++++ .../apache/spark/sql/internal/SQLConf.scala | 14 +++ .../spark/sql/execution/SparkStrategies.scala | 4 + .../adaptive/LogicalQueryStageStrategy.scala | 10 +- .../PlanDynamicPruningFilters.scala | 4 +- .../joins/BroadcastHashJoinExec.scala | 82 +++++++++++++- .../sql/execution/joins/HashedRelation.scala | 71 ++++++++++-- .../sql-tests/inputs/group-by-filter.sql | 3 + .../inputs/subquery/in-subquery/in-basic.sql | 3 + .../inputs/subquery/in-subquery/in-having.sql | 3 + .../inputs/subquery/in-subquery/in-joins.sql | 3 + .../inputs/subquery/in-subquery/in-limit.sql | 3 + .../in-subquery/in-multiple-columns.sql | 3 + .../subquery/in-subquery/in-order-by.sql | 3 + .../subquery/in-subquery/in-with-cte.sql | 3 + .../subquery/in-subquery/nested-not-in.sql | 3 + .../subquery/in-subquery/not-in-joins.sql | 3 + ...not-in-unit-tests-multi-column-literal.sql | 3 + .../not-in-unit-tests-multi-column.sql | 3 + ...ot-in-unit-tests-single-column-literal.sql | 3 + .../not-in-unit-tests-single-column.sql | 3 + .../inputs/subquery/in-subquery/simple-in.sql | 3 + .../sql-tests/results/explain-aqe.sql.out | 8 +- .../sql-tests/results/explain.sql.out | 8 +- .../sql/DynamicPartitionPruningSuite.scala | 2 +- .../org/apache/spark/sql/JoinSuite.scala | 41 +++++++ .../org/apache/spark/sql/SubquerySuite.scala | 107 ++++++++++++++++++ .../execution/WholeStageCodegenSuite.scala | 4 +- .../sql/execution/debug/DebuggingSuite.scala | 3 +- .../execution/joins/HashedRelationSuite.scala | 1 + 30 files changed, 409 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 415ce46788119..5a994f1ad0a39 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -21,8 +21,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.optimizer.JoinSelectionHelper import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf trait OperationHelper { type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan) @@ -388,3 +390,37 @@ object PhysicalWindow { case _ => None } } + +object ExtractSingleColumnNullAwareAntiJoin extends JoinSelectionHelper with PredicateHelper { + + // TODO support multi column NULL-aware anti join in future. + // See. http://www.vldb.org/pvldb/vol2/vldb09-423.pdf Section 6 + // multi-column null aware anti join is much more complicated than single column ones. + + // streamedSideKeys, buildSideKeys + private type ReturnType = (Seq[Expression], Seq[Expression]) + + /** + * See. [SPARK-32290] + * LeftAnti(condition: Or(EqualTo(a=b), IsNull(EqualTo(a=b))) + * will almost certainly be planned as a Broadcast Nested Loop join, + * which is very time consuming because it's an O(M*N) calculation. + * But if it's a single column case O(M*N) calculation could be optimized into O(M) + * using hash lookup instead of loop lookup. + */ + def unapply(join: Join): Option[ReturnType] = join match { + case Join(left, right, LeftAnti, + Some(Or(e @ EqualTo(leftAttr: AttributeReference, rightAttr: AttributeReference), + IsNull(e2 @ EqualTo(_, _)))), _) + if SQLConf.get.optimizeNullAwareAntiJoin && + e.semanticEquals(e2) => + if (canEvaluate(leftAttr, left) && canEvaluate(rightAttr, right)) { + Some(Seq(leftAttr), Seq(rightAttr)) + } else if (canEvaluate(leftAttr, right) && canEvaluate(rightAttr, left)) { + Some(Seq(rightAttr), Seq(leftAttr)) + } else { + None + } + case _ => None + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c68d7ccab4d10..f4bc328b24676 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2672,6 +2672,17 @@ object SQLConf { .checkValue(_ >= 0, "The value must be non-negative.") .createWithDefault(8) + val OPTIMIZE_NULL_AWARE_ANTI_JOIN = + buildConf("spark.sql.optimizeNullAwareAntiJoin") + .internal() + .doc("When true, NULL-aware anti join execution will be planed into " + + "BroadcastHashJoinExec with flag isNullAwareAntiJoin enabled, " + + "optimized from O(M*N) calculation into O(M) calculation " + + "using Hash lookup instead of Looping lookup." + + "Only support for singleColumn NAAJ for now.") + .booleanConf + .createWithDefault(true) + /** * Holds information about keys that have been deprecated. * @@ -3277,6 +3288,9 @@ class SQLConf extends Serializable with Logging { def coalesceBucketsInJoinMaxBucketRatio: Int = getConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO) + def optimizeNullAwareAntiJoin: Boolean = + getConf(SQLConf.OPTIMIZE_NULL_AWARE_ANTI_JOIN) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 78aa258387daa..4f9cb22ee0057 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -232,6 +232,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { .orElse { if (hintToShuffleReplicateNL(hint)) createCartesianProduct() else None } .getOrElse(createJoinWithoutHint()) + case j @ ExtractSingleColumnNullAwareAntiJoin(leftKeys, rightKeys) => + Seq(joins.BroadcastHashJoinExec(leftKeys, rightKeys, LeftAnti, BuildRight, + None, planLater(j.left), planLater(j.right), isNullAwareAntiJoin = true)) + // If it is not an equi-join, we first look at the join hints w.r.t. the following order: // 1. broadcast hint: pick broadcast nested loop join. If both sides have the broadcast // hints, choose the smaller side (based on stats) to broadcast for inner and full joins, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala index ac98342277bc0..bcf9dc1544ce3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala @@ -20,9 +20,10 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.Strategy import org.apache.spark.sql.catalyst.expressions.PredicateHelper import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} -import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys +import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, ExtractSingleColumnNullAwareAntiJoin} +import org.apache.spark.sql.catalyst.plans.LeftAnti import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan} -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{joins, SparkPlan} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} /** @@ -48,6 +49,11 @@ object LogicalQueryStageStrategy extends Strategy with PredicateHelper { Seq(BroadcastHashJoinExec( leftKeys, rightKeys, joinType, buildSide, condition, planLater(left), planLater(right))) + case j @ ExtractSingleColumnNullAwareAntiJoin(leftKeys, rightKeys) + if isBroadcastStage(j.right) => + Seq(joins.BroadcastHashJoinExec(leftKeys, rightKeys, LeftAnti, BuildRight, + None, planLater(j.left), planLater(j.right), isNullAwareAntiJoin = true)) + case j @ Join(left, right, joinType, condition, _) if isBroadcastStage(left) || isBroadcastStage(right) => val buildSide = if (isBroadcastStage(left)) BuildLeft else BuildRight diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala index cfc653a23840d..098576d72f540 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala @@ -59,9 +59,9 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession) // the first to be applied (apart from `InsertAdaptiveSparkPlan`). val canReuseExchange = SQLConf.get.exchangeReuseEnabled && buildKeys.nonEmpty && plan.find { - case BroadcastHashJoinExec(_, _, _, BuildLeft, _, left, _) => + case BroadcastHashJoinExec(_, _, _, BuildLeft, _, left, _, _) => left.sameResult(sparkPlan) - case BroadcastHashJoinExec(_, _, _, BuildRight, _, _, right) => + case BroadcastHashJoinExec(_, _, _, BuildRight, _, _, right, _) => right.sameResult(sparkPlan) case _ => false }.isDefined diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 71faad9829a42..2a283013aceef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -46,14 +46,23 @@ case class BroadcastHashJoinExec( buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) + right: SparkPlan, + isNullAwareAntiJoin: Boolean = false) extends HashJoin with CodegenSupport { + if (isNullAwareAntiJoin) { + require(leftKeys.length == 1, "leftKeys length should be 1") + require(rightKeys.length == 1, "rightKeys length should be 1") + require(joinType == LeftAnti, "joinType must be LeftAnti.") + require(buildSide == BuildRight, "buildSide must be BuildRight.") + require(condition.isEmpty, "null aware anti join optimize condition should be empty.") + } + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { - val mode = HashedRelationBroadcastMode(buildBoundKeys) + val mode = HashedRelationBroadcastMode(buildBoundKeys, isNullAwareAntiJoin) buildSide match { case BuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil @@ -133,10 +142,37 @@ case class BroadcastHashJoinExec( val numOutputRows = longMetric("numOutputRows") val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]() - streamedPlan.execute().mapPartitions { streamedIter => - val hashed = broadcastRelation.value.asReadOnlyCopy() - TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) - join(streamedIter, hashed, numOutputRows) + if (isNullAwareAntiJoin) { + streamedPlan.execute().mapPartitionsInternal { streamedIter => + val hashed = broadcastRelation.value.asReadOnlyCopy() + TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) + if (hashed == EmptyHashedRelation) { + streamedIter + } else if (hashed == EmptyHashedRelationWithAllNullKeys) { + Iterator.empty + } else { + val keyGenerator = UnsafeProjection.create( + BindReferences.bindReferences[Expression]( + leftKeys, + AttributeSeq(left.output)) + ) + streamedIter.filter(row => { + val lookupKey: UnsafeRow = keyGenerator(row) + if (lookupKey.anyNull()) { + false + } else { + // Anti Join: Drop the row on the streamed side if it is a match on the build + hashed.get(lookupKey) == null + } + }) + } + } + } else { + streamedPlan.execute().mapPartitions { streamedIter => + val hashed = broadcastRelation.value.asReadOnlyCopy() + TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) + join(streamedIter, hashed, numOutputRows) + } } } @@ -454,6 +490,40 @@ case class BroadcastHashJoinExec( val (matched, checkCondition, _) = getJoinCondition(ctx, input) val numOutput = metricTerm(ctx, "numOutputRows") + if (isNullAwareAntiJoin) { + if (broadcastRelation.value == EmptyHashedRelation) { + return s""" + |// If the right side is empty, NAAJ simply returns the left side. + |$numOutput.add(1); + |${consume(ctx, input)} + """.stripMargin + } else if (broadcastRelation.value == EmptyHashedRelationWithAllNullKeys) { + return s""" + |// If the right side contains any all-null key, NAAJ simply returns Nothing. + """.stripMargin + } else { + val found = ctx.freshName("found") + return s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |if ($anyNull) { + | $found = true; + |} else { + | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + | if ($matched != null) { + | $found = true; + | } + |} + | + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin + } + } + if (uniqueKeyCodePath) { val found = ctx.freshName("found") s""" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 13180d6b20902..f2835c2fa6626 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -96,7 +96,8 @@ private[execution] object HashedRelation { input: Iterator[InternalRow], key: Seq[Expression], sizeEstimate: Int = 64, - taskMemoryManager: TaskMemoryManager = null): HashedRelation = { + taskMemoryManager: TaskMemoryManager = null, + isNullAware: Boolean = false): HashedRelation = { val mm = Option(taskMemoryManager).getOrElse { new TaskMemoryManager( new UnifiedMemoryManager( @@ -107,10 +108,12 @@ private[execution] object HashedRelation { 0) } - if (key.length == 1 && key.head.dataType == LongType) { - LongHashedRelation(input, key, sizeEstimate, mm) + if (isNullAware && !input.hasNext) { + EmptyHashedRelation + } else if (key.length == 1 && key.head.dataType == LongType) { + LongHashedRelation(input, key, sizeEstimate, mm, isNullAware) } else { - UnsafeHashedRelation(input, key, sizeEstimate, mm) + UnsafeHashedRelation(input, key, sizeEstimate, mm, isNullAware) } } } @@ -310,7 +313,8 @@ private[joins] object UnsafeHashedRelation { input: Iterator[InternalRow], key: Seq[Expression], sizeEstimate: Int, - taskMemoryManager: TaskMemoryManager): HashedRelation = { + taskMemoryManager: TaskMemoryManager, + isNullAware: Boolean = false): HashedRelation = { val pageSizeBytes = Option(SparkEnv.get).map(_.memoryManager.pageSizeBytes) .getOrElse(new SparkConf().get(BUFFER_PAGESIZE).getOrElse(16L * 1024 * 1024)) @@ -338,6 +342,8 @@ private[joins] object UnsafeHashedRelation { throw new SparkOutOfMemoryError("There is not enough memory to build hash map") // scalastyle:on throwerror } + } else if (isNullAware) { + return EmptyHashedRelationWithAllNullKeys } } @@ -889,7 +895,8 @@ private[joins] object LongHashedRelation { input: Iterator[InternalRow], key: Seq[Expression], sizeEstimate: Int, - taskMemoryManager: TaskMemoryManager): LongHashedRelation = { + taskMemoryManager: TaskMemoryManager, + isNullAware: Boolean = false): HashedRelation = { val map = new LongToUnsafeRowMap(taskMemoryManager, sizeEstimate) val keyGenerator = UnsafeProjection.create(key) @@ -903,6 +910,8 @@ private[joins] object LongHashedRelation { if (!rowKey.isNullAt(0)) { val key = rowKey.getLong(0) map.append(key, unsafeRow) + } else if (isNullAware) { + return EmptyHashedRelationWithAllNullKeys } } map.optimize() @@ -910,8 +919,52 @@ private[joins] object LongHashedRelation { } } +/** + * Common trait with dummy implementation for NAAJ special HashedRelation + * EmptyHashedRelation + * EmptyHashedRelationWithAllNullKeys + */ +trait NullAwareHashedRelation extends HashedRelation with Externalizable { + override def get(key: InternalRow): Iterator[InternalRow] = { + throw new UnsupportedOperationException + } + + override def getValue(key: InternalRow): InternalRow = { + throw new UnsupportedOperationException + } + + override def keyIsUnique: Boolean = true + + override def keys(): Iterator[InternalRow] = { + throw new UnsupportedOperationException + } + + override def close(): Unit = {} + + override def writeExternal(out: ObjectOutput): Unit = {} + + override def readExternal(in: ObjectInput): Unit = {} + + override def estimatedSize: Long = 0 +} + +/** + * A special HashedRelation indicates it built from a empty input:Iterator[InternalRow]. + */ +object EmptyHashedRelation extends NullAwareHashedRelation { + override def asReadOnlyCopy(): EmptyHashedRelation.type = this +} + +/** + * A special HashedRelation indicates it built from a non-empty input:Iterator[InternalRow], + * which contains all null columns key. + */ +object EmptyHashedRelationWithAllNullKeys extends NullAwareHashedRelation { + override def asReadOnlyCopy(): EmptyHashedRelationWithAllNullKeys.type = this +} + /** The HashedRelationBroadcastMode requires that rows are broadcasted as a HashedRelation. */ -case class HashedRelationBroadcastMode(key: Seq[Expression]) +case class HashedRelationBroadcastMode(key: Seq[Expression], isNullAware: Boolean = false) extends BroadcastMode { override def transform(rows: Array[InternalRow]): HashedRelation = { @@ -923,9 +976,9 @@ case class HashedRelationBroadcastMode(key: Seq[Expression]) sizeHint: Option[Long]): HashedRelation = { sizeHint match { case Some(numRows) => - HashedRelation(rows, canonicalized.key, numRows.toInt) + HashedRelation(rows, canonicalized.key, numRows.toInt, isNullAware = isNullAware) case None => - HashedRelation(rows, canonicalized.key) + HashedRelation(rows, canonicalized.key, isNullAware = isNullAware) } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql index beb5b9e5fe516..4c1816e93b083 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql @@ -1,5 +1,8 @@ -- Test filter clause for aggregate expression. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-basic.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-basic.sql index f4ffc20086386..5669423148f80 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-basic.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-basic.sql @@ -1,3 +1,6 @@ +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view tab_a as select * from values (1, 1) as tab_a(a1, b1); create temporary view tab_b as select * from values (1, 1) as tab_b(a2, b2); create temporary view struct_tab as select struct(col1 as a, col2 as b) as record from diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql index 8f98ae1155062..750cc42b8641c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql @@ -1,6 +1,9 @@ -- A test suite for IN HAVING in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql index 200a71ebbb622..2353560137d21 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -13,6 +13,9 @@ --CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY --CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN +--CONFIG_DIM3 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM3 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql index 0a16f118f0455..53fc2b8be7501 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql @@ -1,6 +1,9 @@ -- A test suite for IN LIMIT in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql index 4643605148a0c..1a6c06f9dad49 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql @@ -1,6 +1,9 @@ -- A test suite for multiple columns in predicate in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql index 001c49c460b06..568854ebe2d9b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql @@ -6,6 +6,9 @@ --CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY --CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN +--CONFIG_DIM2 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM2 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql index e65cb9106c1d4..fa4ae87f041cf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql @@ -1,6 +1,9 @@ -- A test suite for in with cte in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql index 2f6835b59fdd5..e2d4ad522d446 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql @@ -1,5 +1,8 @@ -- Tests NOT-IN subqueries nested inside OR expression(s). +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", 10), (200, "emp 2", NULL), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql index fcdb667ad4523..2d11c5da20633 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql @@ -1,6 +1,9 @@ -- A test suite for not-in-joins in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql index 8eea84f4f5272..a061e495f51b8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql @@ -4,6 +4,9 @@ -- This file has the same test cases as not-in-unit-tests-multi-column.sql with literals instead of -- subqueries. Small changes have been made to the literals to make them typecheck. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, null), (null, 1.0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql index 9f8dc7fca3b94..28ab75121573a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql @@ -15,6 +15,9 @@ -- This can be generalized to include more tests for more columns, but it covers the main cases -- when there is more than one column. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, null), (null, 1.0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql index b261363d1dde7..79747022eb1e8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql @@ -4,6 +4,9 @@ -- This file has the same test cases as not-in-unit-tests-single-column.sql with literals instead of -- subqueries. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, 1.0), (2, 3.0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql index 2cc08e10acf67..8060246bf3a3f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql @@ -31,6 +31,9 @@ -- cause cases 2, 3, or 4 to be reduced to case 1 by limiting the number of rows returned by the -- subquery, so the row from the parent table should always be included in the output. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, 1.0), (2, 3.0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql index 2748a959cbef8..d8a58afa344db 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql @@ -1,6 +1,9 @@ -- A test suite for simple IN predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index 36757863ffcb5..79f4d48101ca4 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -308,7 +308,7 @@ Input [2]: [key#x, val#x] (7) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (8) BroadcastHashJoin Left keys [1]: [key#x] @@ -362,7 +362,7 @@ Input [2]: [key#x, val#x] (5) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (6) BroadcastHashJoin Left keys [1]: [key#x] @@ -533,7 +533,7 @@ Input [2]: [key#x, val#x] (7) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (8) BroadcastHashJoin Left keys [1]: [key#x] @@ -643,7 +643,7 @@ Results [2]: [key#x, max(val#x)#x AS max(val)#x] (13) BroadcastExchange Input [2]: [key#x, max(val)#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (14) BroadcastHashJoin Left keys [1]: [key#x] diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index 2b07dac0e5d0a..614cb2a137d01 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -316,7 +316,7 @@ Input [2]: [key#x, val#x] (9) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [key#x] @@ -373,7 +373,7 @@ Input [2]: [key#x, val#x] (7) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [key#x] @@ -771,7 +771,7 @@ Input [2]: [key#x, val#x] (9) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [key#x] @@ -853,7 +853,7 @@ Results [2]: [key#x, max(val#x)#x AS max(val)#x] (10) BroadcastExchange Input [2]: [key#x, max(val)#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (11) BroadcastHashJoin [codegen id : 4] Left keys [1]: [key#x] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index cdf9ea4b31ee7..0b754e9e3ec0b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -396,7 +396,7 @@ abstract class DynamicPartitionPruningSuiteBase """.stripMargin) val found = df.queryExecution.executedPlan.find { - case BroadcastHashJoinExec(_, _, p: ExistenceJoin, _, _, _, _) => true + case BroadcastHashJoinExec(_, _, p: ExistenceJoin, _, _, _, _, _) => true case _ => false } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index c42d4c6f74a93..bedfbffc789ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -89,6 +89,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan if (operators.head.getClass != c) { fail(s"$sqlString expected operator: $c, but got ${operators.head}\n physical: \n$physical") } + operators.head } test("join operator selection") { @@ -1147,4 +1148,44 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan }) } } + + test("SPARK-32290: SingleColumn Null Aware Anti Join Optimize") { + withSQLConf(SQLConf.OPTIMIZE_NULL_AWARE_ANTI_JOIN.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString) { + // positive not in subquery case + var joinExec = assertJoin(( + "select * from testData where key not in (select a from testData2)", + classOf[BroadcastHashJoinExec])) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + + // negative not in subquery case since multi-column is not supported + assertJoin(( + "select * from testData where (key, key + 1) not in (select * from testData2)", + classOf[BroadcastNestedLoopJoinExec])) + + // positive hand-written left anti join + // testData.key nullable false + // testData3.b nullable true + joinExec = assertJoin(( + "select * from testData left anti join testData3 ON key = b or isnull(key = b)", + classOf[BroadcastHashJoinExec])) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + + // negative hand-written left anti join + // testData.key nullable false + // testData2.a nullable false + // isnull(key = a) will be optimized to true literal and removed + joinExec = assertJoin(( + "select * from testData left anti join testData2 ON key = a or isnull(key = a)", + classOf[BroadcastHashJoinExec])) + assert(!joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + + // negative hand-written left anti join + // not match pattern Or(EqualTo(a=b), IsNull(EqualTo(a=b)) + assertJoin(( + "select * from testData2 left anti join testData3 ON testData2.a = testData3.b or " + + "isnull(testData2.b = testData3.b)", + classOf[BroadcastNestedLoopJoinExec])) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 2bb9aa55e4579..a21c461e84588 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, FileSourceScanExec, InputAdapter, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.FileScanRDD +import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -1646,4 +1647,110 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkAnswer(df, df2) checkAnswer(df, Nil) } + + test("SPARK-32290: SingleColumn Null Aware Anti Join Optimize") { + Seq(true, false).foreach { enableNAAJ => + Seq(true, false).foreach { enableAQE => + Seq(true, false).foreach { enableCodegen => + withSQLConf( + SQLConf.OPTIMIZE_NULL_AWARE_ANTI_JOIN.key -> enableNAAJ.toString, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> enableCodegen.toString) { + + def findJoinExec(df: DataFrame): BaseJoinExec = { + df.queryExecution.sparkPlan.collectFirst { + case j: BaseJoinExec => j + }.get + } + + var df: DataFrame = null + var joinExec: BaseJoinExec = null + + // single column not in subquery -- empty sub-query + df = sql("select * from l where a not in (select c from r where c > 10)") + checkAnswer(df, spark.table("l")) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- sub-query include null + df = sql("select * from l where a not in (select c from r where d < 6.0)") + checkAnswer(df, Seq.empty) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- streamedSide row is null + df = + sql("select * from l where b = 5.0 and a not in(select c from r where c is not null)") + checkAnswer(df, Seq.empty) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- streamedSide row is not null, match found + df = + sql("select * from l where a = 6 and a not in (select c from r where c is not null)") + checkAnswer(df, Seq.empty) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- streamedSide row is not null, match not found + df = + sql("select * from l where a = 1 and a not in (select c from r where c is not null)") + checkAnswer(df, Row(1, 2.0) :: Row(1, 2.0) :: Nil) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- d = b + 10 joinKey found, match ExtractEquiJoinKeys + df = sql("select * from l where a not in (select c from r where d = b + 10)") + checkAnswer(df, spark.table("l")) + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(!joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + + // single column not in subquery -- d = b + 10 and b = 5.0 => d = 15, joinKey not found + // match ExtractSingleColumnNullAwareAntiJoin + df = + sql("select * from l where b = 5.0 and a not in (select c from r where d = b + 10)") + checkAnswer(df, Row(null, 5.0) :: Nil) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // multi column not in subquery + df = sql("select * from l where (a, b) not in (select c, d from r where c > 10)") + checkAnswer(df, spark.table("l")) + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index f7396ee2a89c8..03596d8654c66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -342,7 +342,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(baseTable, "idx") assert(distinctWithId.queryExecution.executedPlan.collectFirst { case WholeStageCodegenExec( - ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _))) => true + ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _, _))) => true }.isDefined) checkAnswer(distinctWithId, Seq(Row(1, 0), Row(1, 0))) @@ -353,7 +353,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(baseTable, "idx") assert(groupByWithId.queryExecution.executedPlan.collectFirst { case WholeStageCodegenExec( - ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _))) => true + ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _, _))) => true }.isDefined) checkAnswer(groupByWithId, Seq(Row(1, 2, 0), Row(1, 2, 0))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index e9ef7c1a0c540..d2c9322685d94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -69,8 +69,9 @@ class DebuggingSuite extends SharedSparkSession with DisableAdaptiveExecutionSui } val output = captured.toString() + val hashedModeString = "HashedRelationBroadcastMode(List(input[0, bigint, false]),false)" assert(output.replaceAll("\\[id=#\\d+\\]", "[id=#x]").contains( - """== BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])), [id=#x] == + s"""== BroadcastExchange $hashedModeString, [id=#x] == |Tuples output: 0 | id LongType: {} |== WholeStageCodegen (1) == diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 3526aa254c280..21ee88f0d7426 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -144,6 +144,7 @@ class HashedRelationSuite extends SharedSparkSession { } val longRelation2 = LongHashedRelation(rows.iterator ++ rows.iterator, key, 100, mm) + .asInstanceOf[LongHashedRelation] assert(!longRelation2.keyIsUnique) (0 until 100).foreach { i => val rows = longRelation2.get(i).toArray From ca1ecf7f9ff46e99f321bbe899866d5bce2c289a Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 28 Jul 2020 12:24:15 +0000 Subject: [PATCH 306/384] [SPARK-32459][SQL] Support WrappedArray as customCollectionCls in MapObjects ### What changes were proposed in this pull request? This PR supports `WrappedArray` as `customCollectionCls` in `MapObjects`. ### Why are the changes needed? This helps fix the regression caused by SPARK-31826. For the following test, it can pass in branch-3.0 but fail in master branch: ```scala test("WrappedArray") { val myUdf = udf((a: WrappedArray[Int]) => WrappedArray.make[Int](Array(a.head + 99))) checkAnswer(Seq(Array(1)) .toDF("col") .select(myUdf(Column("col"))), Row(ArrayBuffer(100))) } ``` In SPARK-31826, we've changed the catalyst-to-scala converter from `CatalystTypeConverters` to `ExpressionEncoder.deserializer`. However, `CatalystTypeConverters` supports `WrappedArray` while `ExpressionEncoder.deserializer` doesn't. ### Does this PR introduce _any_ user-facing change? No, SPARK-31826 is merged into master and branch-3.1, which haven't been released. ### How was this patch tested? Added a new test for `WrappedArray` in `UDFSuite`; Also updated `ObjectExpressionsSuite` for `MapObjects`. Closes #29261 from Ngone51/fix-wrappedarray. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../expressions/objects/objects.scala | 19 ++++++++++++++++++- .../expressions/ObjectExpressionsSuite.scala | 7 ++++++- .../scala/org/apache/spark/sql/UDFSuite.scala | 11 +++++++++++ 3 files changed, 35 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 361bcd492965b..7cf2c73f9c941 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.objects import java.lang.reflect.{Method, Modifier} import scala.collection.JavaConverters._ -import scala.collection.mutable.Builder +import scala.collection.mutable.{Builder, IndexedSeq, WrappedArray} import scala.reflect.ClassTag import scala.util.Try @@ -755,6 +755,9 @@ case class MapObjects private( } private lazy val mapElements: Seq[_] => Any = customCollectionCls match { + case Some(cls) if classOf[WrappedArray[_]].isAssignableFrom(cls) => + // Scala WrappedArray + inputCollection => WrappedArray.make(executeFuncOnCollection(inputCollection).toArray) case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) => // Scala sequence executeFuncOnCollection(_).toSeq @@ -912,6 +915,20 @@ case class MapObjects private( val (initCollection, addElement, getResult): (String, String => String, String) = customCollectionCls match { + case Some(cls) if classOf[WrappedArray[_]].isAssignableFrom(cls) => + // Scala WrappedArray + val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()" + val builder = ctx.freshName("collectionBuilder") + ( + s""" + ${classOf[Builder[_, _]].getName} $builder = $getBuilder; + $builder.sizeHint($dataLength); + """, + (genValue: String) => s"$builder.$$plus$$eq($genValue);", + s"(${cls.getName}) ${classOf[WrappedArray[_]].getName}$$." + + s"MODULE$$.make(((${classOf[IndexedSeq[_]].getName})$builder" + + s".result()).toArray(scala.reflect.ClassTag$$.MODULE$$.Object()));" + ) case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) || classOf[scala.collection.Set[_]].isAssignableFrom(cls) => // Scala sequence or set diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index 0fea84bb183e0..02c5c9ab89088 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import scala.collection.JavaConverters._ +import scala.collection.mutable.WrappedArray import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.Random @@ -330,6 +331,8 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(result.asInstanceOf[ArrayData].array.toSeq == expected) case l if classOf[java.util.List[_]].isAssignableFrom(l) => assert(result.asInstanceOf[java.util.List[_]].asScala == expected) + case a if classOf[WrappedArray[Int]].isAssignableFrom(a) => + assert(result == WrappedArray.make[Int](expected.toArray)) case s if classOf[Seq[_]].isAssignableFrom(s) => assert(result.asInstanceOf[Seq[_]] == expected) case s if classOf[scala.collection.Set[_]].isAssignableFrom(s) => @@ -337,7 +340,8 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - val customCollectionClasses = Seq(classOf[Seq[Int]], classOf[scala.collection.Set[Int]], + val customCollectionClasses = Seq(classOf[WrappedArray[Int]], + classOf[Seq[Int]], classOf[scala.collection.Set[Int]], classOf[java.util.List[Int]], classOf[java.util.AbstractList[Int]], classOf[java.util.AbstractSequentialList[Int]], classOf[java.util.Vector[Int]], classOf[java.util.Stack[Int]], null) @@ -357,6 +361,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { stack.add(3) Seq( + (Seq(1, 2, 3), ObjectType(classOf[WrappedArray[Int]])), (Seq(1, 2, 3), ObjectType(classOf[Seq[Int]])), (Array(1, 2, 3), ObjectType(classOf[Array[Int]])), (Seq(1, 2, 3), ObjectType(classOf[Object])), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index f0d5a61ad8006..db85ae613eaa1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -22,6 +22,8 @@ import java.sql.Timestamp import java.time.{Instant, LocalDate} import java.time.format.DateTimeFormatter +import scala.collection.mutable.{ArrayBuffer, WrappedArray} + import org.apache.spark.SparkException import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -787,4 +789,13 @@ class UDFSuite extends QueryTest with SharedSparkSession { Seq(Array(1)).toDF("a").createOrReplaceTempView("t") checkAnswer(sql("SELECT key(a) AS k FROM t GROUP BY key(a)"), Row(1) :: Nil) } + + test("SPARK-32459: UDF should not fail on WrappedArray") { + val myUdf = udf((a: WrappedArray[Int]) => + WrappedArray.make[Int](Array(a.head + 99))) + checkAnswer(Seq(Array(1)) + .toDF("col") + .select(myUdf(Column("col"))), + Row(ArrayBuffer(100))) + } } From c28da672f862765f80adfcdd21dfd7265608af11 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 28 Jul 2020 12:34:10 +0000 Subject: [PATCH 307/384] [SPARK-32382][SQL] Override table renaming in JDBC dialects ### What changes were proposed in this pull request? Override the default implementation of `JdbcDialect.renameTable()`: ```scala s"ALTER TABLE $oldTable RENAME TO $newTable" ``` in the following JDBC dialects according to official documentation: - DB2 - Derby - MS SQL Server - Teradata Other dialects follow the default implementation: - MySQL: https://dev.mysql.com/doc/refman/8.0/en/alter-table.html - Oracle: https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/ALTER-TABLE.html#GUID-552E7373-BF93-477D-9DA3-B2C9386F2877 - PostgreSQL: https://www.postgresql.org/docs/12/sql-altertable.html ### Why are the changes needed? To have correct implementation of table renaming for all supported JDBC dialects. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Manually Closes #29237 from MaxGekk/jdbc-rename-table. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala | 7 +++++++ .../scala/org/apache/spark/sql/jdbc/DerbyDialect.scala | 5 +++++ .../org/apache/spark/sql/jdbc/MsSqlServerDialect.scala | 7 +++++++ .../scala/org/apache/spark/sql/jdbc/TeradataDialect.scala | 5 +++++ 4 files changed, 24 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala index ab574df4557a5..430ca9edab799 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala @@ -51,4 +51,11 @@ private object DB2Dialect extends JdbcDialect { } override def isCascadingTruncateTable(): Option[Boolean] = Some(false) + + // scalastyle:off line.size.limit + // See https://www.ibm.com/support/knowledgecenter/en/SSEPGG_11.5.0/com.ibm.db2.luw.sql.ref.doc/doc/r0000980.html + // scalastyle:on line.size.limit + override def renameTable(oldTable: String, newTable: String): String = { + s"RENAME TABLE $oldTable TO $newTable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala index d528d5a9fef5a..9ca8879be31e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala @@ -45,4 +45,9 @@ private object DerbyDialect extends JdbcDialect { } override def isCascadingTruncateTable(): Option[Boolean] = Some(false) + + // See https://db.apache.org/derby/docs/10.5/ref/rrefsqljrenametablestatement.html + override def renameTable(oldTable: String, newTable: String): String = { + s"RENAME TABLE $oldTable TO $newTable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala index 72284b5996201..1c6e8c359aa15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala @@ -57,4 +57,11 @@ private object MsSqlServerDialect extends JdbcDialect { } override def isCascadingTruncateTable(): Option[Boolean] = Some(false) + + // scalastyle:off line.size.limit + // See https://docs.microsoft.com/en-us/sql/relational-databases/system-stored-procedures/sp-rename-transact-sql?view=sql-server-ver15 + // scalastyle:on line.size.limit + override def renameTable(oldTable: String, newTable: String): String = { + s"EXEC sp_rename $oldTable, $newTable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala index 552d7a484f3fd..58fe62cb6e088 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala @@ -50,4 +50,9 @@ private case object TeradataDialect extends JdbcDialect { cascade: Option[Boolean] = isCascadingTruncateTable): String = { s"DELETE FROM $table ALL" } + + // See https://docs.teradata.com/reader/scPHvjfglIlB8F70YliLAw/wysTNUMsP~0aGzksLCl1kg + override def renameTable(oldTable: String, newTable: String): String = { + s"RENAME TABLE $oldTable TO $newTable" + } } From 44c868b73a7cb293ec81927c28991677bf33ea90 Mon Sep 17 00:00:00 2001 From: Xiaochang Wu Date: Tue, 28 Jul 2020 08:36:11 -0700 Subject: [PATCH 308/384] [SPARK-32339][ML][DOC] Improve MLlib BLAS native acceleration docs ### What changes were proposed in this pull request? Rewrite a clearer and complete BLAS native acceleration enabling guide. ### Why are the changes needed? The document of enabling BLAS native acceleration in ML guide (https://spark.apache.org/docs/latest/ml-guide.html#dependencies) is incomplete and unclear to the user. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? N/A Closes #29139 from xwu99/blas-doc. Lead-authored-by: Xiaochang Wu Co-authored-by: Wu, Xiaochang Signed-off-by: Huaxin Gao --- docs/ml-guide.md | 22 +++------ docs/ml-linalg-guide.md | 103 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 109 insertions(+), 16 deletions(-) create mode 100644 docs/ml-linalg-guide.md diff --git a/docs/ml-guide.md b/docs/ml-guide.md index ddce98b32f941..1b4a3e4eee557 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -62,23 +62,13 @@ The primary Machine Learning API for Spark is now the [DataFrame](sql-programmin # Dependencies -MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on -[netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing. -If native libraries[^1] are not available at runtime, you will see a warning message and a pure JVM -implementation will be used instead. +MLlib uses linear algebra packages [Breeze](http://www.scalanlp.org/) and [netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing[^1]. Those packages may call native acceleration libraries such as [Intel MKL](https://software.intel.com/content/www/us/en/develop/tools/math-kernel-library.html) or [OpenBLAS](http://www.openblas.net) if they are available as system libraries or in runtime library paths. -Due to licensing issues with runtime proprietary binaries, we do not include `netlib-java`'s native -proxies by default. -To configure `netlib-java` / Breeze to use system optimised binaries, include -`com.github.fommil.netlib:all:1.1.2` (or build Spark with `-Pnetlib-lgpl`) as a dependency of your -project and read the [netlib-java](https://github.com/fommil/netlib-java) documentation for your -platform's additional installation instructions. - -The most popular native BLAS such as [Intel MKL](https://software.intel.com/en-us/mkl), [OpenBLAS](http://www.openblas.net), can use multiple threads in a single operation, which can conflict with Spark's execution model. - -Configuring these BLAS implementations to use a single thread for operations may actually improve performance (see [SPARK-21305](https://issues.apache.org/jira/browse/SPARK-21305)). It is usually optimal to match this to the number of cores each Spark task is configured to use, which is 1 by default and typically left at 1. - -Please refer to resources like the following to understand how to configure the number of threads these BLAS implementations use: [Intel MKL](https://software.intel.com/en-us/articles/recommended-settings-for-calling-intel-mkl-routines-from-multi-threaded-applications) or [Intel oneMKL](https://software.intel.com/en-us/onemkl-linux-developer-guide-improving-performance-with-threading) and [OpenBLAS](https://github.com/xianyi/OpenBLAS/wiki/faq#multi-threaded). Note that if nativeBLAS is not properly configured in system, java implementation(f2jBLAS) will be used as fallback option. +Due to differing OSS licenses, `netlib-java`'s native proxies can't be distributed with Spark. See [MLlib Linear Algebra Acceleration Guide](ml-linalg-guide.html) for how to enable accelerated linear algebra processing. If accelerated native libraries are not enabled, you will see a warning message like below and a pure JVM implementation will be used instead: +``` +WARN BLAS: Failed to load implementation from:com.github.fommil.netlib.NativeSystemBLAS +WARN BLAS: Failed to load implementation from:com.github.fommil.netlib.NativeRefBLAS +``` To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. diff --git a/docs/ml-linalg-guide.md b/docs/ml-linalg-guide.md new file mode 100644 index 0000000000000..739091363473f --- /dev/null +++ b/docs/ml-linalg-guide.md @@ -0,0 +1,103 @@ +--- +layout: global +title: MLlib Linear Algebra Acceleration Guide +displayTitle: MLlib Linear Algebra Acceleration Guide +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +## Introduction + +This guide provides necessary information to enable accelerated linear algebra processing for Spark MLlib. + +Spark MLlib defines Vector and Matrix as basic data types for machine learning algorithms. On top of them, [BLAS](https://en.wikipedia.org/wiki/Basic_Linear_Algebra_Subprograms) and [LAPACK](https://en.wikipedia.org/wiki/LAPACK) operations are implemented and supported by [netlib-java](https://github.com/fommil/netlib-Java) (the algorithms may call [Breeze](https://github.com/scalanlp/breeze) and it will in turn call `netlib-java`). `netlib-java` can use optimized native linear algebra libraries (refered to as "native libraries" or "BLAS libraries" hereafter) for faster numerical processing. [Intel MKL](https://software.intel.com/content/www/us/en/develop/tools/math-kernel-library.html) and [OpenBLAS](http://www.openblas.net) are two popular ones. + +However due to license differences, the official released Spark binaries by default don't contain native libraries support for `netlib-java`. + +The following sections describe how to enable `netlib-java` with native libraries support for Spark MLlib and how to install native libraries and configure them properly. + +## Enable `netlib-java` with native library proxies + +`netlib-java` depends on `libgfortran`. It requires GFORTRAN 1.4 or above. This can be obtained by installing `libgfortran` package. After installation, the following command can be used to verify if it is installed properly. +``` +strings /path/to/libgfortran.so.3.0.0 | grep GFORTRAN_1.4 +``` + +To build Spark with `netlib-java` native library proxies, you need to add `-Pnetlib-lgpl` to Maven build command line. For example: +``` +$SPARK_SOURCE_HOME/build/mvn -Pnetlib-lgpl -DskipTests -Pyarn -Phadoop-2.7 clean package +``` + +If you only want to enable it in your project, include `com.github.fommil.netlib:all:1.1.2` as a dependency of your project. + +## Install native linear algebra libraries + +Intel MKL and OpenBLAS are two popular native linear algebra libraries. You can choose one of them based on your preference. We provide basic instructions as below. You can refer to [netlib-java documentation](https://github.com/fommil/netlib-java) for more advanced installation instructions. + +### Intel MKL + +- Download and install Intel MKL. The installation should be done on all nodes of the cluster. We assume the installation location is $MKLROOT (e.g. /opt/intel/mkl). +- Create soft links to `libmkl_rt.so` with specific names in system library search paths. For instance, make sure `/usr/local/lib` is in system library search paths and run the following commands: +``` +$ ln -sf $MKLROOT/lib/intel64/libmkl_rt.so /usr/local/lib/libblas.so.3 +$ ln -sf $MKLROOT/lib/intel64/libmkl_rt.so /usr/local/lib/liblapack.so.3 +``` + +### OpenBLAS + +The installation should be done on all nodes of the cluster. Generic version of OpenBLAS are available with most distributions. You can install it with a distribution package manager like `apt` or `yum`. + +For Debian / Ubuntu: +``` +sudo apt-get install libopenblas-base +sudo update-alternatives --config libblas.so.3 +``` +For CentOS / RHEL: +``` +sudo yum install openblas +``` + +## Check if native libraries are enabled for MLlib + +To verify native libraries are properly loaded, start `spark-shell` and run the following code: +``` +scala> import com.github.fommil.netlib.BLAS; +scala> System.out.println(BLAS.getInstance().getClass().getName()); +``` + +If they are correctly loaded, it should print `com.github.fommil.netlib.NativeSystemBLAS`. Otherwise the warnings should be printed: +``` +WARN BLAS: Failed to load implementation from:com.github.fommil.netlib.NativeSystemBLAS +WARN BLAS: Failed to load implementation from:com.github.fommil.netlib.NativeRefBLAS +``` + +If native libraries are not properly configured in the system, the Java implementation (f2jBLAS) will be used as fallback option. + +## Spark Configuration + +The default behavior of multi-threading in either Intel MKL or OpenBLAS may not be optimal with Spark's execution model [^1]. + +Therefore configuring these native libraries to use a single thread for operations may actually improve performance (see [SPARK-21305](https://issues.apache.org/jira/browse/SPARK-21305)). It is usually optimal to match this to the number of `spark.task.cpus`, which is `1` by default and typically left at `1`. + +You can use the options in `config/spark-env.sh` to set thread number for Intel MKL or OpenBLAS: +* For Intel MKL: +``` +MKL_NUM_THREADS=1 +``` +* For OpenBLAS: +``` +OPENBLAS_NUM_THREADS=1 +``` + +[^1]: Please refer to the following resources to understand how to configure the number of threads for these BLAS implementations: [Intel MKL](https://software.intel.com/en-us/articles/recommended-settings-for-calling-intel-mkl-routines-from-multi-threaded-applications) or [Intel oneMKL](https://software.intel.com/en-us/onemkl-linux-developer-guide-improving-performance-with-threading) and [OpenBLAS](https://github.com/xianyi/OpenBLAS/wiki/faq#multi-threaded). From a3d80564ad4d09c3c2f95896ca2765904b1abe8f Mon Sep 17 00:00:00 2001 From: Michael Munday Date: Tue, 28 Jul 2020 10:36:20 -0700 Subject: [PATCH 309/384] [SPARK-32458][SQL][TESTS] Fix incorrectly sized row value reads ### What changes were proposed in this pull request? Updates to tests to use correctly sized `getInt` or `getLong` calls. ### Why are the changes needed? The reads were incorrectly sized (i.e. `putLong` paired with `getInt` and `putInt` paired with `getLong`). This causes test failures on big-endian systems. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Tests were run on a big-endian system (s390x). This change is unlikely to have any practical effect on little-endian systems. Closes #29258 from mundaym/fix-row. Authored-by: Michael Munday Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/encoders/RowEncoderSuite.scala | 2 +- .../apache/spark/sql/catalyst/util/UnsafeMapSuite.scala | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index fd24f058f357c..d20a9ba3f0f68 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -336,7 +336,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { val encoder = RowEncoder(schema).resolveAndBind() val localDate = java.time.LocalDate.parse("2019-02-27") val row = toRow(encoder, Row(localDate)) - assert(row.getLong(0) === DateTimeUtils.localDateToDays(localDate)) + assert(row.getInt(0) === DateTimeUtils.localDateToDays(localDate)) val readback = fromRow(encoder, row) assert(readback.get(0).equals(localDate)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala index ebc88612be22a..443534fd0a06a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala @@ -48,8 +48,8 @@ class UnsafeMapSuite extends SparkFunSuite { val ser = new JavaSerializer(new SparkConf).newInstance() val mapDataSer = ser.deserialize[UnsafeMapData](ser.serialize(unsafeMapData)) assert(mapDataSer.numElements() == 1) - assert(mapDataSer.keyArray().getInt(0) == 19285) - assert(mapDataSer.valueArray().getInt(0) == 19286) + assert(mapDataSer.keyArray().getLong(0) == 19285) + assert(mapDataSer.valueArray().getLong(0) == 19286) assert(mapDataSer.getBaseObject.asInstanceOf[Array[Byte]].length == 1024) } @@ -57,8 +57,8 @@ class UnsafeMapSuite extends SparkFunSuite { val ser = new KryoSerializer(new SparkConf).newInstance() val mapDataSer = ser.deserialize[UnsafeMapData](ser.serialize(unsafeMapData)) assert(mapDataSer.numElements() == 1) - assert(mapDataSer.keyArray().getInt(0) == 19285) - assert(mapDataSer.valueArray().getInt(0) == 19286) + assert(mapDataSer.keyArray().getLong(0) == 19285) + assert(mapDataSer.valueArray().getLong(0) == 19286) assert(mapDataSer.getBaseObject.asInstanceOf[Array[Byte]].length == 1024) } } From 5491c08bf1d3472e712c0dd88c2881d6496108c0 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 29 Jul 2020 12:07:35 +0900 Subject: [PATCH 310/384] Revert "[SPARK-31525][SQL] Return an empty list for df.head() when df is empty" This reverts commit 44a5258ac28f843dc7385868b9a86db77eeff266. --- docs/pyspark-migration-guide.md | 3 --- python/pyspark/sql/dataframe.py | 5 ++--- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md index efc2175a0459d..2c9ea410f217e 100644 --- a/docs/pyspark-migration-guide.md +++ b/docs/pyspark-migration-guide.md @@ -26,9 +26,6 @@ Note that this migration guide describes the items specific to PySpark. Many items of SQL migration can be applied when migrating PySpark to higher versions. Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). -## Upgrading from PySpark 3.0 to 3.1 -- In Spark 3.1, PySpark `DataFrame.head()` will return `[]` if the PySpark DataFrame is empty. In Spark 3.0 or prior, it will return `None`. The bahavior remains the same for non-empty PySpark DataFrame. - ## Upgrading from PySpark 2.4 to 3.0 - In Spark 3.0, PySpark requires a pandas version of 0.23.2 or higher to use pandas related functionality, such as `toPandas`, `createDataFrame` from pandas DataFrame, and so on. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f216f0b4dcd25..1027918adbe15 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1323,8 +1323,7 @@ def head(self, n=None): :param n: int, default 1. Number of rows to return. :return: If n is greater than 1, return a list of :class:`Row`. - If n is 1, return a single Row if it exists. Otherwise, we will return an - empty list to match the behavior of `head(1)` when the dataframe is empty. + If n is 1, return a single Row. >>> df.head() Row(age=2, name='Alice') @@ -1333,7 +1332,7 @@ def head(self, n=None): """ if n is None: rs = self.head(1) - return rs[0] if rs else [] + return rs[0] if rs else None return self.take(n) @since(1.3) From b2180c0950f2d8f3e866cb7b1ba0dd7f287f876d Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 29 Jul 2020 12:14:13 +0900 Subject: [PATCH 311/384] [SPARK-32471][SQL][DOCS][TESTS][PYTHON][SS] Describe JSON option `allowNonNumericNumbers` ### What changes were proposed in this pull request? 1. Describe the JSON option `allowNonNumericNumbers` which is used in read 2. Add new test cases for allowed JSON field values: NaN, +INF, +Infinity, Infinity, -INF and -Infinity ### Why are the changes needed? To improve UX with Spark SQL and to provide users full info about the supported option. ### Does this PR introduce _any_ user-facing change? Yes, in PySpark. ### How was this patch tested? Added new test to `JsonParsingOptionsSuite` Closes #29275 from MaxGekk/allowNonNumericNumbers-doc. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- python/pyspark/sql/readwriter.py | 13 +++++++++++-- python/pyspark/sql/streaming.py | 13 +++++++++++-- .../apache/spark/sql/DataFrameReader.scala | 8 ++++++++ .../sql/streaming/DataStreamReader.scala | 8 ++++++++ .../json/JsonParsingOptionsSuite.scala | 19 +++++++++++++++---- 5 files changed, 53 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index e5553a8bb162b..8560ef78c1fb0 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -184,7 +184,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, samplingRatio=None, dropFieldIfAllNull=None, encoding=None, locale=None, pathGlobFilter=None, - recursiveFileLookup=None): + recursiveFileLookup=None, allowNonNumericNumbers=None): """ Loads JSON files and returns the results as a :class:`DataFrame`. @@ -264,6 +264,14 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + :param allowNonNumericNumbers: allows JSON parser to recognize set of "Not-a-Number" (NaN) + tokens as legal floating number values. If None is set, + it uses the default value, ``true``. + + * ``+INF``: for positive infinity, as well as alias of + ``+Infinity`` and ``Infinity``. + * ``-INF``: for negative infinity, alias ``-Infinity``. + * ``NaN``: for other not-a-numbers, like result of division by zero. .. _partition discovery: https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery @@ -287,7 +295,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, samplingRatio=samplingRatio, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, - locale=locale, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) + locale=locale, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup, + allowNonNumericNumbers=allowNonNumericNumbers) if isinstance(path, str): path = [path] if type(path) == list: diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 4ec47305aa13e..b1832f420e389 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -420,7 +420,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, locale=None, dropFieldIfAllNull=None, encoding=None, pathGlobFilter=None, - recursiveFileLookup=None): + recursiveFileLookup=None, allowNonNumericNumbers=None): """ Loads a JSON file stream and returns the results as a :class:`DataFrame`. @@ -500,6 +500,14 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + :param allowNonNumericNumbers: allows JSON parser to recognize set of "Not-a-Number" (NaN) + tokens as legal floating number values. If None is set, + it uses the default value, ``true``. + + * ``+INF``: for positive infinity, as well as alias of + ``+Infinity`` and ``Infinity``. + * ``-INF``: for negative infinity, alias ``-Infinity``. + * ``NaN``: for other not-a-numbers, like result of division by zero. .. _partition discovery: https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery @@ -520,7 +528,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, - pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) + pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup, + allowNonNumericNumbers=allowNonNumericNumbers) if isinstance(path, str): return self._df(self._jreader.json(path)) else: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 5b78690b3267c..38031002cfad3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -469,6 +469,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * It does not change the behavior of partition discovery.
            • *
            • `recursiveFileLookup`: recursively scan a directory for files. Using this option * disables partition discovery
            • + *
            • `allowNonNumericNumbers` (default `true`): allows JSON parser to recognize set of + * "Not-a-Number" (NaN) tokens as legal floating number values: + *
                + *
              • `+INF` for positive infinity, as well as alias of `+Infinity` and `Infinity`. + *
              • `-INF` for negative infinity), alias `-Infinity`. + *
              • `NaN` for other not-a-numbers, like result of division by zero. + *
              + *
            • *
          • *
          • `recursiveFileLookup`: recursively scan a directory for files. Using this option * disables partition discovery
          • + *
          • `allowNonNumericNumbers` (default `true`): allows JSON parser to recognize set of + * "Not-a-Number" (NaN) tokens as legal floating number values: + *
              + *
            • `+INF` for positive infinity, as well as alias of `+Infinity` and `Infinity`. + *
            • `-INF` for negative infinity, alias `-Infinity`. + *
            • `NaN` for other not-a-numbers, like result of division by zero. + *
            + *
          • *
          * * @since 2.0.0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index 8c1afbfa7ec8f..d27b5c4737a11 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -112,12 +112,23 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession { } test("allowNonNumericNumbers on") { - val str = """{"age": NaN}""" + val str = """{"c0":NaN, "c1":+INF, "c2":+Infinity, "c3":Infinity, "c4":-INF, "c5":-Infinity}""" val df = spark.read.option("allowNonNumericNumbers", true).json(Seq(str).toDS()) - assert(df.schema.head.name == "age") - assert(df.schema.head.dataType == DoubleType) - assert(df.first().getDouble(0).isNaN) + assert(df.schema === + new StructType() + .add("c0", "double") + .add("c1", "double") + .add("c2", "double") + .add("c3", "double") + .add("c4", "double") + .add("c5", "double")) + checkAnswer( + df, + Row( + Double.NaN, + Double.PositiveInfinity, Double.PositiveInfinity, Double.PositiveInfinity, + Double.NegativeInfinity, Double.NegativeInfinity)) } test("allowBackslashEscapingAnyCharacter off") { From 45b7212fd3d60f537aa034a6255188e0283d05e7 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 29 Jul 2020 03:55:48 +0000 Subject: [PATCH 312/384] [SPARK-32401][SQL] Migrate function related commands to use UnresolvedFunc to resolve function identifier ### What changes were proposed in this pull request? This PR proposes to migrate the following function related commands to use `UnresolvedFunc` to resolve function identifier: - DROP FUNCTION - DESCRIBE FUNCTION - SHOW FUNCTIONS `DropFunctionStatement`, `DescribeFunctionStatement` and `ShowFunctionsStatement` logical plans are replaced with `DropFunction`, `DescribeFunction` and `ShowFunctions` logical plans respectively, and each contains `UnresolvedFunc` as its child so that it can be resolved in `Analyzer`. ### Why are the changes needed? Migrating to the new resolution framework, which resolves `UnresolvedFunc` in `Analyzer`. ### Does this PR introduce _any_ user-facing change? The message of exception thrown when a catalog is resolved to v2 has been merged to: `function is only supported in v1 catalog` Previously, it printed out the command used. E.g.,: `CREATE FUNCTION is only supported in v1 catalog` ### How was this patch tested? Updated existing tests. Closes #29198 from imback82/function_framework. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 12 ++++--- .../catalyst/plans/logical/statements.scala | 24 -------------- .../catalyst/plans/logical/v2Commands.scala | 28 ++++++++++++++++ .../sql/connector/catalog/LookupCatalog.scala | 7 ++-- .../sql/catalyst/parser/DDLParserSuite.scala | 32 +++++++++--------- .../analysis/ResolveSessionCatalog.scala | 33 ++++++++----------- .../sql/connector/DataSourceV2SQLSuite.scala | 10 +++--- .../hive/execution/HiveComparisonTest.scala | 4 +-- 9 files changed, 75 insertions(+), 77 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 13d98d8ce9b00..e58b0ae64784d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1939,7 +1939,7 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { // Resolve functions with concrete relations from v2 catalog. case UnresolvedFunc(multipartIdent) => - val funcIdent = parseSessionCatalogFunctionIdentifier(multipartIdent, "function lookup") + val funcIdent = parseSessionCatalogFunctionIdentifier(multipartIdent) ResolvedFunc(Identifier.of(funcIdent.database.toArray, funcIdent.funcName)) case q: LogicalPlan => 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 d6ae89f49c57a..fe99a8ea3cc12 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 @@ -3600,7 +3600,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } else { Seq(describeFuncName.getText) } - DescribeFunctionStatement(functionName, EXTENDED != null) + DescribeFunction(UnresolvedFunc(functionName), EXTENDED != null) } /** @@ -3615,8 +3615,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case Some(x) => throw new ParseException(s"SHOW $x FUNCTIONS not supported", ctx) } val pattern = Option(ctx.pattern).map(string(_)) - val functionName = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) - ShowFunctionsStatement(userScope, systemScope, pattern, functionName) + val unresolvedFuncOpt = Option(ctx.multipartIdentifier) + .map(visitMultipartIdentifier) + .map(UnresolvedFunc(_)) + ShowFunctions(unresolvedFuncOpt, userScope, systemScope, pattern) } /** @@ -3629,8 +3631,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitDropFunction(ctx: DropFunctionContext): LogicalPlan = withOrigin(ctx) { val functionName = visitMultipartIdentifier(ctx.multipartIdentifier) - DropFunctionStatement( - functionName, + DropFunction( + UnresolvedFunc(functionName), ctx.EXISTS != null, ctx.TEMPORARY != null) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index b1129e741221b..19831a7b5ef84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -446,30 +446,6 @@ case class ShowColumnsStatement( */ case class ShowCurrentNamespaceStatement() extends ParsedStatement -/** - * A DESCRIBE FUNCTION statement, as parsed from SQL - */ -case class DescribeFunctionStatement( - functionName: Seq[String], - isExtended: Boolean) extends ParsedStatement - -/** - * SHOW FUNCTIONS statement, as parsed from SQL - */ -case class ShowFunctionsStatement( - userScope: Boolean, - systemScope: Boolean, - pattern: Option[String], - functionName: Option[Seq[String]]) extends ParsedStatement - -/** - * DROP FUNCTION statement, as parsed from SQL - */ -case class DropFunctionStatement( - functionName: Seq[String], - ifExists: Boolean, - isTemp: Boolean) extends ParsedStatement - /** * CREATE FUNCTION statement, as parsed from SQL */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 137fc70397642..70e03c23fd115 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -523,3 +523,31 @@ case class CommentOnTable(child: LogicalPlan, comment: String) extends Command { case class RefreshFunction(child: LogicalPlan) extends Command { override def children: Seq[LogicalPlan] = child :: Nil } + +/** + * The logical plan of the DESCRIBE FUNCTION command that works for v2 catalogs. + */ +case class DescribeFunction(child: LogicalPlan, isExtended: Boolean) extends Command { + override def children: Seq[LogicalPlan] = child :: Nil +} + +/** + * The logical plan of the DROP FUNCTION command that works for v2 catalogs. + */ +case class DropFunction( + child: LogicalPlan, + ifExists: Boolean, + isTemp: Boolean) extends Command { + override def children: Seq[LogicalPlan] = child :: Nil +} + +/** + * The logical plan of the SHOW FUNCTIONS command that works for v2 catalogs. + */ +case class ShowFunctions( + child: Option[LogicalPlan], + userScope: Boolean, + systemScope: Boolean, + pattern: Option[String]) extends Command { + override def children: Seq[LogicalPlan] = child.toSeq +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index b84bf3e2786bc..d8cdecce0d172 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -156,10 +156,7 @@ private[sql] trait LookupCatalog extends Logging { } } - // TODO: move function related v2 statements to the new framework. - def parseSessionCatalogFunctionIdentifier( - nameParts: Seq[String], - sql: String): FunctionIdentifier = { + def parseSessionCatalogFunctionIdentifier(nameParts: Seq[String]): FunctionIdentifier = { if (nameParts.length == 1 && catalogManager.v1SessionCatalog.isTempFunction(nameParts.head)) { return FunctionIdentifier(nameParts.head) } @@ -179,7 +176,7 @@ private[sql] trait LookupCatalog extends Logging { } } - case _ => throw new AnalysisException(s"$sql is only supported in v1 catalog") + case _ => throw new AnalysisException("function is only supported in v1 catalog") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 02e086d5d7895..ac6af4f4e3231 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2026,40 +2026,40 @@ class DDLParserSuite extends AnalysisTest { test("DESCRIBE FUNCTION") { comparePlans( parsePlan("DESC FUNCTION a"), - DescribeFunctionStatement(Seq("a"), false)) + DescribeFunction(UnresolvedFunc(Seq("a")), false)) comparePlans( parsePlan("DESCRIBE FUNCTION a"), - DescribeFunctionStatement(Seq("a"), false)) + DescribeFunction(UnresolvedFunc(Seq("a")), false)) comparePlans( parsePlan("DESCRIBE FUNCTION a.b.c"), - DescribeFunctionStatement(Seq("a", "b", "c"), false)) + DescribeFunction(UnresolvedFunc(Seq("a", "b", "c")), false)) comparePlans( parsePlan("DESCRIBE FUNCTION EXTENDED a.b.c"), - DescribeFunctionStatement(Seq("a", "b", "c"), true)) + DescribeFunction(UnresolvedFunc(Seq("a", "b", "c")), true)) } test("SHOW FUNCTIONS") { comparePlans( parsePlan("SHOW FUNCTIONS"), - ShowFunctionsStatement(true, true, None, None)) + ShowFunctions(None, true, true, None)) comparePlans( parsePlan("SHOW USER FUNCTIONS"), - ShowFunctionsStatement(true, false, None, None)) + ShowFunctions(None, true, false, None)) comparePlans( parsePlan("SHOW user FUNCTIONS"), - ShowFunctionsStatement(true, false, None, None)) + ShowFunctions(None, true, false, None)) comparePlans( parsePlan("SHOW SYSTEM FUNCTIONS"), - ShowFunctionsStatement(false, true, None, None)) + ShowFunctions(None, false, true, None)) comparePlans( parsePlan("SHOW ALL FUNCTIONS"), - ShowFunctionsStatement(true, true, None, None)) + ShowFunctions(None, true, true, None)) comparePlans( parsePlan("SHOW FUNCTIONS LIKE 'funct*'"), - ShowFunctionsStatement(true, true, Some("funct*"), None)) + ShowFunctions(None, true, true, Some("funct*"))) comparePlans( parsePlan("SHOW FUNCTIONS LIKE a.b.c"), - ShowFunctionsStatement(true, true, None, Some(Seq("a", "b", "c")))) + ShowFunctions(Some(UnresolvedFunc(Seq("a", "b", "c"))), true, true, None)) val sql = "SHOW other FUNCTIONS" intercept(sql, s"$sql not supported") } @@ -2067,19 +2067,19 @@ class DDLParserSuite extends AnalysisTest { test("DROP FUNCTION") { comparePlans( parsePlan("DROP FUNCTION a"), - DropFunctionStatement(Seq("a"), false, false)) + DropFunction(UnresolvedFunc(Seq("a")), false, false)) comparePlans( parsePlan("DROP FUNCTION a.b.c"), - DropFunctionStatement(Seq("a", "b", "c"), false, false)) + DropFunction(UnresolvedFunc(Seq("a", "b", "c")), false, false)) comparePlans( parsePlan("DROP TEMPORARY FUNCTION a.b.c"), - DropFunctionStatement(Seq("a", "b", "c"), false, true)) + DropFunction(UnresolvedFunc(Seq("a", "b", "c")), false, true)) comparePlans( parsePlan("DROP FUNCTION IF EXISTS a.b.c"), - DropFunctionStatement(Seq("a", "b", "c"), true, false)) + DropFunction(UnresolvedFunc(Seq("a", "b", "c")), true, false)) comparePlans( parsePlan("DROP TEMPORARY FUNCTION IF EXISTS a.b.c"), - DropFunctionStatement(Seq("a", "b", "c"), true, true)) + DropFunction(UnresolvedFunc(Seq("a", "b", "c")), true, true)) } test("CREATE FUNCTION") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 5717013b2eba2..8923d5c86e19a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -577,25 +577,20 @@ class ResolveSessionCatalog( case ShowTableProperties(r: ResolvedView, propertyKey) => ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey) - case DescribeFunctionStatement(nameParts, extended) => - val functionIdent = - parseSessionCatalogFunctionIdentifier(nameParts, "DESCRIBE FUNCTION") - DescribeFunctionCommand(functionIdent, extended) - - case ShowFunctionsStatement(userScope, systemScope, pattern, fun) => - val (database, function) = fun match { - case Some(nameParts) => - val FunctionIdentifier(fn, db) = - parseSessionCatalogFunctionIdentifier(nameParts, "SHOW FUNCTIONS") - (db, Some(fn)) - case None => (None, pattern) - } - ShowFunctionsCommand(database, function, userScope, systemScope) + case DescribeFunction(ResolvedFunc(identifier), extended) => + DescribeFunctionCommand(identifier.asFunctionIdentifier, extended) + + case ShowFunctions(None, userScope, systemScope, pattern) => + ShowFunctionsCommand(None, pattern, userScope, systemScope) - case DropFunctionStatement(nameParts, ifExists, isTemp) => - val FunctionIdentifier(function, database) = - parseSessionCatalogFunctionIdentifier(nameParts, "DROP FUNCTION") - DropFunctionCommand(database, function, ifExists, isTemp) + case ShowFunctions(Some(ResolvedFunc(identifier)), userScope, systemScope, _) => + val funcIdentifier = identifier.asFunctionIdentifier + ShowFunctionsCommand( + funcIdentifier.database, Some(funcIdentifier.funcName), userScope, systemScope) + + case DropFunction(ResolvedFunc(identifier), ifExists, isTemp) => + val funcIdentifier = identifier.asFunctionIdentifier + DropFunctionCommand(funcIdentifier.database, funcIdentifier.funcName, ifExists, isTemp) case CreateFunctionStatement(nameParts, className, resources, isTemp, ignoreIfExists, replace) => @@ -618,7 +613,7 @@ class ResolveSessionCatalog( replace) } else { val FunctionIdentifier(function, database) = - parseSessionCatalogFunctionIdentifier(nameParts, "CREATE FUNCTION") + parseSessionCatalogFunctionIdentifier(nameParts) CreateFunctionCommand(database, function, className, resources, isTemp, ignoreIfExists, replace) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index d6c24e47e8652..ffc115e6b7600 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2185,7 +2185,7 @@ class DataSourceV2SQLSuite val e = intercept[AnalysisException] { sql("DESCRIBE FUNCTION testcat.ns1.ns2.fun") } - assert(e.message.contains("DESCRIBE FUNCTION is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) val e1 = intercept[AnalysisException] { sql("DESCRIBE FUNCTION default.ns1.ns2.fun") @@ -2200,14 +2200,14 @@ class DataSourceV2SQLSuite val e = intercept[AnalysisException] { sql(s"SHOW FUNCTIONS LIKE $function") } - assert(e.message.contains("SHOW FUNCTIONS is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) } test("DROP FUNCTION: only support session catalog") { val e = intercept[AnalysisException] { sql("DROP FUNCTION testcat.ns1.ns2.fun") } - assert(e.message.contains("DROP FUNCTION is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) val e1 = intercept[AnalysisException] { sql("DROP FUNCTION default.ns1.ns2.fun") @@ -2220,7 +2220,7 @@ class DataSourceV2SQLSuite val e = intercept[AnalysisException] { sql("CREATE FUNCTION testcat.ns1.ns2.fun as 'f'") } - assert(e.message.contains("CREATE FUNCTION is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) val e1 = intercept[AnalysisException] { sql("CREATE FUNCTION default.ns1.ns2.fun as 'f'") @@ -2233,7 +2233,7 @@ class DataSourceV2SQLSuite val e = intercept[AnalysisException] { sql("REFRESH FUNCTION testcat.ns1.ns2.fun") } - assert(e.message.contains("function lookup is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) val e1 = intercept[AnalysisException] { sql("REFRESH FUNCTION default.ns1.ns2.fun") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 2e4c01830432f..7f198632a1cd6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -371,8 +371,8 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && - (!hiveQuery.logical.isInstanceOf[ShowFunctionsStatement]) && - (!hiveQuery.logical.isInstanceOf[DescribeFunctionStatement]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && (!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) && From 26e6574d58429add645db820a83b70ef9dcd49fe Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Wed, 29 Jul 2020 03:58:03 +0000 Subject: [PATCH 313/384] [SPARK-32283][CORE] Kryo should support multiple user registrators ### What changes were proposed in this pull request? `spark.kryo.registrator` in 3.0 has a regression problem. From [SPARK-12080](https://issues.apache.org/jira/browse/SPARK-12080), it supports multiple user registrators by ```scala private val userRegistrators = conf.get("spark.kryo.registrator", "") .split(',').map(_.trim) .filter(!_.isEmpty) ``` But it donsn't work in 3.0. Fix it by `toSequence` in `Kryo.scala` ### Why are the changes needed? In previous Spark version (2.x), it supported multiple user registrators by ```scala private val userRegistrators = conf.get("spark.kryo.registrator", "") .split(',').map(_.trim) .filter(!_.isEmpty) ``` But it doesn't work in 3.0. It's should be a regression. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existed unit tests. Closes #29123 from LantaoJin/SPARK-32283. Authored-by: LantaoJin Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/internal/config/Kryo.scala | 3 ++- .../test/scala/org/apache/spark/SparkConfSuite.scala | 2 +- .../org/apache/spark/serializer/KryoBenchmark.scala | 2 +- .../spark/serializer/KryoSerializerBenchmark.scala | 2 +- .../serializer/KryoSerializerDistributedSuite.scala | 2 +- .../apache/spark/serializer/KryoSerializerSuite.scala | 10 +++++----- .../spark/serializer/SerializerPropertiesSuite.scala | 2 +- .../spark/sql/DatasetSerializerRegistratorSuite.scala | 2 +- 8 files changed, 13 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala index 646d8556538c2..90c59b079461c 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala @@ -29,7 +29,8 @@ private[spark] object Kryo { val KRYO_USER_REGISTRATORS = ConfigBuilder("spark.kryo.registrator") .version("0.5.0") .stringConf - .createOptional + .toSequence + .createWithDefault(Nil) val KRYO_CLASSES_TO_REGISTER = ConfigBuilder("spark.kryo.classesToRegister") .version("1.2.0") diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 3bc2061c4f2ad..72e7ee0214187 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -221,7 +221,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst conf.registerKryoClasses(Array(classOf[Class1])) assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === Seq(classOf[Class1].getName).toSet) - conf.set(KRYO_USER_REGISTRATORS, classOf[CustomRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[CustomRegistrator].getName)) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala index fd228cded783a..525e682dd5d42 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -125,7 +125,7 @@ object KryoBenchmark extends BenchmarkBase { def createSerializer(useUnsafe: Boolean): SerializerInstance = { val conf = new SparkConf() conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[MyRegistrator].getName)) conf.set(KRYO_USE_UNSAFE, useUnsafe) new KryoSerializer(conf).newInstance() diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala index 953b651c72a83..dde0c98704d00 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala @@ -76,7 +76,7 @@ object KryoSerializerBenchmark extends BenchmarkBase { conf.set(EXECUTOR_EXTRA_JAVA_OPTIONS, "-XX:+UseParallelGC -XX:-UseDynamicNumberOfGCThreads") conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[MyRegistrator].getName)) conf.set(KRYO_USE_POOL, usePool) if (sc != null) { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index d4fafab4a5d64..397fdce8ae6e3 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -29,7 +29,7 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) .set(config.SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - .set(config.Kryo.KRYO_USER_REGISTRATORS, classOf[AppJarRegistrator].getName) + .set(config.Kryo.KRYO_USER_REGISTRATORS, Seq(classOf[AppJarRegistrator].getName)) .set(config.TASK_MAX_FAILURES, 1) .set(config.BLACKLIST_ENABLED, false) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 4c47a67ee9ffc..229ef69973775 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.ThreadUtils class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[MyRegistrator].getName)) conf.set(KRYO_USE_UNSAFE, false) test("SPARK-7392 configuration limits") { @@ -313,7 +313,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { import org.apache.spark.SparkException val conf = new SparkConf(false) - conf.set(KRYO_USER_REGISTRATORS, "this.class.does.not.exist") + conf.set(KRYO_USER_REGISTRATORS, Seq("this.class.does.not.exist")) val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance().serialize(1)) assert(thrown.getMessage.contains("Failed to register classes with Kryo")) @@ -412,7 +412,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance] assert(ser.getAutoReset) val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, - classOf[RegistratorWithoutAutoReset].getName) + Seq(classOf[RegistratorWithoutAutoReset].getName)) val ser2 = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance] assert(!ser2.getAutoReset) } @@ -443,7 +443,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { .set(KRYO_REFERENCE_TRACKING, referenceTracking) .set(KRYO_USE_POOL, usePool) if (!autoReset) { - conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[RegistratorWithoutAutoReset].getName)) } val ser = new KryoSerializer(conf) val serInstance = ser.newInstance().asInstanceOf[KryoSerializerInstance] @@ -530,7 +530,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSparkContext { conf.set(SERIALIZER, classOf[KryoSerializer].getName) - conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[RegistratorWithoutAutoReset].getName)) conf.set(KRYO_REFERENCE_TRACKING, true) conf.set(SHUFFLE_MANAGER, "sort") conf.set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 200) diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala index dad080c5fc161..9747f5780dd1e 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala @@ -52,7 +52,7 @@ class SerializerPropertiesSuite extends SparkFunSuite { test("KryoSerializer does not support relocation when auto-reset is disabled") { val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, - classOf[RegistratorWithoutAutoReset].getName) + Seq(classOf[RegistratorWithoutAutoReset].getName)) val ser = new KryoSerializer(conf) assert(!ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()) testSupportsRelocationOfSerializedObjects(ser, generateRandomItem) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala index 43de2663b1236..b20d050f2fc4a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala @@ -34,7 +34,7 @@ class DatasetSerializerRegistratorSuite extends QueryTest with SharedSparkSessio override protected def sparkConf: SparkConf = { // Make sure we use the KryoRegistrator - super.sparkConf.set(KRYO_USER_REGISTRATORS, TestRegistrator().getClass.getCanonicalName) + super.sparkConf.set(KRYO_USER_REGISTRATORS, Seq(TestRegistrator().getClass.getCanonicalName)) } test("Kryo registrator") { From 77987a222c8edf52a2a813d315cfe429ed1c9c9c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 28 Jul 2020 22:31:39 -0700 Subject: [PATCH 314/384] [SPARK-32473][CORE][TESTS] Use === instead IndexSeqView ### What changes were proposed in this pull request? This PR aims to fix `SorterSuite` and `RadixSortSuite` in Scala 2.13 by using `===` instead of `IndexSeqView`. ``` $ git grep "\.view ==" core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala: assert(data0.view === data1.view) core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala: assert(data0.view === data2.view) core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala: assert(ref.view == result.view) core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala: assert(res1.view == res2.view) core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala: assert(ref.view == result.view) core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala: assert(res1.view == res2.view) ``` ### Why are the changes needed? Scala 2.13 reimplements `IndexSeqView` and the behavior is different. - https://docs.scala-lang.org/overviews/core/collections-migration-213.html **Scala 2.12** ```scala Welcome to Scala 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_262). Type in expressions for evaluation. Or try :help. scala> Seq(1,2,3).toArray.view == Seq(1,2,3).toArray.view res0: Boolean = true ``` **Scala 2.13** ```scala Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262). Type in expressions for evaluation. Or try :help. scala> Seq(1,2,3).toArray.view == Seq(1,2,3).toArray.view val res0: Boolean = false ``` ### Does this PR introduce _any_ user-facing change? No. This is a test-only fix. ### How was this patch tested? - Scala 2.12: Pass the GitHub Action or Jenkins. - Scala 2.13: Manually test the following. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.collection.unsafe.sort.RadixSortSuite $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.collection.SorterSuite ``` **BEFORE** ``` Tests: succeeded 9, failed 36, canceled 0, ignored 0, pending 0 *** 36 TESTS FAILED *** Tests: succeeded 3, failed 1, canceled 0, ignored 2, pending 0 *** 1 TEST FAILED *** ``` **AFTER** ``` Tests: succeeded 45, failed 0, canceled 0, ignored 0, pending 0 All tests passed. Tests: succeeded 4, failed 0, canceled 0, ignored 2, pending 0 All tests passed. ``` Closes #29280 from dongjoon-hyun/SPARK-32473. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/util/collection/SorterSuite.scala | 4 ++-- .../util/collection/unsafe/sort/RadixSortSuite.scala | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index bb03f0d3cdc20..82b4f2eac6248 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -39,8 +39,8 @@ class SorterSuite extends SparkFunSuite with Logging { new Sorter(new KeyReuseIntArraySortDataFormat) .sort(data2, 0, data2.length, Ordering[IntWrapper]) - assert(data0.view === data1.view) - assert(data0.view === data2.view) + assert(data0 === data1) + assert(data0 === data2) } test("KVArraySorter") { diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 9ae6a8ef879f3..b3e5e0a73dd9f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -157,7 +157,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { buffer, N, sortType.startByteIdx, sortType.endByteIdx, sortType.descending, sortType.signed) val result = collectToArray(buffer, outOffset, N) - assert(ref.view == result.view) + assert(ref === result) } test("sort key prefix " + sortType.name) { @@ -169,7 +169,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { sortType.descending, sortType.signed) val res1 = collectToArray(buf1, 0, N * 2) val res2 = collectToArray(buf2, outOffset, N * 2) - assert(res1.view == res2.view) + assert(res1 === res2) } fuzzTest(s"fuzz test ${sortType.name} with random bitmasks") { seed => @@ -181,7 +181,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { buffer, N, sortType.startByteIdx, sortType.endByteIdx, sortType.descending, sortType.signed) val result = collectToArray(buffer, outOffset, N) - assert(ref.view == result.view) + assert(ref === result) } fuzzTest(s"fuzz test key prefix ${sortType.name} with random bitmasks") { seed => @@ -194,7 +194,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { sortType.descending, sortType.signed) val res1 = collectToArray(buf1, 0, N * 2) val res2 = collectToArray(buf2, outOffset, N * 2) - assert(res1.view == res2.view) + assert(res1 ===res2) } } } From 9be088357eff4328248b29a3a49a816756745345 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 29 Jul 2020 08:44:56 -0500 Subject: [PATCH 315/384] [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread ### What changes were proposed in this pull request? This PR changes the order between initialization for ExecutorPlugin and starting heartbeat thread in Executor. ### Why are the changes needed? In the current master, heartbeat thread in a executor starts after plugin initialization so if the initialization takes long time, heartbeat is not sent to driver and the executor will be removed from cluster. ### Does this PR introduce _any_ user-facing change? Yes. Plugins for executors will be allowed to take long time for initialization. ### How was this patch tested? New testcase. Closes #29002 from sarutak/fix-heartbeat-issue. Authored-by: Kousuke Saruta Signed-off-by: Thomas Graves --- .../scala/org/apache/spark/TestUtils.scala | 15 +++- .../org/apache/spark/executor/Executor.scala | 12 ++-- .../apache/spark/executor/ExecutorSuite.scala | 72 ++++++++++++++++++- 3 files changed, 89 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 259cc43cdfdbb..6947d1c72f12b 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -179,11 +179,20 @@ private[spark] object TestUtils { destDir: File, toStringValue: String = "", baseClass: String = null, - classpathUrls: Seq[URL] = Seq.empty): File = { + classpathUrls: Seq[URL] = Seq.empty, + implementsClasses: Seq[String] = Seq.empty, + extraCodeBody: String = ""): File = { val extendsText = Option(baseClass).map { c => s" extends ${c}" }.getOrElse("") + val implementsText = + "implements " + (implementsClasses :+ "java.io.Serializable").mkString(", ") val sourceFile = new JavaSourceFromString(className, - "public class " + className + extendsText + " implements java.io.Serializable {" + - " @Override public String toString() { return \"" + toStringValue + "\"; }}") + s""" + |public class $className $extendsText $implementsText { + | @Override public String toString() { return "$toStringValue"; } + | + | $extraCodeBody + |} + """.stripMargin) createCompiledClass(className, destDir, sourceFile, classpathUrls) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bc0f0c0a7b705..d22002917472a 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -154,11 +154,6 @@ private[spark] class Executor( // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. env.serializerManager.setDefaultClassLoader(replClassLoader) - // Plugins need to load using a class loader that includes the executor's user classpath - private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { - PluginContainer(env, resources.asJava) - } - // Max size of direct result. If task result is bigger than this, we use the block manager // to send the result back. private val maxDirectResultSize = Math.min( @@ -225,6 +220,13 @@ private[spark] class Executor( heartbeater.start() + // Plugins need to load using a class loader that includes the executor's user classpath. + // Plugins also needs to be initialized after the heartbeater started + // to avoid blocking to send heartbeat (see SPARK-32175). + private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { + PluginContainer(env, resources.asJava) + } + metricsPoller.start() private[executor] def numRunningTasks: Int = runningTasks.size() diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 31049d104e63d..b198448a299cf 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.io.{Externalizable, File, ObjectInput, ObjectOutput} import java.lang.Thread.UncaughtExceptionHandler import java.nio.ByteBuffer import java.util.Properties @@ -41,6 +41,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.broadcast.Broadcast +import org.apache.spark.deploy.{SimpleApplicationTest, SparkSubmitSuite} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.memory.TestMemoryManager @@ -52,7 +53,7 @@ import org.apache.spark.scheduler.{DirectTaskResult, FakeTask, ResultTask, Task, import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockManager, BlockManagerId} -import org.apache.spark.util.{LongAccumulator, UninterruptibleThread} +import org.apache.spark.util.{LongAccumulator, UninterruptibleThread, Utils} class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar with Eventually with PrivateMethodTester { @@ -402,6 +403,73 @@ class ExecutorSuite extends SparkFunSuite assert(taskMetrics.getMetricValue("JVMHeapMemory") > 0) } + test("SPARK-32175: Plugin initialization should start after heartbeater started") { + withTempDir { tempDir => + val sparkPluginCodeBody = + """ + |@Override + |public org.apache.spark.api.plugin.ExecutorPlugin executorPlugin() { + | return new TestExecutorPlugin(); + |} + | + |@Override + |public org.apache.spark.api.plugin.DriverPlugin driverPlugin() { return null; } + """.stripMargin + val executorPluginBody = + """ + |@Override + |public void init( + | org.apache.spark.api.plugin.PluginContext ctx, + | java.util.Map extraConf) { + | try { + | Thread.sleep(8 * 1000); + | } catch (InterruptedException e) { + | throw new RuntimeException(e); + | } + |} + """.stripMargin + + val compiledExecutorPlugin = TestUtils.createCompiledClass( + "TestExecutorPlugin", + tempDir, + "", + null, + Seq.empty, + Seq("org.apache.spark.api.plugin.ExecutorPlugin"), + executorPluginBody) + + val thisClassPath = + sys.props("java.class.path").split(File.pathSeparator).map(p => new File(p).toURI.toURL) + val compiledSparkPlugin = TestUtils.createCompiledClass( + "TestSparkPlugin", + tempDir, + "", + null, + Seq(tempDir.toURI.toURL) ++ thisClassPath, + Seq("org.apache.spark.api.plugin.SparkPlugin"), + sparkPluginCodeBody) + + val jarUrl = TestUtils.createJar( + Seq(compiledSparkPlugin, compiledExecutorPlugin), + new File(tempDir, "testPlugin.jar")) + + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[1,1,1024]", + "--conf", "spark.plugins=TestSparkPlugin", + "--conf", "spark.storage.blockManagerSlaveTimeoutMs=" + 5 * 1000, + "--conf", "spark.network.timeoutInterval=" + 1000, + "--conf", "spark.executor.heartbeatInterval=" + 1000, + "--conf", "spark.executor.extraClassPath=" + jarUrl.toString, + "--conf", "spark.driver.extraClassPath=" + jarUrl.toString, + "--conf", "spark.ui.enabled=false", + unusedJar.toString) + SparkSubmitSuite.runSparkSubmit(args, timeout = 30.seconds) + } + } + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer): SparkEnv = { val mockEnv = mock[SparkEnv] val mockRpcEnv = mock[RpcEnv] From 5eab8d27e686fac5688ba4482599e3652ec17882 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 29 Jul 2020 07:48:23 -0700 Subject: [PATCH 316/384] [SPARK-32477][CORE] JsonProtocol.accumulablesToJson should be deterministic ### What changes were proposed in this pull request? This PR aims to make `JsonProtocol.accumulablesToJson` deterministic. ### Why are the changes needed? Currently, `JsonProtocol.accumulablesToJson` is indeterministic. So, `JsonProtocolSuite` itself is also using mixed test cases in terms of `"Accumulables": [ ... ]`. Not only this is indeterministic, but also this causes a UT failure in `JsonProtocolSuite` in Scala 2.13. ### Does this PR introduce _any_ user-facing change? Yes. However, this is a fix on indeterministic behavior. ### How was this patch tested? - Scala 2.12: Pass the GitHub Action or Jenkins. - Scala 2.13: Do the following. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.JsonProtocolSuite ``` **BEFORE** ``` *** 1 TEST FAILED *** ``` **AFTER** ``` All tests passed. ``` Closes #29282 from dongjoon-hyun/SPARK-32477. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 96 +++++++++---------- 2 files changed, 49 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index ceaddb4306579..13f7cb453346f 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -333,7 +333,7 @@ private[spark] object JsonProtocol { def accumulablesToJson(accumulables: Iterable[AccumulableInfo]): JArray = { JArray(accumulables .filterNot(_.name.exists(accumulableExcludeList.contains)) - .toList.map(accumulableInfoToJson)) + .toList.sortBy(_.id).map(accumulableInfoToJson)) } def accumulableInfoToJson(accumulableInfo: AccumulableInfo): JValue = { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index c75e98f39758d..f3ed233da7122 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1111,20 +1111,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1170,20 +1170,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1694,20 +1694,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1757,20 +1757,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1837,20 +1837,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1934,20 +1934,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 From 40e6a5bbb0dbedae4a270f830aafd4cb310a8fe2 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 29 Jul 2020 09:58:25 -0500 Subject: [PATCH 317/384] [SPARK-32449][ML][PYSPARK] Add summary to MultilayerPerceptronClassificationModel ### What changes were proposed in this pull request? Add training summary to MultilayerPerceptronClassificationModel... ### Why are the changes needed? so that user can get the training process status, such as loss value of each iteration and total iteration number. ### Does this PR introduce _any_ user-facing change? Yes MultilayerPerceptronClassificationModel.summary MultilayerPerceptronClassificationModel.evaluate ### How was this patch tested? new tests Closes #29250 from huaxingao/mlp_summary. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../scala/org/apache/spark/ml/ann/Layer.scala | 11 ++- .../MultilayerPerceptronClassifier.scala | 93 ++++++++++++++++++- .../org/apache/spark/ml/ann/ANNSuite.scala | 4 +- .../MultilayerPerceptronClassifierSuite.scala | 32 +++++++ python/docs/source/reference/pyspark.ml.rst | 2 + python/pyspark/ml/classification.py | 49 +++++++++- .../pyspark/ml/tests/test_training_summary.py | 45 ++++++++- 7 files changed, 222 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 2b4b0fc55b955..253d4083de7d4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -838,7 +838,7 @@ private[ml] class FeedForwardTrainer( * @param data RDD of input and output vector pairs * @return model */ - def train(data: RDD[(Vector, Vector)]): TopologyModel = { + def train(data: RDD[(Vector, Vector)]): (TopologyModel, Array[Double]) = { val w = if (getWeights == null) { // TODO: will make a copy if vector is a subvector of BDV (see Vectors code) topology.model(_seed).weights @@ -851,9 +851,14 @@ private[ml] class FeedForwardTrainer( } val handlePersistence = trainData.getStorageLevel == StorageLevel.NONE if (handlePersistence) trainData.persist(StorageLevel.MEMORY_AND_DISK) - val newWeights = optimizer.optimize(trainData, w) + val (newWeights, lossHistory) = optimizer match { + case lbfgs: LBFGS => lbfgs.optimizeWithLossReturned(trainData, w) + case sgd: GradientDescent => sgd.optimizeWithLossReturned(trainData, w) + case other => throw new UnsupportedOperationException( + s"Only LBFGS and GradientDescent are supported but got ${other.getClass}.") + } if (handlePersistence) trainData.unpersist() - topology.model(newWeights) + (topology.model(newWeights), lossHistory) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index d23fafb4d79e5..58fc53517c9b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -27,7 +27,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql._ import org.apache.spark.util.VersionUtils.majorMinorVersion /** Params for Multilayer Perceptron. */ @@ -225,8 +225,24 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( s"The solver $solver is not supported by MultilayerPerceptronClassifier.") } trainer.setStackSize($(blockSize)) - val mlpModel = trainer.train(data) - new MultilayerPerceptronClassificationModel(uid, mlpModel.weights) + val (mlpModel, objectiveHistory) = trainer.train(data) + createModel(dataset, mlpModel.weights, objectiveHistory) + } + + private def createModel( + dataset: Dataset[_], + weights: Vector, + objectiveHistory: Array[Double]): MultilayerPerceptronClassificationModel = { + val model = copyValues(new MultilayerPerceptronClassificationModel(uid, weights)) + + val (summaryModel, _, predictionColName) = model.findSummaryModel() + val summary = new MultilayerPerceptronClassificationTrainingSummaryImpl( + summaryModel.transform(dataset), + predictionColName, + $(labelCol), + "", + objectiveHistory) + model.setSummary(Some(summary)) } } @@ -259,7 +275,8 @@ class MultilayerPerceptronClassificationModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("2.0.0") val weights: Vector) extends ProbabilisticClassificationModel[Vector, MultilayerPerceptronClassificationModel] - with MultilayerPerceptronParams with Serializable with MLWritable { + with MultilayerPerceptronParams with Serializable with MLWritable + with HasTrainingSummary[MultilayerPerceptronClassificationTrainingSummary]{ @Since("1.6.0") override lazy val numFeatures: Int = $(layers).head @@ -268,6 +285,26 @@ class MultilayerPerceptronClassificationModel private[ml] ( .multiLayerPerceptron($(layers), softmaxOnTop = true) .model(weights) + /** + * Gets summary of model on training set. An exception is thrown + * if `hasSummary` is false. + */ + @Since("3.1.0") + override def summary: MultilayerPerceptronClassificationTrainingSummary = super.summary + + /** + * Evaluates the model on a test dataset. + * + * @param dataset Test dataset to evaluate model on. + */ + @Since("3.1.0") + def evaluate(dataset: Dataset[_]): MultilayerPerceptronClassificationSummary = { + // Handle possible missing or invalid probability or prediction columns + val (summaryModel, _, predictionColName) = findSummaryModel() + new MultilayerPerceptronClassificationSummaryImpl(summaryModel.transform(dataset), + predictionColName, $(labelCol), "") + } + /** * Predict label for the given features. * This internal method is used to implement `transform()` and output [[predictionCol]]. @@ -359,3 +396,51 @@ object MultilayerPerceptronClassificationModel } } } + + +/** + * Abstraction for MultilayerPerceptronClassification results for a given model. + */ +sealed trait MultilayerPerceptronClassificationSummary extends ClassificationSummary + +/** + * Abstraction for MultilayerPerceptronClassification training results. + */ +sealed trait MultilayerPerceptronClassificationTrainingSummary + extends MultilayerPerceptronClassificationSummary with TrainingSummary + +/** + * MultilayerPerceptronClassification training results. + * + * @param predictions dataframe output by the model's `transform` method. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. + */ +private class MultilayerPerceptronClassificationTrainingSummaryImpl( + predictions: DataFrame, + predictionCol: String, + labelCol: String, + weightCol: String, + override val objectiveHistory: Array[Double]) + extends MultilayerPerceptronClassificationSummaryImpl( + predictions, predictionCol, labelCol, weightCol) + with MultilayerPerceptronClassificationTrainingSummary + +/** + * MultilayerPerceptronClassification results for a given model. + * + * @param predictions dataframe output by the model's `transform` method. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + */ +private class MultilayerPerceptronClassificationSummaryImpl( + @transient override val predictions: DataFrame, + override val predictionCol: String, + override val labelCol: String, + override val weightCol: String) + extends MultilayerPerceptronClassificationSummary diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala index 35586320cb82b..fdd6e352fa639 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala @@ -45,7 +45,7 @@ class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { val trainer = new FeedForwardTrainer(topology, 2, 1) trainer.setWeights(initialWeights) trainer.LBFGSOptimizer.setNumIterations(20) - val model = trainer.train(rddData) + val (model, _) = trainer.train(rddData) val predictionAndLabels = rddData.map { case (input, label) => (model.predict(input)(0), label(0)) }.collect() @@ -80,7 +80,7 @@ class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { // TODO: add a test for SGD trainer.LBFGSOptimizer.setConvergenceTol(1e-4).setNumIterations(20) trainer.setWeights(initialWeights).setStackSize(1) - val model = trainer.train(rddData) + val (model, _) = trainer.train(rddData) val predictionAndLabels = rddData.map { case (input, label) => (model.predict(input), label) }.collect() diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index 902af71e42f86..c909e72c689bc 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -242,4 +242,36 @@ class MultilayerPerceptronClassifierSuite extends MLTest with DefaultReadWriteTe val sparkVersionStr = metadata.select("sparkVersion").first().getString(0) assert(sparkVersionStr == "2.4.4") } + + test("summary and training summary") { + val mlp = new MultilayerPerceptronClassifier() + val model = mlp.setMaxIter(5).setLayers(Array(2, 3, 2)).fit(dataset) + val summary = model.evaluate(dataset) + + assert(model.summary.truePositiveRateByLabel === summary.truePositiveRateByLabel) + assert(model.summary.falsePositiveRateByLabel === summary.falsePositiveRateByLabel) + assert(model.summary.precisionByLabel === summary.precisionByLabel) + assert(model.summary.recallByLabel === summary.recallByLabel) + assert(model.summary.fMeasureByLabel === summary.fMeasureByLabel) + assert(model.summary.accuracy === summary.accuracy) + assert(model.summary.weightedFalsePositiveRate === summary.weightedFalsePositiveRate) + assert(model.summary.weightedTruePositiveRate === summary.weightedTruePositiveRate) + assert(model.summary.weightedPrecision === summary.weightedPrecision) + assert(model.summary.weightedRecall === summary.weightedRecall) + assert(model.summary.weightedFMeasure === summary.weightedFMeasure) + } + + test("MultilayerPerceptron training summary totalIterations") { + Seq(1, 5, 10, 20, 100).foreach { maxIter => + val trainer = new MultilayerPerceptronClassifier() + .setMaxIter(maxIter) + .setLayers(Array(2, 3, 2)) + val model = trainer.fit(dataset) + if (maxIter == 1) { + assert(model.summary.totalIterations === maxIter) + } else { + assert(model.summary.totalIterations <= maxIter) + } + } + } } diff --git a/python/docs/source/reference/pyspark.ml.rst b/python/docs/source/reference/pyspark.ml.rst index b6e7d10276603..00ee7b9078329 100644 --- a/python/docs/source/reference/pyspark.ml.rst +++ b/python/docs/source/reference/pyspark.ml.rst @@ -153,6 +153,8 @@ Classification NaiveBayesModel MultilayerPerceptronClassifier MultilayerPerceptronClassificationModel + MultilayerPerceptronClassificationSummary + MultilayerPerceptronClassificationTrainingSummary OneVsRest OneVsRestModel FMClassifier diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index e192e8c252d50..3bc9dc0628aa8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -51,6 +51,8 @@ 'BinaryRandomForestClassificationTrainingSummary', 'NaiveBayes', 'NaiveBayesModel', 'MultilayerPerceptronClassifier', 'MultilayerPerceptronClassificationModel', + 'MultilayerPerceptronClassificationSummary', + 'MultilayerPerceptronClassificationTrainingSummary', 'OneVsRest', 'OneVsRestModel', 'FMClassifier', 'FMClassificationModel', 'FMClassificationSummary', 'FMClassificationTrainingSummary'] @@ -2622,7 +2624,7 @@ def setSolver(self, value): class MultilayerPerceptronClassificationModel(_JavaProbabilisticClassificationModel, _MultilayerPerceptronParams, JavaMLWritable, - JavaMLReadable): + JavaMLReadable, HasTrainingSummary): """ Model fitted by MultilayerPerceptronClassifier. @@ -2637,6 +2639,51 @@ def weights(self): """ return self._call_java("weights") + @since("3.1.0") + def summary(self): + """ + Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + trained on the training set. An exception is thrown if `trainingSummary is None`. + """ + if self.hasSummary: + return MultilayerPerceptronClassificationTrainingSummary( + super(MultilayerPerceptronClassificationModel, self).summary) + else: + raise RuntimeError("No training summary available for this %s" % + self.__class__.__name__) + + @since("3.1.0") + def evaluate(self, dataset): + """ + Evaluates the model on a test dataset. + + :param dataset: + Test dataset to evaluate model on, where dataset is an + instance of :py:class:`pyspark.sql.DataFrame` + """ + if not isinstance(dataset, DataFrame): + raise ValueError("dataset must be a DataFrame but got %s." % type(dataset)) + java_mlp_summary = self._call_java("evaluate", dataset) + return MultilayerPerceptronClassificationSummary(java_mlp_summary) + + +class MultilayerPerceptronClassificationSummary(_ClassificationSummary): + """ + Abstraction for MultilayerPerceptronClassifier Results for a given model. + .. versionadded:: 3.1.0 + """ + pass + + +@inherit_doc +class MultilayerPerceptronClassificationTrainingSummary(MultilayerPerceptronClassificationSummary, + _TrainingSummary): + """ + Abstraction for MultilayerPerceptronClassifier Training results. + .. versionadded:: 3.1.0 + """ + pass + class _OneVsRestParams(_ClassifierParams, HasWeightCol): """ diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index d305be8b96cd4..6b05ffaa7d52f 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -18,11 +18,12 @@ import sys import unittest -from pyspark.ml.classification import BinaryLogisticRegressionSummary, FMClassifier, \ +from pyspark.ml.classification import BinaryLogisticRegressionSummary, \ + BinaryRandomForestClassificationSummary, FMClassifier, \ FMClassificationSummary, LinearSVC, LinearSVCSummary, \ - BinaryRandomForestClassificationSummary, LogisticRegression, \ - LogisticRegressionSummary, RandomForestClassificationSummary, \ - RandomForestClassifier + LogisticRegression, LogisticRegressionSummary, \ + MultilayerPerceptronClassifier, MultilayerPerceptronClassificationSummary, \ + RandomForestClassificationSummary, RandomForestClassifier from pyspark.ml.clustering import BisectingKMeans, GaussianMixture, KMeans from pyspark.ml.linalg import Vectors from pyspark.ml.regression import GeneralizedLinearRegression, LinearRegression @@ -354,6 +355,42 @@ def test_fm_classification_summary(self): self.assertTrue(isinstance(sameSummary, FMClassificationSummary)) self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) + def test_mlp_classification_summary(self): + df = self.spark.createDataFrame([(0.0, Vectors.dense([0.0, 0.0])), + (1.0, Vectors.dense([0.0, 1.0])), + (1.0, Vectors.dense([1.0, 0.0])), + (0.0, Vectors.dense([1.0, 1.0])) + ], + ["label", "features"]) + mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], seed=123) + model = mlp.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary() + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.predictionCol, "prediction") + self.assertGreater(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertAlmostEqual(s.accuracy, 1.0, 2) + self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2) + self.assertAlmostEqual(s.weightedRecall, 1.0, 2) + self.assertAlmostEqual(s.weightedPrecision, 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2) + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, MultilayerPerceptronClassificationSummary)) + self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) + def test_gaussian_mixture_summary(self): data = [(Vectors.dense(1.0),), (Vectors.dense(5.0),), (Vectors.dense(10.0),), (Vectors.sparse(1, [], []),)] From d897825d2d0430cb52ae9ac0f6fd742582041682 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 30 Jul 2020 01:37:42 +0800 Subject: [PATCH 318/384] [SPARK-32346][SQL] Support filters pushdown in Avro datasource ### What changes were proposed in this pull request? In the PR, I propose to support pushed down filters in Avro datasource V1 and V2. 1. Added new SQL config `spark.sql.avro.filterPushdown.enabled` to control filters pushdown to Avro datasource. It is on by default. 2. Renamed `CSVFilters` to `OrderedFilters`. 3. `OrderedFilters` is used in `AvroFileFormat` (DSv1) and in `AvroPartitionReaderFactory` (DSv2) 4. Modified `AvroDeserializer` to return None from the `deserialize` method when pushdown filters return `false`. ### Why are the changes needed? The changes improve performance on synthetic benchmarks up to **2** times on JDK 11: ``` 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 9614 9669 54 0.1 9614.1 1.0X pushdown disabled 10077 10141 66 0.1 10077.2 1.0X w/ filters 4681 4713 29 0.2 4681.5 2.1X ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? - Added UT to `AvroCatalystDataConversionSuite` and `AvroSuite` - Re-running `AvroReadBenchmark` using Amazon EC2: | Item | Description | | ---- | ----| | Region | us-west-2 (Oregon) | | Instance | r3.xlarge (spot instance) | | AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) | | Java | OpenJDK8/11 installed by`sudo add-apt-repository ppa:openjdk-r/ppa` & `sudo apt install openjdk-11-jdk`| and `./dev/run-benchmarks`: ```python #!/usr/bin/env python3 import os from sparktestsupport.shellutils import run_cmd benchmarks = [ ['avro/test', 'org.apache.spark.sql.execution.benchmark.AvroReadBenchmark'] ] 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 #29145 from MaxGekk/avro-filters-pushdown. Authored-by: Max Gekk Signed-off-by: Gengliang Wang --- .../AvroReadBenchmark-jdk11-results.txt | 72 ++++++++++--------- .../benchmarks/AvroReadBenchmark-results.txt | 72 ++++++++++--------- .../spark/sql/avro/AvroDataToCatalyst.scala | 5 +- .../spark/sql/avro/AvroDeserializer.scala | 40 +++++++---- .../spark/sql/avro/AvroFileFormat.scala | 44 +++++------- .../org/apache/spark/sql/avro/AvroUtils.scala | 36 +++++++++- .../v2/avro/AvroPartitionReaderFactory.scala | 48 ++++++------- .../apache/spark/sql/v2/avro/AvroScan.scala | 24 +++++-- .../spark/sql/v2/avro/AvroScanBuilder.scala | 27 ++++++- .../AvroCatalystDataConversionSuite.scala | 71 ++++++++++++++---- .../org/apache/spark/sql/avro/AvroSuite.scala | 34 ++++++++- .../benchmark/AvroReadBenchmark.scala | 64 ++++++++++++++++- .../CSVFilters.scala => OrderedFilters.scala} | 68 +++++++++--------- .../sql/catalyst/csv/UnivocityParser.scala | 9 ++- .../apache/spark/sql/internal/SQLConf.scala | 8 +++ ...sSuite.scala => OrderedFiltersSuite.scala} | 7 +- 16 files changed, 430 insertions(+), 199 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/{csv/CSVFilters.scala => OrderedFilters.scala} (60%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/{csv/CSVFiltersSuite.scala => OrderedFiltersSuite.scala} (83%) diff --git a/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt b/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt index 3c1b5af0d5986..b70b1446f6f64 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt @@ -2,121 +2,129 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2689 2694 7 5.8 170.9 1.0X +Sum 2872 2936 90 5.5 182.6 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2741 2759 26 5.7 174.2 1.0X +Sum 2810 2838 40 5.6 178.6 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2736 2748 17 5.7 173.9 1.0X +Sum 2901 2922 30 5.4 184.4 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3305 3317 17 4.8 210.2 1.0X +Sum 3387 3391 5 4.6 215.4 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2904 2952 68 5.4 184.6 1.0X +Sum 2890 2960 99 5.4 183.7 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3090 3093 4 5.1 196.5 1.0X +Sum 3067 3088 30 5.1 195.0 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 5351 5365 20 2.0 510.3 1.0X +Sum of columns 4736 4818 116 2.2 451.7 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 3278 3288 14 4.8 208.4 1.0X -Partition column 3149 3193 62 5.0 200.2 1.0X -Both columns 3198 3204 7 4.9 203.4 1.0X +Data column 3383 3400 23 4.6 215.1 1.0X +Partition column 2949 2959 14 5.3 187.5 1.1X +Both columns 3522 3545 33 4.5 223.9 1.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3435 3438 5 3.1 327.6 1.0X +Sum of string length 3332 3355 32 3.1 317.7 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 5634 5650 23 1.9 537.3 1.0X +Sum of string length 5588 5652 90 1.9 532.9 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 4725 4752 39 2.2 450.6 1.0X +Sum of string length 3858 3865 9 2.7 368.0 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3550 3566 23 3.0 338.6 1.0X +Sum of string length 2562 2571 12 4.1 244.3 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 5271 5279 11 0.2 5027.0 1.0X +Sum of single column 5241 5243 3 0.2 4998.0 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 10393 10516 174 0.1 9911.3 1.0X +Sum of single column 10178 10185 10 0.1 9706.5 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +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 Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 15330 15343 19 0.1 14619.6 1.0X +Sum of single column 15201 15232 44 0.1 14496.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 +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 9614 9669 54 0.1 9614.1 1.0X +pushdown disabled 10077 10141 66 0.1 10077.2 1.0X +w/ filters 4681 4713 29 0.2 4681.5 2.1X + diff --git a/external/avro/benchmarks/AvroReadBenchmark-results.txt b/external/avro/benchmarks/AvroReadBenchmark-results.txt index 0ab611a0f9a29..3108a9c8e13fe 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-results.txt @@ -2,121 +2,129 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3049 3071 32 5.2 193.8 1.0X +Sum 2841 2846 7 5.5 180.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2982 2992 13 5.3 189.6 1.0X +Sum 2777 2799 30 5.7 176.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2984 2989 7 5.3 189.7 1.0X +Sum 2730 2753 33 5.8 173.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3262 3353 128 4.8 207.4 1.0X +Sum 3278 3284 9 4.8 208.4 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2716 2723 10 5.8 172.7 1.0X +Sum 2801 2805 6 5.6 178.1 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2868 2870 3 5.5 182.4 1.0X +Sum 2976 2984 12 5.3 189.2 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 4714 4739 35 2.2 449.6 1.0X +Sum of columns 4674 4686 17 2.2 445.8 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 3257 3286 41 4.8 207.1 1.0X -Partition column 3258 3277 27 4.8 207.2 1.0X -Both columns 3399 3405 9 4.6 216.1 1.0X +Data column 3273 3284 17 4.8 208.1 1.0X +Partition column 2934 2935 2 5.4 186.6 1.1X +Both columns 3395 3405 14 4.6 215.8 1.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3292 3316 33 3.2 314.0 1.0X +Sum of string length 3340 3353 19 3.1 318.5 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 5450 5456 9 1.9 519.7 1.0X +Sum of string length 5484 5493 12 1.9 523.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 4410 4435 35 2.4 420.6 1.0X +Sum of string length 3817 3833 22 2.7 364.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3074 3122 68 3.4 293.2 1.0X +Sum of string length 2340 2354 20 4.5 223.2 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 5120 5136 23 0.2 4882.7 1.0X +Sum of single column 4709 4719 14 0.2 4491.1 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 9952 10002 71 0.1 9490.7 1.0X +Sum of single column 9159 9171 18 0.1 8734.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +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 Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 14973 14978 7 0.1 14279.8 1.0X +Sum of single column 13645 13751 151 0.1 13012.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 +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 9215 9309 146 0.1 9215.2 1.0X +pushdown disabled 9535 9637 96 0.1 9534.9 1.0X +w/ filters 3969 3994 22 0.3 3969.5 2.3X + diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 79c72057c5823..285a30bcd046e 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -98,7 +98,10 @@ case class AvroDataToCatalyst( try { decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) result = reader.read(result, decoder) - deserializer.deserialize(result) + val deserialized = deserializer.deserialize(result) + assert(deserialized.isDefined, + "Avro deserializer cannot return an empty result because filters are not pushed down") + deserialized.get } catch { // There could be multiple possible exceptions here, e.g. java.io.IOException, // AvroRuntimeException, ArrayIndexOutOfBoundsException, etc. diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index 1d18594fd349c..29385b78e3490 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -30,7 +30,7 @@ import org.apache.avro.Schema.Type._ import org.apache.avro.generic._ import org.apache.avro.util.Utf8 -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, StructFilters} import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY @@ -45,12 +45,15 @@ import org.apache.spark.unsafe.types.UTF8String class AvroDeserializer( rootAvroType: Schema, rootCatalystType: DataType, - datetimeRebaseMode: LegacyBehaviorPolicy.Value) { + datetimeRebaseMode: LegacyBehaviorPolicy.Value, + filters: StructFilters) { def this(rootAvroType: Schema, rootCatalystType: DataType) { - this(rootAvroType, rootCatalystType, - LegacyBehaviorPolicy.withName( - SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ))) + this( + rootAvroType, + rootCatalystType, + LegacyBehaviorPolicy.withName(SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ)), + new NoopFilters) } private lazy val decimalConversions = new DecimalConversion() @@ -61,19 +64,20 @@ class AvroDeserializer( private val timestampRebaseFunc = DataSourceUtils.creteTimestampRebaseFuncInRead( datetimeRebaseMode, "Avro") - private val converter: Any => Any = rootCatalystType match { + private val converter: Any => Option[Any] = rootCatalystType match { // A shortcut for empty schema. case st: StructType if st.isEmpty => - (data: Any) => InternalRow.empty + (data: Any) => Some(InternalRow.empty) case st: StructType => val resultRow = new SpecificInternalRow(st.map(_.dataType)) val fieldUpdater = new RowUpdater(resultRow) - val writer = getRecordWriter(rootAvroType, st, Nil) + val applyFilters = filters.skipRow(resultRow, _) + val writer = getRecordWriter(rootAvroType, st, Nil, applyFilters) (data: Any) => { val record = data.asInstanceOf[GenericRecord] - writer(fieldUpdater, record) - resultRow + val skipRow = writer(fieldUpdater, record) + if (skipRow) None else Some(resultRow) } case _ => @@ -82,11 +86,11 @@ class AvroDeserializer( val writer = newWriter(rootAvroType, rootCatalystType, Nil) (data: Any) => { writer(fieldUpdater, 0, data) - tmpRow.get(0, rootCatalystType) + Some(tmpRow.get(0, rootCatalystType)) } } - def deserialize(data: Any): Any = converter(data) + def deserialize(data: Any): Option[Any] = converter(data) /** * Creates a writer to write avro values to Catalyst values at the given ordinal with the given @@ -178,7 +182,9 @@ class AvroDeserializer( updater.setDecimal(ordinal, decimal) case (RECORD, st: StructType) => - val writeRecord = getRecordWriter(avroType, st, path) + // Avro datasource doesn't accept filters with nested attributes. See SPARK-32328. + // We can always return `false` from `applyFilters` for nested records. + val writeRecord = getRecordWriter(avroType, st, path, applyFilters = _ => false) (updater, ordinal, value) => val row = new SpecificInternalRow(st) writeRecord(new RowUpdater(row), value.asInstanceOf[GenericRecord]) @@ -315,7 +321,8 @@ class AvroDeserializer( private def getRecordWriter( avroType: Schema, sqlType: StructType, - path: List[String]): (CatalystDataUpdater, GenericRecord) => Unit = { + path: List[String], + applyFilters: Int => Boolean): (CatalystDataUpdater, GenericRecord) => Boolean = { val validFieldIndexes = ArrayBuffer.empty[Int] val fieldWriters = ArrayBuffer.empty[(CatalystDataUpdater, Any) => Unit] @@ -350,10 +357,13 @@ class AvroDeserializer( (fieldUpdater, record) => { var i = 0 - while (i < validFieldIndexes.length) { + var skipRow = false + while (i < validFieldIndexes.length && !skipRow) { fieldWriters(i)(fieldUpdater, record.get(validFieldIndexes(i))) + skipRow = applyFilters(i) i += 1 } + skipRow } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 59d54bc433f8b..fa4b6b829bdde 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, OrderedFilters} import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{DataSourceRegister, Filter} @@ -122,38 +122,28 @@ private[sql] class AvroFileFormat extends FileFormat } reader.sync(file.start) - val stop = file.start + file.length val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( reader.asInstanceOf[DataFileReader[_]].getMetaString, SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ)) - val deserializer = new AvroDeserializer( - userProvidedSchema.getOrElse(reader.getSchema), requiredSchema, datetimeRebaseMode) - - new Iterator[InternalRow] { - private[this] var completed = false - - override def hasNext: Boolean = { - if (completed) { - false - } else { - val r = reader.hasNext && !reader.pastSync(stop) - if (!r) { - reader.close() - completed = true - } - r - } - } + val avroFilters = if (SQLConf.get.avroFilterPushDown) { + new OrderedFilters(filters, requiredSchema) + } else { + new NoopFilters + } - override def next(): InternalRow = { - if (!hasNext) { - throw new NoSuchElementException("next on empty iterator") - } - val record = reader.next() - deserializer.deserialize(record).asInstanceOf[InternalRow] - } + new Iterator[InternalRow] with AvroUtils.RowReader { + override val fileReader = reader + override val deserializer = new AvroDeserializer( + userProvidedSchema.getOrElse(reader.getSchema), + requiredSchema, + datetimeRebaseMode, + avroFilters) + override val stopPosition = file.start + file.length + + override def hasNext: Boolean = hasNextRow + override def next(): InternalRow = nextRow } } else { Iterator.empty diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala index 70dcd58a600fc..51cc51e70cd18 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.avro import java.io.{FileNotFoundException, IOException} import org.apache.avro.Schema +import org.apache.avro.file.{DataFileReader, FileReader} import org.apache.avro.file.DataFileConstants.{BZIP2_CODEC, DEFLATE_CODEC, SNAPPY_CODEC, XZ_CODEC} -import org.apache.avro.file.DataFileReader import org.apache.avro.generic.{GenericDatumReader, GenericRecord} import org.apache.avro.mapred.{AvroOutputFormat, FsInput} import org.apache.avro.mapreduce.AvroJob @@ -32,6 +32,7 @@ import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.AvroOptions.ignoreExtensionKey +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.OutputWriterFactory import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -161,4 +162,37 @@ object AvroUtils extends Logging { "No Avro files found. If files don't have .avro extension, set ignoreExtension to true") } } + + // The trait provides iterator-like interface for reading records from an Avro file, + // deserializing and returning them as internal rows. + trait RowReader { + protected val fileReader: FileReader[GenericRecord] + protected val deserializer: AvroDeserializer + protected val stopPosition: Long + + private[this] var completed = false + private[this] var currentRow: Option[InternalRow] = None + + def hasNextRow: Boolean = { + do { + val r = fileReader.hasNext && !fileReader.pastSync(stopPosition) + if (!r) { + fileReader.close() + completed = true + currentRow = None + } else { + val record = fileReader.next() + currentRow = deserializer.deserialize(record).asInstanceOf[Option[InternalRow]] + } + } while (!completed && currentRow.isEmpty) + + currentRow.isDefined + } + + def nextRow: InternalRow = { + currentRow.getOrElse { + throw new NoSuchElementException("next on empty iterator") + } + } + } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala index 15918f46a83bb..1e6c382041efb 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala @@ -29,12 +29,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions} -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, AvroUtils} +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, OrderedFilters} import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile} import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory, PartitionReaderWithPartitionValues} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -54,7 +55,8 @@ case class AvroPartitionReaderFactory( dataSchema: StructType, readDataSchema: StructType, partitionSchema: StructType, - parsedOptions: AvroOptions) extends FilePartitionReaderFactory with Logging { + parsedOptions: AvroOptions, + filters: Seq[Filter]) extends FilePartitionReaderFactory with Logging { override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { val conf = broadcastedConf.value.value @@ -86,38 +88,28 @@ case class AvroPartitionReaderFactory( } reader.sync(partitionedFile.start) - val stop = partitionedFile.start + partitionedFile.length val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( reader.asInstanceOf[DataFileReader[_]].getMetaString, SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ)) - val deserializer = new AvroDeserializer( - userProvidedSchema.getOrElse(reader.getSchema), readDataSchema, datetimeRebaseMode) - val fileReader = new PartitionReader[InternalRow] { - private[this] var completed = false - - override def next(): Boolean = { - if (completed) { - false - } else { - val r = reader.hasNext && !reader.pastSync(stop) - if (!r) { - reader.close() - completed = true - } - r - } - } + val avroFilters = if (SQLConf.get.avroFilterPushDown) { + new OrderedFilters(filters, readDataSchema) + } else { + new NoopFilters + } - override def get(): InternalRow = { - if (!next) { - throw new NoSuchElementException("next on empty iterator") - } - val record = reader.next() - deserializer.deserialize(record).asInstanceOf[InternalRow] - } + val fileReader = new PartitionReader[InternalRow] with AvroUtils.RowReader { + override val fileReader = reader + override val deserializer = new AvroDeserializer( + userProvidedSchema.getOrElse(reader.getSchema), + readDataSchema, + datetimeRebaseMode, + avroFilters) + override val stopPosition = partitionedFile.start + partitionedFile.length + override def next(): Boolean = hasNextRow + override def get(): InternalRow = nextRow override def close(): Unit = reader.close() } new PartitionReaderWithPartitionValues(fileReader, readDataSchema, diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala index fe7315c739296..e94bef2f8bebe 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -37,6 +38,7 @@ case class AvroScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, + pushedFilters: Array[Filter], partitionFilters: Seq[Expression] = Seq.empty, dataFilters: Seq[Expression] = Seq.empty) extends FileScan { override def isSplitable(path: Path): Boolean = true @@ -50,8 +52,14 @@ case class AvroScan( val parsedOptions = new AvroOptions(caseSensitiveMap, hadoopConf) // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. - AvroPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, parsedOptions) + AvroPartitionReaderFactory( + sparkSession.sessionState.conf, + broadcastedConf, + dataSchema, + readDataSchema, + readPartitionSchema, + parsedOptions, + pushedFilters) } override def withFilters( @@ -59,10 +67,18 @@ case class AvroScan( this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) override def equals(obj: Any): Boolean = obj match { - case a: AvroScan => super.equals(a) && dataSchema == a.dataSchema && options == a.options - + case a: AvroScan => super.equals(a) && dataSchema == a.dataSchema && options == a.options && + equivalentFilters(pushedFilters, a.pushedFilters) case _ => false } override def hashCode(): Int = super.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") + } + + override def getMetaData(): Map[String, String] = { + super.getMetaData() ++ Map("PushedFilers" -> seqToString(pushedFilters)) + } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala index e36c71ef4b1f7..9420608bb22ce 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.v2.avro import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.catalyst.StructFilters +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -29,8 +31,27 @@ class AvroScanBuilder ( schema: StructType, dataSchema: StructType, options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) with SupportsPushDownFilters { + override def build(): Scan = { - AvroScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + AvroScan( + sparkSession, + fileIndex, + dataSchema, + readDataSchema(), + readPartitionSchema(), + options, + pushedFilters()) } + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + if (sparkSession.sessionState.conf.avroFilterPushDown) { + _pushedFilters = StructFilters.pushedFilters(filters, dataSchema) + } + filters + } + + override def pushedFilters(): Array[Filter] = _pushedFilters } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala index c8a1f670bda9e..2d3209f8daa26 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala @@ -26,11 +26,14 @@ import org.apache.avro.message.{BinaryMessageDecoder, BinaryMessageEncoder} import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.{RandomDataGenerator, Row} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, NoopFilters, OrderedFilters, StructFilters} import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, GenericInternalRow, Literal} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.{EqualTo, Not} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String class AvroCatalystDataConversionSuite extends SparkFunSuite with SharedSparkSession @@ -272,6 +275,25 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite assert(message == "Cannot convert Catalyst type StringType to Avro type \"long\".") } + private def checkDeserialization( + schema: Schema, + data: GenericData.Record, + expected: Option[Any], + filters: StructFilters = new NoopFilters): Unit = { + val dataType = SchemaConverters.toSqlType(schema).dataType + val deserializer = new AvroDeserializer( + schema, + dataType, + SQLConf.LegacyBehaviorPolicy.CORRECTED, + filters) + val deserialized = deserializer.deserialize(data) + expected match { + case None => assert(deserialized == None) + case Some(d) => + assert(checkResult(d, deserialized.get, dataType, exprNullable = false)) + } + } + test("avro array can be generic java collection") { val jsonFormatSchema = """ @@ -287,30 +309,53 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite |} """.stripMargin val avroSchema = new Schema.Parser().parse(jsonFormatSchema) - val dataType = SchemaConverters.toSqlType(avroSchema).dataType - val deserializer = new AvroDeserializer(avroSchema, dataType) - - def checkDeserialization(data: GenericData.Record, expected: Any): Unit = { - assert(checkResult( - expected, - deserializer.deserialize(data), - dataType, exprNullable = false - )) - } def validateDeserialization(array: java.util.Collection[Integer]): Unit = { val data = new GenericRecordBuilder(avroSchema) .set("array", array) .build() val expected = InternalRow(new GenericArrayData(new util.ArrayList[Any](array))) - checkDeserialization(data, expected) + checkDeserialization(avroSchema, data, Some(expected)) val reEncoded = new BinaryMessageDecoder[GenericData.Record](new GenericData(), avroSchema) .decode(new BinaryMessageEncoder(new GenericData(), avroSchema).encode(data)) - checkDeserialization(reEncoded, expected) + checkDeserialization(avroSchema, reEncoded, Some(expected)) } validateDeserialization(Collections.emptySet()) validateDeserialization(util.Arrays.asList(1, null, 3)) } + + test("SPARK-32346: filter pushdown to Avro deserializer") { + val schema = + """ + |{ + | "type" : "record", + | "name" : "test_schema", + | "fields" : [ + | {"name": "Age", "type": "int"}, + | {"name": "Name", "type": "string"} + | ] + |} + """.stripMargin + val avroSchema = new Schema.Parser().parse(schema) + val sqlSchema = new StructType().add("Age", "int").add("Name", "string") + val data = new GenericRecordBuilder(avroSchema) + .set("Age", 39) + .set("Name", "Maxim") + .build() + val expectedRow = Some(InternalRow(39, UTF8String.fromString("Maxim"))) + + checkDeserialization(avroSchema, data, expectedRow) + checkDeserialization( + avroSchema, + data, + expectedRow, + new OrderedFilters(Seq(EqualTo("Age", 39)), sqlSchema)) + checkDeserialization( + avroSchema, + data, + None, + new OrderedFilters(Seq(Not(EqualTo("Age", 39))), sqlSchema)) + } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 83a7ef0061fb2..46fe9b2c44529 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1920,6 +1920,7 @@ class AvroV2Suite extends AvroSuite with ExplainSuiteHelper { |Format: avro |Location: InMemoryFileIndex\\[.*\\] |PartitionFilters: \\[isnotnull\\(id#x\\), \\(id#x > 1\\)\\] + |PushedFilers: \\[IsNotNull\\(value\\), GreaterThan\\(value,2\\)\\] |ReadSchema: struct\\ |""".stripMargin.trim spark.range(10) @@ -1933,7 +1934,38 @@ class AvroV2Suite extends AvroSuite with ExplainSuiteHelper { .format("avro") .load(basePath).where($"id" > 1 && $"value" > 2) val normalizedOutput = getNormalizedExplain(df, FormattedMode) - assert(expected_plan_fragment.r.findAllMatchIn(normalizedOutput).length == 1) + assert(expected_plan_fragment.r.findAllMatchIn(normalizedOutput).length == 1, + normalizedOutput) + } + } + + test("SPARK-32346: filters pushdown to Avro datasource v2") { + Seq(true, false).foreach { filtersPushdown => + withSQLConf(SQLConf.AVRO_FILTER_PUSHDOWN_ENABLED.key -> filtersPushdown.toString) { + withTempPath { dir => + Seq(("a", 1, 2), ("b", 1, 2), ("c", 2, 1)) + .toDF("value", "p1", "p2") + .write + .format("avro") + .save(dir.getCanonicalPath) + val df = spark + .read + .format("avro") + .load(dir.getCanonicalPath) + .where("value = 'a'") + + val fileScan = df.queryExecution.executedPlan collectFirst { + case BatchScanExec(_, f: AvroScan) => f + } + assert(fileScan.nonEmpty) + if (filtersPushdown) { + assert(fileScan.get.pushedFilters.nonEmpty) + } else { + assert(fileScan.get.pushedFilters.isEmpty) + } + checkAnswer(df, Row("a", 1, 2)) + } + } } } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala index dc9606f405191..fde858e0a7419 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.execution.benchmark import java.io.File +import java.time.Instant import scala.util.Random import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -36,6 +39,8 @@ import org.apache.spark.sql.types._ * }}} */ object AvroReadBenchmark extends SqlBasedBenchmark { + import spark.implicits._ + def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) } @@ -186,6 +191,60 @@ object AvroReadBenchmark extends SqlBasedBenchmark { } } + private def filtersPushdownBenchmark(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("Filters pushdown", rowsNum, output = output) + val colsNum = 100 + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", TimestampType)) + val schema = StructType(StructField("key", LongType) +: fields) + def columns(): Seq[Column] = { + val ts = Seq.tabulate(colsNum) { i => + lit(Instant.ofEpochSecond(-30610224000L + i * 123456)).as(s"col$i") + } + ($"id" % 1000).as("key") +: ts + } + withTempPath { path => + // Write and read timestamp in the LEGACY mode to make timestamp conversions more expensive + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> "LEGACY") { + spark.range(rowsNum).select(columns(): _*) + .write + .format("avro") + .save(path.getAbsolutePath) + } + def readback = { + spark.read + .schema(schema) + .format("avro") + .load(path.getAbsolutePath) + } + + benchmark.addCase("w/o filters", numIters) { _ => + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY") { + readback.noop() + } + } + + def withFilter(configEnabled: Boolean): Unit = { + withSQLConf( + SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY", + SQLConf.AVRO_FILTER_PUSHDOWN_ENABLED.key -> configEnabled.toString()) { + readback.filter($"key" === 0).noop() + } + } + + benchmark.addCase("pushdown disabled", numIters) { _ => + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY") { + withFilter(configEnabled = false) + } + } + + benchmark.addCase("w/ filters", numIters) { _ => + withFilter(configEnabled = true) + } + + benchmark.run() + } + } + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("SQL Single Numeric Column Scan") { Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType).foreach { dataType => @@ -211,5 +270,8 @@ object AvroReadBenchmark extends SqlBasedBenchmark { columnsBenchmark(1024 * 1024 * 1, 200) columnsBenchmark(1024 * 1024 * 1, 300) } + // Benchmark pushdown filters that refer to top-level columns. + // TODO (SPARK-32328): Add benchmarks for filters with nested column attributes. + filtersPushdownBenchmark(rowsNum = 1000 * 1000, numIters = 3) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/OrderedFilters.scala similarity index 60% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/OrderedFilters.scala index d2cb2c4d8134a..b7c8a0140ea66 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/OrderedFilters.scala @@ -15,23 +15,21 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.csv +package org.apache.spark.sql.catalyst -import org.apache.spark.sql.catalyst.{InternalRow, StructFilters} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType /** - * An instance of the class compiles filters to predicates and allows to - * apply the predicates to an internal row with partially initialized values - * converted from parsed CSV fields. + * An instance of the class compiles filters to predicates and sorts them in + * the order which allows to apply the predicates to an internal row with partially + * initialized values, for instance converted from parsed CSV fields. * - * @param filters The filters pushed down to CSV datasource. + * @param filters The filters pushed down to a datasource. * @param requiredSchema The schema with only fields requested by the upper layer. */ -class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) +class OrderedFilters(filters: Seq[sources.Filter], requiredSchema: StructType) extends StructFilters(filters, requiredSchema) { /** * Converted filters to predicates and grouped by maximum field index @@ -48,33 +46,31 @@ class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) private val predicates: Array[BasePredicate] = { val len = requiredSchema.fields.length val groupedPredicates = Array.fill[BasePredicate](len)(null) - if (SQLConf.get.csvFilterPushDown) { - val groupedFilters = Array.fill(len)(Seq.empty[sources.Filter]) - for (filter <- filters) { - val refs = filter.references - val index = if (refs.isEmpty) { - // For example, `AlwaysTrue` and `AlwaysFalse` doesn't have any references - // Filters w/o refs always return the same result. Taking into account - // that predicates are combined via `And`, we can apply such filters only - // once at the position 0. - 0 - } else { - // readSchema must contain attributes of all filters. - // Accordingly, `fieldIndex()` returns a valid index always. - refs.map(requiredSchema.fieldIndex).max - } - groupedFilters(index) :+= filter + val groupedFilters = Array.fill(len)(Seq.empty[sources.Filter]) + for (filter <- filters) { + val refs = filter.references + val index = if (refs.isEmpty) { + // For example, `AlwaysTrue` and `AlwaysFalse` doesn't have any references + // Filters w/o refs always return the same result. Taking into account + // that predicates are combined via `And`, we can apply such filters only + // once at the position 0. + 0 + } else { + // readSchema must contain attributes of all filters. + // Accordingly, `fieldIndex()` returns a valid index always. + refs.map(requiredSchema.fieldIndex).max } - if (len > 0 && !groupedFilters(0).isEmpty) { - // We assume that filters w/o refs like `AlwaysTrue` and `AlwaysFalse` - // can be evaluated faster that others. We put them in front of others. - val (literals, others) = groupedFilters(0).partition(_.references.isEmpty) - groupedFilters(0) = literals ++ others - } - for (i <- 0 until len) { - if (!groupedFilters(i).isEmpty) { - groupedPredicates(i) = toPredicate(groupedFilters(i)) - } + groupedFilters(index) :+= filter + } + if (len > 0 && groupedFilters(0).nonEmpty) { + // We assume that filters w/o refs like `AlwaysTrue` and `AlwaysFalse` + // can be evaluated faster that others. We put them in front of others. + val (literals, others) = groupedFilters(0).partition(_.references.isEmpty) + groupedFilters(0) = literals ++ others + } + for (i <- 0 until len) { + if (groupedFilters(i).nonEmpty) { + groupedPredicates(i) = toPredicate(groupedFilters(i)) } } groupedPredicates @@ -90,11 +86,13 @@ class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) * otherwise `false` if at least one of the filters returns `false`. */ def skipRow(row: InternalRow, index: Int): Boolean = { + assert(0 <= index && index < requiredSchema.fields.length, + "Index is out of the valid range: it must point out to a field of the required schema.") val predicate = predicates(index) predicate != null && !predicate.eval(row) } - // CSV filters are applied sequentially, and no need to track which filter references + // The filters are applied sequentially, and no need to track which filter references // point out to already set row values. The `reset()` method is trivial because // the filters don't have any states. def reset(): Unit = {} 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 898b963fd0ab5..b5c14a193ddee 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 @@ -25,10 +25,11 @@ import com.univocity.parsers.csv.CsvParser import org.apache.spark.SparkUpgradeException import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, OrderedFilters} import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -98,7 +99,11 @@ class UnivocityParser( legacyFormat = FAST_DATE_FORMAT, isParsing = true) - private val csvFilters = new CSVFilters(filters, requiredSchema) + private val csvFilters = if (SQLConf.get.csvFilterPushDown) { + new OrderedFilters(filters, requiredSchema) + } else { + new NoopFilters + } // Retrieve the raw record string. private def getCurrentInput: UTF8String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f4bc328b24676..1304a021386f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2536,6 +2536,12 @@ object SQLConf { .booleanConf .createWithDefault(true) + val AVRO_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.avro.filterPushdown.enabled") + .doc("When true, enable filter pushdown to Avro datasource.") + .version("3.1.0") + .booleanConf + .createWithDefault(true) + val ADD_PARTITION_BATCH_SIZE = buildConf("spark.sql.addPartitionInBatch.size") .internal() @@ -3276,6 +3282,8 @@ class SQLConf extends Serializable with Logging { def jsonFilterPushDown: Boolean = getConf(JSON_FILTER_PUSHDOWN_ENABLED) + def avroFilterPushDown: Boolean = getConf(AVRO_FILTER_PUSHDOWN_ENABLED) + def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID) def legacyAllowCastNumericToTimestamp: Boolean = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/OrderedFiltersSuite.scala similarity index 83% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/OrderedFiltersSuite.scala index 21bef20d7d4d9..b156cb52e921c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/OrderedFiltersSuite.scala @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.csv +package org.apache.spark.sql.catalyst -import org.apache.spark.sql.catalyst.{StructFilters, StructFiltersSuite} import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType -class CSVFiltersSuite extends StructFiltersSuite { +class OrderedFiltersSuite extends StructFiltersSuite { override def createFilters(filters: Seq[sources.Filter], schema: StructType): StructFilters = { - new CSVFilters(filters, schema) + new OrderedFilters(filters, schema) } } From 9dc02378518b4b0b8a069684f575ed40813fa417 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 29 Jul 2020 10:47:32 -0700 Subject: [PATCH 319/384] [SPARK-32476][CORE] ResourceAllocator.availableAddrs should be deterministic ### What changes were proposed in this pull request? This PR aims to make `ResourceAllocator.availableAddrs` deterministic. ### Why are the changes needed? Currently, this function returns indeterministically due to the underlying `HashMap`. So, the test case itself is creating a list `[0, 1, 2]` initially, but ends up with comparing `[2, 1, 0]`. Not only this happens in the 3.0.0, but also this causes UT failures on Scala 2.13 environment. ### Does this PR introduce _any_ user-facing change? Yes, but this fixes the in-deterministic behavior. ### How was this patch tested? - Scala 2.12: This should pass the UT with the modified test case. - Scala 2.13: This can be tested like the following (at least `JsonProtocolSuite`) ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.deploy.JsonProtocolSuite ``` **BEFORE** ``` *** 2 TESTS FAILED *** ``` **AFTER** ``` All tests passed. ``` Closes #29281 from dongjoon-hyun/SPARK-32476. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/resource/ResourceAllocator.scala | 4 ++-- .../scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 22272a0f98a6c..482d9e94c6dd9 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -56,7 +56,7 @@ trait ResourceAllocator { def availableAddrs: Seq[String] = addressAvailabilityMap .flatMap { case (addr, available) => (0 until available).map(_ => addr) - }.toSeq + }.toSeq.sorted /** * Sequence of currently assigned resource addresses. @@ -68,7 +68,7 @@ trait ResourceAllocator { private[spark] def assignedAddrs: Seq[String] = addressAvailabilityMap .flatMap { case (addr, available) => (0 until slotsPerAddress - available).map(_ => addr) - }.toSeq + }.toSeq.sorted /** * Acquire a sequence of resource addresses (to a launched task), these addresses must be diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 354e6eb2138d9..7d3eb7c6b0f6e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -128,8 +128,8 @@ object JsonConstants { |:["3","4","5"]}},"resourcesused":{"gpu": |{"name":"gpu","addresses":[]},"fpga": |{"name":"fpga","addresses":[]}},"resourcesfree": - |{"gpu":{"name":"gpu","addresses":["2","1","0"]}, - |"fpga":{"name":"fpga","addresses":["5","4","3"]}}, + |{"gpu":{"name":"gpu","addresses":["0","1","2"]}, + |"fpga":{"name":"fpga","addresses":["3","4","5"]}}, |"state":"ALIVE","lastheartbeat":%d} """.format(currTimeInMillis).stripMargin From e926d419d305c9400f6f2426ca3e8d04a9180005 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 29 Jul 2020 13:46:28 -0500 Subject: [PATCH 320/384] [SPARK-30322][DOCS] Add stage level scheduling docs ### What changes were proposed in this pull request? Document the stage level scheduling feature. ### Why are the changes needed? Document the stage level scheduling feature. ### Does this PR introduce _any_ user-facing change? Documentation. ### How was this patch tested? n/a docs only Closes #29292 from tgravescs/SPARK-30322. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- docs/configuration.md | 7 +++++++ docs/running-on-yarn.md | 4 ++++ 2 files changed, 11 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index abf76105ae77d..62799db8becd0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3028,3 +3028,10 @@ There are configurations available to request resources for the driver: sp Spark will use the configurations specified to first request containers with the corresponding resources from the cluster manager. Once it gets the container, Spark launches an Executor in that container which will discover what resources the container has and the addresses associated with each resource. The Executor will register with the Driver and report back the resources available to that Executor. The Spark scheduler can then schedule tasks to each Executor and assign specific resource addresses based on the resource requirements the user specified. The user can see the resources assigned to a task using the TaskContext.get().resources api. On the driver, the user can see the resources assigned with the SparkContext resources call. It's then up to the user to use the assignedaddresses to do the processing they want or pass those into the ML/AI framework they are using. See your cluster manager specific page for requirements and details on each of - [YARN](running-on-yarn.html#resource-allocation-and-configuration-overview), [Kubernetes](running-on-kubernetes.html#resource-allocation-and-configuration-overview) and [Standalone Mode](spark-standalone.html#resource-allocation-and-configuration-overview). It is currently not available with Mesos or local mode. And please also note that local-cluster mode with multiple workers is not supported(see Standalone documentation). + +# Stage Level Scheduling Overview + +The stage level scheduling feature allows users to specify task and executor resource requirements at the stage level. This allows for different stages to run with executors that have different resources. A prime example of this is one ETL stage runs with executors with just CPUs, the next stage is an ML stage that needs GPUs. Stage level scheduling allows for user to request different executors that have GPUs when the ML stage runs rather then having to acquire executors with GPUs at the start of the application and them be idle while the ETL stage is being run. +This is only available for the RDD API in Scala, Java, and Python and requires dynamic allocation to be enabled. It is only available on YARN at this time. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page for more implementation details. + +See the `RDD.withResources` and `ResourceProfileBuilder` API's for using this feature. The current implementation acquires new executors for each `ResourceProfile` created and currently has to be an exact match. Spark does not try to fit tasks into an executor that require a different ResourceProfile than the executor was created with. Executors that are not in use will idle timeout with the dynamic allocation logic. The default configuration for this feature is to only allow one ResourceProfile per stage. If the user associates more then 1 ResourceProfile to an RDD, Spark will throw an exception by default. See config `spark.scheduler.resource.profileMergeConflicts` to control that behavior. The current merge strategy Spark implements when `spark.scheduler.resource.profileMergeConflicts` is enabled is a simple max of each resource within the conflicting ResourceProfiles. Spark will create a new ResourceProfile with the max of each of the resources. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 36d8f0b7ab8df..6f7aaf2baeccd 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -641,6 +641,10 @@ If the user has a user defined YARN resource, lets call it `acceleratorX` then t YARN does not tell Spark the addresses of the resources allocated to each container. For that reason, the user must specify a discovery script that gets run by the executor on startup to discover what resources are available to that executor. You can find an example scripts in `examples/src/main/scripts/getGpusResources.sh`. The script must have execute permissions set and the user should setup permissions to not allow malicious users to modify it. The script should write to STDOUT a JSON string in the format of the ResourceInformation class. This has the resource name and an array of resource addresses available to just that executor. +# Stage Level Scheduling Overview + +Stage level scheduling is supported on YARN when dynamic allocation is enabled. One thing to note that is YARN specific is that each ResourceProfile requires a different container priority on YARN. The mapping is simply the ResourceProfile id becomes the priority, on YARN lower numbers are higher priority. This means that profiles created earlier will have a higher priority in YARN. Normally this won't matter as Spark finishes one stage before starting another one, the only case this might have an affect is in a job server type scenario, so its something to keep in mind. + # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. From a025a89f4ef3a05d7e70c02f03a9826bb97eceac Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Jul 2020 14:21:47 -0500 Subject: [PATCH 321/384] [SPARK-32332][SQL] Support columnar exchanges ### What changes were proposed in this pull request? This PR adds abstract classes for shuffle and broadcast, so that users can provide their columnar implementations. This PR updates several places to use the abstract exchange classes, and also update `AdaptiveSparkPlanExec` so that the columnar rules can see exchange nodes. This is an alternative of https://github.com/apache/spark/pull/29134 . Close https://github.com/apache/spark/pull/29134 ### Why are the changes needed? To allow columnar exchanges. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? new tests Closes #29262 from cloud-fan/columnar. Authored-by: Wenchen Fan Signed-off-by: Thomas Graves --- .../adaptive/AdaptiveSparkPlanExec.scala | 30 ++++- .../adaptive/CustomShuffleReaderExec.scala | 21 +-- .../adaptive/OptimizeLocalShuffleReader.scala | 5 +- .../adaptive/OptimizeSkewedJoin.scala | 4 +- .../execution/adaptive/QueryStageExec.scala | 37 ++++-- .../execution/adaptive/simpleCosting.scala | 6 +- .../exchange/BroadcastExchangeExec.scala | 46 +++++-- .../exchange/ShuffleExchangeExec.scala | 57 ++++++++- .../streaming/IncrementalExecution.scala | 4 +- .../sql/SparkSessionExtensionSuite.scala | 120 +++++++++++++++--- 10 files changed, 260 insertions(+), 70 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 34db0a334f67f..b160b8ac2ed68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -100,7 +100,12 @@ case class AdaptiveSparkPlanExec( // The following two rules need to make use of 'CustomShuffleReaderExec.partitionSpecs' // added by `CoalesceShufflePartitions`. So they must be executed after it. OptimizeSkewedJoin(conf), - OptimizeLocalShuffleReader(conf), + OptimizeLocalShuffleReader(conf) + ) + + // A list of physical optimizer rules to be applied right after a new stage is created. The input + // plan to these rules has exchange as its root node. + @transient private val postStageCreationRules = Seq( ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules), CollapseCodegenStages(conf) ) @@ -227,7 +232,8 @@ case class AdaptiveSparkPlanExec( } // Run the final plan when there's no more unfinished stages. - currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) + currentPhysicalPlan = applyPhysicalRules( + result.newPlan, queryStageOptimizerRules ++ postStageCreationRules) isFinalPlan = true executionId.foreach(onUpdatePlan(_, Seq(currentPhysicalPlan))) currentPhysicalPlan @@ -376,10 +382,22 @@ case class AdaptiveSparkPlanExec( private def newQueryStage(e: Exchange): QueryStageExec = { val optimizedPlan = applyPhysicalRules(e.child, queryStageOptimizerRules) val queryStage = e match { - case s: ShuffleExchangeExec => - ShuffleQueryStageExec(currentStageId, s.copy(child = optimizedPlan)) - case b: BroadcastExchangeExec => - BroadcastQueryStageExec(currentStageId, b.copy(child = optimizedPlan)) + case s: ShuffleExchangeLike => + val newShuffle = applyPhysicalRules( + s.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) + if (!newShuffle.isInstanceOf[ShuffleExchangeLike]) { + throw new IllegalStateException( + "Custom columnar rules cannot transform shuffle node to something else.") + } + ShuffleQueryStageExec(currentStageId, newShuffle) + case b: BroadcastExchangeLike => + val newBroadcast = applyPhysicalRules( + b.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) + if (!newBroadcast.isInstanceOf[BroadcastExchangeLike]) { + throw new IllegalStateException( + "Custom columnar rules cannot transform broadcast node to something else.") + } + BroadcastQueryStageExec(currentStageId, newBroadcast) } currentStageId += 1 setLogicalLinkForNewQueryStage(queryStage, e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala index af18ee065aa86..49a4c25fa637f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD @@ -25,8 +24,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -45,6 +45,8 @@ case class CustomShuffleReaderExec private( assert(partitionSpecs.forall(_.isInstanceOf[PartialMapperPartitionSpec])) } + override def supportsColumnar: Boolean = child.supportsColumnar + override def output: Seq[Attribute] = child.output override lazy val outputPartitioning: Partitioning = { // If it is a local shuffle reader with one mapper per task, then the output partitioning is @@ -55,9 +57,9 @@ case class CustomShuffleReaderExec private( partitionSpecs.map(_.asInstanceOf[PartialMapperPartitionSpec].mapIndex).toSet.size == partitionSpecs.length) { child match { - case ShuffleQueryStageExec(_, s: ShuffleExchangeExec) => + case ShuffleQueryStageExec(_, s: ShuffleExchangeLike) => s.child.outputPartitioning - case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeExec)) => + case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeLike)) => s.child.outputPartitioning match { case e: Expression => r.updateAttr(e).asInstanceOf[Partitioning] case other => other @@ -176,18 +178,21 @@ case class CustomShuffleReaderExec private( } } - private lazy val cachedShuffleRDD: RDD[InternalRow] = { + private lazy val shuffleRDD: RDD[_] = { sendDriverMetrics() shuffleStage.map { stage => - new ShuffledRowRDD( - stage.shuffle.shuffleDependency, stage.shuffle.readMetrics, partitionSpecs.toArray) + stage.shuffle.getShuffleRDD(partitionSpecs.toArray) }.getOrElse { throw new IllegalStateException("operating on canonicalized plan") } } override protected def doExecute(): RDD[InternalRow] = { - cachedShuffleRDD + shuffleRDD.asInstanceOf[RDD[InternalRow]] + } + + override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { + shuffleRDD.asInstanceOf[RDD[ColumnarBatch]] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 3620f27058af2..45fb36420e770 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -78,10 +78,9 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { private def getPartitionSpecs( shuffleStage: ShuffleQueryStageExec, advisoryParallelism: Option[Int]): Seq[ShufflePartitionSpec] = { - val shuffleDep = shuffleStage.shuffle.shuffleDependency - val numReducers = shuffleDep.partitioner.numPartitions + val numMappers = shuffleStage.shuffle.numMappers + val numReducers = shuffleStage.shuffle.numPartitions val expectedParallelism = advisoryParallelism.getOrElse(numReducers) - val numMappers = shuffleDep.rdd.getNumPartitions val splitPoints = if (numMappers == 0) { Seq.empty } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index 627f0600f2383..a85b188727ba4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -202,7 +202,7 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { val leftParts = if (isLeftSkew && !isLeftCoalesced) { val reducerId = leftPartSpec.startReducerIndex val skewSpecs = createSkewPartitionSpecs( - left.shuffleStage.shuffle.shuffleDependency.shuffleId, reducerId, leftTargetSize) + left.mapStats.shuffleId, reducerId, leftTargetSize) if (skewSpecs.isDefined) { logDebug(s"Left side partition $partitionIndex " + s"(${FileUtils.byteCountToDisplaySize(leftActualSize)}) is skewed, " + @@ -218,7 +218,7 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { val rightParts = if (isRightSkew && !isRightCoalesced) { val reducerId = rightPartSpec.startReducerIndex val skewSpecs = createSkewPartitionSpecs( - right.shuffleStage.shuffle.shuffleDependency.shuffleId, reducerId, rightTargetSize) + right.mapStats.shuffleId, reducerId, rightTargetSize) if (skewSpecs.isDefined) { logDebug(s"Right side partition $partitionIndex " + s"(${FileUtils.byteCountToDisplaySize(rightActualSize)}) is skewed, " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 4e83b4344fbf0..0927ef5b0b3c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.ThreadUtils /** @@ -81,17 +82,19 @@ abstract class QueryStageExec extends LeafExecNode { def newReuseInstance(newStageId: Int, newOutput: Seq[Attribute]): QueryStageExec + /** + * Returns the runtime statistics after stage materialization. + */ + def getRuntimeStatistics: Statistics + /** * Compute the statistics of the query stage if executed, otherwise None. */ def computeStats(): Option[Statistics] = resultOption.get().map { _ => - // Metrics `dataSize` are available in both `ShuffleExchangeExec` and `BroadcastExchangeExec`. - val exchange = plan match { - case r: ReusedExchangeExec => r.child - case e: Exchange => e - case _ => throw new IllegalStateException("wrong plan for query stage:\n " + plan.treeString) - } - Statistics(sizeInBytes = exchange.metrics("dataSize").value) + val runtimeStats = getRuntimeStatistics + val dataSize = runtimeStats.sizeInBytes.max(0) + val numOutputRows = runtimeStats.rowCount.map(_.max(0)) + Statistics(dataSize, numOutputRows) } @transient @@ -110,6 +113,8 @@ abstract class QueryStageExec extends LeafExecNode { protected override def doPrepare(): Unit = plan.prepare() protected override def doExecute(): RDD[InternalRow] = plan.execute() + override def supportsColumnar: Boolean = plan.supportsColumnar + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = plan.executeColumnar() override def doExecuteBroadcast[T](): Broadcast[T] = plan.executeBroadcast() override def doCanonicalize(): SparkPlan = plan.canonicalized @@ -138,15 +143,15 @@ abstract class QueryStageExec extends LeafExecNode { } /** - * A shuffle query stage whose child is a [[ShuffleExchangeExec]] or [[ReusedExchangeExec]]. + * A shuffle query stage whose child is a [[ShuffleExchangeLike]] or [[ReusedExchangeExec]]. */ case class ShuffleQueryStageExec( override val id: Int, override val plan: SparkPlan) extends QueryStageExec { @transient val shuffle = plan match { - case s: ShuffleExchangeExec => s - case ReusedExchangeExec(_, s: ShuffleExchangeExec) => s + case s: ShuffleExchangeLike => s + case ReusedExchangeExec(_, s: ShuffleExchangeLike) => s case _ => throw new IllegalStateException("wrong plan for shuffle stage:\n " + plan.treeString) } @@ -177,22 +182,24 @@ case class ShuffleQueryStageExec( * this method returns None, as there is no map statistics. */ def mapStats: Option[MapOutputStatistics] = { - assert(resultOption.get().isDefined, "ShuffleQueryStageExec should already be ready") + assert(resultOption.get().isDefined, s"${getClass.getSimpleName} should already be ready") val stats = resultOption.get().get.asInstanceOf[MapOutputStatistics] Option(stats) } + + override def getRuntimeStatistics: Statistics = shuffle.runtimeStatistics } /** - * A broadcast query stage whose child is a [[BroadcastExchangeExec]] or [[ReusedExchangeExec]]. + * A broadcast query stage whose child is a [[BroadcastExchangeLike]] or [[ReusedExchangeExec]]. */ case class BroadcastQueryStageExec( override val id: Int, override val plan: SparkPlan) extends QueryStageExec { @transient val broadcast = plan match { - case b: BroadcastExchangeExec => b - case ReusedExchangeExec(_, b: BroadcastExchangeExec) => b + case b: BroadcastExchangeLike => b + case ReusedExchangeExec(_, b: BroadcastExchangeLike) => b case _ => throw new IllegalStateException("wrong plan for broadcast stage:\n " + plan.treeString) } @@ -231,6 +238,8 @@ case class BroadcastQueryStageExec( broadcast.relationFuture.cancel(true) } } + + override def getRuntimeStatistics: Statistics = broadcast.runtimeStatistics } object BroadcastQueryStageExec { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/simpleCosting.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/simpleCosting.scala index 67cd720bb5b33..cdc57dbc7dcc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/simpleCosting.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/simpleCosting.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} /** * A simple implementation of [[Cost]], which takes a number of [[Long]] as the cost value. @@ -35,13 +35,13 @@ case class SimpleCost(value: Long) extends Cost { /** * A simple implementation of [[CostEvaluator]], which counts the number of - * [[ShuffleExchangeExec]] nodes in the plan. + * [[ShuffleExchangeLike]] nodes in the plan. */ object SimpleCostEvaluator extends CostEvaluator { override def evaluateCost(plan: SparkPlan): Cost = { val cost = plan.collect { - case s: ShuffleExchangeExec => s + case s: ShuffleExchangeLike => s }.size SimpleCost(cost) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index d35bbe9b8adc0..6d8d37022ea42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -29,6 +29,7 @@ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPartitioning, Partitioning} import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.joins.HashedRelation @@ -37,16 +38,43 @@ import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.util.{SparkFatalException, ThreadUtils} +/** + * Common trait for all broadcast exchange implementations to facilitate pattern matching. + */ +trait BroadcastExchangeLike extends Exchange { + + /** + * The broadcast job group ID + */ + def runId: UUID = UUID.randomUUID + + /** + * The asynchronous job that prepares the broadcast relation. + */ + def relationFuture: Future[broadcast.Broadcast[Any]] + + /** + * For registering callbacks on `relationFuture`. + * Note that calling this method may not start the execution of broadcast job. + */ + def completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] + + /** + * Returns the runtime statistics after broadcast materialization. + */ + def runtimeStatistics: Statistics +} + /** * A [[BroadcastExchangeExec]] collects, transforms and finally broadcasts the result of * a transformed SparkPlan. */ case class BroadcastExchangeExec( mode: BroadcastMode, - child: SparkPlan) extends Exchange { + child: SparkPlan) extends BroadcastExchangeLike { import BroadcastExchangeExec._ - private[sql] val runId: UUID = UUID.randomUUID + override val runId: UUID = UUID.randomUUID override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), @@ -60,21 +88,23 @@ case class BroadcastExchangeExec( BroadcastExchangeExec(mode.canonicalized, child.canonicalized) } + override def runtimeStatistics: Statistics = { + val dataSize = metrics("dataSize").value + Statistics(dataSize) + } + @transient private lazy val promise = Promise[broadcast.Broadcast[Any]]() - /** - * For registering callbacks on `relationFuture`. - * Note that calling this field will not start the execution of broadcast job. - */ @transient - lazy val completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] = promise.future + override lazy val completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] = + promise.future @transient private val timeout: Long = SQLConf.get.broadcastTimeout @transient - private[sql] lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { + override lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { SQLExecution.withThreadLocalCaptured[broadcast.Broadcast[Any]]( sqlContext.sparkSession, BroadcastExchangeExec.executionContext) { try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index b06742e8470c7..30c9f0ae1282d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -30,8 +30,9 @@ import org.apache.spark.shuffle.{ShuffleWriteMetricsReporter, ShuffleWriteProces import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Divide, Literal, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering +import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} @@ -40,13 +41,49 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.util.MutablePair import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordComparator} +/** + * Common trait for all shuffle exchange implementations to facilitate pattern matching. + */ +trait ShuffleExchangeLike extends Exchange { + + /** + * Returns the number of mappers of this shuffle. + */ + def numMappers: Int + + /** + * Returns the shuffle partition number. + */ + def numPartitions: Int + + /** + * Returns whether the shuffle partition number can be changed. + */ + def canChangeNumPartitions: Boolean + + /** + * The asynchronous job that materializes the shuffle. + */ + def mapOutputStatisticsFuture: Future[MapOutputStatistics] + + /** + * Returns the shuffle RDD with specified partition specs. + */ + def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] + + /** + * Returns the runtime statistics after shuffle materialization. + */ + def runtimeStatistics: Statistics +} + /** * Performs a shuffle that will result in the desired partitioning. */ case class ShuffleExchangeExec( override val outputPartitioning: Partitioning, child: SparkPlan, - canChangeNumPartitions: Boolean = true) extends Exchange { + canChangeNumPartitions: Boolean = true) extends ShuffleExchangeLike { private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) @@ -64,7 +101,7 @@ case class ShuffleExchangeExec( @transient lazy val inputRDD: RDD[InternalRow] = child.execute() // 'mapOutputStatisticsFuture' is only needed when enable AQE. - @transient lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { + @transient override lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { if (inputRDD.getNumPartitions == 0) { Future.successful(null) } else { @@ -72,6 +109,20 @@ case class ShuffleExchangeExec( } } + override def numMappers: Int = shuffleDependency.rdd.getNumPartitions + + override def numPartitions: Int = shuffleDependency.partitioner.numPartitions + + override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[InternalRow] = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionSpecs) + } + + override def runtimeStatistics: Statistics = { + val dataSize = metrics("dataSize").value + val rowCount = metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN).value + Statistics(dataSize, Some(rowCount)) + } + /** * A [[ShuffleDependency]] that will partition rows of its child based on * the partitioning scheme defined in `newPartitioning`. Those partitions of diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 7773ac71c4954..bfa60cf7dfd78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, HashPartitioning, SinglePartition} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{LeafExecNode, LocalLimitExec, QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.util.Utils @@ -118,7 +118,7 @@ class IncrementalExecution( case s: StatefulOperator => statefulOpFound = true - case e: ShuffleExchangeExec => + case e: ShuffleExchangeLike => // Don't search recursively any further as any child stateful operator as we // are only looking for stateful subplans that this plan has narrow dependencies on. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 44e784de5164f..e5e8bc6917799 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -16,19 +16,24 @@ */ package org.apache.spark.sql -import java.util.Locale +import java.util.{Locale, UUID} -import org.apache.spark.{SparkFunSuite, TaskContext} +import scala.concurrent.Future + +import org.apache.spark.{MapOutputStatistics, SparkFunSuite, TaskContext} +import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, UnresolvedHint} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Statistics, UnresolvedHint} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, BroadcastExchangeLike, ShuffleExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.COLUMN_BATCH_SIZE @@ -169,33 +174,61 @@ class SparkSessionExtensionSuite extends SparkFunSuite { } } - test("inject columnar") { + test("inject columnar AQE on") { + testInjectColumnar(true) + } + + test("inject columnar AQE off") { + testInjectColumnar(false) + } + + private def testInjectColumnar(enableAQE: Boolean): Unit = { + def collectPlanSteps(plan: SparkPlan): Seq[Int] = plan match { + case a: AdaptiveSparkPlanExec => + assert(a.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) + collectPlanSteps(a.executedPlan) + case _ => plan.collect { + case _: ReplacedRowToColumnarExec => 1 + case _: ColumnarProjectExec => 10 + case _: ColumnarToRowExec => 100 + case s: QueryStageExec => collectPlanSteps(s.plan).sum + case _: MyShuffleExchangeExec => 1000 + case _: MyBroadcastExchangeExec => 10000 + } + } + val extensions = create { extensions => extensions.injectColumnar(session => MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) } withSession(extensions) { session => - // The ApplyColumnarRulesAndInsertTransitions rule is not applied when enable AQE - session.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, false) + session.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, enableAQE) assert(session.sessionState.columnarRules.contains( MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) import session.sqlContext.implicits._ - // repartitioning avoids having the add operation pushed up into the LocalTableScan - val data = Seq((100L), (200L), (300L)).toDF("vals").repartition(1) - val df = data.selectExpr("vals + 1") - // Verify that both pre and post processing of the plan worked. - val found = df.queryExecution.executedPlan.collect { - case rep: ReplacedRowToColumnarExec => 1 - case proj: ColumnarProjectExec => 10 - case c2r: ColumnarToRowExec => 100 - }.sum - assert(found == 111) + // perform a join to inject a broadcast exchange + val left = Seq((1, 50L), (2, 100L), (3, 150L)).toDF("l1", "l2") + val right = Seq((1, 50L), (2, 100L), (3, 150L)).toDF("r1", "r2") + val data = left.join(right, $"l1" === $"r1") + // repartitioning avoids having the add operation pushed up into the LocalTableScan + .repartition(1) + val df = data.selectExpr("l2 + r2") + // execute the plan so that the final adaptive plan is available when AQE is on + df.collect() + val found = collectPlanSteps(df.queryExecution.executedPlan).sum + // 1 MyBroadcastExchangeExec + // 1 MyShuffleExchangeExec + // 1 ColumnarToRowExec + // 2 ColumnarProjectExec + // 1 ReplacedRowToColumnarExec + // so 11121 is expected. + assert(found == 11121) // Verify that we get back the expected, wrong, result val result = df.collect() - assert(result(0).getLong(0) == 102L) // Check that broken columnar Add was used. - assert(result(1).getLong(0) == 202L) - assert(result(2).getLong(0) == 302L) + assert(result(0).getLong(0) == 101L) // Check that broken columnar Add was used. + assert(result(1).getLong(0) == 201L) + assert(result(2).getLong(0) == 301L) } } @@ -695,6 +728,16 @@ case class PreRuleReplaceAddWithBrokenVersion() extends Rule[SparkPlan] { def replaceWithColumnarPlan(plan: SparkPlan): SparkPlan = try { plan match { + case e: ShuffleExchangeExec => + // note that this is not actually columnar but demonstrates that exchanges can + // be replaced. + val replaced = e.withNewChildren(e.children.map(replaceWithColumnarPlan)) + MyShuffleExchangeExec(replaced.asInstanceOf[ShuffleExchangeExec]) + case e: BroadcastExchangeExec => + // note that this is not actually columnar but demonstrates that exchanges can + // be replaced. + val replaced = e.withNewChildren(e.children.map(replaceWithColumnarPlan)) + MyBroadcastExchangeExec(replaced.asInstanceOf[BroadcastExchangeExec]) case plan: ProjectExec => new ColumnarProjectExec(plan.projectList.map((exp) => replaceWithColumnarExpression(exp).asInstanceOf[NamedExpression]), @@ -713,6 +756,41 @@ case class PreRuleReplaceAddWithBrokenVersion() extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = replaceWithColumnarPlan(plan) } +/** + * Custom Exchange used in tests to demonstrate that shuffles can be replaced regardless of + * whether AQE is enabled. + */ +case class MyShuffleExchangeExec(delegate: ShuffleExchangeExec) extends ShuffleExchangeLike { + override def numMappers: Int = delegate.numMappers + override def numPartitions: Int = delegate.numPartitions + override def canChangeNumPartitions: Boolean = delegate.canChangeNumPartitions + override def mapOutputStatisticsFuture: Future[MapOutputStatistics] = + delegate.mapOutputStatisticsFuture + override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] = + delegate.getShuffleRDD(partitionSpecs) + override def runtimeStatistics: Statistics = delegate.runtimeStatistics + override def child: SparkPlan = delegate.child + override protected def doExecute(): RDD[InternalRow] = delegate.execute() + override def outputPartitioning: Partitioning = delegate.outputPartitioning +} + +/** + * Custom Exchange used in tests to demonstrate that broadcasts can be replaced regardless of + * whether AQE is enabled. + */ +case class MyBroadcastExchangeExec(delegate: BroadcastExchangeExec) extends BroadcastExchangeLike { + override def runId: UUID = delegate.runId + override def relationFuture: java.util.concurrent.Future[Broadcast[Any]] = + delegate.relationFuture + override def completionFuture: Future[Broadcast[Any]] = delegate.completionFuture + override def runtimeStatistics: Statistics = delegate.runtimeStatistics + override def child: SparkPlan = delegate.child + override protected def doPrepare(): Unit = delegate.prepare() + override protected def doExecute(): RDD[InternalRow] = delegate.execute() + override def doExecuteBroadcast[T](): Broadcast[T] = delegate.executeBroadcast() + override def outputPartitioning: Partitioning = delegate.outputPartitioning +} + class ReplacedRowToColumnarExec(override val child: SparkPlan) extends RowToColumnarExec(child) { From 50911df08eb7a27494dc83bcec3d09701c2babfe Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 29 Jul 2020 21:39:14 +0000 Subject: [PATCH 322/384] [SPARK-32397][BUILD] Allow specifying of time for build to keep time consistent between modules ### What changes were proposed in this pull request? Upgrade codehaus maven build helper to allow people to specify a time during the build to avoid snapshot artifacts with different version strings. ### Why are the changes needed? During builds of snapshots the maven may assign different versions to different artifacts based on the time each individual sub-module starts building. The timestamp is used as part of the version string when run `maven deploy` on a snapshot build. This results in different sub-modules having different version strings. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual build while specifying the current time, ensured the time is consistent in the sub components. Open question: Ideally I'd like to backport this as well since it's sort of a bug fix and while it does change a dependency version it's not one that is propagated. I'd like to hear folks thoughts about this. Closes #29274 from holdenk/SPARK-32397-snapshot-artifact-timestamp-differences. Authored-by: Holden Karau Signed-off-by: DB Tsai --- pom.xml | 40 +++++++++++++++++++++++++++++++++++----- 1 file changed, 35 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 065b8f2a9abc5..d07813c2787de 100644 --- a/pom.xml +++ b/pom.xml @@ -251,6 +251,8 @@ 1g + + yyyy-MM-dd HH:mm:ss z @@ -2405,11 +2407,39 @@ - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - + + org.codehaus.mojo + build-helper-maven-plugin + 3.2.0 + + + module-timestamp-property + validate + + timestamp-property + + + module.build.timestamp + ${maven.build.timestamp.format} + current + America/Los_Angeles + + + + local-timestamp-property + validate + + timestamp-property + + + local.build.timestamp + ${maven.build.timestamp.format} + build + America/Los_Angeles + + + + net.alchim31.maven scala-maven-plugin From 163867435a6af1e9a37521e34ea41b07168f4730 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 29 Jul 2020 17:57:46 -0700 Subject: [PATCH 323/384] [SPARK-32487][CORE] Remove j.w.r.NotFoundException from `import` in [Stages|OneApplication]Resource ### What changes were proposed in this pull request? This PR aims to remove `java.ws.rs.NotFoundException` from two problematic `import` statements. All the other use cases are correct. ### Why are the changes needed? In `StagesResource` and `OneApplicationResource`, there exist two `NotFoundException`s. - javax.ws.rs.NotFoundException - org.apache.spark.status.api.v1.NotFoundException To use `org.apache.spark.status.api.v1.NotFoundException` correctly, we should not import `java.ws.rs.NotFoundException`. This causes UT failures in Scala 2.13 environment. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Scala 2.12: Pass the GitHub Action or Jenkins. - Scala 2.13: Do the following manually. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.deploy.history.HistoryServerSuite ``` **BEFORE** ``` *** 4 TESTS FAILED *** ``` **AFTER** ``` *** 1 TEST FAILED *** ``` Closes #29293 from dongjoon-hyun/SPARK-32487. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/status/api/v1/OneApplicationResource.scala | 2 +- .../scala/org/apache/spark/status/api/v1/StagesResource.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala index e0c85fdf6fb5d..05a33f17a5d95 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -19,7 +19,7 @@ package org.apache.spark.status.api.v1 import java.io.OutputStream import java.util.{List => JList} import java.util.zip.ZipOutputStream -import javax.ws.rs._ +import javax.ws.rs.{NotFoundException => _, _} import javax.ws.rs.core.{MediaType, Response, StreamingOutput} import scala.util.control.NonFatal diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 05a7e96882d77..84bd430d9abef 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -17,7 +17,7 @@ package org.apache.spark.status.api.v1 import java.util.{HashMap, List => JList, Locale} -import javax.ws.rs._ +import javax.ws.rs.{NotFoundException => _, _} import javax.ws.rs.core.{Context, MediaType, MultivaluedMap, UriInfo} import org.apache.spark.ui.UIUtils From 08a66f8fd0df38280dfd54bb79aa8a8ae1272fc9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 29 Jul 2020 18:05:53 -0700 Subject: [PATCH 324/384] [SPARK-32248][BUILD] Recover Java 11 build in Github Actions ### What changes were proposed in this pull request? This PR aims to recover Java 11 build in `GitHub Action`. ### Why are the changes needed? This test coverage is removed before. Now, it's time to recover it. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the GitHub Action. Closes #29295 from dongjoon-hyun/SPARK-32248. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 29f98ed36e2ec..7ff960bf9b7be 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -241,3 +241,28 @@ jobs: run: | cd docs jekyll build + + java11: + name: Java 11 build + runs-on: ubuntu-latest + steps: + - name: Checkout Spark repository + uses: actions/checkout@v2 + - name: Cache Maven local repository + uses: actions/cache@v2 + with: + path: ~/.m2/repository + key: java11-maven-${{ hashFiles('**/pom.xml') }} + restore-keys: | + java11-maven- + - name: Install Java 11 + uses: actions/setup-java@v1 + with: + java-version: 11 + - name: Build with Maven + run: | + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" + export MAVEN_CLI_OPTS="--no-transfer-progress" + mkdir -p ~/.m2 + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Djava.version=11 install + rm -rf ~/.m2/repository/org/apache/spark From 89d9b7cc64f01de9b3e88352d6a1979852873a77 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 30 Jul 2020 10:15:25 +0900 Subject: [PATCH 325/384] [SPARK-32010][PYTHON][CORE] Add InheritableThread for local properties and fixing a thread leak issue in pinned thread mode ### What changes were proposed in this pull request? This PR proposes: 1. To introduce `InheritableThread` class, that works identically with `threading.Thread` but it can inherit the inheritable attributes of a JVM thread such as `InheritableThreadLocal`. This was a problem from the pinned thread mode, see also https://github.com/apache/spark/pull/24898. Now it works as below: ```python import pyspark spark.sparkContext.setLocalProperty("a", "hi") def print_prop(): print(spark.sparkContext.getLocalProperty("a")) pyspark.InheritableThread(target=print_prop).start() ``` ``` hi ``` 2. Also, it adds the resource leak fix into `InheritableThread`. Py4J leaks the thread and does not close the connection from Python to JVM. In `InheritableThread`, it manually closes the connections when PVM garbage collection happens. So, JVM threads finish safely. I manually verified by profiling but there's also another easy way to verify: ```bash PYSPARK_PIN_THREAD=true ./bin/pyspark ``` ```python >>> from threading import Thread >>> Thread(target=lambda: spark.range(1000).collect()).start() >>> Thread(target=lambda: spark.range(1000).collect()).start() >>> Thread(target=lambda: spark.range(1000).collect()).start() >>> spark._jvm._gateway_client.deque deque([, , , , ]) >>> Thread(target=lambda: spark.range(1000).collect()).start() >>> spark._jvm._gateway_client.deque deque([, , , , , ]) ``` This issue is fixed now. 3. Because now we have a fix for the issue here, it also proposes to deprecate `collectWithJobGroup` which was a temporary workaround added to avoid this leak issue. ### Why are the changes needed? To support pinned thread mode properly without a resource leak, and a proper inheritable local properties. ### Does this PR introduce _any_ user-facing change? Yes, it adds an API `InheritableThread` class for pinned thread mode. ### How was this patch tested? Manually tested as described above, and unit test was added as well. Closes #28968 from HyukjinKwon/SPARK-32010. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- docs/job-scheduling.md | 8 ++-- python/pyspark/__init__.py | 5 +- python/pyspark/context.py | 18 +++++--- python/pyspark/rdd.py | 10 +++- python/pyspark/tests/test_pin_thread.py | 23 +++++++++- python/pyspark/util.py | 61 +++++++++++++++++++++++++ 6 files changed, 110 insertions(+), 15 deletions(-) diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 5c19c77f37a81..1eed0ff3ee5e8 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -297,11 +297,9 @@ via `sc.setJobGroup` in a separate PVM thread, which also disallows to cancel th later. In order to synchronize PVM threads with JVM threads, you should set `PYSPARK_PIN_THREAD` environment variable -to `true`. This pinned thread mode allows one PVM thread has one corresponding JVM thread. - -However, currently it cannot inherit the local properties from the parent thread although it isolates -each thread with its own local properties. To work around this, you should manually copy and set the -local properties from the parent thread to the child thread when you create another thread in PVM. +to `true`. This pinned thread mode allows one PVM thread has one corresponding JVM thread. With this mode, +`pyspark.InheritableThread` is recommanded to use together for a PVM thread to inherit the interitable attributes + such as local properties in a JVM thread. Note that `PYSPARK_PIN_THREAD` is currently experimental and not recommended for use in production. diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index ee153af18c88c..61e38fdb2a57b 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -42,6 +42,8 @@ A :class:`TaskContext` that provides extra info and tooling for barrier execution. - :class:`BarrierTaskInfo`: Information about a barrier task. + - :class:`InheritableThread`: + A inheritable thread to use in Spark when the pinned thread mode is on. """ from functools import wraps @@ -51,6 +53,7 @@ from pyspark.context import SparkContext from pyspark.rdd import RDD, RDDBarrier from pyspark.files import SparkFiles +from pyspark.util import InheritableThread from pyspark.storagelevel import StorageLevel from pyspark.accumulators import Accumulator, AccumulatorParam from pyspark.broadcast import Broadcast @@ -118,5 +121,5 @@ def wrapper(self, *args, **kwargs): "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", - "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", + "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "InheritableThread", ] diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 2e105cc38260d..5ddce9f4584c4 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -1013,8 +1013,10 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): .. note:: Currently, setting a group ID (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local - properties for each thread on PVM. To work around this, You can use - :meth:`RDD.collectWithJobGroup` for now. + properties for each thread on PVM. + + To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` + environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. """ self._jsc.setJobGroup(groupId, description, interruptOnCancel) @@ -1026,8 +1028,10 @@ def setLocalProperty(self, key, value): .. note:: Currently, setting a local property with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. To work around this, You can use - :meth:`RDD.collectWithJobGroup`. + for each thread on PVM. + + To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` + environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. """ self._jsc.setLocalProperty(key, value) @@ -1045,8 +1049,10 @@ def setJobDescription(self, value): .. note:: Currently, setting a job description (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate - local properties for each thread on PVM. To work around this, You can use - :meth:`RDD.collectWithJobGroup` for now. + local properties for each thread on PVM. + + To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` + environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. """ self._jsc.setJobDescription(value) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 437b2c446529a..4ee486800f882 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -859,12 +859,18 @@ def collect(self): def collectWithJobGroup(self, groupId, description, interruptOnCancel=False): """ - .. note:: Experimental - When collect rdd, use this method to specify job group. + .. note:: Deprecated in 3.1.0. Use :class:`pyspark.InheritableThread` with + the pinned thread mode enabled. + .. versionadded:: 3.0.0 """ + warnings.warn( + "Deprecated in 3.1, Use pyspark.InheritableThread with " + "the pinned thread mode enabled.", + DeprecationWarning) + with SCCallSiteSync(self.context) as css: sock_info = self.ctx._jvm.PythonRDD.collectAndServeWithJobGroup( self._jrdd.rdd(), groupId, description, interruptOnCancel) diff --git a/python/pyspark/tests/test_pin_thread.py b/python/pyspark/tests/test_pin_thread.py index 657d129fe63bb..50eb8e0ec8b50 100644 --- a/python/pyspark/tests/test_pin_thread.py +++ b/python/pyspark/tests/test_pin_thread.py @@ -20,7 +20,7 @@ import threading import unittest -from pyspark import SparkContext, SparkConf +from pyspark import SparkContext, SparkConf, InheritableThread class PinThreadTests(unittest.TestCase): @@ -143,6 +143,27 @@ def run_job(job_group, index): is_job_cancelled[i], "Thread {i}: Job in group B did not succeeded.".format(i=i)) + def test_inheritable_local_property(self): + self.sc.setLocalProperty("a", "hi") + expected = [] + + def get_inner_local_prop(): + expected.append(self.sc.getLocalProperty("b")) + + def get_outer_local_prop(): + expected.append(self.sc.getLocalProperty("a")) + self.sc.setLocalProperty("b", "hello") + t2 = InheritableThread(target=get_inner_local_prop) + t2.start() + t2.join() + + t1 = InheritableThread(target=get_outer_local_prop) + t1.start() + t1.join() + + self.assertEqual(self.sc.getLocalProperty("b"), None) + self.assertEqual(expected, ["hi", "hello"]) + if __name__ == "__main__": import unittest diff --git a/python/pyspark/util.py b/python/pyspark/util.py index c003586e9c03b..86e5ab5a01585 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -16,10 +16,13 @@ # limitations under the License. # +import threading import re import sys import traceback +from py4j.clientserver import ClientServer + __all__ = [] @@ -114,6 +117,64 @@ def _parse_memory(s): raise ValueError("invalid format: " + s) return int(float(s[:-1]) * units[s[-1].lower()]) + +class InheritableThread(threading.Thread): + """ + Thread that is recommended to be used in PySpark instead of :class:`threading.Thread` + when the pinned thread mode is enabled. The usage of this class is exactly same as + :class:`threading.Thread` but correctly inherits the inheritable properties specific + to JVM thread such as ``InheritableThreadLocal``. + + Also, note that pinned thread mode does not close the connection from Python + to JVM when the thread is finished in the Python side. With this class, Python + garbage-collects the Python thread instance and also closes the connection + which finishes JVM thread correctly. + + When the pinned thread mode is off, this works as :class:`threading.Thread`. + + .. note:: Experimental + + .. versionadded:: 3.1.0 + """ + def __init__(self, target, *args, **kwargs): + from pyspark import SparkContext + + sc = SparkContext._active_spark_context + + if isinstance(sc._gateway, ClientServer): + # Here's when the pinned-thread mode (PYSPARK_PIN_THREAD) is on. + properties = sc._jsc.sc().getLocalProperties().clone() + self._sc = sc + + def copy_local_properties(*a, **k): + sc._jsc.sc().setLocalProperties(properties) + return target(*a, **k) + + super(InheritableThread, self).__init__( + target=copy_local_properties, *args, **kwargs) + else: + super(InheritableThread, self).__init__(target=target, *args, **kwargs) + + def __del__(self): + from pyspark import SparkContext + + if isinstance(SparkContext._gateway, ClientServer): + thread_connection = self._sc._jvm._gateway_client.thread_connection.connection() + if thread_connection is not None: + connections = self._sc._jvm._gateway_client.deque + + # Reuse the lock for Py4J in PySpark + with SparkContext._lock: + for i in range(len(connections)): + if connections[i] is thread_connection: + connections[i].close() + del connections[i] + break + else: + # Just in case the connection was not closed but removed from the queue. + thread_connection.close() + + if __name__ == "__main__": import doctest (failure_count, test_count) = doctest.testmod() From 81b0785fb2d9a2d45d4366a58a3c30fe478c299a Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Wed, 29 Jul 2020 19:53:28 -0700 Subject: [PATCH 326/384] [SPARK-32455][ML] LogisticRegressionModel prediction optimization ### What changes were proposed in this pull request? for binary `LogisticRegressionModel`: 1, keep variables `_threshold` and `_rawThreshold` instead of computing them on each instance; 2, in `raw2probabilityInPlace`, make use of the characteristic that the sum of probability is 1.0; ### Why are the changes needed? for better performance ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? existing testsuite and performace test in REPL Closes #29255 from zhengruifeng/pred_opt. Authored-by: zhengruifeng Signed-off-by: Huaxin Gao --- .../classification/LogisticRegression.scala | 89 ++++++++++--------- 1 file changed, 49 insertions(+), 40 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index e7dfaec0e37e2..4d763cbd29d3c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -1098,16 +1098,42 @@ class LogisticRegressionModel private[spark] ( _intercept } - private lazy val _intercept = interceptVector.toArray.head + private lazy val _intercept = interceptVector(0) + private lazy val _interceptVector = interceptVector.toDense + private var _threshold = Double.NaN + private var _rawThreshold = Double.NaN + + updateBinaryThreshold() + + private def updateBinaryThreshold(): Unit = { + if (!isMultinomial) { + _threshold = getThreshold + if (_threshold == 0.0) { + _rawThreshold = Double.NegativeInfinity + } else if (_threshold == 1.0) { + _rawThreshold = Double.PositiveInfinity + } else { + _rawThreshold = math.log(_threshold / (1.0 - _threshold)) + } + } + } @Since("1.5.0") - override def setThreshold(value: Double): this.type = super.setThreshold(value) + override def setThreshold(value: Double): this.type = { + super.setThreshold(value) + updateBinaryThreshold() + this + } @Since("1.5.0") override def getThreshold: Double = super.getThreshold @Since("1.5.0") - override def setThresholds(value: Array[Double]): this.type = super.setThresholds(value) + override def setThresholds(value: Array[Double]): this.type = { + super.setThresholds(value) + updateBinaryThreshold() + this + } @Since("1.5.0") override def getThresholds: Array[Double] = super.getThresholds @@ -1119,7 +1145,7 @@ class LogisticRegressionModel private[spark] ( /** Margin (rawPrediction) for each class label. */ private val margins: Vector => Vector = (features) => { - val m = interceptVector.toDense.copy + val m = _interceptVector.copy BLAS.gemv(1.0, coefficientMatrix, features, 1.0, m) m } @@ -1178,52 +1204,43 @@ class LogisticRegressionModel private[spark] ( override def predict(features: Vector): Double = if (isMultinomial) { super.predict(features) } else { - // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. - if (score(features) > getThreshold) 1 else 0 + // Note: We should use _threshold instead of $(threshold) since getThreshold is overridden. + if (score(features) > _threshold) 1 else 0 } override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { rawPrediction match { case dv: DenseVector => + val values = dv.values if (isMultinomial) { - val size = dv.size - val values = dv.values - // get the maximum margin val maxMarginIndex = rawPrediction.argmax val maxMargin = rawPrediction(maxMarginIndex) if (maxMargin == Double.PositiveInfinity) { var k = 0 - while (k < size) { + while (k < numClasses) { values(k) = if (k == maxMarginIndex) 1.0 else 0.0 k += 1 } } else { - val sum = { - var temp = 0.0 - var k = 0 - while (k < numClasses) { - values(k) = if (maxMargin > 0) { - math.exp(values(k) - maxMargin) - } else { - math.exp(values(k)) - } - temp += values(k) - k += 1 + var sum = 0.0 + var k = 0 + while (k < numClasses) { + values(k) = if (maxMargin > 0) { + math.exp(values(k) - maxMargin) + } else { + math.exp(values(k)) } - temp + sum += values(k) + k += 1 } BLAS.scal(1 / sum, dv) } dv } else { - var i = 0 - val size = dv.size - while (i < size) { - dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i))) - i += 1 - } + values(0) = 1.0 / (1.0 + math.exp(-values(0))) + values(1) = 1.0 - values(0) dv } case sv: SparseVector => @@ -1253,16 +1270,8 @@ class LogisticRegressionModel private[spark] ( if (isMultinomial) { super.raw2prediction(rawPrediction) } else { - // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. - val t = getThreshold - val rawThreshold = if (t == 0.0) { - Double.NegativeInfinity - } else if (t == 1.0) { - Double.PositiveInfinity - } else { - math.log(t / (1.0 - t)) - } - if (rawPrediction(1) > rawThreshold) 1 else 0 + // Note: We should use _threshold instead of $(threshold) since getThreshold is overridden. + if (rawPrediction(1) > _rawThreshold) 1.0 else 0.0 } } @@ -1270,8 +1279,8 @@ class LogisticRegressionModel private[spark] ( if (isMultinomial) { super.probability2prediction(probability) } else { - // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. - if (probability(1) > getThreshold) 1 else 0 + // Note: We should use _threshold instead of $(threshold) since getThreshold is overridden. + if (probability(1) > _threshold) 1.0 else 0.0 } } From 99a855575c3a5554443a27385caf49661cc7f139 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 30 Jul 2020 06:05:55 +0000 Subject: [PATCH 327/384] [SPARK-32431][SQL] Check duplicate nested columns in read from in-built datasources ### What changes were proposed in this pull request? When `spark.sql.caseSensitive` is `false` (by default), check that there are not duplicate column names on the same level (top level or nested levels) in reading from in-built datasources Parquet, ORC, Avro and JSON. If such duplicate columns exist, throw the exception: ``` org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: ``` ### Why are the changes needed? To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error when `spark.sql.caseSensitive` is `false`: ```Scala org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase` ``` Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758. ### Does this PR introduce _any_ user-facing change? Yes. For the example from SPARK-32431: ORC: ```scala java.io.IOException: Error reading file: file:/private/var/folders/p3/dfs6mf655d7fnjrsjvldh0tc0000gn/T/spark-c02c2f9a-0cdc-4859-94fc-b9c809ca58b1/part-00001-63e8c3f0-7131-4ec9-be02-30b3fdd276f4-c000.snappy.orc at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1329) at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78) ... Caused by: java.io.EOFException: Read past end of RLE integer from compressed stream Stream for column 3 kind DATA position: 6 length: 6 range: 0 offset: 12 limit: 12 range 0 = 0 to 6 uncompressed: 3 to 3 at org.apache.orc.impl.RunLengthIntegerReaderV2.readValues(RunLengthIntegerReaderV2.java:61) at org.apache.orc.impl.RunLengthIntegerReaderV2.next(RunLengthIntegerReaderV2.java:323) ``` JSON: ```scala +------------+ |StructColumn| +------------+ | [,,]| +------------+ ``` Parquet: ```scala +------------+ |StructColumn| +------------+ | [0,, 1]| +------------+ ``` Avro: ```scala +------------+ |StructColumn| +------------+ | [,,]| +------------+ ``` After the changes, Parquet, ORC, JSON and Avro output the same error: ```scala Found duplicate column(s) in the data schema: `camelcase`; org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`; at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:112) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:51) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:67) ``` ### How was this patch tested? Run modified test suites: ``` $ build/sbt "sql/test:testOnly org.apache.spark.sql.FileBasedDataSourceSuite" $ build/sbt "avro/test:testOnly org.apache.spark.sql.avro.*" ``` and added new UT to `SchemaUtilsSuite`. Closes #29234 from MaxGekk/nested-case-insensitive-column. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + .../org/apache/spark/sql/avro/AvroSuite.scala | 3 +- .../apache/spark/sql/util/SchemaUtils.scala | 34 +++++++- .../spark/sql/util/SchemaUtilsSuite.scala | 26 +++++- .../execution/datasources/DataSource.scala | 12 +-- .../execution/datasources/v2/FileTable.scala | 4 +- .../spark/sql/NestedDataSourceSuite.scala | 83 +++++++++++++++++++ 7 files changed, 152 insertions(+), 12 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 46d330e095865..7c9c0a726c881 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -29,6 +29,8 @@ license: | - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. + + - In Spark 3.1, the Parquet, ORC, Avro and JSON datasources throw the exception `org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema` in read if they detect duplicate names in top-level columns as well in nested structures. The datasources take into account the SQL config `spark.sql.caseSensitive` while detecting column name duplicates. ## Upgrading from Spark SQL 3.0 to 3.0.1 diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 46fe9b2c44529..8a8a7681abd1c 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -50,9 +50,10 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.v2.avro.AvroScan import org.apache.spark.util.Utils -abstract class AvroSuite extends QueryTest with SharedSparkSession { +abstract class AvroSuite extends QueryTest with SharedSparkSession with NestedDataSourceSuiteBase { import testImplicits._ + override val nestedDataSources = Seq("avro") val episodesAvro = testFile("episodes.avro") val testAvro = testFile("test.avro") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 27b5eec27281d..c83cd52250702 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -41,8 +41,38 @@ private[spark] object SchemaUtils { * @param caseSensitiveAnalysis whether duplication checks should be case sensitive or not */ def checkSchemaColumnNameDuplication( - schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { - checkColumnNameDuplication(schema.map(_.name), colType, caseSensitiveAnalysis) + schema: DataType, + colType: String, + caseSensitiveAnalysis: Boolean = false): Unit = { + schema match { + case ArrayType(elementType, _) => + checkSchemaColumnNameDuplication(elementType, colType, caseSensitiveAnalysis) + case MapType(keyType, valueType, _) => + checkSchemaColumnNameDuplication(keyType, colType, caseSensitiveAnalysis) + checkSchemaColumnNameDuplication(valueType, colType, caseSensitiveAnalysis) + case structType: StructType => + val fields = structType.fields + checkColumnNameDuplication(fields.map(_.name), colType, caseSensitiveAnalysis) + fields.foreach { field => + checkSchemaColumnNameDuplication(field.dataType, colType, caseSensitiveAnalysis) + } + case _ => + } + } + + /** + * Checks if an input schema has duplicate column names. This throws an exception if the + * duplication exists. + * + * @param schema schema to check + * @param colType column type name, used in an exception message + * @param resolver resolver used to determine if two identifiers are equal + */ + def checkSchemaColumnNameDuplication( + schema: StructType, + colType: String, + resolver: Resolver): Unit = { + checkSchemaColumnNameDuplication(schema, colType, isCaseSensitiveAnalysis(resolver)) } // Returns true if a given resolver is case-sensitive diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index 2f576a4031e92..02ee634dba1b6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -22,7 +22,7 @@ import java.util.Locale import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{ArrayType, LongType, MapType, StructType} class SchemaUtilsSuite extends SparkFunSuite { @@ -82,4 +82,28 @@ class SchemaUtilsSuite extends SparkFunSuite { checkNoExceptionCases("a INT, b INT, c INT", caseSensitive = false) } + + test("SPARK-32431: duplicated fields in nested schemas") { + val schemaA = new StructType() + .add("LowerCase", LongType) + .add("camelcase", LongType) + .add("CamelCase", LongType) + val schemaB = new StructType() + .add("f1", LongType) + .add("StructColumn1", schemaA) + val schemaC = new StructType() + .add("f2", LongType) + .add("StructColumn2", schemaB) + val schemaD = new StructType() + .add("f3", ArrayType(schemaC)) + val schemaE = MapType(LongType, schemaD) + val schemaF = MapType(schemaD, LongType) + Seq(schemaA, schemaB, schemaC, schemaD, schemaE, schemaF).foreach { schema => + val msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = false) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `camelcase`")) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index db564485be883..36e5eb33e1bae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -421,18 +421,18 @@ case class DataSource( relation match { case hs: HadoopFsRelation => - SchemaUtils.checkColumnNameDuplication( - hs.dataSchema.map(_.name), + SchemaUtils.checkSchemaColumnNameDuplication( + hs.dataSchema, "in the data schema", equality) - SchemaUtils.checkColumnNameDuplication( - hs.partitionSchema.map(_.name), + SchemaUtils.checkSchemaColumnNameDuplication( + hs.partitionSchema, "in the partition schema", equality) DataSourceUtils.verifySchema(hs.fileFormat, hs.dataSchema) case _ => - SchemaUtils.checkColumnNameDuplication( - relation.schema.map(_.name), + SchemaUtils.checkSchemaColumnNameDuplication( + relation.schema, "in the data schema", equality) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala index 59dc3ae56bf25..7bd05f12873f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -79,7 +79,7 @@ abstract class FileTable( override lazy val schema: StructType = { val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis - SchemaUtils.checkColumnNameDuplication(dataSchema.fieldNames, + SchemaUtils.checkSchemaColumnNameDuplication(dataSchema, "in the data schema", caseSensitive) dataSchema.foreach { field => if (!supportsDataType(field.dataType)) { @@ -88,7 +88,7 @@ abstract class FileTable( } } val partitionSchema = fileIndex.partitionSchema - SchemaUtils.checkColumnNameDuplication(partitionSchema.fieldNames, + SchemaUtils.checkSchemaColumnNameDuplication(partitionSchema, "in the partition schema", caseSensitive) val partitionNameSet: Set[String] = partitionSchema.fields.map(PartitioningUtils.getColName(_, caseSensitive)).toSet diff --git a/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala new file mode 100644 index 0000000000000..152d59b7b1908 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{LongType, StructType} + +// Datasource tests for nested schemas +trait NestedDataSourceSuiteBase extends QueryTest with SharedSparkSession { + protected val nestedDataSources: Seq[String] = Seq("orc", "parquet", "json") + + test("SPARK-32431: consistent error for nested and top-level duplicate columns") { + Seq( + Seq("id AS lowercase", "id + 1 AS camelCase") -> + new StructType() + .add("LowerCase", LongType) + .add("camelcase", LongType) + .add("CamelCase", LongType), + Seq("NAMED_STRUCT('lowercase', id, 'camelCase', id + 1) AS StructColumn") -> + new StructType().add("StructColumn", + new StructType() + .add("LowerCase", LongType) + .add("camelcase", LongType) + .add("CamelCase", LongType)) + ).foreach { case (selectExpr: Seq[String], caseInsensitiveSchema: StructType) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + nestedDataSources.map { format => + withClue(s"format = $format select = ${selectExpr.mkString(",")}") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark + .range(1L) + .selectExpr(selectExpr: _*) + .write.mode("overwrite") + .format(format) + .save(path) + val e = intercept[AnalysisException] { + spark + .read + .schema(caseInsensitiveSchema) + .format(format) + .load(path) + .show + } + assert(e.getMessage.contains( + "Found duplicate column(s) in the data schema: `camelcase`")) + } + } + } + } + } + } +} + +class NestedDataSourceV1Suite extends NestedDataSourceSuiteBase { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, nestedDataSources.mkString(",")) +} + +class NestedDataSourceV2Suite extends NestedDataSourceSuiteBase { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} From e1d73210341a314601a953e6ac483112660874e6 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 30 Jul 2020 15:16:02 +0900 Subject: [PATCH 328/384] [SPARK-32478][R][SQL] Error message to show the schema mismatch in gapply with Arrow vectorization ### What changes were proposed in this pull request? This PR proposes to: 1. Fix the error message when the output schema is misbatched with R DataFrame from the given function. For example, ```R df <- createDataFrame(list(list(a=1L, b="2"))) count(gapply(df, "a", function(key, group) { group }, structType("a int, b int"))) ``` **Before:** ``` Error in handleErrors(returnStatus, conn) : ... java.lang.UnsupportedOperationException ... ``` **After:** ``` Error in handleErrors(returnStatus, conn) : ... java.lang.AssertionError: assertion failed: Invalid schema from gapply: expected IntegerType, IntegerType, got IntegerType, StringType ... ``` 2. Update documentation about the schema matching for `gapply` and `dapply`. ### Why are the changes needed? To show which schema is not matched, and let users know what's going on. ### Does this PR introduce _any_ user-facing change? Yes, error message is updated as above, and documentation is updated. ### How was this patch tested? Manually tested and unitttests were added. Closes #29283 from HyukjinKwon/r-vectorized-error. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/pkg/tests/fulltests/test_sparkSQL_arrow.R | 18 ++++++++++++++++++ docs/sparkr.md | 16 ++++++++-------- .../apache/spark/sql/execution/objects.scala | 9 ++++++++- 3 files changed, 34 insertions(+), 9 deletions(-) diff --git a/R/pkg/tests/fulltests/test_sparkSQL_arrow.R b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R index 97972753a78fa..16d93763ff038 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL_arrow.R +++ b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R @@ -312,4 +312,22 @@ test_that("Arrow optimization - unsupported types", { }) }) +test_that("SPARK-32478: gapply() Arrow optimization - error message for schema mismatch", { + skip_if_not_installed("arrow") + df <- createDataFrame(list(list(a = 1L, b = "a"))) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + expect_error( + count(gapply(df, "a", function(key, group) { group }, structType("a int, b int"))), + "expected IntegerType, IntegerType, got IntegerType, StringType") + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + sparkR.session.stop() diff --git a/docs/sparkr.md b/docs/sparkr.md index fa1bb1b851815..05310f89f278d 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -681,12 +681,12 @@ The current supported minimum version is 1.0.0; however, this might change betwe Arrow optimization is available when converting a Spark DataFrame to an R DataFrame using the call `collect(spark_df)`, when creating a Spark DataFrame from an R DataFrame with `createDataFrame(r_df)`, when applying an R native function to each partition via `dapply(...)` and when applying an R native function to grouped data via `gapply(...)`. -To use Arrow when executing these calls, users need to first set the Spark configuration ‘spark.sql.execution.arrow.sparkr.enabled’ -to ‘true’. This is disabled by default. +To use Arrow when executing these, users need to set the Spark configuration ‘spark.sql.execution.arrow.sparkr.enabled’ +to ‘true’ first. This is disabled by default. -In addition, optimizations enabled by ‘spark.sql.execution.arrow.sparkr.enabled’ could fallback automatically to non-Arrow optimization -implementation if an error occurs before the actual computation within Spark during converting a Spark DataFrame to/from an R -DataFrame. +Whether the optimization is enabled or not, SparkR produces the same results. In addition, the conversion +between Spark DataFrame and R DataFrame falls back automatically to non-Arrow optimization implementation +when the optimization fails for any reasons before the actual computation.
          {% highlight r %} @@ -713,9 +713,9 @@ collect(gapply(spark_df, {% endhighlight %}
          -Using the above optimizations with Arrow will produce the same results as when Arrow is not enabled. Note that even with Arrow, -`collect(spark_df)` results in the collection of all records in the DataFrame to the driver program and should be done on a -small subset of the data. +Note that even with Arrow, `collect(spark_df)` results in the collection of all records in the DataFrame to +the driver program and should be done on a small subset of the data. In addition, the specified output schema +in `gapply(...)` and `dapply(...)` should be matched to the R DataFrame's returned by the given function. ## Supported SQL Types diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 4b2d4195ee906..c08db132c946f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -567,7 +567,14 @@ case class FlatMapGroupsInRWithArrowExec( // binary in a batch due to the limitation of R API. See also ARROW-4512. val columnarBatchIter = runner.compute(groupedByRKey, -1) val outputProject = UnsafeProjection.create(output, output) - columnarBatchIter.flatMap(_.rowIterator().asScala).map(outputProject) + val outputTypes = StructType.fromAttributes(output).map(_.dataType) + + columnarBatchIter.flatMap { batch => + val actualDataTypes = (0 until batch.numCols()).map(i => batch.column(i).dataType()) + assert(outputTypes == actualDataTypes, "Invalid schema from gapply(): " + + s"expected ${outputTypes.mkString(", ")}, got ${actualDataTypes.mkString(", ")}") + batch.rowIterator().asScala + }.map(outputProject) } } } From 510a1656e650246a708d3866c8a400b7a1b9f962 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 30 Jul 2020 06:44:13 +0000 Subject: [PATCH 329/384] [SPARK-32412][SQL] Unify error handling for spark thrift server operations ### What changes were proposed in this pull request? Log error/warn message only once at the server-side for both sync and async modes ### Why are the changes needed? In https://github.com/apache/spark/commit/b151194299f5ba15e0d9d8d7d2980fd164fe0822 we make the error logging for SparkExecuteStatementOperation with `runInBackground=true` not duplicated, but the operations with runInBackground=false and other metadata operation still will be log twice which happened in the operation's `runInternal` method and ThriftCLIService. In this PR, I propose to reflect the logic to get a unified error handling approach. ### Does this PR introduce _any_ user-facing change? Yes, when spark.sql.hive.thriftServer.async=false and people call sync APIs the error message will be logged only once at server-side. ### How was this patch tested? locally verified the result in target/unit-test.log add unit tests. Closes #29204 from yaooqinn/SPARK-32412. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../SparkExecuteStatementOperation.scala | 56 +++-------------- .../hive/thriftserver/SparkOperation.scala | 35 ++++++++--- .../ThriftServerWithSparkContextSuite.scala | 60 +++++++++++++------ 3 files changed, 76 insertions(+), 75 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index d30951f89cf6b..922af72604027 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -19,11 +19,9 @@ package org.apache.spark.sql.hive.thriftserver import java.security.PrivilegedExceptionAction import java.util.{Arrays, Map => JMap} -import java.util.concurrent.RejectedExecutionException import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.util.control.NonFatal import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.shims.Utils @@ -38,7 +36,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.VariableSubstitution import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval -import org.apache.spark.util.{Utils => SparkUtils} private[hive] class SparkExecuteStatementOperation( val sqlContext: SQLContext, @@ -113,7 +110,7 @@ private[hive] class SparkExecuteStatementOperation( } def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = withLocalProperties { - log.info(s"Received getNextRowSet request order=${order} and maxRowsL=${maxRowsL} " + + logInfo(s"Received getNextRowSet request order=${order} and maxRowsL=${maxRowsL} " + s"with ${statementId}") validateDefaultFetchOrientation(order) assertState(OperationState.FINISHED) @@ -182,7 +179,7 @@ private[hive] class SparkExecuteStatementOperation( resultOffset += 1 } previousFetchEndOffset = resultOffset - log.info(s"Returning result set with ${curRow} rows from offsets " + + logInfo(s"Returning result set with ${curRow} rows from offsets " + s"[$previousFetchStartOffset, $previousFetchEndOffset) with $statementId") resultRowSet } @@ -219,7 +216,9 @@ private[hive] class SparkExecuteStatementOperation( execute() } } catch { - case e: HiveSQLException => setOperationException(e) + case e: HiveSQLException => + setOperationException(e) + logError(s"Error executing query with $statementId,", e) } } } @@ -239,21 +238,7 @@ private[hive] class SparkExecuteStatementOperation( val backgroundHandle = parentSession.getSessionManager().submitBackgroundOperation(backgroundOperation) setBackgroundHandle(backgroundHandle) - } catch { - case rejected: RejectedExecutionException => - logError("Error submitting query in background, query rejected", rejected) - setState(OperationState.ERROR) - HiveThriftServer2.eventManager.onStatementError( - statementId, rejected.getMessage, SparkUtils.exceptionString(rejected)) - throw new HiveSQLException("The background threadpool cannot accept" + - " new task for execution, please retry the operation", rejected) - case NonFatal(e) => - logError(s"Error executing query in background", e) - setState(OperationState.ERROR) - HiveThriftServer2.eventManager.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw new HiveSQLException(e) - } + } catch onError() } } @@ -294,30 +279,7 @@ private[hive] class SparkExecuteStatementOperation( } dataTypes = result.schema.fields.map(_.dataType) } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - // When cancel() or close() is called very quickly after the query is started, - // then they may both call cleanup() before Spark Jobs are started. But before background - // task interrupted, it may have start some spark job, so we need to cancel again to - // make sure job was cancelled when background thread was interrupted - if (statementId != null) { - sqlContext.sparkContext.cancelJobGroup(statementId) - } - val currentState = getStatus().getState() - if (currentState.isTerminal) { - // This may happen if the execution was cancelled, and then closed from another thread. - logWarning(s"Ignore exception in terminal state with $statementId: $e") - } else { - logError(s"Error executing query with $statementId, currentState $currentState, ", e) - setState(OperationState.ERROR) - HiveThriftServer2.eventManager.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - e match { - case _: HiveSQLException => throw e - case _ => throw new HiveSQLException("Error running query: " + e.toString, e) - } - } + onError(needCancel = true) } finally { synchronized { if (!getStatus.getState.isTerminal) { @@ -348,9 +310,7 @@ private[hive] class SparkExecuteStatementOperation( } } // RDDs will be cleaned automatically upon garbage collection. - if (statementId != null) { - sqlContext.sparkContext.cancelJobGroup(statementId) - } + sqlContext.sparkContext.cancelJobGroup(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index bbfc1b83379aa..8e8b2d7ff774d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.thriftserver +import java.util.concurrent.RejectedExecutionException + import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.Operation @@ -94,15 +96,32 @@ private[hive] trait SparkOperation extends Operation with Logging { throw new IllegalArgumentException(s"Unknown table type is found: $t") } - protected def onError(): PartialFunction[Throwable, Unit] = { + protected def onError(needCancel: Boolean = false): PartialFunction[Throwable, Unit] = { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. case e: Throwable => - logError(s"Error operating $getType with $statementId", e) - super.setState(OperationState.ERROR) - HiveThriftServer2.eventManager.onStatementError( - statementId, e.getMessage, Utils.exceptionString(e)) - e match { - case _: HiveSQLException => throw e - case _ => throw new HiveSQLException(s"Error operating $getType ${e.getMessage}", e) + // When cancel() or close() is called very quickly after the query is started, + // then they may both call cleanup() before Spark Jobs are started. But before background + // task interrupted, it may have start some spark job, so we need to cancel again to + // make sure job was cancelled when background thread was interrupted + if (needCancel) sqlContext.sparkContext.cancelJobGroup(statementId) + val currentState = getStatus.getState + if (currentState.isTerminal) { + // This may happen if the execution was cancelled, and then closed from another thread. + logWarning(s"Ignore exception in terminal state with $statementId: $e") + } else { + super.setState(OperationState.ERROR) + HiveThriftServer2.eventManager.onStatementError( + statementId, e.getMessage, Utils.exceptionString(e)) + e match { + case _: HiveSQLException => throw e + case rejected: RejectedExecutionException => + throw new HiveSQLException("The background threadpool cannot accept" + + " new task for execution, please retry the operation", rejected) + case _ => + val tips = if (shouldRunAsync()) " in background" else "" + throw new HiveSQLException(s"Error operating $getType$tips: ${e.getMessage}", e) + } } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index fd3a638c4fa44..2bb9169693a15 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -21,6 +21,8 @@ import java.sql.SQLException import org.apache.hive.service.cli.HiveSQLException +import org.apache.spark.sql.hive.HiveUtils + trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("the scratch dir will be deleted during server start but recreated with new operation") { @@ -52,31 +54,51 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("Full stack traces as error message for jdbc or thrift client") { val sql = "select date_sub(date'2011-11-11', '1.2')" - withCLIServiceClient { client => - val sessionHandle = client.openSession(user, "") + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] - val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] - val e = intercept[HiveSQLException] { - client.executeStatement( - sessionHandle, - sql, - confOverlay) + withSQLConf((HiveUtils.HIVE_THRIFT_SERVER_ASYNC.key, "false")) { + withCLIServiceClient { client => + val sessionHandle = client.openSession(user, "") + val e = intercept[HiveSQLException] { + client.executeStatement(sessionHandle, sql, confOverlay) + } + assert(e.getMessage + .contains("The second argument of 'date_sub' function needs to be an integer.")) + assert(!e.getMessage + .contains("java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) } + } + + withSQLConf((HiveUtils.HIVE_THRIFT_SERVER_ASYNC.key, "true")) { + withCLIServiceClient { client => + val sessionHandle = client.openSession(user, "") + val opHandle = client.executeStatementAsync(sessionHandle, sql, confOverlay) + var status = client.getOperationStatus(opHandle) + while (!status.getState.isTerminal) { + Thread.sleep(10) + status = client.getOperationStatus(opHandle) + } + val e = status.getOperationException - assert(e.getMessage - .contains("The second argument of 'date_sub' function needs to be an integer.")) - assert(!e.getMessage.contains("" + - "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + assert(e.getMessage + .contains("The second argument of 'date_sub' function needs to be an integer.")) + assert(e.getMessage + .contains("java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + } } - withJdbcStatement { statement => - val e = intercept[SQLException] { - statement.executeQuery(sql) + Seq("true", "false").foreach { value => + withSQLConf((HiveUtils.HIVE_THRIFT_SERVER_ASYNC.key, value)) { + withJdbcStatement { statement => + val e = intercept[SQLException] { + statement.executeQuery(sql) + } + assert(e.getMessage.contains( + "The second argument of 'date_sub' function needs to be an integer.")) + assert(e.getMessage.contains( + "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + } } - assert(e.getMessage - .contains("The second argument of 'date_sub' function needs to be an integer.")) - assert(e.getMessage.contains("" + - "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) } } } From 30e3042dc5febf49123483184e6282fefde8ebc0 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 30 Jul 2020 07:51:27 +0000 Subject: [PATCH 330/384] [SPARK-32488][SQL] Use @parser::members and @lexer::members to avoid generating unused code ### What changes were proposed in this pull request? This PR aims to update `SqlBse.g4` for avoiding generating unused code. Currently, ANTLR generates unused methods and variables; `isValidDecimal` and `isHint` are only used in the generated lexer. This PR changed the code to use `parser::members` and `lexer::members` to avoid it. ### Why are the changes needed? To reduce unnecessary code. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #29296 from maropu/UpdateSqlBase. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 14 ++++++++------ .../spark/sql/catalyst/parser/ParseDriver.scala | 3 --- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 4631165981aa5..39f94651a0cb5 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -16,7 +16,7 @@ grammar SqlBase; -@members { +@parser::members { /** * When false, INTERSECT is given the greater precedence over the other set * operations (UNION, EXCEPT and MINUS) as per the SQL standard. @@ -29,6 +29,13 @@ grammar SqlBase; */ public boolean legacy_exponent_literal_as_decimal_enabled = false; + /** + * When true, the behavior of keywords follows ANSI SQL standard. + */ + public boolean SQL_standard_keyword_behavior = false; +} + +@lexer::members { /** * Verify whether current token is a valid decimal token (which contains dot). * Returns true if the character that follows the token is not a digit or letter or underscore. @@ -51,11 +58,6 @@ grammar SqlBase; } } - /** - * When true, the behavior of keywords follows ANSI SQL standard. - */ - public boolean SQL_standard_keyword_behavior = false; - /** * This method will be called when we see '/*' and try to match it as a bracketed comment. * If the next character is '+', it should be parsed as hint later, and we cannot match diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index fab282f15f215..343b499f7231f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -96,9 +96,6 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) - lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - lexer.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled - lexer.SQL_standard_keyword_behavior = conf.ansiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) From 1f7fe5415e88cc289b44e366cd4e74290784db5f Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 30 Jul 2020 19:28:34 +0900 Subject: [PATCH 331/384] [SPARK-32491][INFRA] Do not install SparkR in test-only mode in testing script ### What changes were proposed in this pull request? This PR proposes to skip SparkR installation that is to run R linters (see SPARK-8505) in the test-only mode at `dev/run-tests.py` script. As of SPARK-32292, the test-only mode in `dev/run-tests.py` was introduced, for example: ``` dev/run-tests.py --modules sql,core ``` which only runs the relevant tests and does not run other tests such as linters. Therefore, we don't need to install SparkR when `--modules` are specified. ### Why are the changes needed? GitHub Actions build is currently failed as below: ``` ERROR: this R is version 3.4.4, package 'SparkR' requires R >= 3.5 [error] running /home/runner/work/spark/spark/R/install-dev.sh ; received return code 1 ##[error]Process completed with exit code 10. ``` For some reasons, looks GitHub Actions started to have R 3.4.4 installed by default; however, R 3.4 was dropped as of SPARK-32073. When SparkR tests are not needed, GitHub Actions still builds SparkR with a low R version and it causes the test failure. This PR partially fixes it by avoid the installation of SparkR. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? GitHub Actions tests should run to confirm this fix is correct. Closes #29300 from HyukjinKwon/install-r. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/run-tests.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 065a27c0e853b..dfb5974427d46 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -609,11 +609,14 @@ def main(): " install one and retry.") sys.exit(2) - # install SparkR - if which("R"): - run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")]) - else: - print("Cannot install SparkR as R was not found in PATH") + # Install SparkR + should_only_test_modules = opts.modules is not None + if not should_only_test_modules: + # If tests modules are specified, we will not run R linter. + if which("R"): + run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")]) + else: + print("Cannot install SparkR as R was not found in PATH") if os.environ.get("AMPLAB_JENKINS"): # if we're on the Amplab Jenkins build servers setup variables @@ -643,7 +646,6 @@ def main(): changed_modules = None test_modules = None changed_files = None - should_only_test_modules = opts.modules is not None included_tags = [] excluded_tags = [] if should_only_test_modules: From e0c8bd07af6ea2873c77ae6428b3ab4ee68e8e32 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 30 Jul 2020 20:06:35 +0900 Subject: [PATCH 332/384] [SPARK-32493][INFRA] Manually install R instead of using setup-r in GitHub Actions ### What changes were proposed in this pull request? This PR proposes to manually install R instead of using `setup-r` which seems broken. Currently, GitHub Actions uses its default R 3.4.4 installed, which we dropped as of SPARK-32073. While I am here, I am also upgrading R version to 4.0. Jenkins will test the old version and GitHub Actions tests the new version. AppVeyor uses R 4.0 but it does not check CRAN which is important when we make a release. ### Why are the changes needed? To recover GitHub Actions build. ### Does this PR introduce _any_ user-facing change? No, dev-only ### How was this patch tested? Manually tested at https://github.com/HyukjinKwon/spark/pull/15 Closes #29302 from HyukjinKwon/SPARK-32493. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .github/workflows/master.yml | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 7ff960bf9b7be..881e11d5ea4ff 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -154,11 +154,13 @@ jobs: python3.8 -m pip install numpy pyarrow pandas scipy python3.8 -m pip list # SparkR - - name: Install R 3.6 - uses: r-lib/actions/setup-r@v1 + - name: Install R 4.0 if: contains(matrix.modules, 'sparkr') - with: - r-version: 3.6 + run: | + sudo sh -c "echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list" + curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add + sudo apt-get update + sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev - name: Install R packages if: contains(matrix.modules, 'sparkr') run: | @@ -203,10 +205,12 @@ jobs: # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. # See also https://github.com/sphinx-doc/sphinx/issues/7551. pip3 install flake8 'sphinx<3.1.0' numpy pydata_sphinx_theme - - name: Install R 3.6 - uses: r-lib/actions/setup-r@v1 - with: - r-version: 3.6 + - name: Install R 4.0 + run: | + sudo sh -c "echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list" + curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add + sudo apt-get update + sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev - name: Install R linter dependencies and SparkR run: | sudo apt-get install -y libcurl4-openssl-dev From 12f443cd99a91689dc5b44b6794205289ef2d998 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 30 Jul 2020 21:07:31 +0900 Subject: [PATCH 333/384] [SPARK-32496][INFRA] Include GitHub Action file as the changes in testing ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/26556 excluded `.github/workflows/master.yml`. So tests are skipped if the GitHub Actions configuration file is changed. As of SPARK-32245, we now run the regular tests via the testing script. We should include it to test to make sure GitHub Actions build does not break due to some changes such as Python versions. ### Why are the changes needed? For better test coverage in GitHub Actions build. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? GitHub Actions in this PR will test. Closes #29305 from HyukjinKwon/SPARK-32496. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/run-tests.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index dfb5974427d46..5d8b7a075826e 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -49,13 +49,12 @@ def determine_modules_for_files(filenames): ['pyspark-core', 'sql'] >>> [x.name for x in determine_modules_for_files(["file_not_matched_by_any_subproject"])] ['root'] - >>> [x.name for x in determine_modules_for_files( \ - [".github/workflows/master.yml", "appveyor.yml"])] + >>> [x.name for x in determine_modules_for_files(["appveyor.yml"])] [] """ changed_modules = set() for filename in filenames: - if filename in (".github/workflows/master.yml", "appveyor.yml"): + if filename in ("appveyor.yml",): continue matched_at_least_one_module = False for module in modules.all_modules: From 743772095273b464f845efefb3eb59284b06b9be Mon Sep 17 00:00:00 2001 From: Warren Zhu Date: Thu, 30 Jul 2020 21:44:49 +0900 Subject: [PATCH 334/384] [SPARK-32227] Fix regression bug in load-spark-env.cmd with Spark 3.0.0 ### What changes were proposed in this pull request? Fix regression bug in load-spark-env.cmd with Spark 3.0.0 ### Why are the changes needed? cmd doesn't support set env twice. So set `SPARK_ENV_CMD=%SPARK_CONF_DIR%\%SPARK_ENV_CMD%` doesn't take effect, which caused regression. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually tested. 1. Create a spark-env.cmd under conf folder. Inside this, `echo spark-env.cmd` 2. Run old load-spark-env.cmd, nothing printed in the output 2. Run fixed load-spark-env.cmd, `spark-env.cmd` showed in the output. Closes #29044 from warrenzhu25/32227. Lead-authored-by: Warren Zhu Co-authored-by: Warren Zhu Signed-off-by: HyukjinKwon --- bin/load-spark-env.cmd | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index 6cb8bf7fc7a21..fe725a4e1a368 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -21,7 +21,6 @@ rem This script loads spark-env.cmd if it exists, and ensures it is only loaded rem spark-env.cmd is loaded from SPARK_CONF_DIR if set, or within the current directory's rem conf\ subdirectory. -set SPARK_ENV_CMD=spark-env.cmd if not defined SPARK_ENV_LOADED ( set SPARK_ENV_LOADED=1 @@ -29,10 +28,7 @@ if not defined SPARK_ENV_LOADED ( set SPARK_CONF_DIR=%~dp0..\conf ) - set SPARK_ENV_CMD=%SPARK_CONF_DIR%\%SPARK_ENV_CMD% - if exist %SPARK_ENV_CMD% ( - call %SPARK_ENV_CMD% - ) + call :LoadSparkEnv ) rem Setting SPARK_SCALA_VERSION if not already set. @@ -59,3 +55,8 @@ if not defined SPARK_SCALA_VERSION ( ) ) exit /b 0 + +:LoadSparkEnv +if exist "%SPARK_CONF_DIR%\spark-env.cmd" ( + call "%SPARK_CONF_DIR%\spark-env.cmd" +) From 32f4ef005fd590e0e7c319b43a459cb3828bba5a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 31 Jul 2020 00:57:24 +0900 Subject: [PATCH 335/384] [SPARK-32497][INFRA] Installs qpdf package for CRAN check in GitHub Actions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? CRAN check fails due to the size of the generated PDF docs as below: ``` ... WARNING ‘qpdf’ is needed for checks on size reduction of PDFs ... Status: 1 WARNING, 1 NOTE See ‘/home/runner/work/spark/spark/R/SparkR.Rcheck/00check.log’ for details. ``` This PR proposes to install `qpdf` in GitHub Actions. Note that I cannot reproduce in my local with the same R version so I am not documenting it for now. Also, while I am here, I piggyback to install SparkR when the module includes `sparkr`. it is rather a followup of SPARK-32491. ### Why are the changes needed? To fix SparkR CRAN check failure. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? GitHub Actions will test it out. Closes #29306 from HyukjinKwon/SPARK-32497. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .github/workflows/master.yml | 3 ++- dev/run-tests.py | 16 +++++++++------- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 881e11d5ea4ff..009ebe90ddf51 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -164,7 +164,8 @@ jobs: - name: Install R packages if: contains(matrix.modules, 'sparkr') run: | - sudo apt-get install -y libcurl4-openssl-dev + # qpdf is required to reduce the size of PDFs to make CRAN check pass. See SPARK-32497. + sudo apt-get install -y libcurl4-openssl-dev qpdf sudo Rscript -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'devtools', 'e1071', 'survival', 'arrow', 'roxygen2'), repos='https://cloud.r-project.org/')" # Show installed packages in R. sudo Rscript -e 'pkg_list <- as.data.frame(installed.packages()[, c(1,3:4)]); pkg_list[is.na(pkg_list$Priority), 1:2, drop = FALSE]' diff --git a/dev/run-tests.py b/dev/run-tests.py index 5d8b7a075826e..6aae3bdaefaff 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -610,8 +610,14 @@ def main(): # Install SparkR should_only_test_modules = opts.modules is not None - if not should_only_test_modules: + test_modules = [] + if should_only_test_modules: + str_test_modules = [m.strip() for m in opts.modules.split(",")] + test_modules = [m for m in modules.all_modules if m.name in str_test_modules] + + if not should_only_test_modules or modules.sparkr in test_modules: # If tests modules are specified, we will not run R linter. + # SparkR needs the manual SparkR installation. if which("R"): run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")]) else: @@ -642,15 +648,11 @@ def main(): "and Hive profile", hive_version, "under environment", test_env) extra_profiles = get_hadoop_profiles(hadoop_version) + get_hive_profiles(hive_version) - changed_modules = None - test_modules = None - changed_files = None + changed_modules = [] + changed_files = [] included_tags = [] excluded_tags = [] if should_only_test_modules: - str_test_modules = [m.strip() for m in opts.modules.split(",")] - test_modules = [m for m in modules.all_modules if m.name in str_test_modules] - # If we're running the tests in Github Actions, attempt to detect and test # only the affected modules. if test_env == "github_actions": From 7cf3b54a2a7528e815841015af50e08ce4515cb9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 30 Jul 2020 10:59:26 -0700 Subject: [PATCH 336/384] [SPARK-32489][CORE] Pass `core` module UTs in Scala 2.13 ### What changes were proposed in this pull request? So far, we fixed many stuffs in `core` module. This PR fixes the remaining UT failures in Scala 2.13. - `OneApplicationResource.environmentInfo` will return a deterministic result for `sparkProperties`, `hadoopProperties`, `systemProperties`, and `classpathEntries`. - `SubmitRestProtocolSuite` has Scala 2.13 answer in addition to the existing Scala 2.12 answer, and uses the expected answer based on the Scala runtime version. ### Why are the changes needed? To support Scala 2.13. ### Does this PR introduce _any_ user-facing change? Yes, `environmentInfo` is changed, but this fixes the indeterministic behavior. ### How was this patch tested? - Scala 2.12: Pass the Jenkins or GitHub Action - Scala 2.13: Do the following. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 ``` **BEFORE** ``` Tests: succeeded 2612, failed 3, canceled 1, ignored 8, pending 0 *** 3 TESTS FAILED *** ``` **AFTER** ``` Tests: succeeded 2615, failed 0, canceled 1, ignored 8, pending 0 All tests passed. ``` Closes #29298 from dongjoon-hyun/SPARK-32489. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../api/v1/OneApplicationResource.scala | 8 +- .../app_environment_expectation.json | 456 +++++++++--------- .../deploy/rest/SubmitRestProtocolSuite.scala | 31 +- 3 files changed, 262 insertions(+), 233 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala index 05a33f17a5d95..536a1fcd59cd0 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -104,10 +104,10 @@ private[v1] class AbstractApplicationResource extends BaseAppResource { val resourceProfileInfo = ui.store.resourceProfileInfo() new v1.ApplicationEnvironmentInfo( envInfo.runtime, - Utils.redact(ui.conf, envInfo.sparkProperties), - Utils.redact(ui.conf, envInfo.hadoopProperties), - Utils.redact(ui.conf, envInfo.systemProperties), - envInfo.classpathEntries, + Utils.redact(ui.conf, envInfo.sparkProperties).sortBy(_._1), + Utils.redact(ui.conf, envInfo.hadoopProperties).sortBy(_._1), + Utils.redact(ui.conf, envInfo.systemProperties).sortBy(_._1), + envInfo.classpathEntries.sortBy(_._1), resourceProfileInfo) } diff --git a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json index 0b617a7d0aced..c2616129de954 100644 --- a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json @@ -5,283 +5,283 @@ "scalaVersion" : "version 2.11.8" }, "sparkProperties" : [ - [ "spark.blacklist.task.maxTaskAttemptsPerExecutor", "3" ], + [ "spark.app.id", "app-20161116163331-0000" ], + [ "spark.app.name", "Spark shell" ], + [ "spark.blacklist.application.maxFailedExecutorsPerNode", "2" ], + [ "spark.blacklist.application.maxFailedTasksPerExecutor", "1" ], [ "spark.blacklist.enabled", "TRUE" ], - [ "spark.driver.host", "172.22.0.167" ], + [ "spark.blacklist.stage.maxFailedExecutorsPerNode", "3" ], + [ "spark.blacklist.stage.maxFailedTasksPerExecutor", "3" ], + [ "spark.blacklist.task.maxTaskAttemptsPerExecutor", "3" ], [ "spark.blacklist.task.maxTaskAttemptsPerNode", "3" ], - [ "spark.eventLog.enabled", "TRUE" ], + [ "spark.blacklist.timeout", "1000000" ], + [ "spark.driver.host", "172.22.0.167" ], [ "spark.driver.port", "51459" ], - [ "spark.repl.class.uri", "spark://172.22.0.167:51459/classes" ], - [ "spark.jars", "" ], - [ "spark.repl.class.outputDir", "/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed" ], - [ "spark.app.name", "Spark shell" ], - [ "spark.blacklist.stage.maxFailedExecutorsPerNode", "3" ], - [ "spark.scheduler.mode", "FIFO" ], + [ "spark.eventLog.compress", "FALSE" ], + [ "spark.eventLog.dir", "/Users/jose/logs" ], + [ "spark.eventLog.enabled", "TRUE" ], [ "spark.eventLog.overwrite", "TRUE" ], - [ "spark.blacklist.stage.maxFailedTasksPerExecutor", "3" ], [ "spark.executor.id", "driver" ], - [ "spark.blacklist.application.maxFailedExecutorsPerNode", "2" ], - [ "spark.submit.deployMode", "client" ], - [ "spark.master", "local-cluster[4,4,1024]" ], [ "spark.home", "/Users/Jose/IdeaProjects/spark" ], - [ "spark.eventLog.dir", "/Users/jose/logs" ], + [ "spark.jars", "" ], + [ "spark.master", "local-cluster[4,4,1024]" ], + [ "spark.repl.class.outputDir", "/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed" ], + [ "spark.repl.class.uri", "spark://172.22.0.167:51459/classes" ], + [ "spark.scheduler.mode", "FIFO" ], [ "spark.sql.catalogImplementation", "in-memory" ], - [ "spark.eventLog.compress", "FALSE" ], - [ "spark.blacklist.application.maxFailedTasksPerExecutor", "1" ], - [ "spark.blacklist.timeout", "1000000" ], - [ "spark.app.id", "app-20161116163331-0000" ], + [ "spark.submit.deployMode", "client" ], [ "spark.task.maxFailures", "4" ] ], "hadoopProperties" : [ + [ "mapreduce.jobhistory.client.thread-count", "10" ], [ "mapreduce.jobtracker.address", "local" ], - [ "yarn.resourcemanager.scheduler.monitor.policies", "org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy" ], - [ "mapreduce.jobhistory.client.thread-count", "10" ] + [ "yarn.resourcemanager.scheduler.monitor.policies", "org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy" ] ], "systemProperties" : [ - [ "java.io.tmpdir", "/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/" ], - [ "line.separator", "\n" ], - [ "path.separator", ":" ], - [ "sun.management.compiler", "HotSpot 64-Bit Tiered Compilers" ], [ "SPARK_SUBMIT", "true" ], - [ "sun.cpu.endian", "little" ], - [ "java.specification.version", "1.8" ], - [ "java.vm.specification.name", "Java Virtual Machine Specification" ], - [ "java.vendor", "Oracle Corporation" ], - [ "java.vm.specification.version", "1.8" ], - [ "user.home", "/Users/Jose" ], + [ "awt.toolkit", "sun.lwawt.macosx.LWCToolkit" ], + [ "file.encoding", "UTF-8" ], [ "file.encoding.pkg", "sun.io" ], - [ "sun.nio.ch.bugLevel", "" ], + [ "file.separator", "/" ], [ "ftp.nonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], - [ "sun.arch.data.model", "64" ], - [ "sun.boot.library.path", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib" ], - [ "user.dir", "/Users/Jose/IdeaProjects/spark" ], - [ "java.library.path", "/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:." ], - [ "sun.cpu.isalist", "" ], - [ "os.arch", "x86_64" ], - [ "java.vm.version", "25.92-b14" ], + [ "gopherProxySet", "false" ], + [ "http.nonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], + [ "io.netty.maxDirectMemory", "0" ], + [ "java.awt.graphicsenv", "sun.awt.CGraphicsEnvironment" ], + [ "java.awt.printerjob", "sun.lwawt.macosx.CPrinterJob" ], + [ "java.class.version", "52.0" ], [ "java.endorsed.dirs", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed" ], - [ "java.runtime.version", "1.8.0_92-b14" ], - [ "java.vm.info", "mixed mode" ], [ "java.ext.dirs", "/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java" ], + [ "java.home", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre" ], + [ "java.io.tmpdir", "/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/" ], + [ "java.library.path", "/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:." ], [ "java.runtime.name", "Java(TM) SE Runtime Environment" ], - [ "file.separator", "/" ], - [ "io.netty.maxDirectMemory", "0" ], - [ "java.class.version", "52.0" ], - [ "scala.usejavacp", "true" ], + [ "java.runtime.version", "1.8.0_92-b14" ], [ "java.specification.name", "Java Platform API Specification" ], - [ "sun.boot.class.path", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes" ], - [ "file.encoding", "UTF-8" ], - [ "user.timezone", "America/Chicago" ], [ "java.specification.vendor", "Oracle Corporation" ], - [ "sun.java.launcher", "SUN_STANDARD" ], + [ "java.specification.version", "1.8" ], + [ "java.vendor", "Oracle Corporation" ], + [ "java.vendor.url", "http://java.oracle.com/" ], + [ "java.vendor.url.bug", "http://bugreport.sun.com/bugreport/" ], + [ "java.version", "1.8.0_92" ], + [ "java.vm.info", "mixed mode" ], + [ "java.vm.name", "Java HotSpot(TM) 64-Bit Server VM" ], + [ "java.vm.specification.name", "Java Virtual Machine Specification" ], + [ "java.vm.specification.vendor", "Oracle Corporation" ], + [ "java.vm.specification.version", "1.8" ], + [ "java.vm.vendor", "Oracle Corporation" ], + [ "java.vm.version", "25.92-b14" ], + [ "line.separator", "\n" ], + [ "os.arch", "x86_64" ], + [ "os.name", "Mac OS X" ], [ "os.version", "10.11.6" ], + [ "path.separator", ":" ], + [ "scala.usejavacp", "true" ], + [ "socksNonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], + [ "sun.arch.data.model", "64" ], + [ "sun.boot.class.path", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes" ], + [ "sun.boot.library.path", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib" ], + [ "sun.cpu.endian", "little" ], + [ "sun.cpu.isalist", "" ], + [ "sun.io.unicode.encoding", "UnicodeBig" ], + [ "sun.java.command", "org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala" ], + [ "sun.java.launcher", "SUN_STANDARD" ], + [ "sun.jnu.encoding", "UTF-8" ], + [ "sun.management.compiler", "HotSpot 64-Bit Tiered Compilers" ], + [ "sun.nio.ch.bugLevel", "" ], [ "sun.os.patch.level", "unknown" ], - [ "gopherProxySet", "false" ], - [ "java.vm.specification.vendor", "Oracle Corporation" ], [ "user.country", "US" ], - [ "sun.jnu.encoding", "UTF-8" ], - [ "http.nonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], + [ "user.dir", "/Users/Jose/IdeaProjects/spark" ], + [ "user.home", "/Users/Jose" ], [ "user.language", "en" ], - [ "socksNonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], - [ "java.vendor.url", "http://java.oracle.com/" ], - [ "java.awt.printerjob", "sun.lwawt.macosx.CPrinterJob" ], - [ "java.awt.graphicsenv", "sun.awt.CGraphicsEnvironment" ], - [ "awt.toolkit", "sun.lwawt.macosx.LWCToolkit" ], - [ "os.name", "Mac OS X" ], - [ "java.vm.vendor", "Oracle Corporation" ], - [ "java.vendor.url.bug", "http://bugreport.sun.com/bugreport/" ], [ "user.name", "jose" ], - [ "java.vm.name", "Java HotSpot(TM) 64-Bit Server VM" ], - [ "sun.java.command", "org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala" ], - [ "java.home", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre" ], - [ "java.version", "1.8.0_92" ], - [ "sun.io.unicode.encoding", "UnicodeBig" ] + [ "user.timezone", "America/Chicago" ] ], "classpathEntries" : [ - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/conf/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/core/target/jars/*", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar", "System Classpath" ] + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/conf/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/core/target/jars/*", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes", "System Classpath" ] ], "resourceProfiles" : [ ] } diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala index 03102fd8c696c..d08052faa0043 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.rest import java.lang.Boolean +import scala.util.Properties.versionNumberString + import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkConf, SparkFunSuite} @@ -232,7 +234,7 @@ class SubmitRestProtocolSuite extends SparkFunSuite { |} """.stripMargin - private val submitDriverRequestJson = + private lazy val submitDriverRequestJson = if (versionNumberString.startsWith("2.12")) { s""" |{ | "action" : "CreateSubmissionRequest", @@ -258,6 +260,33 @@ class SubmitRestProtocolSuite extends SparkFunSuite { | } |} """.stripMargin + } else { + s""" + |{ + | "action" : "CreateSubmissionRequest", + | "appArgs" : [ "two slices", "a hint of cinnamon" ], + | "appResource" : "honey-walnut-cherry.jar", + | "clientSparkVersion" : "1.2.3", + | "environmentVariables" : { + | "PATH" : "/dev/null" + | }, + | "mainClass" : "org.apache.spark.examples.SparkPie", + | "sparkProperties" : { + | "spark.driver.extraLibraryPath" : "pickle.jar", + | "spark.jars" : "mayonnaise.jar,ketchup.jar", + | "spark.driver.supervise" : "false", + | "spark.driver.memory" : "${Utils.DEFAULT_DRIVER_MEM_MB}m", + | "spark.files" : "fireball.png", + | "spark.driver.cores" : "180", + | "spark.driver.extraJavaOptions" : " -Dslices=5 -Dcolor=mostly_red", + | "spark.app.name" : "SparkPie", + | "spark.cores.max" : "10000", + | "spark.executor.memory" : "256m", + | "spark.driver.extraClassPath" : "food-coloring.jar" + | } + |} + """.stripMargin + } private val submitDriverResponseJson = """ From 366a1789333bac97643159857a206bcd773761a4 Mon Sep 17 00:00:00 2001 From: Devesh Agrawal Date: Thu, 30 Jul 2020 11:58:11 -0700 Subject: [PATCH 337/384] [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning ### What changes were proposed in this pull request? This PR reduces the prospect of a job loss during decommissioning. It fixes two holes in the current decommissioning framework: - (a) Loss of decommissioned executors is not treated as a job failure: We know that the decommissioned executor would be dying soon, so its death is clearly not caused by the application. - (b) Shuffle files on the decommissioned host are cleared when the first fetch failure is detected from a decommissioned host: This is a bit tricky in terms of when to clear the shuffle state ? Ideally you want to clear it the millisecond before the shuffle service on the node dies (or the executor dies when there is no external shuffle service) -- too soon and it could lead to some wastage and too late would lead to fetch failures. The approach here is to do this clearing when the very first fetch failure is observed on the decommissioned block manager, without waiting for other blocks to also signal a failure. ### Why are the changes needed? Without them decommissioning a lot of executors at a time leads to job failures. ### Code overview The task scheduler tracks the executors that were decommissioned along with their `ExecutorDecommissionInfo`. This information is used by: (a) For handling a `ExecutorProcessLost` error, or (b) by the `DAGScheduler` when handling a fetch failure. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a new unit test `DecommissionWorkerSuite` to test the new behavior by exercising the Master-Worker decommissioning. I chose to add a new test since the setup logic was quite different from the existing `WorkerDecommissionSuite`. I am open to changing the name of the newly added test suite :-) ### Questions for reviewers - Should I add a feature flag to guard these two behaviors ? They seem safe to me that they should only get triggered by decommissioning, but you never know :-). Closes #29014 from agrawaldevesh/decom_harden. Authored-by: Devesh Agrawal Signed-off-by: Holden Karau --- .../apache/spark/scheduler/DAGScheduler.scala | 19 +- .../spark/scheduler/ExecutorLossReason.scala | 7 +- .../spark/scheduler/TaskScheduler.scala | 5 + .../spark/scheduler/TaskSchedulerImpl.scala | 37 +- .../spark/scheduler/TaskSetManager.scala | 1 + .../deploy/DecommissionWorkerSuite.scala | 424 ++++++++++++++++++ .../spark/scheduler/DAGSchedulerSuite.scala | 4 + .../ExternalClusterManagerSuite.scala | 2 + .../scheduler/TaskSchedulerImplSuite.scala | 47 ++ 9 files changed, 539 insertions(+), 7 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2503ae0856dc7..6b376cdadc66b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1821,10 +1821,19 @@ private[spark] class DAGScheduler( // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { - val hostToUnregisterOutputs = if (env.blockManager.externalShuffleServiceEnabled && - unRegisterOutputOnHostOnFetchFailure) { - // We had a fetch failure with the external shuffle service, so we - // assume all shuffle data on the node is bad. + val externalShuffleServiceEnabled = env.blockManager.externalShuffleServiceEnabled + val isHostDecommissioned = taskScheduler + .getExecutorDecommissionInfo(bmAddress.executorId) + .exists(_.isHostDecommissioned) + + // Shuffle output of all executors on host `bmAddress.host` may be lost if: + // - External shuffle service is enabled, so we assume that all shuffle data on node is + // bad. + // - Host is decommissioned, thus all executors on that host will die. + val shuffleOutputOfEntireHostLost = externalShuffleServiceEnabled || + isHostDecommissioned + val hostToUnregisterOutputs = if (shuffleOutputOfEntireHostLost + && unRegisterOutputOnHostOnFetchFailure) { Some(bmAddress.host) } else { // Unregister shuffle data just for one executor (we don't have any @@ -2339,7 +2348,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case ExecutorLost(execId, reason) => val workerLost = reason match { - case ExecutorProcessLost(_, true) => true + case ExecutorProcessLost(_, true, _) => true case _ => false } dagScheduler.handleExecutorLost(execId, workerLost) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index 4141ed799a4e0..671dedaa5a6e8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -54,9 +54,14 @@ private [spark] object LossReasonPending extends ExecutorLossReason("Pending los /** * @param _message human readable loss reason * @param workerLost whether the worker is confirmed lost too (i.e. including shuffle service) + * @param causedByApp whether the loss of the executor is the fault of the running app. + * (assumed true by default unless known explicitly otherwise) */ private[spark] -case class ExecutorProcessLost(_message: String = "Worker lost", workerLost: Boolean = false) +case class ExecutorProcessLost( + _message: String = "Executor Process Lost", + workerLost: Boolean = false, + causedByApp: Boolean = true) extends ExecutorLossReason(_message) /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index b29458c481413..1101d0616d2bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -103,6 +103,11 @@ private[spark] trait TaskScheduler { */ def executorDecommission(executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit + /** + * If an executor is decommissioned, return its corresponding decommission info + */ + def getExecutorDecommissionInfo(executorId: String): Option[ExecutorDecommissionInfo] + /** * Process a lost executor */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 510318afcb8df..b734d9f72944a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -136,6 +136,8 @@ private[spark] class TaskSchedulerImpl( // IDs of the tasks running on each executor private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]] + private val executorsPendingDecommission = new HashMap[String, ExecutorDecommissionInfo] + def runningTasksByExecutors: Map[String, Int] = synchronized { executorIdToRunningTaskIds.toMap.mapValues(_.size).toMap } @@ -939,12 +941,43 @@ private[spark] class TaskSchedulerImpl( override def executorDecommission( executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = { + synchronized { + // Don't bother noting decommissioning for executors that we don't know about + if (executorIdToHost.contains(executorId)) { + // The scheduler can get multiple decommission updates from multiple sources, + // and some of those can have isHostDecommissioned false. We merge them such that + // if we heard isHostDecommissioned ever true, then we keep that one since it is + // most likely coming from the cluster manager and thus authoritative + val oldDecomInfo = executorsPendingDecommission.get(executorId) + if (oldDecomInfo.isEmpty || !oldDecomInfo.get.isHostDecommissioned) { + executorsPendingDecommission(executorId) = decommissionInfo + } + } + } rootPool.executorDecommission(executorId) backend.reviveOffers() } - override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = { + override def getExecutorDecommissionInfo(executorId: String) + : Option[ExecutorDecommissionInfo] = synchronized { + executorsPendingDecommission.get(executorId) + } + + override def executorLost(executorId: String, givenReason: ExecutorLossReason): Unit = { var failedExecutor: Option[String] = None + val reason = givenReason match { + // Handle executor process loss due to decommissioning + case ExecutorProcessLost(message, origWorkerLost, origCausedByApp) => + val executorDecommissionInfo = getExecutorDecommissionInfo(executorId) + ExecutorProcessLost( + message, + // Also mark the worker lost if we know that the host was decommissioned + origWorkerLost || executorDecommissionInfo.exists(_.isHostDecommissioned), + // Executor loss is certainly not caused by app if we knew that this executor is being + // decommissioned + causedByApp = executorDecommissionInfo.isEmpty && origCausedByApp) + case e => e + } synchronized { if (executorIdToRunningTaskIds.contains(executorId)) { @@ -1033,6 +1066,8 @@ private[spark] class TaskSchedulerImpl( } } + executorsPendingDecommission -= executorId + if (reason != LossReasonPending) { executorIdToHost -= executorId rootPool.executorLost(executorId, host, reason) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 4b31ff0c790da..d69f358cd19de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -985,6 +985,7 @@ private[spark] class TaskSetManager( val exitCausedByApp: Boolean = reason match { case exited: ExecutorExited => exited.exitCausedByApp case ExecutorKilled => false + case ExecutorProcessLost(_, _, false) => false case _ => true } handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure(info.executorId, exitCausedByApp, diff --git a/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala new file mode 100644 index 0000000000000..ee9a6be03868f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala @@ -0,0 +1,424 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.concurrent.duration._ + +import org.scalatest.BeforeAndAfterEach +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark._ +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState, WorkerDecommission} +import org.apache.spark.deploy.master.{ApplicationInfo, Master, WorkerInfo} +import org.apache.spark.deploy.worker.Worker +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.network.TransportContext +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.ExternalBlockHandler +import org.apache.spark.rpc.{RpcAddress, RpcEnv} +import org.apache.spark.scheduler._ +import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.Utils + +class DecommissionWorkerSuite + extends SparkFunSuite + with Logging + with LocalSparkContext + with BeforeAndAfterEach { + + private var masterAndWorkerConf: SparkConf = null + private var masterAndWorkerSecurityManager: SecurityManager = null + private var masterRpcEnv: RpcEnv = null + private var master: Master = null + private var workerIdToRpcEnvs: mutable.HashMap[String, RpcEnv] = null + private var workers: mutable.ArrayBuffer[Worker] = null + + override def beforeEach(): Unit = { + super.beforeEach() + masterAndWorkerConf = new SparkConf() + .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) + masterAndWorkerSecurityManager = new SecurityManager(masterAndWorkerConf) + masterRpcEnv = RpcEnv.create( + Master.SYSTEM_NAME, + "localhost", + 0, + masterAndWorkerConf, + masterAndWorkerSecurityManager) + master = makeMaster() + workerIdToRpcEnvs = mutable.HashMap.empty + workers = mutable.ArrayBuffer.empty + } + + override def afterEach(): Unit = { + try { + masterRpcEnv.shutdown() + workerIdToRpcEnvs.values.foreach(_.shutdown()) + workerIdToRpcEnvs.clear() + master.stop() + workers.foreach(_.stop()) + workers.clear() + masterRpcEnv = null + } finally { + super.afterEach() + } + } + + test("decommission workers should not result in job failure") { + val maxTaskFailures = 2 + val numTimesToKillWorkers = maxTaskFailures + 1 + val numWorkers = numTimesToKillWorkers + 1 + createWorkers(numWorkers) + + // Here we will have a single task job and we will keep decommissioning (and killing) the + // worker running that task K times. Where K is more than the maxTaskFailures. Since the worker + // is notified of the decommissioning, the task failures can be ignored and not fail + // the job. + + sc = createSparkContext(config.TASK_MAX_FAILURES.key -> maxTaskFailures.toString) + val executorIdToWorkerInfo = getExecutorToWorkerAssignments + val taskIdsKilled = new ConcurrentHashMap[Long, Boolean] + val listener = new RootStageAwareListener { + override def handleRootTaskStart(taskStart: SparkListenerTaskStart): Unit = { + val taskInfo = taskStart.taskInfo + if (taskIdsKilled.size() < numTimesToKillWorkers) { + val workerInfo = executorIdToWorkerInfo(taskInfo.executorId) + decommissionWorkerOnMaster(workerInfo, "partition 0 must die") + killWorkerAfterTimeout(workerInfo, 1) + taskIdsKilled.put(taskInfo.taskId, true) + } + } + } + TestUtils.withListener(sc, listener) { _ => + val jobResult = sc.parallelize(1 to 1, 1).map { _ => + Thread.sleep(5 * 1000L); 1 + }.count() + assert(jobResult === 1) + } + // single task job that gets to run numTimesToKillWorkers + 1 times. + assert(listener.getTasksFinished().size === numTimesToKillWorkers + 1) + listener.rootTasksStarted.asScala.foreach { taskInfo => + assert(taskInfo.index == 0, s"Unknown task index ${taskInfo.index}") + } + listener.rootTasksEnded.asScala.foreach { taskInfo => + assert(taskInfo.index === 0, s"Expected task index ${taskInfo.index} to be 0") + // If a task has been killed then it shouldn't be successful + val taskSuccessExpected = !taskIdsKilled.getOrDefault(taskInfo.taskId, false) + val taskSuccessActual = taskInfo.successful + assert(taskSuccessActual === taskSuccessExpected, + s"Expected task success $taskSuccessActual == $taskSuccessExpected") + } + } + + test("decommission workers ensure that shuffle output is regenerated even with shuffle service") { + createWorkers(2) + val ss = new ExternalShuffleServiceHolder() + + sc = createSparkContext( + config.Tests.TEST_NO_STAGE_RETRY.key -> "true", + config.SHUFFLE_MANAGER.key -> "sort", + config.SHUFFLE_SERVICE_ENABLED.key -> "true", + config.SHUFFLE_SERVICE_PORT.key -> ss.getPort.toString + ) + + // Here we will create a 2 stage job: The first stage will have two tasks and the second stage + // will have one task. The two tasks in the first stage will be long and short. We decommission + // and kill the worker after the short task is done. Eventually the driver should get the + // executor lost signal for the short task executor. This should trigger regenerating + // the shuffle output since we cleanly decommissioned the executor, despite running with an + // external shuffle service. + try { + val executorIdToWorkerInfo = getExecutorToWorkerAssignments + val workerForTask0Decommissioned = new AtomicBoolean(false) + // single task job + val listener = new RootStageAwareListener { + override def handleRootTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + val taskInfo = taskEnd.taskInfo + if (taskInfo.index == 0) { + if (workerForTask0Decommissioned.compareAndSet(false, true)) { + val workerInfo = executorIdToWorkerInfo(taskInfo.executorId) + decommissionWorkerOnMaster(workerInfo, "Kill early done map worker") + killWorkerAfterTimeout(workerInfo, 0) + logInfo(s"Killed the node ${workerInfo.hostPort} that was running the early task") + } + } + } + } + TestUtils.withListener(sc, listener) { _ => + val jobResult = sc.parallelize(1 to 2, 2).mapPartitionsWithIndex((pid, _) => { + val sleepTimeSeconds = if (pid == 0) 1 else 10 + Thread.sleep(sleepTimeSeconds * 1000L) + List(1).iterator + }, preservesPartitioning = true).repartition(1).sum() + assert(jobResult === 2) + } + val tasksSeen = listener.getTasksFinished() + // 4 tasks: 2 from first stage, one retry due to decom, one more from the second stage. + assert(tasksSeen.size === 4, s"Expected 4 tasks but got $tasksSeen") + listener.rootTasksStarted.asScala.foreach { taskInfo => + assert(taskInfo.index <= 1, s"Expected ${taskInfo.index} <= 1") + assert(taskInfo.successful, s"Task ${taskInfo.index} should be successful") + } + val tasksEnded = listener.rootTasksEnded.asScala + tasksEnded.filter(_.index != 0).foreach { taskInfo => + assert(taskInfo.attemptNumber === 0, "2nd task should succeed on 1st attempt") + } + val firstTaskAttempts = tasksEnded.filter(_.index == 0) + assert(firstTaskAttempts.size > 1, s"Task 0 should have multiple attempts") + } finally { + ss.close() + } + } + + test("decommission workers ensure that fetch failures lead to rerun") { + createWorkers(2) + sc = createSparkContext( + config.Tests.TEST_NO_STAGE_RETRY.key -> "false", + config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE.key -> "true") + val executorIdToWorkerInfo = getExecutorToWorkerAssignments + val executorToDecom = executorIdToWorkerInfo.keysIterator.next + + // The task code below cannot call executorIdToWorkerInfo, so we need to pre-compute + // the worker to decom to force it to be serialized into the task. + val workerToDecom = executorIdToWorkerInfo(executorToDecom) + + // The setup of this job is similar to the one above: 2 stage job with first stage having + // long and short tasks. Except that we want the shuffle output to be regenerated on a + // fetch failure instead of an executor lost. Since it is hard to "trigger a fetch failure", + // we manually raise the FetchFailed exception when the 2nd stage's task runs and require that + // fetch failure to trigger a recomputation. + logInfo(s"Will try to decommission the task running on executor $executorToDecom") + val listener = new RootStageAwareListener { + override def handleRootTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + val taskInfo = taskEnd.taskInfo + if (taskInfo.executorId == executorToDecom && taskInfo.attemptNumber == 0 && + taskEnd.stageAttemptId == 0) { + decommissionWorkerOnMaster(workerToDecom, + "decommission worker after task on it is done") + } + } + } + TestUtils.withListener(sc, listener) { _ => + val jobResult = sc.parallelize(1 to 2, 2).mapPartitionsWithIndex((_, _) => { + val executorId = SparkEnv.get.executorId + val sleepTimeSeconds = if (executorId == executorToDecom) 10 else 1 + Thread.sleep(sleepTimeSeconds * 1000L) + List(1).iterator + }, preservesPartitioning = true) + .repartition(1).mapPartitions(iter => { + val context = TaskContext.get() + if (context.attemptNumber == 0 && context.stageAttemptNumber() == 0) { + // MapIndex is explicitly -1 to force the entire host to be decommissioned + // However, this will cause both the tasks in the preceding stage since the host here is + // "localhost" (shortcoming of this single-machine unit test in that all the workers + // are actually on the same host) + throw new FetchFailedException(BlockManagerId(executorToDecom, + workerToDecom.host, workerToDecom.port), 0, 0, -1, 0, "Forcing fetch failure") + } + val sumVal: List[Int] = List(iter.sum) + sumVal.iterator + }, preservesPartitioning = true) + .sum() + assert(jobResult === 2) + } + // 6 tasks: 2 from first stage, 2 rerun again from first stage, 2nd stage attempt 1 and 2. + val tasksSeen = listener.getTasksFinished() + assert(tasksSeen.size === 6, s"Expected 6 tasks but got $tasksSeen") + } + + private abstract class RootStageAwareListener extends SparkListener { + private var rootStageId: Option[Int] = None + private val tasksFinished = new ConcurrentLinkedQueue[String]() + private val jobDone = new AtomicBoolean(false) + val rootTasksStarted = new ConcurrentLinkedQueue[TaskInfo]() + val rootTasksEnded = new ConcurrentLinkedQueue[TaskInfo]() + + protected def isRootStageId(stageId: Int): Boolean = + (rootStageId.isDefined && rootStageId.get == stageId) + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + if (stageSubmitted.stageInfo.parentIds.isEmpty && rootStageId.isEmpty) { + rootStageId = Some(stageSubmitted.stageInfo.stageId) + } + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + jobEnd.jobResult match { + case JobSucceeded => jobDone.set(true) + } + } + + protected def handleRootTaskEnd(end: SparkListenerTaskEnd) = {} + + protected def handleRootTaskStart(start: SparkListenerTaskStart) = {} + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + if (isRootStageId(taskStart.stageId)) { + rootTasksStarted.add(taskStart.taskInfo) + handleRootTaskStart(taskStart) + } + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + val taskSignature = s"${taskEnd.stageId}:${taskEnd.stageAttemptId}:" + + s"${taskEnd.taskInfo.index}:${taskEnd.taskInfo.attemptNumber}" + logInfo(s"Task End $taskSignature") + tasksFinished.add(taskSignature) + if (isRootStageId(taskEnd.stageId)) { + rootTasksEnded.add(taskEnd.taskInfo) + handleRootTaskEnd(taskEnd) + } + } + + def getTasksFinished(): Seq[String] = { + assert(jobDone.get(), "Job isn't successfully done yet") + tasksFinished.asScala.toSeq + } + } + + private def getExecutorToWorkerAssignments: Map[String, WorkerInfo] = { + val executorIdToWorkerInfo = mutable.HashMap[String, WorkerInfo]() + master.workers.foreach { wi => + assert(wi.executors.size <= 1, "There should be at most one executor per worker") + // Cast the executorId to string since the TaskInfo.executorId is a string + wi.executors.values.foreach { e => + val executorIdString = e.id.toString + val oldWorkerInfo = executorIdToWorkerInfo.put(executorIdString, wi) + assert(oldWorkerInfo.isEmpty, + s"Executor $executorIdString already present on another worker ${oldWorkerInfo}") + } + } + executorIdToWorkerInfo.toMap + } + + private def makeMaster(): Master = { + val master = new Master( + masterRpcEnv, + masterRpcEnv.address, + 0, + masterAndWorkerSecurityManager, + masterAndWorkerConf) + masterRpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) + master + } + + private def createWorkers(numWorkers: Int, cores: Int = 1, memory: Int = 1024): Unit = { + val workerRpcEnvs = (0 until numWorkers).map { i => + RpcEnv.create( + Worker.SYSTEM_NAME + i, + "localhost", + 0, + masterAndWorkerConf, + masterAndWorkerSecurityManager) + } + workers.clear() + val rpcAddressToRpcEnv: mutable.HashMap[RpcAddress, RpcEnv] = mutable.HashMap.empty + workerRpcEnvs.foreach { rpcEnv => + val workDir = Utils.createTempDir(namePrefix = this.getClass.getSimpleName()).toString + val worker = new Worker(rpcEnv, 0, cores, memory, Array(masterRpcEnv.address), + Worker.ENDPOINT_NAME, workDir, masterAndWorkerConf, masterAndWorkerSecurityManager) + rpcEnv.setupEndpoint(Worker.ENDPOINT_NAME, worker) + workers.append(worker) + val oldRpcEnv = rpcAddressToRpcEnv.put(rpcEnv.address, rpcEnv) + logInfo(s"Created a worker at ${rpcEnv.address} with workdir $workDir") + assert(oldRpcEnv.isEmpty, s"Detected duplicate rpcEnv ${oldRpcEnv} for ${rpcEnv.address}") + } + workerIdToRpcEnvs.clear() + // Wait until all workers register with master successfully + eventually(timeout(1.minute), interval(1.seconds)) { + val workersOnMaster = getMasterState.workers + val numWorkersCurrently = workersOnMaster.length + logInfo(s"Waiting for $numWorkers workers to come up: So far $numWorkersCurrently") + assert(numWorkersCurrently === numWorkers) + workersOnMaster.foreach { workerInfo => + val rpcAddress = RpcAddress(workerInfo.host, workerInfo.port) + val rpcEnv = rpcAddressToRpcEnv(rpcAddress) + assert(rpcEnv != null, s"Cannot find the worker for $rpcAddress") + val oldRpcEnv = workerIdToRpcEnvs.put(workerInfo.id, rpcEnv) + assert(oldRpcEnv.isEmpty, s"Detected duplicate rpcEnv ${oldRpcEnv} for worker " + + s"${workerInfo.id}") + } + } + logInfo(s"Created ${workers.size} workers") + } + + private def getMasterState: MasterStateResponse = { + master.self.askSync[MasterStateResponse](RequestMasterState) + } + + private def getApplications(): Seq[ApplicationInfo] = { + getMasterState.activeApps + } + + def decommissionWorkerOnMaster(workerInfo: WorkerInfo, reason: String): Unit = { + logInfo(s"Trying to decommission worker ${workerInfo.id} for reason `$reason`") + master.self.send(WorkerDecommission(workerInfo.id, workerInfo.endpoint)) + } + + def killWorkerAfterTimeout(workerInfo: WorkerInfo, secondsToWait: Int): Unit = { + val env = workerIdToRpcEnvs(workerInfo.id) + Thread.sleep(secondsToWait * 1000L) + env.shutdown() + env.awaitTermination() + } + + def createSparkContext(extraConfs: (String, String)*): SparkContext = { + val conf = new SparkConf() + .setMaster(masterRpcEnv.address.toSparkURL) + .setAppName("test") + .setAll(extraConfs) + sc = new SparkContext(conf) + val appId = sc.applicationId + eventually(timeout(1.minute), interval(1.seconds)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.getExecutorLimit === Int.MaxValue) + } + sc + } + + private class ExternalShuffleServiceHolder() { + // The external shuffle service can start with default configs and not get polluted by the + // other configs used in this test. + private val transportConf = SparkTransportConf.fromSparkConf(new SparkConf(), + "shuffle", numUsableCores = 2) + private val rpcHandler = new ExternalBlockHandler(transportConf, null) + private val transportContext = new TransportContext(transportConf, rpcHandler) + private val server = transportContext.createServer() + + def getPort: Int = server.getPort + + def close(): Unit = { + Utils.tryLogNonFatalError { + server.close() + } + Utils.tryLogNonFatalError { + rpcHandler.close() + } + Utils.tryLogNonFatalError { + transportContext.close() + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 45af0d086890f..c829006923c4f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -178,6 +178,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def executorDecommission( executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = {} + override def getExecutorDecommissionInfo( + executorId: String): Option[ExecutorDecommissionInfo] = None } /** @@ -785,6 +787,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def executorDecommission( executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = {} + override def getExecutorDecommissionInfo( + executorId: String): Option[ExecutorDecommissionInfo] = None } val noKillScheduler = new DAGScheduler( sc, diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index b2a5f77b4b04c..07d88672290fc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -101,4 +101,6 @@ private class DummyTaskScheduler extends TaskScheduler { override def executorDecommission( executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = {} + override def getExecutorDecommissionInfo( + executorId: String): Option[ExecutorDecommissionInfo] = None } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 9ca3ce9d43ca5..e5836458e7f91 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -1802,6 +1802,53 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(2 == taskDescriptions.head.resources(GPU).addresses.size) } + private def setupSchedulerForDecommissionTests(): TaskSchedulerImpl = { + val taskScheduler = setupSchedulerWithMaster( + s"local[2]", + config.CPUS_PER_TASK.key -> 1.toString) + taskScheduler.submitTasks(FakeTask.createTaskSet(2)) + val multiCoreWorkerOffers = IndexedSeq(WorkerOffer("executor0", "host0", 1), + WorkerOffer("executor1", "host1", 1)) + val taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten + assert(taskDescriptions.map(_.executorId).sorted === Seq("executor0", "executor1")) + taskScheduler + } + + test("scheduler should keep the decommission info where host was decommissioned") { + val scheduler = setupSchedulerForDecommissionTests() + + scheduler.executorDecommission("executor0", ExecutorDecommissionInfo("0", false)) + scheduler.executorDecommission("executor1", ExecutorDecommissionInfo("1", true)) + scheduler.executorDecommission("executor0", ExecutorDecommissionInfo("0 new", false)) + scheduler.executorDecommission("executor1", ExecutorDecommissionInfo("1 new", false)) + + assert(scheduler.getExecutorDecommissionInfo("executor0") + === Some(ExecutorDecommissionInfo("0 new", false))) + assert(scheduler.getExecutorDecommissionInfo("executor1") + === Some(ExecutorDecommissionInfo("1", true))) + assert(scheduler.getExecutorDecommissionInfo("executor2").isEmpty) + } + + test("scheduler should ignore decommissioning of removed executors") { + val scheduler = setupSchedulerForDecommissionTests() + + // executor 0 is decommissioned after loosing + assert(scheduler.getExecutorDecommissionInfo("executor0").isEmpty) + scheduler.executorLost("executor0", ExecutorExited(0, false, "normal")) + assert(scheduler.getExecutorDecommissionInfo("executor0").isEmpty) + scheduler.executorDecommission("executor0", ExecutorDecommissionInfo("", false)) + assert(scheduler.getExecutorDecommissionInfo("executor0").isEmpty) + + // executor 1 is decommissioned before loosing + assert(scheduler.getExecutorDecommissionInfo("executor1").isEmpty) + scheduler.executorDecommission("executor1", ExecutorDecommissionInfo("", false)) + assert(scheduler.getExecutorDecommissionInfo("executor1").isDefined) + scheduler.executorLost("executor1", ExecutorExited(0, false, "normal")) + assert(scheduler.getExecutorDecommissionInfo("executor1").isEmpty) + scheduler.executorDecommission("executor1", ExecutorDecommissionInfo("", false)) + assert(scheduler.getExecutorDecommissionInfo("executor1").isEmpty) + } + /** * Used by tests to simulate a task failure. This calls the failure handler explicitly, to ensure * that all the state is updated when this method returns. Otherwise, there's no way to know when From 6032c5b0320fe70455586f4ce863d5d9361b5e07 Mon Sep 17 00:00:00 2001 From: Devesh Agrawal Date: Thu, 30 Jul 2020 12:00:19 -0700 Subject: [PATCH 338/384] [SPARK-32417] Fix flakyness of BlockManagerDecommissionIntegrationSuite ### What changes were proposed in this pull request? This test tries to fix the flakyness of BlockManagerDecommissionIntegrationSuite. ### Description of the problem Make the block manager decommissioning test be less flaky An interesting failure happens when migrateDuring = true (and persist or shuffle is true): - We schedule the job with tasks on executors 0, 1, 2. - We wait 300 ms and decommission executor 0. - If the task is not yet done on executor 0, it will now fail because the block manager won't be able to save the block. This condition is easy to trigger on a loaded machine where the github checks run. - The task with retry on a different executor (1 or 2) and its shuffle blocks will land there. - No actual block migration happens here because the decommissioned executor technically failed before it could even produce a block. To remove the above race, this change replaces the fixed wait for 300 ms to wait for an actual task to succeed. When a task has succeeded, we know its blocks would have been written for sure and thus its executor would certainly be forced to migrate those blocks when it is decommissioned. The change always decommissions an executor on which a real task finished successfully instead of picking the first executor. Because the system may choose to schedule nothing on the first executor and instead run the two tasks on one executor. ### Why are the changes needed? I have had bad luck with BlockManagerDecommissionIntegrationSuite and it has failed several times on my PRs. So fixing it. ### Does this PR introduce _any_ user-facing change? No, unit test only change. ### How was this patch tested? Github checks. Ran this test 100 times, 10 at a time in parallel in a script. Closes #29226 from agrawaldevesh/block-manager-decom-flaky. Authored-by: Devesh Agrawal Signed-off-by: Holden Karau --- ...kManagerDecommissionIntegrationSuite.scala | 147 ++++++++++++------ 1 file changed, 103 insertions(+), 44 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index 57410103dd080..6a52f72938c6c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -17,8 +17,9 @@ package org.apache.spark.storage -import java.util.concurrent.Semaphore +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, Semaphore} +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ @@ -28,29 +29,40 @@ import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend -import org.apache.spark.util.{ResetSystemProperties, ThreadUtils} +import org.apache.spark.util.{ResetSystemProperties, SystemClock, ThreadUtils} class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalSparkContext with ResetSystemProperties with Eventually { val numExecs = 3 val numParts = 3 + val TaskStarted = "TASK_STARTED" + val TaskEnded = "TASK_ENDED" + val JobEnded = "JOB_ENDED" test(s"verify that an already running task which is going to cache data succeeds " + - s"on a decommissioned executor") { - runDecomTest(true, false, true) + s"on a decommissioned executor after task start") { + runDecomTest(true, false, TaskStarted) } - test(s"verify that shuffle blocks are migrated") { - runDecomTest(false, true, false) + test(s"verify that an already running task which is going to cache data succeeds " + + s"on a decommissioned executor after one task ends but before job ends") { + runDecomTest(true, false, TaskEnded) } - test(s"verify that both migrations can work at the same time.") { - runDecomTest(true, true, false) + test(s"verify that shuffle blocks are migrated") { + runDecomTest(false, true, JobEnded) } - private def runDecomTest(persist: Boolean, shuffle: Boolean, migrateDuring: Boolean) = { + test(s"verify that both migrations can work at the same time") { + runDecomTest(true, true, JobEnded) + } + private def runDecomTest( + persist: Boolean, + shuffle: Boolean, + whenToDecom: String): Unit = { + val migrateDuring = whenToDecom != JobEnded val master = s"local-cluster[${numExecs}, 1, 1024]" val conf = new SparkConf().setAppName("test").setMaster(master) .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) @@ -61,6 +73,10 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS // workload we need to worry about. .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 1L) + if (whenToDecom == TaskStarted) { + // We are using accumulators below, make sure those are reported frequently. + conf.set(config.EXECUTOR_HEARTBEAT_INTERVAL.key, "10ms") + } sc = new SparkContext(master, "test", conf) // Wait for the executors to start @@ -70,15 +86,29 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS val input = sc.parallelize(1 to numParts, numParts) val accum = sc.longAccumulator("mapperRunAccumulator") - input.count() + + val sleepIntervalMs = whenToDecom match { + // Increase the window of time b/w task started and ended so that we can decom within that. + case TaskStarted => 2000 + // Make one task take a really short time so that we can decommission right after it is + // done but before its peers are done. + case TaskEnded => + if (TaskContext.getPartitionId() == 0) { + 100 + } else { + 1000 + } + // No sleep otherwise + case _ => 0 + } // Create a new RDD where we have sleep in each partition, we are also increasing // the value of accumulator in each partition val baseRdd = input.mapPartitions { x => - if (migrateDuring) { - Thread.sleep(1000) - } accum.add(1) + if (sleepIntervalMs > 0) { + Thread.sleep(sleepIntervalMs) + } x.map(y => (y, y)) } val testRdd = shuffle match { @@ -87,35 +117,46 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS } // Listen for the job & block updates - val taskStartSem = new Semaphore(0) - val broadcastSem = new Semaphore(0) val executorRemovedSem = new Semaphore(0) - val taskEndEvents = ArrayBuffer.empty[SparkListenerTaskEnd] + val taskEndEvents = new ConcurrentLinkedQueue[SparkListenerTaskEnd]() + val executorsActuallyStarted = new ConcurrentHashMap[String, Boolean]() val blocksUpdated = ArrayBuffer.empty[SparkListenerBlockUpdated] - sc.addSparkListener(new SparkListener { + def getCandidateExecutorToDecom: Option[String] = if (whenToDecom == TaskStarted) { + executorsActuallyStarted.keySet().asScala.headOption + } else { + taskEndEvents.asScala.filter(_.taskInfo.successful).map(_.taskInfo.executorId).headOption + } + + sc.addSparkListener(new SparkListener { override def onExecutorRemoved(execRemoved: SparkListenerExecutorRemoved): Unit = { executorRemovedSem.release() } - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { - taskStartSem.release() - } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - taskEndEvents.append(taskEnd) + taskEndEvents.add(taskEnd) } override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { - // Once broadcast start landing on the executors we're good to proceed. - // We don't only use task start as it can occur before the work is on the executor. - if (blockUpdated.blockUpdatedInfo.blockId.isBroadcast) { - broadcastSem.release() - } blocksUpdated.append(blockUpdated) } - }) + override def onExecutorMetricsUpdate( + executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { + val executorId = executorMetricsUpdate.execId + if (executorId != SparkContext.DRIVER_IDENTIFIER) { + val validUpdate = executorMetricsUpdate + .accumUpdates + .flatMap(_._4) + .exists { accumInfo => + accumInfo.name == accum.name && accumInfo.update.exists(_.asInstanceOf[Long] >= 1) + } + if (validUpdate) { + executorsActuallyStarted.put(executorId, java.lang.Boolean.TRUE) + } + } + } + }) // Cache the RDD lazily if (persist) { @@ -125,28 +166,32 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS // Start the computation of RDD - this step will also cache the RDD val asyncCount = testRdd.countAsync() - // Wait for the job to have started. - taskStartSem.acquire(1) - // Wait for each executor + driver to have it's broadcast info delivered. - broadcastSem.acquire((numExecs + 1)) - // Make sure the job is either mid run or otherwise has data to migrate. if (migrateDuring) { - // Give Spark a tiny bit to start executing after the broadcast blocks land. - // For me this works at 100, set to 300 for system variance. - Thread.sleep(300) + // Wait for one of the tasks to succeed and finish writing its blocks. + // This way we know that this executor had real data to migrate when it is subsequently + // decommissioned below. + val intervalMs = if (whenToDecom == TaskStarted) { + 3.milliseconds + } else { + 10.milliseconds + } + eventually(timeout(6.seconds), interval(intervalMs)) { + assert(getCandidateExecutorToDecom.isDefined) + } } else { ThreadUtils.awaitResult(asyncCount, 15.seconds) } // Decommission one of the executors. val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] - val execs = sched.getExecutorIds() - assert(execs.size == numExecs, s"Expected ${numExecs} executors but found ${execs.size}") - val execToDecommission = execs.head - logDebug(s"Decommissioning executor ${execToDecommission}") - sched.decommissionExecutor(execToDecommission, ExecutorDecommissionInfo("", false)) + val execToDecommission = getCandidateExecutorToDecom.get + logInfo(s"Decommissioning executor ${execToDecommission}") + sched.decommissionExecutor( + execToDecommission, + ExecutorDecommissionInfo("", isHostDecommissioned = false)) + val decomTime = new SystemClock().getTimeMillis() // Wait for job to finish. val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 15.seconds) @@ -155,16 +200,31 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS assert(accum.value === numParts) sc.listenerBus.waitUntilEmpty() + val taskEndEventsCopy = taskEndEvents.asScala if (shuffle) { // mappers & reducers which succeeded - assert(taskEndEvents.count(_.reason == Success) === 2 * numParts, + assert(taskEndEventsCopy.count(_.reason == Success) === 2 * numParts, s"Expected ${2 * numParts} tasks got ${taskEndEvents.size} (${taskEndEvents})") } else { // only mappers which executed successfully - assert(taskEndEvents.count(_.reason == Success) === numParts, + assert(taskEndEventsCopy.count(_.reason == Success) === numParts, s"Expected ${numParts} tasks got ${taskEndEvents.size} (${taskEndEvents})") } + val minTaskEndTime = taskEndEventsCopy.map(_.taskInfo.finishTime).min + val maxTaskEndTime = taskEndEventsCopy.map(_.taskInfo.finishTime).max + + // Verify that the decom time matched our expectations + val decomAssertMsg = s"$whenToDecom: decomTime: $decomTime, minTaskEnd: $minTaskEndTime," + + s" maxTaskEnd: $maxTaskEndTime" + assert(minTaskEndTime <= maxTaskEndTime, decomAssertMsg) + whenToDecom match { + case TaskStarted => assert(minTaskEndTime > decomTime, decomAssertMsg) + case TaskEnded => assert(minTaskEndTime <= decomTime && + decomTime < maxTaskEndTime, decomAssertMsg) + case JobEnded => assert(maxTaskEndTime <= decomTime, decomAssertMsg) + } + // Wait for our respective blocks to have migrated eventually(timeout(30.seconds), interval(10.milliseconds)) { if (persist) { @@ -224,6 +284,5 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS // should have same value like before assert(testRdd.count() === numParts) assert(accum.value === numParts) - } } From 9d7b1d935f7a2b770d8b2f264cfe4a4db2ad64b6 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 31 Jul 2020 10:37:05 +0900 Subject: [PATCH 339/384] [SPARK-32175][SPARK-32175][FOLLOWUP] Remove flaky test added in ### What changes were proposed in this pull request? This PR removes a test added in SPARK-32175(#29002). ### Why are the changes needed? That test is flaky. It can be mitigated by increasing the timeout but it would rather be simpler to remove the test. See also the [discussion](https://github.com/apache/spark/pull/29002#issuecomment-666746857). ### Does this PR introduce _any_ user-facing change? No. Closes #29314 from sarutak/remove-flaky-test. Authored-by: Kousuke Saruta Signed-off-by: Kousuke Saruta --- .../apache/spark/executor/ExecutorSuite.scala | 67 ------------------- 1 file changed, 67 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index b198448a299cf..8e58beff74290 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -403,73 +403,6 @@ class ExecutorSuite extends SparkFunSuite assert(taskMetrics.getMetricValue("JVMHeapMemory") > 0) } - test("SPARK-32175: Plugin initialization should start after heartbeater started") { - withTempDir { tempDir => - val sparkPluginCodeBody = - """ - |@Override - |public org.apache.spark.api.plugin.ExecutorPlugin executorPlugin() { - | return new TestExecutorPlugin(); - |} - | - |@Override - |public org.apache.spark.api.plugin.DriverPlugin driverPlugin() { return null; } - """.stripMargin - val executorPluginBody = - """ - |@Override - |public void init( - | org.apache.spark.api.plugin.PluginContext ctx, - | java.util.Map extraConf) { - | try { - | Thread.sleep(8 * 1000); - | } catch (InterruptedException e) { - | throw new RuntimeException(e); - | } - |} - """.stripMargin - - val compiledExecutorPlugin = TestUtils.createCompiledClass( - "TestExecutorPlugin", - tempDir, - "", - null, - Seq.empty, - Seq("org.apache.spark.api.plugin.ExecutorPlugin"), - executorPluginBody) - - val thisClassPath = - sys.props("java.class.path").split(File.pathSeparator).map(p => new File(p).toURI.toURL) - val compiledSparkPlugin = TestUtils.createCompiledClass( - "TestSparkPlugin", - tempDir, - "", - null, - Seq(tempDir.toURI.toURL) ++ thisClassPath, - Seq("org.apache.spark.api.plugin.SparkPlugin"), - sparkPluginCodeBody) - - val jarUrl = TestUtils.createJar( - Seq(compiledSparkPlugin, compiledExecutorPlugin), - new File(tempDir, "testPlugin.jar")) - - val unusedJar = TestUtils.createJarWithClasses(Seq.empty) - val args = Seq( - "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), - "--name", "testApp", - "--master", "local-cluster[1,1,1024]", - "--conf", "spark.plugins=TestSparkPlugin", - "--conf", "spark.storage.blockManagerSlaveTimeoutMs=" + 5 * 1000, - "--conf", "spark.network.timeoutInterval=" + 1000, - "--conf", "spark.executor.heartbeatInterval=" + 1000, - "--conf", "spark.executor.extraClassPath=" + jarUrl.toString, - "--conf", "spark.driver.extraClassPath=" + jarUrl.toString, - "--conf", "spark.ui.enabled=false", - unusedJar.toString) - SparkSubmitSuite.runSparkSubmit(args, timeout = 30.seconds) - } - } - private def createMockEnv(conf: SparkConf, serializer: JavaSerializer): SparkEnv = { val mockEnv = mock[SparkEnv] val mockRpcEnv = mock[RpcEnv] From f6027827a49a52f6586f73aee9e4067659f650b6 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Fri, 31 Jul 2020 13:40:33 +0900 Subject: [PATCH 340/384] [SPARK-32482][SS][TESTS] Eliminate deprecated poll(long) API calls to avoid infinite wait in tests ### What changes were proposed in this pull request? Structured Streaming Kafka connector tests are now using a deprecated `poll(long)` API which could cause infinite wait. In this PR I've eliminated these calls and replaced them with `AdminClient`. ### Why are the changes needed? Deprecated `poll(long)` API calls. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #29289 from gaborgsomogyi/SPARK-32482. Authored-by: Gabor Somogyi Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../spark/sql/kafka010/KafkaTestUtils.scala | 47 ++++++------------- 1 file changed, 14 insertions(+), 33 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 275a8170182fe..e4a1e4f77b28e 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -462,32 +462,24 @@ class KafkaTestUtils( server.logManager.cleanupLogs() } + private def getOffsets(topics: Set[String], offsetSpec: OffsetSpec): Map[TopicPartition, Long] = { + val listOffsetsParams = adminClient.describeTopics(topics.asJava).all().get().asScala + .flatMap { topicDescription => + topicDescription._2.partitions().asScala.map { topicPartitionInfo => + new TopicPartition(topicDescription._1, topicPartitionInfo.partition()) + } + }.map(_ -> offsetSpec).toMap.asJava + val partitionOffsets = adminClient.listOffsets(listOffsetsParams).all().get().asScala + .map(result => result._1 -> result._2.offset()).toMap + partitionOffsets + } + def getEarliestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { - val kc = new KafkaConsumer[String, String](consumerConfiguration) - logInfo("Created consumer to get earliest offsets") - kc.subscribe(topics.asJavaCollection) - kc.poll(0) - val partitions = kc.assignment() - kc.pause(partitions) - kc.seekToBeginning(partitions) - val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap - kc.close() - logInfo("Closed consumer to get earliest offsets") - offsets + getOffsets(topics, OffsetSpec.earliest()) } def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { - val kc = new KafkaConsumer[String, String](consumerConfiguration) - logInfo("Created consumer to get latest offsets") - kc.subscribe(topics.asJavaCollection) - kc.poll(0) - val partitions = kc.assignment() - kc.pause(partitions) - kc.seekToEnd(partitions) - val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap - kc.close() - logInfo("Closed consumer to get latest offsets") - offsets + getOffsets(topics, OffsetSpec.latest()) } def listConsumerGroups(): ListConsumerGroupsResult = { @@ -559,17 +551,6 @@ class KafkaTestUtils( } } - private def consumerConfiguration: Properties = { - val props = new Properties() - props.put("bootstrap.servers", brokerAddress) - props.put("group.id", "group-KafkaTestUtils-" + Random.nextInt) - props.put("value.deserializer", classOf[StringDeserializer].getName) - props.put("key.deserializer", classOf[StringDeserializer].getName) - props.put("enable.auto.commit", "false") - setAuthenticationConfigIfNeeded(props) - props - } - private def setAuthenticationConfigIfNeeded(props: Properties): Unit = { if (secure) { val jaasParams = KafkaTokenUtil.getKeytabJaasParams( From ae82768c1396bfe626b52c4eac33241a9eb91f54 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Fri, 31 Jul 2020 05:51:57 +0000 Subject: [PATCH 341/384] [SPARK-32421][SQL] Add code-gen for shuffled hash join ### What changes were proposed in this pull request? Adding codegen for shuffled hash join. Shuffled hash join codegen is very similar to broadcast hash join codegen. So most of code change is to refactor existing codegen in `BroadcastHashJoinExec` to `HashJoin`. Example codegen for query in [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153): ``` def shuffleHashJoin(): Unit = { val N: Long = 4 << 20 withSQLConf( SQLConf.SHUFFLE_PARTITIONS.key -> "2", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000", SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { codegenBenchmark("shuffle hash join", N) { val df1 = spark.range(N).selectExpr(s"id as k1") val df2 = spark.range(N / 3).selectExpr(s"id * 3 as k2") val df = df1.join(df2, col("k1") === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[ShuffledHashJoinExec]).isDefined) df.noop() } } } ``` Shuffled hash join codegen: ``` == Subtree 3 / 3 (maxMethodCodeSize:113; maxConstantPoolSize:126(0.19% used); numInnerClasses:0) == *(3) ShuffledHashJoin [k1#2L], [k2#6L], Inner, BuildRight :- *(1) Project [id#0L AS k1#2L] : +- *(1) Range (0, 4194304, step=1, splits=1) +- *(2) Project [(id#4L * 3) AS k2#6L] +- *(2) Range (0, 1398101, step=1, splits=1) Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIteratorForCodegenStage3(references); /* 003 */ } /* 004 */ /* 005 */ // codegenStageId=3 /* 006 */ final class GeneratedIteratorForCodegenStage3 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.execution.joins.HashedRelation shj_relation_0; /* 011 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] shj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1]; /* 012 */ /* 013 */ public GeneratedIteratorForCodegenStage3(Object[] references) { /* 014 */ this.references = references; /* 015 */ } /* 016 */ /* 017 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 018 */ partitionIndex = index; /* 019 */ this.inputs = inputs; /* 020 */ inputadapter_input_0 = inputs[0]; /* 021 */ shj_relation_0 = ((org.apache.spark.sql.execution.joins.ShuffledHashJoinExec) references[0] /* plan */).buildHashedRelation(inputs[1]); /* 022 */ shj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0); /* 023 */ /* 024 */ } /* 025 */ /* 026 */ private void shj_doConsume_0(InternalRow inputadapter_row_0, long shj_expr_0_0) throws java.io.IOException { /* 027 */ // generate join key for stream side /* 028 */ /* 029 */ // find matches from HashRelation /* 030 */ scala.collection.Iterator shj_matches_0 = false ? /* 031 */ null : (scala.collection.Iterator)shj_relation_0.get(shj_expr_0_0); /* 032 */ if (shj_matches_0 != null) { /* 033 */ while (shj_matches_0.hasNext()) { /* 034 */ UnsafeRow shj_matched_0 = (UnsafeRow) shj_matches_0.next(); /* 035 */ { /* 036 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[1] /* numOutputRows */).add(1); /* 037 */ /* 038 */ long shj_value_1 = shj_matched_0.getLong(0); /* 039 */ shj_mutableStateArray_0[0].reset(); /* 040 */ /* 041 */ shj_mutableStateArray_0[0].write(0, shj_expr_0_0); /* 042 */ /* 043 */ shj_mutableStateArray_0[0].write(1, shj_value_1); /* 044 */ append((shj_mutableStateArray_0[0].getRow()).copy()); /* 045 */ /* 046 */ } /* 047 */ } /* 048 */ } /* 049 */ /* 050 */ } /* 051 */ /* 052 */ protected void processNext() throws java.io.IOException { /* 053 */ while ( inputadapter_input_0.hasNext()) { /* 054 */ InternalRow inputadapter_row_0 = (InternalRow) inputadapter_input_0.next(); /* 055 */ /* 056 */ long inputadapter_value_0 = inputadapter_row_0.getLong(0); /* 057 */ /* 058 */ shj_doConsume_0(inputadapter_row_0, inputadapter_value_0); /* 059 */ if (shouldStop()) return; /* 060 */ } /* 061 */ } /* 062 */ /* 063 */ } ``` Broadcast hash join codegen for the same query (for reference here): ``` == Subtree 2 / 2 (maxMethodCodeSize:280; maxConstantPoolSize:218(0.33% used); numInnerClasses:0) == *(2) BroadcastHashJoin [k1#2L], [k2#6L], Inner, BuildRight, false :- *(2) Project [id#0L AS k1#2L] : +- *(2) Range (0, 4194304, step=1, splits=1) +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#22] +- *(1) Project [(id#4L * 3) AS k2#6L] +- *(1) Range (0, 1398101, step=1, splits=1) Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIteratorForCodegenStage2(references); /* 003 */ } /* 004 */ /* 005 */ // codegenStageId=2 /* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator { /* 007 */ private Object[] references; /* 008 */ private scala.collection.Iterator[] inputs; /* 009 */ private boolean range_initRange_0; /* 010 */ private long range_nextIndex_0; /* 011 */ private TaskContext range_taskContext_0; /* 012 */ private InputMetrics range_inputMetrics_0; /* 013 */ private long range_batchEnd_0; /* 014 */ private long range_numElementsTodo_0; /* 015 */ private org.apache.spark.sql.execution.joins.LongHashedRelation bhj_relation_0; /* 016 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4]; /* 017 */ /* 018 */ public GeneratedIteratorForCodegenStage2(Object[] references) { /* 019 */ this.references = references; /* 020 */ } /* 021 */ /* 022 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 023 */ partitionIndex = index; /* 024 */ this.inputs = inputs; /* 025 */ /* 026 */ range_taskContext_0 = TaskContext.get(); /* 027 */ range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics(); /* 028 */ range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0); /* 029 */ range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0); /* 030 */ range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0); /* 031 */ /* 032 */ bhj_relation_0 = ((org.apache.spark.sql.execution.joins.LongHashedRelation) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcast */).value()).asReadOnlyCopy(); /* 033 */ incPeakExecutionMemory(bhj_relation_0.estimatedSize()); /* 034 */ /* 035 */ range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0); /* 036 */ /* 037 */ } /* 038 */ /* 039 */ private void initRange(int idx) { /* 040 */ java.math.BigInteger index = java.math.BigInteger.valueOf(idx); /* 041 */ java.math.BigInteger numSlice = java.math.BigInteger.valueOf(1L); /* 042 */ java.math.BigInteger numElement = java.math.BigInteger.valueOf(4194304L); /* 043 */ java.math.BigInteger step = java.math.BigInteger.valueOf(1L); /* 044 */ java.math.BigInteger start = java.math.BigInteger.valueOf(0L); /* 045 */ long partitionEnd; /* 046 */ /* 047 */ java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start); /* 048 */ if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) { /* 049 */ range_nextIndex_0 = Long.MAX_VALUE; /* 050 */ } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) { /* 051 */ range_nextIndex_0 = Long.MIN_VALUE; /* 052 */ } else { /* 053 */ range_nextIndex_0 = st.longValue(); /* 054 */ } /* 055 */ range_batchEnd_0 = range_nextIndex_0; /* 056 */ /* 057 */ java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice) /* 058 */ .multiply(step).add(start); /* 059 */ if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) { /* 060 */ partitionEnd = Long.MAX_VALUE; /* 061 */ } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) { /* 062 */ partitionEnd = Long.MIN_VALUE; /* 063 */ } else { /* 064 */ partitionEnd = end.longValue(); /* 065 */ } /* 066 */ /* 067 */ java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract( /* 068 */ java.math.BigInteger.valueOf(range_nextIndex_0)); /* 069 */ range_numElementsTodo_0 = startToEnd.divide(step).longValue(); /* 070 */ if (range_numElementsTodo_0 < 0) { /* 071 */ range_numElementsTodo_0 = 0; /* 072 */ } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) { /* 073 */ range_numElementsTodo_0++; /* 074 */ } /* 075 */ } /* 076 */ /* 077 */ private void bhj_doConsume_0(long bhj_expr_0_0) throws java.io.IOException { /* 078 */ // generate join key for stream side /* 079 */ /* 080 */ // find matches from HashedRelation /* 081 */ UnsafeRow bhj_matched_0 = false ? null: (UnsafeRow)bhj_relation_0.getValue(bhj_expr_0_0); /* 082 */ if (bhj_matched_0 != null) { /* 083 */ { /* 084 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1); /* 085 */ /* 086 */ long bhj_value_2 = bhj_matched_0.getLong(0); /* 087 */ range_mutableStateArray_0[3].reset(); /* 088 */ /* 089 */ range_mutableStateArray_0[3].write(0, bhj_expr_0_0); /* 090 */ /* 091 */ range_mutableStateArray_0[3].write(1, bhj_value_2); /* 092 */ append((range_mutableStateArray_0[3].getRow())); /* 093 */ /* 094 */ } /* 095 */ } /* 096 */ /* 097 */ } /* 098 */ /* 099 */ protected void processNext() throws java.io.IOException { /* 100 */ // initialize Range /* 101 */ if (!range_initRange_0) { /* 102 */ range_initRange_0 = true; /* 103 */ initRange(partitionIndex); /* 104 */ } /* 105 */ /* 106 */ while (true) { /* 107 */ if (range_nextIndex_0 == range_batchEnd_0) { /* 108 */ long range_nextBatchTodo_0; /* 109 */ if (range_numElementsTodo_0 > 1000L) { /* 110 */ range_nextBatchTodo_0 = 1000L; /* 111 */ range_numElementsTodo_0 -= 1000L; /* 112 */ } else { /* 113 */ range_nextBatchTodo_0 = range_numElementsTodo_0; /* 114 */ range_numElementsTodo_0 = 0; /* 115 */ if (range_nextBatchTodo_0 == 0) break; /* 116 */ } /* 117 */ range_batchEnd_0 += range_nextBatchTodo_0 * 1L; /* 118 */ } /* 119 */ /* 120 */ int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L); /* 121 */ for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) { /* 122 */ long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0; /* 123 */ /* 124 */ bhj_doConsume_0(range_value_0); /* 125 */ /* 126 */ if (shouldStop()) { /* 127 */ range_nextIndex_0 = range_value_0 + 1L; /* 128 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1); /* 129 */ range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1); /* 130 */ return; /* 131 */ } /* 132 */ /* 133 */ } /* 134 */ range_nextIndex_0 = range_batchEnd_0; /* 135 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0); /* 136 */ range_inputMetrics_0.incRecordsRead(range_localEnd_0); /* 137 */ range_taskContext_0.killTaskIfInterrupted(); /* 138 */ } /* 139 */ } /* 140 */ /* 141 */ } ``` ### Why are the changes needed? Codegen shuffled hash join can help save CPU cost. We added shuffled hash join codegen internally in our fork, and seeing obvious improvement in benchmark compared to current non-codegen code path. Test example query in [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153), seeing 30% wall clock time improvement compared to existing non-codegen code path: Enable shuffled hash join code-gen: ``` Running benchmark: shuffle hash join Running case: shuffle hash join wholestage off Stopped after 2 iterations, 1358 ms Running case: shuffle hash join wholestage on Stopped after 5 iterations, 2323 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4 Intel(R) Core(TM) i9-9980HK CPU 2.40GHz shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ shuffle hash join wholestage off 649 679 43 6.5 154.7 1.0X shuffle hash join wholestage on 436 465 45 9.6 103.9 1.5X ``` Disable shuffled hash join codegen: ``` Running benchmark: shuffle hash join Running case: shuffle hash join wholestage off Stopped after 2 iterations, 1345 ms Running case: shuffle hash join wholestage on Stopped after 5 iterations, 2967 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4 Intel(R) Core(TM) i9-9980HK CPU 2.40GHz shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ shuffle hash join wholestage off 646 673 37 6.5 154.1 1.0X shuffle hash join wholestage on 549 594 47 7.6 130.9 1.2X ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit test in `WholeStageCodegenSuite`. Closes #29277 from c21/codegen. Authored-by: Cheng Su Signed-off-by: Wenchen Fan --- .../sql/execution/WholeStageCodegenExec.scala | 7 +- .../joins/BroadcastHashJoinExec.scala | 447 ++---------------- .../spark/sql/execution/joins/HashJoin.scala | 411 +++++++++++++++- .../joins/ShuffledHashJoinExec.scala | 22 +- .../execution/WholeStageCodegenSuite.scala | 28 +- .../execution/metric/SQLMetricsSuite.scala | 4 +- 6 files changed, 499 insertions(+), 420 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 558d990e8c4bb..59a42d8931925 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -50,6 +50,7 @@ trait CodegenSupport extends SparkPlan { private def variablePrefix: String = this match { case _: HashAggregateExec => "agg" case _: BroadcastHashJoinExec => "bhj" + case _: ShuffledHashJoinExec => "shj" case _: SortMergeJoinExec => "smj" case _: RDDScanExec => "rdd" case _: DataSourceScanExec => "scan" @@ -903,6 +904,10 @@ case class CollapseCodegenStages( // The children of SortMergeJoin should do codegen separately. j.withNewChildren(j.children.map( child => InputAdapter(insertWholeStageCodegen(child)))) + case j: ShuffledHashJoinExec => + // The children of ShuffledHashJoin should do codegen separately. + j.withNewChildren(j.children.map( + child => InputAdapter(insertWholeStageCodegen(child)))) case p => p.withNewChildren(p.children.map(insertInputAdapter)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 2a283013aceef..e4935c8c72228 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -25,13 +25,11 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, HashPartitioning, Partitioning, PartitioningCollection, UnspecifiedDistribution} import org.apache.spark.sql.execution.{CodegenSupport, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.{BooleanType, LongType} /** * Performs an inner hash join of two child relations. When the output RDD of this operator is @@ -197,23 +195,6 @@ case class BroadcastHashJoinExec( override def needCopyResult: Boolean = streamedPlan.asInstanceOf[CodegenSupport].needCopyResult || multipleOutputForOneInput - override def doProduce(ctx: CodegenContext): String = { - streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this) - } - - override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { - joinType match { - case _: InnerLike => codegenInner(ctx, input) - case LeftOuter | RightOuter => codegenOuter(ctx, input) - case LeftSemi => codegenSemi(ctx, input) - case LeftAnti => codegenAnti(ctx, input) - case j: ExistenceJoin => codegenExistence(ctx, input) - case x => - throw new IllegalArgumentException( - s"BroadcastHashJoin should not take $x as the JoinType") - } - } - /** * Returns a tuple of Broadcast of HashedRelation and the variable name for it. */ @@ -232,411 +213,55 @@ case class BroadcastHashJoinExec( (broadcastRelation, relationTerm) } - /** - * Returns the code for generating join key for stream side, and expression of whether the key - * has any null in it or not. - */ - private def genStreamSideJoinKey( - ctx: CodegenContext, - input: Seq[ExprCode]): (ExprCode, String) = { - ctx.currentVars = input - if (streamedBoundKeys.length == 1 && streamedBoundKeys.head.dataType == LongType) { - // generate the join key as Long - val ev = streamedBoundKeys.head.genCode(ctx) - (ev, ev.isNull) - } else { - // generate the join key as UnsafeRow - val ev = GenerateUnsafeProjection.createCode(ctx, streamedBoundKeys) - (ev, s"${ev.value}.anyNull()") - } - } - - /** - * Generates the code for variable of build side. - */ - private def genBuildSideVars(ctx: CodegenContext, matched: String): Seq[ExprCode] = { - ctx.currentVars = null - ctx.INPUT_ROW = matched - buildPlan.output.zipWithIndex.map { case (a, i) => - val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx) - if (joinType.isInstanceOf[InnerLike]) { - ev - } else { - // the variables are needed even there is no matched rows - val isNull = ctx.freshName("isNull") - val value = ctx.freshName("value") - val javaType = CodeGenerator.javaType(a.dataType) - val code = code""" - |boolean $isNull = true; - |$javaType $value = ${CodeGenerator.defaultValue(a.dataType)}; - |if ($matched != null) { - | ${ev.code} - | $isNull = ${ev.isNull}; - | $value = ${ev.value}; - |} - """.stripMargin - ExprCode(code, JavaCode.isNullVariable(isNull), JavaCode.variable(value, a.dataType)) - } - } - } - - /** - * Generate the (non-equi) condition used to filter joined rows. This is used in Inner, Left Semi - * and Left Anti joins. - */ - private def getJoinCondition( - ctx: CodegenContext, - input: Seq[ExprCode]): (String, String, Seq[ExprCode]) = { - val matched = ctx.freshName("matched") - val buildVars = genBuildSideVars(ctx, matched) - val checkCondition = if (condition.isDefined) { - val expr = condition.get - // evaluate the variables from build side that used by condition - val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) - // filter the output via condition - ctx.currentVars = input ++ buildVars - val ev = - BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) - val skipRow = s"${ev.isNull} || !${ev.value}" - s""" - |$eval - |${ev.code} - |if (!($skipRow)) - """.stripMargin - } else { - "" - } - (matched, checkCondition, buildVars) - } - - /** - * Generates the code for Inner join. - */ - private def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = { - val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input) - val numOutput = metricTerm(ctx, "numOutputRows") - - val resultVars = buildSide match { - case BuildLeft => buildVars ++ input - case BuildRight => input ++ buildVars - } - if (broadcastRelation.value.keyIsUnique) { - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashedRelation - |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - |if ($matched != null) { - | $checkCondition { - | $numOutput.add(1); - | ${consume(ctx, resultVars)} - | } - |} - """.stripMargin - - } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashRelation - |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); - |if ($matches != null) { - | while ($matches.hasNext()) { - | UnsafeRow $matched = (UnsafeRow) $matches.next(); - | $checkCondition { - | $numOutput.add(1); - | ${consume(ctx, resultVars)} - | } - | } - |} - """.stripMargin - } - } - - /** - * Generates the code for left or right outer join. - */ - private def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = { - val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val matched = ctx.freshName("matched") - val buildVars = genBuildSideVars(ctx, matched) - val numOutput = metricTerm(ctx, "numOutputRows") - - // filter the output via condition - val conditionPassed = ctx.freshName("conditionPassed") - val checkCondition = if (condition.isDefined) { - val expr = condition.get - // evaluate the variables from build side that used by condition - val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) - ctx.currentVars = input ++ buildVars - val ev = - BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) - s""" - |boolean $conditionPassed = true; - |${eval.trim} - |if ($matched != null) { - | ${ev.code} - | $conditionPassed = !${ev.isNull} && ${ev.value}; - |} - """.stripMargin - } else { - s"final boolean $conditionPassed = true;" - } - - val resultVars = buildSide match { - case BuildLeft => buildVars ++ input - case BuildRight => input ++ buildVars - } - if (broadcastRelation.value.keyIsUnique) { - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashedRelation - |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - |${checkCondition.trim} - |if (!$conditionPassed) { - | $matched = null; - | // reset the variables those are already evaluated. - | ${buildVars.filter(_.code.isEmpty).map(v => s"${v.isNull} = true;").mkString("\n")} - |} - |$numOutput.add(1); - |${consume(ctx, resultVars)} - """.stripMargin - - } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - val found = ctx.freshName("found") - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashRelation - |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); - |boolean $found = false; - |// the last iteration of this loop is to emit an empty row if there is no matched rows. - |while ($matches != null && $matches.hasNext() || !$found) { - | UnsafeRow $matched = $matches != null && $matches.hasNext() ? - | (UnsafeRow) $matches.next() : null; - | ${checkCondition.trim} - | if ($conditionPassed) { - | $found = true; - | $numOutput.add(1); - | ${consume(ctx, resultVars)} - | } - |} - """.stripMargin - } - } - - /** - * Generates the code for left semi join. - */ - private def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = { + protected override def prepareRelation(ctx: CodegenContext): (String, Boolean) = { val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val (matched, checkCondition, _) = getJoinCondition(ctx, input) - val numOutput = metricTerm(ctx, "numOutputRows") - if (broadcastRelation.value.keyIsUnique) { - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashedRelation - |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - |if ($matched != null) { - | $checkCondition { - | $numOutput.add(1); - | ${consume(ctx, input)} - | } - |} - """.stripMargin - } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - val found = ctx.freshName("found") - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashRelation - |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); - |if ($matches != null) { - | boolean $found = false; - | while (!$found && $matches.hasNext()) { - | UnsafeRow $matched = (UnsafeRow) $matches.next(); - | $checkCondition { - | $found = true; - | } - | } - | if ($found) { - | $numOutput.add(1); - | ${consume(ctx, input)} - | } - |} - """.stripMargin - } + (relationTerm, broadcastRelation.value.keyIsUnique) } /** * Generates the code for anti join. + * Handles NULL-aware anti join (NAAJ) separately here. */ - private def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = { - val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val uniqueKeyCodePath = broadcastRelation.value.keyIsUnique - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val (matched, checkCondition, _) = getJoinCondition(ctx, input) - val numOutput = metricTerm(ctx, "numOutputRows") - + protected override def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = { if (isNullAwareAntiJoin) { + val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, _, _) = getJoinCondition(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + if (broadcastRelation.value == EmptyHashedRelation) { - return s""" - |// If the right side is empty, NAAJ simply returns the left side. - |$numOutput.add(1); - |${consume(ctx, input)} - """.stripMargin + s""" + |// If the right side is empty, NAAJ simply returns the left side. + |$numOutput.add(1); + |${consume(ctx, input)} + """.stripMargin } else if (broadcastRelation.value == EmptyHashedRelationWithAllNullKeys) { - return s""" - |// If the right side contains any all-null key, NAAJ simply returns Nothing. - """.stripMargin + s""" + |// If the right side contains any all-null key, NAAJ simply returns Nothing. + """.stripMargin } else { val found = ctx.freshName("found") - return s""" - |boolean $found = false; - |// generate join key for stream side - |${keyEv.code} - |if ($anyNull) { - | $found = true; - |} else { - | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - | if ($matched != null) { - | $found = true; - | } - |} - | - |if (!$found) { - | $numOutput.add(1); - | ${consume(ctx, input)} - |} - """.stripMargin + s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |if ($anyNull) { + | $found = true; + |} else { + | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + | if ($matched != null) { + | $found = true; + | } + |} + | + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin } - } - - if (uniqueKeyCodePath) { - val found = ctx.freshName("found") - s""" - |boolean $found = false; - |// generate join key for stream side - |${keyEv.code} - |// Check if the key has nulls. - |if (!($anyNull)) { - | // Check if the HashedRelation exists. - | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - | if ($matched != null) { - | // Evaluate the condition. - | $checkCondition { - | $found = true; - | } - | } - |} - |if (!$found) { - | $numOutput.add(1); - | ${consume(ctx, input)} - |} - """.stripMargin - } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - val found = ctx.freshName("found") - s""" - |boolean $found = false; - |// generate join key for stream side - |${keyEv.code} - |// Check if the key has nulls. - |if (!($anyNull)) { - | // Check if the HashedRelation exists. - | $iteratorCls $matches = ($iteratorCls)$relationTerm.get(${keyEv.value}); - | if ($matches != null) { - | // Evaluate the condition. - | while (!$found && $matches.hasNext()) { - | UnsafeRow $matched = (UnsafeRow) $matches.next(); - | $checkCondition { - | $found = true; - | } - | } - | } - |} - |if (!$found) { - | $numOutput.add(1); - | ${consume(ctx, input)} - |} - """.stripMargin - } - } - - /** - * Generates the code for existence join. - */ - private def codegenExistence(ctx: CodegenContext, input: Seq[ExprCode]): String = { - val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val numOutput = metricTerm(ctx, "numOutputRows") - val existsVar = ctx.freshName("exists") - - val matched = ctx.freshName("matched") - val buildVars = genBuildSideVars(ctx, matched) - val checkCondition = if (condition.isDefined) { - val expr = condition.get - // evaluate the variables from build side that used by condition - val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) - // filter the output via condition - ctx.currentVars = input ++ buildVars - val ev = - BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) - s""" - |$eval - |${ev.code} - |$existsVar = !${ev.isNull} && ${ev.value}; - """.stripMargin - } else { - s"$existsVar = true;" - } - - val resultVar = input ++ Seq(ExprCode.forNonNullValue( - JavaCode.variable(existsVar, BooleanType))) - if (broadcastRelation.value.keyIsUnique) { - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashedRelation - |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - |boolean $existsVar = false; - |if ($matched != null) { - | $checkCondition - |} - |$numOutput.add(1); - |${consume(ctx, resultVar)} - """.stripMargin } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashRelation - |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); - |boolean $existsVar = false; - |if ($matches != null) { - | while (!$existsVar && $matches.hasNext()) { - | UnsafeRow $matched = (UnsafeRow) $matches.next(); - | $checkCondition - | } - |} - |$numOutput.add(1); - |${consume(ctx, resultVar)} - """.stripMargin + super.codegenAnti(ctx, input) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 4f22007b65845..1c6504b141890 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -20,14 +20,16 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{ExplainUtils, RowIterator} +import org.apache.spark.sql.execution.{CodegenSupport, ExplainUtils, RowIterator} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.types.{IntegralType, LongType} +import org.apache.spark.sql.types.{BooleanType, IntegralType, LongType} -trait HashJoin extends BaseJoinExec { +trait HashJoin extends BaseJoinExec with CodegenSupport { def buildSide: BuildSide override def simpleStringWithNodeId(): String = { @@ -316,6 +318,409 @@ trait HashJoin extends BaseJoinExec { resultProj(r) } } + + override def doProduce(ctx: CodegenContext): String = { + streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this) + } + + override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { + joinType match { + case _: InnerLike => codegenInner(ctx, input) + case LeftOuter | RightOuter => codegenOuter(ctx, input) + case LeftSemi => codegenSemi(ctx, input) + case LeftAnti => codegenAnti(ctx, input) + case _: ExistenceJoin => codegenExistence(ctx, input) + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType") + } + } + + /** + * Returns the code for generating join key for stream side, and expression of whether the key + * has any null in it or not. + */ + protected def genStreamSideJoinKey( + ctx: CodegenContext, + input: Seq[ExprCode]): (ExprCode, String) = { + ctx.currentVars = input + if (streamedBoundKeys.length == 1 && streamedBoundKeys.head.dataType == LongType) { + // generate the join key as Long + val ev = streamedBoundKeys.head.genCode(ctx) + (ev, ev.isNull) + } else { + // generate the join key as UnsafeRow + val ev = GenerateUnsafeProjection.createCode(ctx, streamedBoundKeys) + (ev, s"${ev.value}.anyNull()") + } + } + + /** + * Generates the code for variable of build side. + */ + private def genBuildSideVars(ctx: CodegenContext, matched: String): Seq[ExprCode] = { + ctx.currentVars = null + ctx.INPUT_ROW = matched + buildPlan.output.zipWithIndex.map { case (a, i) => + val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx) + if (joinType.isInstanceOf[InnerLike]) { + ev + } else { + // the variables are needed even there is no matched rows + val isNull = ctx.freshName("isNull") + val value = ctx.freshName("value") + val javaType = CodeGenerator.javaType(a.dataType) + val code = code""" + |boolean $isNull = true; + |$javaType $value = ${CodeGenerator.defaultValue(a.dataType)}; + |if ($matched != null) { + | ${ev.code} + | $isNull = ${ev.isNull}; + | $value = ${ev.value}; + |} + """.stripMargin + ExprCode(code, JavaCode.isNullVariable(isNull), JavaCode.variable(value, a.dataType)) + } + } + } + + /** + * Generate the (non-equi) condition used to filter joined rows. This is used in Inner, Left Semi + * and Left Anti joins. + */ + protected def getJoinCondition( + ctx: CodegenContext, + input: Seq[ExprCode]): (String, String, Seq[ExprCode]) = { + val matched = ctx.freshName("matched") + val buildVars = genBuildSideVars(ctx, matched) + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + // filter the output via condition + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + val skipRow = s"${ev.isNull} || !${ev.value}" + s""" + |$eval + |${ev.code} + |if (!($skipRow)) + """.stripMargin + } else { + "" + } + (matched, checkCondition, buildVars) + } + + /** + * Generates the code for Inner join. + */ + protected def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + + val resultVars = buildSide match { + case BuildLeft => buildVars ++ input + case BuildRight => input ++ buildVars + } + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |if ($matched != null) { + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? + | null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |if ($matches != null) { + | while ($matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for left or right outer join. + */ + protected def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val matched = ctx.freshName("matched") + val buildVars = genBuildSideVars(ctx, matched) + val numOutput = metricTerm(ctx, "numOutputRows") + + // filter the output via condition + val conditionPassed = ctx.freshName("conditionPassed") + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + s""" + |boolean $conditionPassed = true; + |${eval.trim} + |if ($matched != null) { + | ${ev.code} + | $conditionPassed = !${ev.isNull} && ${ev.value}; + |} + """.stripMargin + } else { + s"final boolean $conditionPassed = true;" + } + + val resultVars = buildSide match { + case BuildLeft => buildVars ++ input + case BuildRight => input ++ buildVars + } + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |${checkCondition.trim} + |if (!$conditionPassed) { + | $matched = null; + | // reset the variables those are already evaluated. + | ${buildVars.filter(_.code.isEmpty).map(v => s"${v.isNull} = true;").mkString("\n")} + |} + |$numOutput.add(1); + |${consume(ctx, resultVars)} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |boolean $found = false; + |// the last iteration of this loop is to emit an empty row if there is no matched rows. + |while ($matches != null && $matches.hasNext() || !$found) { + | UnsafeRow $matched = $matches != null && $matches.hasNext() ? + | (UnsafeRow) $matches.next() : null; + | ${checkCondition.trim} + | if ($conditionPassed) { + | $found = true; + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for left semi join. + */ + protected def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, _) = getJoinCondition(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |if ($matched != null) { + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, input)} + | } + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |if ($matches != null) { + | boolean $found = false; + | while (!$found && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $found = true; + | } + | } + | if ($found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for anti join. + */ + protected def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, _) = getJoinCondition(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + + if (keyIsUnique) { + val found = ctx.freshName("found") + s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |// Check if the key has nulls. + |if (!($anyNull)) { + | // Check if the HashedRelation exists. + | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + | if ($matched != null) { + | // Evaluate the condition. + | $checkCondition { + | $found = true; + | } + | } + |} + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |// Check if the key has nulls. + |if (!($anyNull)) { + | // Check if the HashedRelation exists. + | $iteratorCls $matches = ($iteratorCls)$relationTerm.get(${keyEv.value}); + | if ($matches != null) { + | // Evaluate the condition. + | while (!$found && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $found = true; + | } + | } + | } + |} + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin + } + } + + /** + * Generates the code for existence join. + */ + protected def codegenExistence(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + val existsVar = ctx.freshName("exists") + + val matched = ctx.freshName("matched") + val buildVars = genBuildSideVars(ctx, matched) + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + // filter the output via condition + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + s""" + |$eval + |${ev.code} + |$existsVar = !${ev.isNull} && ${ev.value}; + """.stripMargin + } else { + s"$existsVar = true;" + } + + val resultVar = input ++ Seq(ExprCode.forNonNullValue( + JavaCode.variable(existsVar, BooleanType))) + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |boolean $existsVar = false; + |if ($matched != null) { + | $checkCondition + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |boolean $existsVar = false; + |if ($matches != null) { + | while (!$existsVar && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition + | } + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } + } + + /** + * Returns a tuple of variable name for HashedRelation, + * and a boolean to indicate whether keys of HashedRelation + * known to be unique in code-gen time. + */ + protected def prepareRelation(ctx: CodegenContext): (String, Boolean) } object HashJoin { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 3b398dd7120c2..9f811cddef6a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -23,6 +23,7 @@ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.optimizer.BuildSide import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -49,7 +50,10 @@ case class ShuffledHashJoinExec( override def outputPartitioning: Partitioning = super[ShuffledJoin].outputPartitioning - private def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = { + /** + * This is called by generated Java class, should be public. + */ + def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = { val buildDataSize = longMetric("buildDataSize") val buildTime = longMetric("buildTime") val start = System.nanoTime() @@ -70,4 +74,20 @@ case class ShuffledHashJoinExec( join(streamIter, hashed, numOutputRows) } } + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + streamedPlan.execute() :: buildPlan.execute() :: Nil + } + + override def needCopyResult: Boolean = true + + protected override def prepareRelation(ctx: CodegenContext): (String, Boolean) = { + val thisPlan = ctx.addReferenceObj("plan", this) + val clsName = classOf[HashedRelation].getName + + // Inline mutable state since not many join operations in a task + val relationTerm = ctx.addMutableState(clsName, "relation", + v => s"$v = $thisPlan.buildHashedRelation(inputs[1]);", forceInline = true) + (relationTerm, false) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 03596d8654c66..fe40d7dce344d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -22,8 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAnd import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec -import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -71,6 +70,31 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) } + test("ShuffledHashJoin should be included in WholeStageCodegen") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { + val df1 = spark.range(5).select($"id".as("k1")) + val df2 = spark.range(15).select($"id".as("k2")) + val df3 = spark.range(6).select($"id".as("k3")) + + // test one shuffled hash join + val oneJoinDF = df1.join(df2, $"k1" === $"k2") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) + + // test two shuffled hash joins + val twoJoinsDF = df1.join(df2, $"k1" === $"k2").join(df3, $"k1" === $"k3") + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) + } + } + test("Sort should be included in WholeStageCodegen") { val df = spark.range(3, 0, -1).toDF().sort(col("id")) val plan = df.queryExecution.executedPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 50652690339a8..078a3ba029e4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -346,8 +346,8 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils val rightDf = (1 to 10).map(i => (i, i.toString)).toSeq.toDF("key2", "value") Seq((0L, "right_outer", leftDf, rightDf, 10L, false), (0L, "left_outer", rightDf, leftDf, 10L, false), - (0L, "right_outer", leftDf, rightDf, 10L, true), - (0L, "left_outer", rightDf, leftDf, 10L, true), + (1L, "right_outer", leftDf, rightDf, 10L, true), + (1L, "left_outer", rightDf, leftDf, 10L, true), (2L, "left_anti", rightDf, leftDf, 8L, true), (2L, "left_semi", rightDf, leftDf, 2L, true), (1L, "left_anti", rightDf, leftDf, 8L, false), From 813532d10310027fee9e12680792cee2e1c2b7c7 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Fri, 31 Jul 2020 14:52:33 +0900 Subject: [PATCH 342/384] [SPARK-32468][SS][TESTS] Fix timeout config issue in Kafka connector tests ### What changes were proposed in this pull request? While I'm implementing SPARK-32032 I've found a bug in Kafka: https://issues.apache.org/jira/browse/KAFKA-10318. This will cause issues only later when it's fixed but it would be good to fix it now because SPARK-32032 would like to bring in `AdminClient` where the code blows up with the mentioned `ConfigException`. This would reduce the code changes in the mentioned jira. In this PR I've changed `default.api.timeout.ms` to `request.timeout.ms` which fulfils this condition. ### Why are the changes needed? Solve later problems and reduce SPARK-32032 PR size. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #29272 from gaborgsomogyi/SPARK-32468. Authored-by: Gabor Somogyi Signed-off-by: Jungtaek Lim (HeartSaVioR) --- .../spark/sql/kafka010/KafkaContinuousSourceSuite.scala | 2 +- .../spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala | 2 +- .../spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index 76c25980fc33f..099d6ff13051b 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -193,7 +193,7 @@ class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") - .option("kafka.default.api.timeout.ms", "3000") + .option("kafka.request.timeout.ms", "3000") .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala index a1c7b5ae13474..5b634e4d50641 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala @@ -223,7 +223,7 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with KafkaM .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") - .option("kafka.default.api.timeout.ms", "3000") + .option("kafka.request.timeout.ms", "3000") .option("subscribePattern", "failOnDataLoss.*") .option("startingOffsets", "earliest") .option("failOnDataLoss", "false") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index ee31652eaf1f4..517d153ca3c91 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -363,7 +363,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") - .option("kafka.default.api.timeout.ms", "3000") + .option("kafka.request.timeout.ms", "3000") .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") @@ -400,7 +400,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") - .option("kafka.default.api.timeout.ms", "3000") + .option("kafka.request.timeout.ms", "3000") .option("startingOffsets", "earliest") .option("subscribePattern", s"$topicPrefix-.*") @@ -589,7 +589,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") - .option("kafka.default.api.timeout.ms", "3000") + .option("kafka.request.timeout.ms", "3000") .option("subscribe", topic) // If a topic is deleted and we try to poll data starting from offset 0, // the Kafka consumer will just block until timeout and return an empty result. @@ -1860,7 +1860,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest { .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", "stress.*") .option("failOnDataLoss", "false") - .option("kafka.default.api.timeout.ms", "3000") + .option("kafka.request.timeout.ms", "3000") .load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .as[(String, String)] From 8014b0b5d61237dc4851d4ae9927778302d692da Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 31 Jul 2020 17:28:35 +0900 Subject: [PATCH 343/384] [SPARK-32160][CORE][PYSPARK] Add a config to switch allow/disallow to create SparkContext in executors ### What changes were proposed in this pull request? This is a follow-up of #28986. This PR adds a config to switch allow/disallow to create `SparkContext` in executors. - `spark.driver.allowSparkContextInExecutors` ### Why are the changes needed? Some users or libraries actually create `SparkContext` in executors. We shouldn't break their workloads. ### Does this PR introduce _any_ user-facing change? Yes, users will be able to create `SparkContext` in executors with the config enabled. ### How was this patch tested? More tests are added. Closes #29278 from ueshin/issues/SPARK-32160/add_configs. Authored-by: Takuya UESHIN Signed-off-by: HyukjinKwon --- .../scala/org/apache/spark/SparkContext.scala | 6 +++-- .../spark/internal/config/package.scala | 7 +++++ .../org/apache/spark/SparkContextSuite.scala | 9 +++++++ docs/core-migration-guide.md | 4 +++ python/pyspark/context.py | 6 +++-- python/pyspark/tests/test_context.py | 11 ++++++++ .../org/apache/spark/sql/SparkSession.scala | 12 ++++++--- .../spark/sql/SparkSessionBuilderSuite.scala | 26 ++++++++++++++++++- 8 files changed, 72 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 06abc0541a9a9..9ecf316beeaa1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -83,8 +83,10 @@ class SparkContext(config: SparkConf) extends Logging { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() - // In order to prevent SparkContext from being created in executors. - SparkContext.assertOnDriver() + if (!config.get(ALLOW_SPARK_CONTEXT_IN_EXECUTORS)) { + // In order to prevent SparkContext from being created in executors. + SparkContext.assertOnDriver() + } // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having started construction. diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index e1b598e670495..fdc9253ce9b02 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1908,4 +1908,11 @@ package object config { .version("3.1.0") .booleanConf .createWithDefault(false) + + private[spark] val ALLOW_SPARK_CONTEXT_IN_EXECUTORS = + ConfigBuilder("spark.driver.allowSparkContextInExecutors") + .doc("If set to true, SparkContext can be created in executors.") + .version("3.0.1") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 132e994c37a50..1f7aa8eec8942 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -946,6 +946,15 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(error.contains("SparkContext should only be created and accessed on the driver.")) } + + test("SPARK-32160: Allow to create SparkContext in executors if the config is set") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) + + sc.range(0, 1).foreach { _ => + new SparkContext(new SparkConf().setAppName("test").setMaster("local") + .set(ALLOW_SPARK_CONTEXT_IN_EXECUTORS, true)).stop() + } + } } object SparkContextSuite { diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 63baef145f01e..b2a08502d0d6f 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -22,6 +22,10 @@ license: | * Table of contents {:toc} +## Upgrading from Core 3.0 to 3.1 + +- In Spark 3.0 and below, `SparkContext` can be created in executors. Since Spark 3.1, an exception will be thrown when creating `SparkContext` in executors. You can allow it by setting the configuration `spark.driver.allowSparkContextInExecutors` when creating `SparkContext` in executors. + ## Upgrading from Core 2.4 to 3.0 - The `org.apache.spark.ExecutorPlugin` interface and related configuration has been replaced with diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5ddce9f4584c4..0816657692826 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -117,8 +117,10 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ... ValueError:... """ - # In order to prevent SparkContext from being created in executors. - SparkContext._assert_on_driver() + if (conf is None or + conf.get("spark.driver.allowSparkContextInExecutors", "false").lower() != "true"): + # In order to prevent SparkContext from being created in executors. + SparkContext._assert_on_driver() self._callsite = first_spark_call() or CallSite(None, None, None) if gateway is not None and gateway.gateway_parameters.auth_token is None: diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 168299e385e78..64fe3837e7697 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -275,6 +275,17 @@ def test_disallow_to_create_spark_context_in_executors(self): self.assertIn("SparkContext should only be created and accessed on the driver.", str(context.exception)) + def test_allow_to_create_spark_context_in_executors(self): + # SPARK-32160: SparkContext can be created in executors if the config is set. + + def create_spark_context(): + conf = SparkConf().set("spark.driver.allowSparkContextInExecutors", "true") + with SparkContext(conf=conf): + pass + + with SparkContext("local-cluster[3, 1, 1024]") as sc: + sc.range(2).foreach(lambda _: create_spark_context()) + class ContextTestsWithResources(unittest.TestCase): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 08b0a1c6a60a2..306c3235b0bc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -29,6 +29,7 @@ import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.ALLOW_SPARK_CONTEXT_IN_EXECUTORS import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog @@ -900,7 +901,13 @@ object SparkSession extends Logging { * @since 2.0.0 */ def getOrCreate(): SparkSession = synchronized { - assertOnDriver() + val sparkConf = new SparkConf() + options.foreach { case (k, v) => sparkConf.set(k, v) } + + if (!sparkConf.get(ALLOW_SPARK_CONTEXT_IN_EXECUTORS)) { + assertOnDriver() + } + // Get the session from current thread's active session. var session = activeThreadSession.get() if ((session ne null) && !session.sparkContext.isStopped) { @@ -919,9 +926,6 @@ object SparkSession extends Logging { // No active nor global default session. Create a new one. val sparkContext = userSuppliedContext.getOrElse { - val sparkConf = new SparkConf() - options.foreach { case (k, v) => sparkConf.set(k, v) } - // set a random app name if not given. if (!sparkConf.contains("spark.app.name")) { sparkConf.setAppName(java.util.UUID.randomUUID().toString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index e914d8398e921..cc261a9ed3598 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark.internal.config.ALLOW_SPARK_CONTEXT_IN_EXECUTORS import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf._ @@ -257,4 +258,27 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { context.stop() } } + + test("SPARK-32160: Disallow to create SparkSession in executors") { + val session = SparkSession.builder().master("local-cluster[3, 1, 1024]").getOrCreate() + + val error = intercept[SparkException] { + session.range(1).foreach { v => + SparkSession.builder.master("local").getOrCreate() + () + } + }.getMessage() + + assert(error.contains("SparkSession should only be created and accessed on the driver.")) + } + + test("SPARK-32160: Allow to create SparkSession in executors if the config is set") { + val session = SparkSession.builder().master("local-cluster[3, 1, 1024]").getOrCreate() + + session.range(1).foreach { v => + SparkSession.builder.master("local") + .config(ALLOW_SPARK_CONTEXT_IN_EXECUTORS.key, true).getOrCreate().stop() + () + } + } } From f4800406a455b33afa7b1d62d10f236da4cd1f83 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 31 Jul 2020 22:11:05 +0900 Subject: [PATCH 344/384] [SPARK-32406][SQL][FOLLOWUP] Make RESET fail against static and core configs ### What changes were proposed in this pull request? This followup addresses comments from https://github.com/apache/spark/pull/29202#discussion_r462054784 1. make RESET static SQL configs/spark core configs fail as same as the SET command. Not that, for core ones, they have to be pre-registered, otherwise, they are still able to be SET/RESET 2. add test cases for configurations w/ optional default values ### Why are the changes needed? behavior change with suggestions from PMCs ### Does this PR introduce _any_ user-facing change? Yes, RESET will fail after this PR, before it just does nothing because the static ones are static. ### How was this patch tested? add more tests. Closes #29297 from yaooqinn/SPARK-32406-F. Authored-by: Kent Yao Signed-off-by: HyukjinKwon --- .../sql/execution/command/SetCommand.scala | 11 ++++---- .../spark/sql/internal/SQLConfSuite.scala | 27 +++++++++++++++++++ 2 files changed, 33 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 24c43ee407caa..fd89e361fe3d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -172,15 +172,16 @@ object SetCommand { case class ResetCommand(config: Option[String]) extends RunnableCommand with IgnoreCachedData { override def run(sparkSession: SparkSession): Seq[Row] = { - val conf = sparkSession.sessionState.conf val defaults = sparkSession.sparkContext.conf config match { case Some(key) => - conf.unsetConf(key) - defaults.getOption(key).foreach(conf.setConfString(key, _)) + sparkSession.conf.unset(key) + defaults.getOption(key).foreach(sparkSession.conf.set(key, _)) case None => - conf.clear() - defaults.getAll.foreach { case (k, v) => conf.setConfString(k, v) } + sparkSession.sessionState.conf.clear() + defaults.getAll.foreach { case (k, v) => + sparkSession.sessionState.conf.setConfString(k, v) + } } Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 0ecc5ee04ce16..565a8351d9c06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -142,9 +142,12 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) + assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES).isEmpty) sql(s"reset") assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL)) assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) + assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES) === + Some("org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation")) } finally { sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") } @@ -182,18 +185,42 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } test("SPARK-32406: reset - single configuration") { + spark.sessionState.conf.clear() + // spark core conf w/o entry registered val appId = spark.sparkContext.getConf.getAppId sql("RESET spark.app.id") assert(spark.conf.get("spark.app.id") === appId, "Should not change spark core ones") + // spark core conf w/ entry registered + val e1 = intercept[AnalysisException](sql("RESET spark.executor.cores")) + assert(e1.getMessage === "Cannot modify the value of a Spark config: spark.executor.cores;") + + // user defined settings sql("SET spark.abc=xyz") assert(spark.conf.get("spark.abc") === "xyz") sql("RESET spark.abc") intercept[NoSuchElementException](spark.conf.get("spark.abc")) sql("RESET spark.abc") // ignore nonexistent keys + + // runtime sql configs val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) sql(s"SET ${SQLConf.GROUP_BY_ORDINAL.key}=false") sql(s"RESET ${SQLConf.GROUP_BY_ORDINAL.key}") assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === original) + + // runtime sql configs with optional defaults + assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES).isEmpty) + sql(s"RESET ${SQLConf.OPTIMIZER_EXCLUDED_RULES.key}") + assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES) === + Some("org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation")) + sql(s"SET ${SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES.key}=abc") + sql(s"RESET ${SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES.key}") + assert(spark.conf.get(SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES).isEmpty) + + // static sql configs + val e2 = intercept[AnalysisException](sql(s"RESET ${StaticSQLConf.WAREHOUSE_PATH.key}")) + assert(e2.getMessage === + s"Cannot modify the value of a static config: ${StaticSQLConf.WAREHOUSE_PATH.key};") + } test("invalid conf value") { From 4eaf3a0a23d66f234e98062852223d81ec770fbe Mon Sep 17 00:00:00 2001 From: Venkata krishnan Sowrirajan Date: Fri, 31 Jul 2020 22:12:24 +0900 Subject: [PATCH 345/384] [SPARK-31418][CORE][FOLLOW-UP][MINOR] Fix log messages to print stage id instead of the object name ### What changes were proposed in this pull request? Just few log lines fixes which are logging the object name instead of the stage IDs ### Why are the changes needed? This would make it easier later for debugging. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Just log messages. Existing tests should be enough Closes #29279 from venkata91/SPARK-31418-follow-up. Authored-by: Venkata krishnan Sowrirajan Signed-off-by: HyukjinKwon --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index b734d9f72944a..2551e497a165a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -664,8 +664,9 @@ private[spark] class TaskSchedulerImpl( // in order to provision more executors to make them schedulable if (Utils.isDynamicAllocationEnabled(conf)) { if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { - logInfo(s"Notifying ExecutorAllocationManager to allocate more executors to" + - s" schedule the unschedulable task before aborting $taskSet.") + logInfo("Notifying ExecutorAllocationManager to allocate more executors to" + + " schedule the unschedulable task before aborting" + + " stage ${taskSet.stageId}.") dagScheduler.unschedulableTaskSetAdded(taskSet.taskSet.stageId, taskSet.taskSet.stageAttemptId) updateUnschedulableTaskSetTimeoutAndStartAbortTimer(taskSet, taskIndex) @@ -673,7 +674,7 @@ private[spark] class TaskSchedulerImpl( } else { // Abort Immediately logInfo("Cannot schedule any task because of complete blacklisting. No idle" + - s" executors can be found to kill. Aborting $taskSet.") + s" executors can be found to kill. Aborting stage ${taskSet.stageId}.") taskSet.abortSinceCompletelyBlacklisted(taskIndex) } } @@ -744,7 +745,7 @@ private[spark] class TaskSchedulerImpl( val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout logInfo(s"Waiting for $timeout ms for completely " + - s"blacklisted task to be schedulable again before aborting $taskSet.") + s"blacklisted task to be schedulable again before aborting stage ${taskSet.stageId}.") abortTimer.schedule( createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout) } @@ -757,7 +758,7 @@ private[spark] class TaskSchedulerImpl( if (unschedulableTaskSetToExpiryTime.contains(taskSet) && unschedulableTaskSetToExpiryTime(taskSet) <= clock.getTimeMillis()) { logInfo("Cannot schedule any task because of complete blacklisting. " + - s"Wait time for scheduling expired. Aborting $taskSet.") + s"Wait time for scheduling expired. Aborting stage ${taskSet.stageId}.") taskSet.abortSinceCompletelyBlacklisted(taskIndex) } else { this.cancel() From 354313b6bc89149f97b7cebf6249abd9e3e87724 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Fri, 31 Jul 2020 14:16:41 +0000 Subject: [PATCH 346/384] [SPARK-31894][SS][FOLLOW-UP] Rephrase the config doc ### What changes were proposed in this pull request? Address comment in https://github.com/apache/spark/pull/28707#discussion_r461102749 ### Why are the changes needed? Hide the implementation details in the config doc. ### Does this PR introduce _any_ user-facing change? Config doc change. ### How was this patch tested? Document only. Closes #29315 from xuanyuanking/SPARK-31894-follow. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1304a021386f0..6b407e65a9a72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1236,9 +1236,9 @@ object SQLConf { val STATE_STORE_FORMAT_VALIDATION_ENABLED = buildConf("spark.sql.streaming.stateStore.formatValidation.enabled") .internal() - .doc("When true, check if the UnsafeRow from the state store is valid or not when running " + - "streaming queries. This can happen if the state store format has been changed. Note, " + - "the feature is only effective in the build-in HDFS state store provider now.") + .doc("When true, check if the data from state store is valid or not when running streaming " + + "queries. This can happen if the state store format has been changed. Note, the feature " + + "is only effective in the build-in HDFS state store provider now.") .version("3.1.0") .booleanConf .createWithDefault(true) From 1c6dff7b5fc171c190feea0d8f7d323e330d9151 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 31 Jul 2020 14:20:20 +0000 Subject: [PATCH 347/384] [SPARK-32083][SQL] AQE coalesce should at least return one partition ### What changes were proposed in this pull request? This PR updates the AQE framework to at least return one partition during coalescing. This PR also updates `ShuffleExchangeExec.canChangeNumPartitions` to not coalesce for `SinglePartition`. ### Why are the changes needed? It's a bit risky to return 0 partitions, as sometimes it's different from empty data. For example, global aggregate will return one result row even if the input table is empty. If there is 0 partition, no task will be run and no result will be returned. More specifically, the global aggregate requires `AllTuples` and we can't coalesce to 0 partitions. This is not a real bug for now. The global aggregate will be planned as partial and final physical agg nodes. The partial agg will return at least one row, so that the shuffle still have data. But it's better to fix this issue to avoid potential bugs in the future. According to https://github.com/apache/spark/pull/28916, this change also fix some perf problems. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? updated test. Closes #29307 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../spark/sql/execution/SparkStrategies.scala | 7 ++++--- .../adaptive/OptimizeLocalShuffleReader.scala | 4 ++-- .../adaptive/ShufflePartitionsUtil.scala | 7 ++++--- .../execution/exchange/ShuffleExchangeExec.scala | 7 ++++++- .../execution/ShufflePartitionsUtilSuite.scala | 15 ++++++++++----- .../adaptive/AdaptiveQueryExecSuite.scala | 10 +++++----- 6 files changed, 31 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 4f9cb22ee0057..eb32bfcecae7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -665,7 +665,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Repartition(numPartitions, shuffle, child) => if (shuffle) { ShuffleExchangeExec(RoundRobinPartitioning(numPartitions), - planLater(child), canChangeNumPartitions = false) :: Nil + planLater(child), noUserSpecifiedNumPartition = false) :: Nil } else { execution.CoalesceExec(numPartitions, planLater(child)) :: Nil } @@ -698,9 +698,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: logical.Range => execution.RangeExec(r) :: Nil case r: logical.RepartitionByExpression => - val canChangeNumParts = r.optNumPartitions.isEmpty exchange.ShuffleExchangeExec( - r.partitioning, planLater(r.child), canChangeNumParts) :: Nil + r.partitioning, + planLater(r.child), + noUserSpecifiedNumPartition = r.optNumPartitions.isEmpty) :: Nil case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 45fb36420e770..7bb9265e1717a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -140,9 +140,9 @@ object OptimizeLocalShuffleReader { def canUseLocalShuffleReader(plan: SparkPlan): Boolean = plan match { case s: ShuffleQueryStageExec => - s.shuffle.canChangeNumPartitions + s.shuffle.canChangeNumPartitions && s.mapStats.isDefined case CustomShuffleReaderExec(s: ShuffleQueryStageExec, partitionSpecs) => - s.shuffle.canChangeNumPartitions && partitionSpecs.nonEmpty + s.shuffle.canChangeNumPartitions && s.mapStats.isDefined && partitionSpecs.nonEmpty case _ => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala index 83fdafbadcb60..ed92af6adc186 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala @@ -92,9 +92,9 @@ object ShufflePartitionsUtil extends Logging { var coalescedSize = 0L var i = 0 - def createPartitionSpec(): Unit = { + def createPartitionSpec(forceCreate: Boolean = false): Unit = { // Skip empty inputs, as it is a waste to launch an empty task. - if (coalescedSize > 0) { + if (coalescedSize > 0 || forceCreate) { partitionSpecs += CoalescedPartitionSpec(latestSplitPoint, i) } } @@ -120,7 +120,8 @@ object ShufflePartitionsUtil extends Logging { } i += 1 } - createPartitionSpec() + // Create at least one partition if all partitions are empty. + createPartitionSpec(partitionSpecs.isEmpty) partitionSpecs.toSeq } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 30c9f0ae1282d..6af4b098bee2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -83,7 +83,12 @@ trait ShuffleExchangeLike extends Exchange { case class ShuffleExchangeExec( override val outputPartitioning: Partitioning, child: SparkPlan, - canChangeNumPartitions: Boolean = true) extends ShuffleExchangeLike { + noUserSpecifiedNumPartition: Boolean = true) extends ShuffleExchangeLike { + + // If users specify the num partitions via APIs like `repartition`, we shouldn't change it. + // For `SinglePartition`, it requires exactly one partition and we can't change it either. + override def canChangeNumPartitions: Boolean = + noUserSpecifiedNumPartition && outputPartitioning != SinglePartition private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala index f5c3b7816f5ea..94e22a414a628 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala @@ -198,9 +198,11 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite { // the size of data is 0. val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) + // Create at least one partition spec + val expectedPartitionSpecs = Seq(CoalescedPartitionSpec(0, 5)) checkEstimation( Array(bytesByPartitionId1, bytesByPartitionId2), - Seq.empty, targetSize, minNumPartitions) + expectedPartitionSpecs, targetSize, minNumPartitions) } @@ -248,16 +250,19 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite { val minNumPartitions = 2 { - // 1 shuffle: All bytes per partition are 0, no partition spec created. + // 1 shuffle: All bytes per partition are 0, 1 empty partition spec created. val bytesByPartitionId = Array[Long](0, 0, 0, 0, 0) - checkEstimation(Array(bytesByPartitionId), Seq.empty, targetSize) + val expectedPartitionSpecs = Seq(CoalescedPartitionSpec(0, 5)) + checkEstimation(Array(bytesByPartitionId), expectedPartitionSpecs, targetSize) } { - // 2 shuffles: All bytes per partition are 0, no partition spec created. + // 2 shuffles: All bytes per partition are 0, 1 empty partition spec created. val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) - checkEstimation(Array(bytesByPartitionId1, bytesByPartitionId2), Seq.empty, targetSize) + val expectedPartitionSpecs = Seq(CoalescedPartitionSpec(0, 5)) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionSpecs, targetSize) } { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 03471fb047260..7fdcbd0d089cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -213,7 +213,7 @@ class AdaptiveQueryExecSuite } } - test("Empty stage coalesced to 0-partition RDD") { + test("Empty stage coalesced to 1-partition RDD") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true") { @@ -227,8 +227,8 @@ class AdaptiveQueryExecSuite val coalescedReaders = collect(plan) { case r: CustomShuffleReaderExec => r } - assert(coalescedReaders.length == 2) - coalescedReaders.foreach(r => assert(r.partitionSpecs.isEmpty)) + assert(coalescedReaders.length == 3) + coalescedReaders.foreach(r => assert(r.partitionSpecs.length == 1)) } withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { @@ -239,8 +239,8 @@ class AdaptiveQueryExecSuite val coalescedReaders = collect(plan) { case r: CustomShuffleReaderExec => r } - assert(coalescedReaders.length == 2, s"$plan") - coalescedReaders.foreach(r => assert(r.partitionSpecs.isEmpty)) + assert(coalescedReaders.length == 3, s"$plan") + coalescedReaders.foreach(r => assert(r.isLocalReader || r.partitionSpecs.length == 1)) } } } From 71aea02e9ffb0c6f7c72c91054c2a4653e22e801 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sat, 1 Aug 2020 13:09:26 +0800 Subject: [PATCH 348/384] [SPARK-32467][UI] Avoid encoding URL twice on https redirect ### What changes were proposed in this pull request? When https is enabled for Spark UI, an HTTP request will be redirected as an encoded HTTPS URL: https://github.com/apache/spark/pull/10238/files#diff-f79a5ead735b3d0b34b6b94486918e1cR312 When we create the redirect url, we will call getRequestURI and getQueryString. Both two methods may return an encoded string. However, we pass them directly to the following URI constructor ``` URI(String scheme, String authority, String path, String query, String fragment) ``` As this URI constructor assumes both path and query parameters are decoded strings, it will encode them again. This makes the redirect URL encoded twice. This problem is on stage page with HTTPS enabled. The URL of "/taskTable" contains query parameter `order%5B0%5D%5Bcolumn%5D`. After encoded it becomes `order%255B0%255D%255Bcolumn%255D` and it will be decoded as `order%5B0%5D%5Bcolumn%5D` instead of `order[0][dir]`. When the parameter `order[0][dir]` is missing, there will be an excetpion from: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala#L176 and the stage page fail to load. To fix the problem, we can try decoding the query parameters before encoding it. This is to make sure we encode the URL ### Why are the changes needed? Fix a UI issue when HTTPS is enabled ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? A new Unit test + manually test on a cluster Closes #29271 from gengliangwang/urlEncode. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang --- .../org/apache/spark/ui/JettyUtils.scala | 29 +++++++++++++++---- .../scala/org/apache/spark/ui/UISuite.scala | 21 ++++++++++++++ 2 files changed, 44 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index f1962ef39fc06..2a3597e323543 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui -import java.net.{URI, URL} +import java.net.{URI, URL, URLDecoder} import java.util.EnumSet import javax.servlet.DispatcherType import javax.servlet.http._ @@ -377,8 +377,7 @@ private[spark] object JettyUtils extends Logging { if (baseRequest.isSecure) { return } - val httpsURI = createRedirectURI(scheme, baseRequest.getServerName, securePort, - baseRequest.getRequestURI, baseRequest.getQueryString) + val httpsURI = createRedirectURI(scheme, securePort, baseRequest) response.setContentLength(0) response.sendRedirect(response.encodeRedirectURL(httpsURI)) baseRequest.setHandled(true) @@ -440,16 +439,34 @@ private[spark] object JettyUtils extends Logging { handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) } + private def decodeURL(url: String, encoding: String): String = { + if (url == null) { + null + } else { + URLDecoder.decode(url, encoding) + } + } + // Create a new URI from the arguments, handling IPv6 host encoding and default ports. - private def createRedirectURI( - scheme: String, server: String, port: Int, path: String, query: String) = { + private def createRedirectURI(scheme: String, port: Int, request: Request): String = { + val server = request.getServerName val redirectServer = if (server.contains(":") && !server.startsWith("[")) { s"[${server}]" } else { server } val authority = s"$redirectServer:$port" - new URI(scheme, authority, path, query, null).toString + val queryEncoding = if (request.getQueryEncoding != null) { + request.getQueryEncoding + } else { + // By default decoding the URI as "UTF-8" should be enough for SparkUI + "UTF-8" + } + // The request URL can be raw or encoded here. To avoid the request URL being + // encoded twice, let's decode it here. + val requestURI = decodeURL(request.getRequestURI, queryEncoding) + val queryString = decodeURL(request.getQueryString, queryEncoding) + new URI(scheme, authority, requestURI, queryString, null).toString } def toVirtualHosts(connectors: String*): Array[String] = connectors.map("@" + _).toArray diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 2ad4a634cd9a7..56026eaa0072b 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -262,6 +262,27 @@ class UISuite extends SparkFunSuite { } } + test("SPARK-32467: Avoid encoding URL twice on https redirect") { + val (conf, securityMgr, sslOptions) = sslEnabledConf() + val serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, conf) + try { + val serverAddr = s"http://localhost:${serverInfo.boundPort}" + + val (_, ctx) = newContext("/ctx1") + serverInfo.addHandler(ctx, securityMgr) + + TestUtils.withHttpConnection(new URL(s"$serverAddr/ctx%281%29?a%5B0%5D=b")) { conn => + assert(conn.getResponseCode() === HttpServletResponse.SC_FOUND) + val location = Option(conn.getHeaderFields().get("Location")) + .map(_.get(0)).orNull + val expectedLocation = s"https://localhost:${serverInfo.securePort.get}/ctx(1)?a[0]=b" + assert(location == expectedLocation) + } + } finally { + stopServer(serverInfo) + } + } + test("http -> https redirect applies to all URIs") { val (conf, securityMgr, sslOptions) = sslEnabledConf() val serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, conf) From 0693d8bbf2942ab96ffe705ef0fc3fe4b0d9ec11 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Sun, 2 Aug 2020 16:46:11 -0700 Subject: [PATCH 349/384] [SPARK-32490][BUILD] Upgrade netty-all to 4.1.51.Final ### What changes were proposed in this pull request? This PR aims to bring the bug fixes from the latest netty version. ### Why are the changes needed? - 4.1.48.Final: [https://github.com/netty/netty/milestone/223?closed=1](https://github.com/netty/netty/milestone/223?closed=1)(14 patches or issues) - 4.1.49.Final: [https://github.com/netty/netty/milestone/224?closed=1](https://github.com/netty/netty/milestone/224?closed=1)(48 patches or issues) - 4.1.50.Final: [https://github.com/netty/netty/milestone/225?closed=1](https://github.com/netty/netty/milestone/225?closed=1)(38 patches or issues) - 4.1.51.Final: [https://github.com/netty/netty/milestone/226?closed=1](https://github.com/netty/netty/milestone/226?closed=1)(53 patches or issues) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #29299 from LuciferYang/upgrade-netty-version. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 2 +- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 45c346fa5ecc4..4936d808db85a 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -156,7 +156,7 @@ 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 -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar objenesis/2.6//objenesis-2.6.jar okhttp/3.12.6//okhttp-3.12.6.jar okio/1.14.0//okio-1.14.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 4dc36d65f8632..82cd20bf77191 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -171,7 +171,7 @@ 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 -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar objenesis/2.6//objenesis-2.6.jar okhttp/3.12.6//okhttp-3.12.6.jar okio/1.14.0//okio-1.14.0.jar 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 40feed25d3a6d..7257fb8722422 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,7 @@ 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 -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar nimbus-jose-jwt/4.41.1//nimbus-jose-jwt-4.41.1.jar objenesis/2.6//objenesis-2.6.jar okhttp/2.7.5//okhttp-2.7.5.jar diff --git a/pom.xml b/pom.xml index d07813c2787de..6b72e7d13643b 100644 --- a/pom.xml +++ b/pom.xml @@ -726,7 +726,7 @@ io.netty netty-all - 4.1.47.Final + 4.1.51.Final org.apache.derby From 713124d5e32fb5984ef3c15ed655284b58399032 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 3 Aug 2020 03:15:54 +0000 Subject: [PATCH 350/384] [SPARK-32274][SQL] Make SQL cache serialization pluggable ### What changes were proposed in this pull request? Add a config to let users change how SQL/Dataframe data is compressed when cached. This adds a few new classes/APIs for use with this config. 1. `CachedBatch` is a trait used to tag data that is intended to be cached. It has a few APIs that lets us keep the compression/serialization of the data separate from the metrics about it. 2. `CachedBatchSerializer` provides the APIs that must be implemented to cache data. * `convertForCache` is an API that runs a cached spark plan and turns its result into an `RDD[CachedBatch]`. The actual caching is done outside of this API * `buildFilter` is an API that takes a set of predicates and builds a filter function that can be used to filter the `RDD[CachedBatch]` returned by `convertForCache` * `decompressColumnar` decompresses an `RDD[CachedBatch]` into an `RDD[ColumnarBatch]` This is only used for a limited set of data types. These data types may expand in the future. If they do we can add in a new API with a default value that says which data types this serializer supports. * `decompressToRows` decompresses an `RDD[CachedBatch]` into an `RDD[InternalRow]` this API, like `decompressColumnar` decompresses the data in `CachedBatch` but turns it into `InternalRow`s, typically using code generation for performance reasons. There is also an API that lets you reuse the current filtering based on min/max values. `SimpleMetricsCachedBatch` and `SimpleMetricsCachedBatchSerializer`. ### Why are the changes needed? This lets users explore different types of compression and compression ratios. ### Does this PR introduce _any_ user-facing change? This adds in a single config, and exposes some developer API classes described above. ### How was this patch tested? I ran the unit tests around this and I also did some manual performance tests. I could find any performance difference between the old and new code, and if there is any it is within error. Closes #29067 from revans2/pluggable_cache_serializer. Authored-by: Robert (Bobby) Evans Signed-off-by: Wenchen Fan --- .../spark/sql/internal/StaticSQLConf.scala | 10 + .../sql/columnar/CachedBatchSerializer.scala | 343 ++++++++++++++++++ .../spark/sql/execution/CacheManager.scala | 15 +- .../apache/spark/sql/execution/Columnar.scala | 13 +- .../sql/execution/columnar/ColumnStats.scala | 4 +- .../columnar/GenerateColumnAccessor.scala | 5 +- .../execution/columnar/InMemoryRelation.scala | 299 +++++++++++---- .../columnar/InMemoryTableScanExec.scala | 256 ++----------- .../columnar/CachedBatchSerializerSuite.scala | 134 +++++++ .../columnar/InMemoryColumnarQuerySuite.scala | 54 +-- 10 files changed, 813 insertions(+), 320 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 10f54f856a193..ca1074fcf6fc0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -127,6 +127,16 @@ object StaticSQLConf { .toSequence .createOptional + val SPARK_CACHE_SERIALIZER = buildStaticConf("spark.sql.cache.serializer") + .doc("The name of a class that implements " + + "org.apache.spark.sql.columnar.CachedBatchSerializer. It will be used to " + + "translate SQL data into a format that can more efficiently be cached. The underlying " + + "API is subject to change so use with caution. Multiple classes cannot be specified. " + + "The class must have a no-arg constructor.") + .version("3.1.0") + .stringConf + .createWithDefault("org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer") + val QUERY_EXECUTION_LISTENERS = buildStaticConf("spark.sql.queryExecutionListeners") .doc("List of class names implementing QueryExecutionListener that will be automatically " + "added to newly created sessions. The classes should have either a no-arg constructor, " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala new file mode 100644 index 0000000000000..1113e63cab332 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -0,0 +1,343 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, LessThanOrEqual, Literal, Or, Predicate, StartsWith} +import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, PartitionStatistics} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage.StorageLevel + +/** + * Basic interface that all cached batches of data must support. This is primarily to allow + * for metrics to be handled outside of the encoding and decoding steps in a standard way. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatch { + def numRows: Int + def sizeInBytes: Long +} + +/** + * Provides APIs that handle transformations of SQL data associated with the cache/persist APIs. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatchSerializer extends Serializable { + /** + * Can `convertColumnarBatchToCachedBatch()` be called instead of + * `convertInternalRowToCachedBatch()` for this given schema? True if it can and false if it + * cannot. Columnar input is only supported if the plan could produce columnar output. Currently + * this is mostly supported by input formats like parquet and orc, but more operations are likely + * to be supported soon. + * @param schema the schema of the data being stored. + * @return True if columnar input can be supported, else false. + */ + def supportsColumnarInput(schema: Seq[Attribute]): Boolean + + /** + * Convert an `RDD[InternalRow]` into an `RDD[CachedBatch]` in preparation for caching the data. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Convert an `RDD[ColumnarBatch]` into an `RDD[CachedBatch]` in preparation for caching the data. + * This will only be called if `supportsColumnarInput()` returned true for the given schema and + * the plan up to this point would could produce columnar output without modifying it. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Builds a function that can be used to filter batches prior to being decompressed. + * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will provide the filter logic + * necessary. You will need to provide metrics for this to work. [[SimpleMetricsCachedBatch]] + * provides the APIs to hold those metrics and explains the metrics used, really just min and max. + * Note that this is intended to skip batches that are not needed, and the actual filtering of + * individual rows is handled later. + * @param predicates the set of expressions to use for filtering. + * @param cachedAttributes the schema/attributes of the data that is cached. This can be helpful + * if you don't store it with the data. + * @return a function that takes the partition id and the iterator of batches in the partition. + * It returns an iterator of batches that should be decompressed. + */ + def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] + + /** + * Can `convertCachedBatchToColumnarBatch()` be called instead of + * `convertCachedBatchToInternalRow()` for this given schema? True if it can and false if it + * cannot. Columnar output is typically preferred because it is more efficient. Note that + * `convertCachedBatchToInternalRow()` must always be supported as there are other checks that + * can force row based output. + * @param schema the schema of the data being checked. + * @return true if columnar output should be used for this schema, else false. + */ + def supportsColumnarOutput(schema: StructType): Boolean + + /** + * The exact java types of the columns that are output in columnar processing mode. This + * is a performance optimization for code generation and is optional. + * @param attributes the attributes to be output. + * @param conf the config for the query that will read the data. + */ + def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = None + + /** + * Convert the cached data into a ColumnarBatch. This currently is only used if + * `supportsColumnarOutput()` returns true for the associated schema, but there are other checks + * that can force row based output. One of the main advantages of doing columnar output over row + * based output is that the code generation is more standard and can be combined with code + * generation for downstream operations. + * @param input the cached batches that should be converted. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the fields that should be loaded from the data and the order they + * should appear in the output batch. + * @param conf the configuration for the job. + * @return an RDD of the input cached batches transformed into the ColumnarBatch format. + */ + def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] + + /** + * Convert the cached batch into `InternalRow`s. If you want this to be performant, code + * generation is advised. + * @param input the cached batches that should be converted. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the field that should be loaded from the data and the order they + * should appear in the output rows. + * @param conf the configuration for the job. + * @return RDD of the rows that were stored in the cached batches. + */ + def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] +} + +/** + * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with + * the [[SimpleMetricsCachedBatchSerializer]]. + * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata + * are needed in the row. + */ +@DeveloperApi +@Since("3.1.0") +trait SimpleMetricsCachedBatch extends CachedBatch { + /** + * Holds stats for each cached column. The optional `upperBound` and `lowerBound` should be + * of the same type as the original column. If they are null, then it is assumed that they + * are not provided, and will not be used for filtering. + *
            + *
          • `upperBound` (optional)
          • + *
          • `lowerBound` (Optional)
          • + *
          • `nullCount`: `Int`
          • + *
          • `rowCount`: `Int`
          • + *
          • `sizeInBytes`: `Long`
          • + *
          + * These are repeated for each column in the original cached data. + */ + val stats: InternalRow + override def sizeInBytes: Long = + Range.apply(4, stats.numFields, 5).map(stats.getLong).sum +} + +// Currently, uses statistics for all atomic types that are not `BinaryType`. +private object ExtractableLiteral { + def unapply(expr: Expression): Option[Literal] = expr match { + case lit: Literal => lit.dataType match { + case BinaryType => None + case _: AtomicType => Some(lit) + case _ => None + } + case _ => None + } +} + +/** + * Provides basic filtering for [[CachedBatchSerializer]] implementations. + * The requirement to extend this is that all of the batches produced by your serializer are + * instances of [[SimpleMetricsCachedBatch]]. + * This does not calculate the metrics needed to be stored in the batches. That is up to each + * implementation. The metrics required are really just min and max values and those are optional + * especially for complex types. Because those metrics are simple and it is likely that compression + * will also be done on the data we thought it best to let each implementation decide on the most + * efficient way to calculate the metrics, possibly combining them with compression passes that + * might also be done across the data. + */ +@DeveloperApi +@Since("3.1.0") +abstract class SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { + override def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + val stats = new PartitionStatistics(cachedAttributes) + val statsSchema = stats.schema + + def statsFor(a: Attribute): ColumnStatisticsSchema = { + stats.forAttribute(a) + } + + // Returned filter predicate should return false iff it is impossible for the input expression + // to evaluate to `true` based on statistics collected about this partition batch. + @transient lazy val buildFilter: PartialFunction[Expression, Expression] = { + case And(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => + (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) + + case Or(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => + buildFilter(lhs) || buildFilter(rhs) + + case EqualTo(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualTo(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l + case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound + + case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l + case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => + l <= statsFor(a).upperBound + + case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound + case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l + + case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + l <= statsFor(a).upperBound + case GreaterThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l + + case IsNull(a: Attribute) => statsFor(a).nullCount > 0 + case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 + + case In(a: AttributeReference, list: Seq[Expression]) + if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => + list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && + l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) + // This is an example to explain how it works, imagine that the id column stored as follows: + // __________________________________________ + // | Partition ID | lowerBound | upperBound | + // |--------------|------------|------------| + // | p1 | '1' | '9' | + // | p2 | '10' | '19' | + // | p3 | '20' | '29' | + // | p4 | '30' | '39' | + // | p5 | '40' | '49' | + // |______________|____________|____________| + // + // A filter: df.filter($"id".startsWith("2")). + // In this case it substr lowerBound and upperBound: + // ________________________________________________________________________________________ + // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | + // |--------------|-----------------------------------|-----------------------------------| + // | p1 | '1' | '9' | + // | p2 | '1' | '1' | + // | p3 | '2' | '2' | + // | p4 | '3' | '3' | + // | p5 | '4' | '4' | + // |______________|___________________________________|___________________________________| + // + // We can see that we only need to read p1 and p3. + case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound.substr(0, Length(l)) <= l && + l <= statsFor(a).upperBound.substr(0, Length(l)) + } + + // When we bind the filters we need to do it against the stats schema + val partitionFilters: Seq[Expression] = { + predicates.flatMap { p => + val filter = buildFilter.lift(p) + val boundFilter = + filter.map( + BindReferences.bindReference( + _, + statsSchema, + allowFailures = true)) + + boundFilter.foreach(_ => + filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) + + // If the filter can't be resolved then we are missing required statistics. + boundFilter.filter(_.resolved) + } + } + + def ret(index: Int, cachedBatchIterator: Iterator[CachedBatch]): Iterator[CachedBatch] = { + val partitionFilter = Predicate.create( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + cachedAttributes) + + partitionFilter.initialize(index) + val schemaIndex = cachedAttributes.zipWithIndex + + cachedBatchIterator.filter { cb => + val cachedBatch = cb.asInstanceOf[SimpleMetricsCachedBatch] + if (!partitionFilter.eval(cachedBatch.stats)) { + logDebug { + val statsString = schemaIndex.map { case (a, i) => + val value = cachedBatch.stats.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + s"Skipping partition based on stats $statsString" + } + false + } else { + true + } + } + } + ret + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 7d86c48015406..7201026b11b6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -27,11 +27,10 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.columnar.{DefaultCachedBatchSerializer, InMemoryRelation} import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -85,11 +84,9 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val inMemoryRelation = sessionWithAqeOff.withActive { val qe = sessionWithAqeOff.sessionState.executePlan(planToCache) InMemoryRelation( - sessionWithAqeOff.sessionState.conf.useCompression, - sessionWithAqeOff.sessionState.conf.columnBatchSize, storageLevel, - qe.executedPlan, - tableName, - optimizedPlan = qe.optimizedPlan) + storageLevel, + qe, + tableName) } this.synchronized { @@ -195,9 +192,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val sessionWithAqeOff = getOrCloneSessionWithAqeOff(spark) val newCache = sessionWithAqeOff.withActive { val qe = sessionWithAqeOff.sessionState.executePlan(cd.plan) - InMemoryRelation( - cacheBuilder = cd.cachedRepresentation.cacheBuilder.copy(cachedPlan = qe.executedPlan), - optimizedPlan = qe.optimizedPlan) + InMemoryRelation(cd.cachedRepresentation.cacheBuilder, qe) } val recomputedPlan = cd.copy(cachedRepresentation = newCache) this.synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index e01cd8598db0c..13ea609f7bfa6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -49,6 +49,13 @@ class ColumnarRule { def postColumnarTransitions: Rule[SparkPlan] = plan => plan } +/** + * A trait that is used as a tag to indicate a transition from columns to rows. This allows plugins + * to replace the current [[ColumnarToRowExec]] with an optimized version and still have operations + * that walk a spark plan looking for this type of transition properly match it. + */ +trait ColumnarToRowTransition extends UnaryExecNode + /** * Provides a common executor to translate an [[RDD]] of [[ColumnarBatch]] into an [[RDD]] of * [[InternalRow]]. This is inserted whenever such a transition is determined to be needed. @@ -57,7 +64,7 @@ class ColumnarRule { * [[org.apache.spark.sql.execution.python.ArrowEvalPythonExec]] and * [[MapPartitionsInRWithArrowExec]]. Eventually this should replace those implementations. */ -case class ColumnarToRowExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { +case class ColumnarToRowExec(child: SparkPlan) extends ColumnarToRowTransition with CodegenSupport { assert(child.supportsColumnar) override def output: Seq[Attribute] = child.output @@ -479,7 +486,9 @@ case class RowToColumnarExec(child: SparkPlan) extends UnaryExecNode { * Apply any user defined [[ColumnarRule]]s and find the correct place to insert transitions * to/from columnar formatted data. */ -case class ApplyColumnarRulesAndInsertTransitions(conf: SQLConf, columnarRules: Seq[ColumnarRule]) +case class ApplyColumnarRulesAndInsertTransitions( + conf: SQLConf, + columnarRules: Seq[ColumnarRule]) extends Rule[SparkPlan] { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index 20ecc57c49e75..45557bfbada6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializable { +class ColumnStatisticsSchema(a: Attribute) extends Serializable { val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = true)() val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() @@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes) } -private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { +class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { val (forAttribute: AttributeMap[ColumnStatisticsSchema], schema: Seq[AttributeReference]) = { val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) (AttributeMap(allStats), allStats.flatMap(_._2.schema)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index bd2d06665a910..eb0663830dd67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.types.CalendarInterval * An Iterator to walk through the InternalRows from a CachedBatch */ abstract class ColumnarIterator extends Iterator[InternalRow] { - def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], + def initialize(input: Iterator[DefaultCachedBatch], columnTypes: Array[DataType], columnIndexes: Array[Int]): Unit } @@ -203,7 +203,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera return false; } - ${classOf[CachedBatch].getName} batch = (${classOf[CachedBatch].getName}) input.next(); + ${classOf[DefaultCachedBatch].getName} batch = + (${classOf[DefaultCachedBatch].getName}) input.next(); currentRow = 0; numRowsInBatch = batch.numRows(); for (int i = 0; i < columnIndexes.length; i ++) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index cf9f3ddeb42a2..be3dc5934e84f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -19,33 +19,189 @@ package org.apache.spark.sql.execution.columnar import org.apache.commons.lang3.StringUtils +import org.apache.spark.TaskContext import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer} +import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.LongAccumulator - +import org.apache.spark.util.{LongAccumulator, Utils} /** - * CachedBatch is a cached batch of rows. + * The default implementation of CachedBatch. * * @param numRows The total number of rows in this batch * @param buffers The buffers for serialized columns * @param stats The stat of columns */ -private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) +case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) + extends SimpleMetricsCachedBatch + +/** + * The default implementation of CachedBatchSerializer. + */ +class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false + + override def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = + throw new IllegalStateException("Columnar input is not supported") + + override def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + val batchSize = conf.columnBatchSize + val useCompression = conf.useCompression + convertForCacheInternal(input, schema, batchSize, useCompression) + } + + def convertForCacheInternal( + input: RDD[InternalRow], + output: Seq[Attribute], + batchSize: Int, + useCompression: Boolean): RDD[CachedBatch] = { + input.mapPartitionsInternal { rowIterator => + new Iterator[DefaultCachedBatch] { + def next(): DefaultCachedBatch = { + val columnBuilders = output.map { attribute => + ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) + }.toArray + + var rowCount = 0 + var totalSize = 0L + while (rowIterator.hasNext && rowCount < batchSize + && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { + val row = rowIterator.next() + + // Added for SPARK-6082. This assertion can be useful for scenarios when something + // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM + // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat + // hard to decipher. + assert( + row.numFields == columnBuilders.length, + s"Row column number mismatch, expected ${output.size} columns, " + + s"but got ${row.numFields}." + + s"\nRow content: $row") + + var i = 0 + totalSize = 0 + while (i < row.numFields) { + columnBuilders(i).appendFrom(row, i) + totalSize += columnBuilders(i).columnStats.sizeInBytes + i += 1 + } + rowCount += 1 + } + + val stats = InternalRow.fromSeq( + columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq) + DefaultCachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build()) + }, stats) + } + + def hasNext: Boolean = rowIterator.hasNext + } + } + } + + override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall(f => + f.dataType match { + // More types can be supported, but this is to match the original implementation that + // only supported primitive types "for ease of review" + case BooleanType | ByteType | ShortType | IntegerType | LongType | + FloatType | DoubleType => true + case _ => false + }) + + override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = + Option(Seq.fill(attributes.length)( + if (!conf.offHeapColumnVectorEnabled) { + classOf[OnHeapColumnVector].getName + } else { + classOf[OffHeapColumnVector].getName + } + )) + + override def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled + val outputSchema = StructType.fromAttributes(selectedAttributes) + val columnIndices = + selectedAttributes.map(a => cacheAttributes.map(o => o.exprId).indexOf(a.exprId)).toArray + + def createAndDecompressColumn(cb: CachedBatch): ColumnarBatch = { + val cachedColumnarBatch = cb.asInstanceOf[DefaultCachedBatch] + val rowCount = cachedColumnarBatch.numRows + val taskContext = Option(TaskContext.get()) + val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) { + OnHeapColumnVector.allocateColumns(rowCount, outputSchema) + } else { + OffHeapColumnVector.allocateColumns(rowCount, outputSchema) + } + val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]]) + columnarBatch.setNumRows(rowCount) + + for (i <- selectedAttributes.indices) { + ColumnAccessor.decompress( + cachedColumnarBatch.buffers(columnIndices(i)), + columnarBatch.column(i).asInstanceOf[WritableColumnVector], + outputSchema.fields(i).dataType, rowCount) + } + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close())) + columnarBatch + } + + input.map(createAndDecompressColumn) + } + + override def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + // Find the ordinals and data types of the requested columns. + val (requestedColumnIndices, requestedColumnDataTypes) = + selectedAttributes.map { a => + cacheAttributes.map(_.exprId).indexOf(a.exprId) -> a.dataType + }.unzip + + val columnTypes = requestedColumnDataTypes.map { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + }.toArray + + input.mapPartitionsInternal { cachedBatchIterator => + val columnarIterator = GenerateColumnAccessor.generate(columnTypes) + columnarIterator.initialize(cachedBatchIterator.asInstanceOf[Iterator[DefaultCachedBatch]], + columnTypes, + requestedColumnIndices.toArray) + columnarIterator + } + } +} +private[sql] case class CachedRDDBuilder( - useCompression: Boolean, - batchSize: Int, + serializer: CachedBatchSerializer, storageLevel: StorageLevel, @transient cachedPlan: SparkPlan, tableName: Option[String]) { @@ -85,54 +241,24 @@ case class CachedRDDBuilder( } private def buildBuffers(): RDD[CachedBatch] = { - val output = cachedPlan.output - val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator => - new Iterator[CachedBatch] { - def next(): CachedBatch = { - val columnBuilders = output.map { attribute => - ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) - }.toArray - - var rowCount = 0 - var totalSize = 0L - while (rowIterator.hasNext && rowCount < batchSize - && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { - val row = rowIterator.next() - - // Added for SPARK-6082. This assertion can be useful for scenarios when something - // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM - // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat - // hard to decipher. - assert( - row.numFields == columnBuilders.length, - s"Row column number mismatch, expected ${output.size} columns, " + - s"but got ${row.numFields}." + - s"\nRow content: $row") - - var i = 0 - totalSize = 0 - while (i < row.numFields) { - columnBuilders(i).appendFrom(row, i) - totalSize += columnBuilders(i).columnStats.sizeInBytes - i += 1 - } - rowCount += 1 - } - - sizeInBytesStats.add(totalSize) - rowCountStats.add(rowCount) - - val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq) - CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, stats) - } - - def hasNext: Boolean = rowIterator.hasNext - } + val cb = if (cachedPlan.supportsColumnar) { + serializer.convertColumnarBatchToCachedBatch( + cachedPlan.executeColumnar(), + cachedPlan.output, + storageLevel, + cachedPlan.conf) + } else { + serializer.convertInternalRowToCachedBatch( + cachedPlan.execute(), + cachedPlan.output, + storageLevel, + cachedPlan.conf) + } + val cached = cb.map { batch => + sizeInBytesStats.add(batch.sizeInBytes) + rowCountStats.add(batch.numRows) + batch }.persist(storageLevel) - cached.setName(cachedName) cached } @@ -140,22 +266,71 @@ case class CachedRDDBuilder( object InMemoryRelation { + private[this] var ser: Option[CachedBatchSerializer] = None + private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized { + if (ser.isEmpty) { + val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) + val serClass = Utils.classForName(serName) + val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer] + ser = Some(instance) + } + ser.get + } + + def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match { + case gen: WholeStageCodegenExec => gen.child match { + case c2r: ColumnarToRowTransition => c2r.child match { + case ia: InputAdapter => ia.child + case _ => plan + } + case _ => plan + } + case c2r: ColumnarToRowTransition => // This matches when whole stage code gen is disabled. + c2r.child + case _ => plan + } + def apply( - useCompression: Boolean, - batchSize: Int, + storageLevel: StorageLevel, + qe: QueryExecution, + tableName: Option[String]): InMemoryRelation = { + val optimizedPlan = qe.optimizedPlan + val serializer = getSerializer(optimizedPlan.conf) + val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) { + convertToColumnarIfPossible(qe.executedPlan) + } else { + qe.executedPlan + } + val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName) + val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) + relation.statsOfPlanToCache = optimizedPlan.stats + relation + } + + /** + * This API is intended only to be used for testing. + */ + def apply( + serializer: CachedBatchSerializer, storageLevel: StorageLevel, child: SparkPlan, tableName: Option[String], optimizedPlan: LogicalPlan): InMemoryRelation = { - val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, storageLevel, child, tableName) + val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName) val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation } - def apply(cacheBuilder: CachedRDDBuilder, optimizedPlan: LogicalPlan): InMemoryRelation = { + def apply(cacheBuilder: CachedRDDBuilder, qe: QueryExecution): InMemoryRelation = { + val optimizedPlan = qe.optimizedPlan + val newBuilder = if (cacheBuilder.serializer.supportsColumnarInput(optimizedPlan.output)) { + cacheBuilder.copy(cachedPlan = convertToColumnarIfPossible(qe.executedPlan)) + } else { + cacheBuilder.copy(cachedPlan = qe.executedPlan) + } val relation = new InMemoryRelation( - cacheBuilder.cachedPlan.output, cacheBuilder, optimizedPlan.outputOrdering) + newBuilder.cachedPlan.output, newBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index f03c2586048bd..e4194562b7a5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,19 +17,15 @@ package org.apache.spark.sql.execution.columnar -import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.columnar.CachedBatch import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized._ -import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} - +import org.apache.spark.sql.vectorized.ColumnarBatch case class InMemoryTableScanExec( attributes: Seq[Attribute], @@ -57,68 +53,29 @@ case class InMemoryTableScanExec( relation = relation.canonicalized.asInstanceOf[InMemoryRelation]) override def vectorTypes: Option[Seq[String]] = - Option(Seq.fill(attributes.length)( - if (!conf.offHeapColumnVectorEnabled) { - classOf[OnHeapColumnVector].getName - } else { - classOf[OffHeapColumnVector].getName - } - )) + relation.cacheBuilder.serializer.vectorTypes(attributes, conf) /** * If true, get data from ColumnVector in ColumnarBatch, which are generally faster. * If false, get data from UnsafeRow build from CachedBatch */ override val supportsColumnar: Boolean = { - // In the initial implementation, for ease of review - // support only primitive data types and # of fields is less than wholeStageMaxNumFields - conf.cacheVectorizedReaderEnabled && relation.schema.fields.forall(f => f.dataType match { - case BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType => true - case _ => false - }) && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) - } - - private val columnIndices = - attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray - - private val relationSchema = relation.schema.toArray - - private lazy val columnarBatchSchema = new StructType(columnIndices.map(i => relationSchema(i))) - - private def createAndDecompressColumn( - cachedColumnarBatch: CachedBatch, - offHeapColumnVectorEnabled: Boolean): ColumnarBatch = { - val rowCount = cachedColumnarBatch.numRows - val taskContext = Option(TaskContext.get()) - val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) { - OnHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema) - } else { - OffHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema) - } - val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]]) - columnarBatch.setNumRows(rowCount) - - for (i <- attributes.indices) { - ColumnAccessor.decompress( - cachedColumnarBatch.buffers(columnIndices(i)), - columnarBatch.column(i).asInstanceOf[WritableColumnVector], - columnarBatchSchema.fields(i).dataType, rowCount) - } - taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close())) - columnarBatch + conf.cacheVectorizedReaderEnabled && + !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) && + relation.cacheBuilder.serializer.supportsColumnarOutput(relation.schema) } private lazy val columnarInputRDD: RDD[ColumnarBatch] = { val numOutputRows = longMetric("numOutputRows") val buffers = filteredCachedBatches() - val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled - buffers - .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled)) - .map { buffer => - numOutputRows += buffer.numRows() - buffer - } + relation.cacheBuilder.serializer.convertCachedBatchToColumnarBatch( + buffers, + relation.output, + attributes, + conf).map { cb => + numOutputRows += cb.numRows() + cb + } } private lazy val inputRDD: RDD[InternalRow] = { @@ -130,35 +87,24 @@ case class InMemoryTableScanExec( val numOutputRows = longMetric("numOutputRows") // Using these variables here to avoid serialization of entire objects (if referenced // directly) within the map Partitions closure. - val relOutput: AttributeSeq = relation.output - - filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator => - // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnDataTypes) = - attributes.map { a => - relOutput.indexOf(a.exprId) -> a.dataType - }.unzip - - // update SQL metrics - val withMetrics = cachedBatchIterator.map { batch => - if (enableAccumulatorsForTest) { - readBatches.add(1) + val relOutput = relation.output + val serializer = relation.cacheBuilder.serializer + + // update SQL metrics + val withMetrics = + filteredCachedBatches().mapPartitionsInternal { iter => + if (enableAccumulatorsForTest && iter.hasNext) { + readPartitions.add(1) + } + iter.map { batch => + if (enableAccumulatorsForTest) { + readBatches.add(1) + } + numOutputRows += batch.numRows + batch } - numOutputRows += batch.numRows - batch - } - - val columnTypes = requestedColumnDataTypes.map { - case udt: UserDefinedType[_] => udt.sqlType - case other => other - }.toArray - val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) - if (enableAccumulatorsForTest && columnarIterator.hasNext) { - readPartitions.add(1) } - columnarIterator - } + serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf) } override def output: Seq[Attribute] = attributes @@ -186,114 +132,6 @@ case class InMemoryTableScanExec( override def outputOrdering: Seq[SortOrder] = relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder]) - // Keeps relation's partition statistics because we don't serialize relation. - private val stats = relation.partitionStatistics - private def statsFor(a: Attribute) = stats.forAttribute(a) - - // Currently, only use statistics from atomic types except binary type only. - private object ExtractableLiteral { - def unapply(expr: Expression): Option[Literal] = expr match { - case lit: Literal => lit.dataType match { - case BinaryType => None - case _: AtomicType => Some(lit) - case _ => None - } - case _ => None - } - } - - // Returned filter predicate should return false iff it is impossible for the input expression - // to evaluate to `true` based on statistics collected about this partition batch. - @transient lazy val buildFilter: PartialFunction[Expression, Expression] = { - case And(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => - (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) - - case Or(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => - buildFilter(lhs) || buildFilter(rhs) - - case EqualTo(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualTo(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l - case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound - - case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l - case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => - l <= statsFor(a).upperBound - - case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound - case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l - - case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - l <= statsFor(a).upperBound - case GreaterThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l - - case IsNull(a: Attribute) => statsFor(a).nullCount > 0 - case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 - - case In(a: AttributeReference, list: Seq[Expression]) - if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => - list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && - l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) - - // This is an example to explain how it works, imagine that the id column stored as follows: - // __________________________________________ - // | Partition ID | lowerBound | upperBound | - // |--------------|------------|------------| - // | p1 | '1' | '9' | - // | p2 | '10' | '19' | - // | p3 | '20' | '29' | - // | p4 | '30' | '39' | - // | p5 | '40' | '49' | - // |______________|____________|____________| - // - // A filter: df.filter($"id".startsWith("2")). - // In this case it substr lowerBound and upperBound: - // ________________________________________________________________________________________ - // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | - // |--------------|-----------------------------------|-----------------------------------| - // | p1 | '1' | '9' | - // | p2 | '1' | '1' | - // | p3 | '2' | '2' | - // | p4 | '3' | '3' | - // | p5 | '4' | '4' | - // |______________|___________________________________|___________________________________| - // - // We can see that we only need to read p1 and p3. - case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound.substr(0, Length(l)) <= l && - l <= statsFor(a).upperBound.substr(0, Length(l)) - } - - lazy val partitionFilters: Seq[Expression] = { - predicates.flatMap { p => - val filter = buildFilter.lift(p) - val boundFilter = - filter.map( - BindReferences.bindReference( - _, - stats.schema, - allowFailures = true)) - - boundFilter.foreach(_ => - filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) - - // If the filter can't be resolved then we are missing required statistics. - boundFilter.filter(_.resolved) - } - } - lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled // Accumulators used for testing purposes @@ -303,37 +141,13 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning private def filteredCachedBatches(): RDD[CachedBatch] = { - // Using these variables here to avoid serialization of entire objects (if referenced directly) - // within the map Partitions closure. - val schema = stats.schema - val schemaIndex = schema.zipWithIndex val buffers = relation.cacheBuilder.cachedColumnBuffers - buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = Predicate.create( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(index) - - // Do partition batch pruning if enabled - if (inMemoryPartitionPruningEnabled) { - cachedBatchIterator.filter { cachedBatch => - if (!partitionFilter.eval(cachedBatch.stats)) { - logDebug { - val statsString = schemaIndex.map { case (a, i) => - val value = cachedBatch.stats.get(i, a.dataType) - s"${a.name}: $value" - }.mkString(", ") - s"Skipping partition based on stats $statsString" - } - false - } else { - true - } - } - } else { - cachedBatchIterator - } + if (inMemoryPartitionPruningEnabled) { + val filterFunc = relation.cacheBuilder.serializer.buildFilter(predicates, relation.output) + buffers.mapPartitionsWithIndexInternal(filterFunc) + } else { + buffers } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala new file mode 100644 index 0000000000000..72eba7f6e6907 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.columnar + +import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.storage.StorageLevel + +case class SingleIntCachedBatch(data: Array[Int]) extends CachedBatch { + override def numRows: Int = data.length + override def sizeInBytes: Long = 4 * data.length +} + +/** + * Very simple serializer that only supports a single int column, but does support columnar. + */ +class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true + + override def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + if (schema.length != 1 || schema.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing $schema") + } + input.map { cb => + val column = cb.column(0) + val data = column.getInts(0, cb.numRows()) + SingleIntCachedBatch(data) + } + } + + override def supportsColumnarOutput(schema: StructType): Boolean = true + override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = + Some(attributes.map(_ => classOf[OnHeapColumnVector].getName)) + + override def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + if (selectedAttributes.isEmpty) { + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + new ColumnarBatch(new Array[ColumnVector](0), single.numRows) + } + } else { + if (selectedAttributes.length > 1 || + selectedAttributes.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing") + } + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + val cv = OnHeapColumnVector.allocateColumns(single.numRows, selectedAttributes.toStructType) + val data = single.data + cv(0).putInts(0, data.length, data, 0) + new ColumnarBatch(cv.toArray, single.numRows) + } + } + } + + override def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + def ret(index: Int, cb: Iterator[CachedBatch]): Iterator[CachedBatch] = cb + ret + } +} + +class CachedBatchSerializerSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + override protected def sparkConf: SparkConf = { + super.sparkConf.set( + StaticSQLConf.SPARK_CACHE_SERIALIZER.key, + classOf[TestSingleIntColumnarCachedBatchSerializer].getName) + } + + test("Columnar Cache Plugin") { + withTempPath { workDir => + val workDirPath = workDir.getAbsolutePath + val input = Seq(100, 200, 300).toDF("count") + input.write.parquet(workDirPath) + val data = spark.read.parquet(workDirPath) + data.cache() + assert(data.count() == 3) + checkAnswer(data, Row(100) :: Row(200) :: Row(300) :: Nil) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 18f29f7b90ad5..b8f73f4563ef1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -20,18 +20,32 @@ package org.apache.spark.sql.execution.columnar import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, WholeStageCodegenExec} +import org.apache.spark.sql.columnar.CachedBatch +import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ -import org.apache.spark.util.Utils + +class TestCachedBatchSerializer( + useCompression: Boolean, + batchSize: Int) extends DefaultCachedBatchSerializer { + + override def convertInternalRowToCachedBatch(input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + convertForCacheInternal(input, schema, batchSize, useCompression) + } +} class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -42,12 +56,12 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { data.createOrReplaceTempView(s"testData$dataType") val storageLevel = MEMORY_ONLY val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None, - data.logicalPlan) + val inMemoryRelation = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + storageLevel, plan, None, data.logicalPlan) assert(inMemoryRelation.cacheBuilder.cachedColumnBuffers.getStorageLevel == storageLevel) inMemoryRelation.cacheBuilder.cachedColumnBuffers.collect().head match { - case _: CachedBatch => + case _: DefaultCachedBatch => case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") } checkAnswer(inMemoryRelation, data.collect().toSeq) @@ -119,8 +133,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("simple columnar query") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - testData.logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, testData.logicalPlan) checkAnswer(scan, testData.collect().toSeq) } @@ -140,8 +154,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("projection") { val logicalPlan = testData.select('value, 'key).logicalPlan val plan = spark.sessionState.executePlan(logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, logicalPlan) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -157,8 +171,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - testData.logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, testData.logicalPlan) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) @@ -336,7 +350,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-17549: cached table size should be correctly calculated") { val data = spark.sparkContext.parallelize(1 to 10, 5).toDF() val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5), + MEMORY_ONLY, plan, None, data.logicalPlan) // Materialize the data. val expectedAnswer = data.collect() @@ -349,7 +364,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("cached row count should be calculated") { val data = spark.range(6).toDF val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5), + MEMORY_ONLY, plan, None, data.logicalPlan) // Materialize the data. val expectedAnswer = data.collect() @@ -474,12 +490,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-22249: buildFilter should not throw exception when In contains an empty list") { val attribute = AttributeReference("a", IntegerType)() - val localTableScanExec = LocalTableScanExec(Seq(attribute), Nil) - val testRelation = InMemoryRelation(false, 1, MEMORY_ONLY, localTableScanExec, None, - LocalRelation(Seq(attribute), Nil)) - val tableScanExec = InMemoryTableScanExec(Seq(attribute), - Seq(In(attribute, Nil)), testRelation) - assert(tableScanExec.partitionFilters.isEmpty) + val testSerializer = new TestCachedBatchSerializer(false, 1) + testSerializer.buildFilter(Seq(In(attribute, Nil)), Seq(attribute)) } testWithWholeStageCodegenOnAndOff("SPARK-22348: table cache " + From fda397d9c8ff52f9e0785da8f470c37ed40af616 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 3 Aug 2020 03:20:17 +0000 Subject: [PATCH 351/384] [SPARK-32510][SQL] Check duplicate nested columns in read from JDBC datasource ### What changes were proposed in this pull request? Check that there are not duplicate column names on the same level (top level or nested levels) in reading from JDBC datasource. If such duplicate columns exist, throw the exception: ``` org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the customSchema option value: ``` The check takes into account the SQL config `spark.sql.caseSensitive` (`false` by default). ### Why are the changes needed? To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error: ```Scala org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the customSchema option value: `camelcase` ``` Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758, and duplicates in nested structures by https://github.com/apache/spark/pull/29234. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Added new test suite `JdbcNestedDataSourceSuite`. Closes #29317 from MaxGekk/jdbc-dup-nested-columns. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../datasources/jdbc/JdbcUtils.scala | 6 ++- .../spark/sql/NestedDataSourceSuite.scala | 21 +++++--- .../sql/jdbc/JDBCNestedDataSourceSuite.scala | 51 +++++++++++++++++++ 3 files changed, 68 insertions(+), 10 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 3c6649b26ecd2..816f5f45860c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -819,8 +819,10 @@ object JdbcUtils extends Logging { if (null != customSchema && customSchema.nonEmpty) { val userSchema = CatalystSqlParser.parseTableSchema(customSchema) - SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "in the customSchema option value", nameEquality) + SchemaUtils.checkSchemaColumnNameDuplication( + userSchema, + "in the customSchema option value", + nameEquality) // This is resolved by names, use the custom filed dataType to replace the default dataType. val newSchema = tableSchema.map { col => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala index 152d59b7b1908..78b314272aa07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala @@ -24,6 +24,16 @@ import org.apache.spark.sql.types.{LongType, StructType} // Datasource tests for nested schemas trait NestedDataSourceSuiteBase extends QueryTest with SharedSparkSession { protected val nestedDataSources: Seq[String] = Seq("orc", "parquet", "json") + protected def readOptions(schema: StructType): Map[String, String] = Map.empty + protected def save(selectExpr: Seq[String], format: String, path: String): Unit = { + spark + .range(1L) + .selectExpr(selectExpr: _*) + .write.mode("overwrite") + .format(format) + .save(path) + } + protected val colType: String = "in the data schema" test("SPARK-32431: consistent error for nested and top-level duplicate columns") { Seq( @@ -44,22 +54,17 @@ trait NestedDataSourceSuiteBase extends QueryTest with SharedSparkSession { withClue(s"format = $format select = ${selectExpr.mkString(",")}") { withTempPath { dir => val path = dir.getCanonicalPath - spark - .range(1L) - .selectExpr(selectExpr: _*) - .write.mode("overwrite") - .format(format) - .save(path) + save(selectExpr, format, path) val e = intercept[AnalysisException] { spark .read + .options(readOptions(caseInsensitiveSchema)) .schema(caseInsensitiveSchema) .format(format) .load(path) .show } - assert(e.getMessage.contains( - "Found duplicate column(s) in the data schema: `camelcase`")) + assert(e.getMessage.contains(s"Found duplicate column(s) $colType: `camelcase`")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala new file mode 100644 index 0000000000000..46bdb1918147c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import org.apache.spark.sql.NestedDataSourceSuiteBase +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils + +class JDBCNestedDataSourceSuite extends NestedDataSourceSuiteBase { + override val nestedDataSources: Seq[String] = Seq("jdbc") + private val tempDir = Utils.createTempDir() + private val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" + override val colType: String = "in the customSchema option value" + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDir) + super.afterAll() + } + + override def readOptions(schema: StructType): Map[String, String] = { + Map("url" -> url, "dbtable" -> "t1", "customSchema" -> schema.toDDL) + } + + override def save(selectExpr: Seq[String], format: String, path: String): Unit = { + // We ignore `selectExpr` because: + // 1. H2 doesn't support nested columns + // 2. JDBC datasource checks duplicates before comparing of user's schema with + // actual schema of `t1`. + spark + .range(1L) + .write.mode("overwrite") + .options(Map("url" -> url, "dbtable" -> "t1")) + .format(format) + .save() + } +} From 7a09e71198a094250f04e0f82f0c7c9860169540 Mon Sep 17 00:00:00 2001 From: Prakhar Jain Date: Mon, 3 Aug 2020 03:26:03 +0000 Subject: [PATCH 352/384] [SPARK-32509][SQL] Ignore unused DPP True Filter in Canonicalization ### What changes were proposed in this pull request? This PR fixes issues relate to Canonicalization of FileSourceScanExec when it contains unused DPP Filter. ### Why are the changes needed? As part of PlanDynamicPruningFilter rule, the unused DPP Filter are simply replaced by `DynamicPruningExpression(TrueLiteral)` so that they can be avoided. But these unnecessary`DynamicPruningExpression(TrueLiteral)` partition filter inside the FileSourceScanExec affects the canonicalization of the node and so in many cases, this can prevent ReuseExchange from happening. This PR fixes this issue by ignoring the unused DPP filter in the `def doCanonicalize` method. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added UT. Closes #29318 from prakharjain09/SPARK-32509_df_reuse. Authored-by: Prakhar Jain Signed-off-by: Wenchen Fan --- .../sql/execution/DataSourceScanExec.scala | 10 +++++++- .../sql/DynamicPartitionPruningSuite.scala | 24 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 78808ff21394c..bef9f4b46c628 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -608,12 +608,20 @@ case class FileSourceScanExec( new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } + // Filters unused DynamicPruningExpression expressions - one which has been replaced + // with DynamicPruningExpression(Literal.TrueLiteral) during Physical Planning + private def filterUnusedDynamicPruningExpressions( + predicates: Seq[Expression]): Seq[Expression] = { + predicates.filterNot(_ == DynamicPruningExpression(Literal.TrueLiteral)) + } + override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, output.map(QueryPlan.normalizeExpressions(_, output)), requiredSchema, - QueryPlan.normalizePredicates(partitionFilters, output), + QueryPlan.normalizePredicates( + filterUnusedDynamicPruningExpressions(partitionFilters), output), optionalBucketSet, optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, output), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 0b754e9e3ec0b..8edfb91d15fff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1240,6 +1240,30 @@ abstract class DynamicPartitionPruningSuiteBase } } + test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + + "canonicalization and exchange reuse") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = sql( + """ WITH view1 as ( + | SELECT f.store_id FROM fact_stats f WHERE f.units_sold = 70 + | ) + | + | SELECT * FROM view1 v1 join view1 v2 WHERE v1.store_id = v2.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + val reuseExchangeNodes = df.queryExecution.executedPlan.collect { + case se: ReusedExchangeExec => se + } + assert(reuseExchangeNodes.size == 1, "Expected plan to contain 1 ReusedExchangeExec " + + s"nodes. Found ${reuseExchangeNodes.size}") + + checkAnswer(df, Row(15, 15) :: Nil) + } + } + } + test("Plan broadcast pruning only when the broadcast can be reused") { Given("dynamic pruning filter on the build side") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { From 42f9ee4c7deb83a915fad070af5eedab56399382 Mon Sep 17 00:00:00 2001 From: beliefer Date: Mon, 3 Aug 2020 06:03:55 +0000 Subject: [PATCH 353/384] [SPARK-24884][SQL] Support regexp function regexp_extract_all ### What changes were proposed in this pull request? `regexp_extract_all` is a very useful function expanded the capabilities of `regexp_extract`. There are some description of this function. ``` SELECT regexp_extract('1a 2b 14m', '\d+', 0); -- 1 SELECT regexp_extract_all('1a 2b 14m', '\d+', 0); -- [1, 2, 14] SELECT regexp_extract('1a 2b 14m', '(\d+)([a-z]+)', 2); -- 'a' SELECT regexp_extract_all('1a 2b 14m', '(\d+)([a-z]+)', 2); -- ['a', 'b', 'm'] ``` There are some mainstream database support the syntax. **Presto:** https://prestodb.io/docs/current/functions/regexp.html **Pig:** https://pig.apache.org/docs/latest/api/org/apache/pig/builtin/REGEX_EXTRACT_ALL.html Note: This PR pick up the work of https://github.com/apache/spark/pull/21985 ### Why are the changes needed? `regexp_extract_all` is a very useful function and make work easier. ### Does this PR introduce any user-facing change? No ### How was this patch tested? New UT Closes #27507 from beliefer/support-regexp_extract_all. Lead-authored-by: beliefer Co-authored-by: gengjiaan Co-authored-by: Jiaan Geng Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/regexpExpressions.scala | 180 +++++++++++++++--- .../expressions/RegexpExpressionsSuite.scala | 50 +++++ .../org/apache/spark/sql/functions.scala | 15 ++ .../sql-functions/sql-expression-schema.md | 1 + .../sql-tests/inputs/regexp-functions.sql | 22 +++ .../results/regexp-functions.sql.out | 171 ++++++++++++++++- .../spark/sql/StringFunctionsSuite.scala | 27 +++ 8 files changed, 440 insertions(+), 27 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 3989df5d29467..7e73667e4b85f 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 @@ -354,6 +354,7 @@ object FunctionRegistry { expression[StringLocate]("position", true), expression[FormatString]("printf", true), expression[RegExpExtract]("regexp_extract"), + expression[RegExpExtractAll]("regexp_extract_all"), expression[RegExpReplace]("regexp_replace"), expression[StringRepeat]("repeat"), expression[StringReplace]("replace"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index 1af1636e1df75..8eb7f463e049c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Locale -import java.util.regex.{MatchResult, Pattern} +import java.util.regex.{Matcher, MatchResult, Pattern} + +import scala.collection.mutable.ArrayBuffer import org.apache.commons.text.StringEscapeUtils @@ -410,7 +412,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio } } -object RegExpExtract { +object RegExpExtractBase { def checkGroupIndex(groupCount: Int, groupIndex: Int): Unit = { if (groupIndex < 0) { throw new IllegalArgumentException("The specified group index cannot be less than zero") @@ -421,20 +423,58 @@ object RegExpExtract { } } +abstract class RegExpExtractBase + extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant { + def subject: Expression + def regexp: Expression + def idx: Expression + + // last regex in string, we will update the pattern iff regexp value changed. + @transient private var lastRegex: UTF8String = _ + // last regex pattern, we cache it for performance concern + @transient private var pattern: Pattern = _ + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) + override def children: Seq[Expression] = subject :: regexp :: idx :: Nil + + protected def getLastMatcher(s: Any, p: Any): Matcher = { + if (p != lastRegex) { + // regex value changed + lastRegex = p.asInstanceOf[UTF8String].clone() + pattern = Pattern.compile(lastRegex.toString) + } + pattern.matcher(s.toString) + } +} + /** * Extract a specific(idx) group identified by a Java regex. * * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. */ @ExpressionDescription( - usage = "_FUNC_(str, regexp[, idx]) - Extracts a group that matches `regexp`.", + usage = """ + _FUNC_(str, regexp[, idx]) - Extract the first string in the `str` that match the `regexp` + expression and corresponding to the regex group index. + """, arguments = """ Arguments: * str - a string expression. - * regexp - a string representing a regular expression. - The regex string should be a Java regular expression. - * idx - an integer expression that representing the group index. The group index should be - non-negative. If `idx` is not specified, the default group index value is 1. + * regexp - a string representing a regular expression. The regex string should be a + Java regular expression. + + Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL + parser. For example, to match "\abc", a regular expression for `regexp` can be + "^\\abc$". + + There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to + fallback to the Spark 1.6 behavior regarding string literal parsing. For example, + if the config is enabled, the `regexp` that can match "\abc" is "^\abc$". + * idx - an integer expression that representing the group index. The regex maybe contains + multiple groups. `idx` indicates which regex group to extract. The group index should + be non-negative. The minimum value of `idx` is 0, which means matching the entire + regular expression. If `idx` is not specified, the default group index value is 1. The + `idx` parameter is the Java regex Matcher group() method index. """, examples = """ Examples: @@ -443,27 +483,17 @@ object RegExpExtract { """, since = "1.5.0") case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expression) - extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant { + extends RegExpExtractBase { def this(s: Expression, r: Expression) = this(s, r, Literal(1)) - // last regex in string, we will update the pattern iff regexp value changed. - @transient private var lastRegex: UTF8String = _ - // last regex pattern, we cache it for performance concern - @transient private var pattern: Pattern = _ - override def nullSafeEval(s: Any, p: Any, r: Any): Any = { - if (!p.equals(lastRegex)) { - // regex value changed - lastRegex = p.asInstanceOf[UTF8String].clone() - pattern = Pattern.compile(lastRegex.toString) - } - val m = pattern.matcher(s.toString) + val m = getLastMatcher(s, p) if (m.find) { val mr: MatchResult = m.toMatchResult val index = r.asInstanceOf[Int] - RegExpExtract.checkGroupIndex(mr.groupCount, index) + RegExpExtractBase.checkGroupIndex(mr.groupCount, index) val group = mr.group(index) - if (group == null) { // Pattern matched, but not optional group + if (group == null) { // Pattern matched, but it's an optional group UTF8String.EMPTY_UTF8 } else { UTF8String.fromString(group) @@ -474,13 +504,11 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio } override def dataType: DataType = StringType - override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) - override def children: Seq[Expression] = subject :: regexp :: idx :: Nil override def prettyName: String = "regexp_extract" override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val classNamePattern = classOf[Pattern].getCanonicalName - val classNameRegExpExtract = classOf[RegExpExtract].getCanonicalName + val classNameRegExpExtractBase = classOf[RegExpExtractBase].getCanonicalName val matcher = ctx.freshName("matcher") val matchResult = ctx.freshName("matchResult") @@ -504,7 +532,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio $termPattern.matcher($subject.toString()); if ($matcher.find()) { java.util.regex.MatchResult $matchResult = $matcher.toMatchResult(); - $classNameRegExpExtract.checkGroupIndex($matchResult.groupCount(), $idx); + $classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx); if ($matchResult.group($idx) == null) { ${ev.value} = UTF8String.EMPTY_UTF8; } else { @@ -518,3 +546,105 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio }) } } + +/** + * Extract all specific(idx) groups identified by a Java regex. + * + * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. + */ +@ExpressionDescription( + usage = """ + _FUNC_(str, regexp[, idx]) - Extract all strings in the `str` that match the `regexp` + expression and corresponding to the regex group index. + """, + arguments = """ + Arguments: + * str - a string expression. + * regexp - a string representing a regular expression. The regex string should be a + Java regular expression. + + Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL + parser. For example, to match "\abc", a regular expression for `regexp` can be + "^\\abc$". + + There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to + fallback to the Spark 1.6 behavior regarding string literal parsing. For example, + if the config is enabled, the `regexp` that can match "\abc" is "^\abc$". + * idx - an integer expression that representing the group index. The regex may contains + multiple groups. `idx` indicates which regex group to extract. The group index should + be non-negative. The minimum value of `idx` is 0, which means matching the entire + regular expression. If `idx` is not specified, the default group index value is 1. The + `idx` parameter is the Java regex Matcher group() method index. + """, + examples = """ + Examples: + > SELECT _FUNC_('100-200, 300-400', '(\\d+)-(\\d+)', 1); + ["100","300"] + """, + since = "3.1.0") +case class RegExpExtractAll(subject: Expression, regexp: Expression, idx: Expression) + extends RegExpExtractBase { + def this(s: Expression, r: Expression) = this(s, r, Literal(1)) + + override def nullSafeEval(s: Any, p: Any, r: Any): Any = { + val m = getLastMatcher(s, p) + val matchResults = new ArrayBuffer[UTF8String]() + while(m.find) { + val mr: MatchResult = m.toMatchResult + val index = r.asInstanceOf[Int] + RegExpExtractBase.checkGroupIndex(mr.groupCount, index) + val group = mr.group(index) + if (group == null) { // Pattern matched, but it's an optional group + matchResults += UTF8String.EMPTY_UTF8 + } else { + matchResults += UTF8String.fromString(group) + } + } + + new GenericArrayData(matchResults.toArray.asInstanceOf[Array[Any]]) + } + + override def dataType: DataType = ArrayType(StringType) + override def prettyName: String = "regexp_extract_all" + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val classNamePattern = classOf[Pattern].getCanonicalName + val classNameRegExpExtractBase = classOf[RegExpExtractBase].getCanonicalName + val arrayClass = classOf[GenericArrayData].getName + val matcher = ctx.freshName("matcher") + val matchResult = ctx.freshName("matchResult") + val matchResults = ctx.freshName("matchResults") + + val termLastRegex = ctx.addMutableState("UTF8String", "lastRegex") + val termPattern = ctx.addMutableState(classNamePattern, "pattern") + + val setEvNotNull = if (nullable) { + s"${ev.isNull} = false;" + } else { + "" + } + nullSafeCodeGen(ctx, ev, (subject, regexp, idx) => { + s""" + | if (!$regexp.equals($termLastRegex)) { + | // regex value changed + | $termLastRegex = $regexp.clone(); + | $termPattern = $classNamePattern.compile($termLastRegex.toString()); + | } + | java.util.regex.Matcher $matcher = $termPattern.matcher($subject.toString()); + | java.util.ArrayList $matchResults = new java.util.ArrayList(); + | while ($matcher.find()) { + | java.util.regex.MatchResult $matchResult = $matcher.toMatchResult(); + | $classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx); + | if ($matchResult.group($idx) == null) { + | $matchResults.add(UTF8String.EMPTY_UTF8); + | } else { + | $matchResults.add(UTF8String.fromString($matchResult.group($idx))); + | } + | } + | ${ev.value} = + | new $arrayClass($matchResults.toArray(new UTF8String[$matchResults.size()])); + | $setEvNotNull + """ + }) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index cab81f85fda06..205dc10efc8a8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -322,6 +322,56 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { RegExpExtract(Literal("\"quote"), Literal("\"quote"), Literal(1)) :: Nil) } + test("RegexExtractAll") { + val row1 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 0) + val row2 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 1) + val row3 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 2) + val row4 = create_row("100-200,300-400,500-600", "(\\d+).*", 1) + val row5 = create_row("100-200,300-400,500-600", "([a-z])", 1) + val row6 = create_row(null, "([a-z])", 1) + val row7 = create_row("100-200,300-400,500-600", null, 1) + val row8 = create_row("100-200,300-400,500-600", "([a-z])", null) + + val s = 's.string.at(0) + val p = 'p.string.at(1) + val r = 'r.int.at(2) + + val expr = RegExpExtractAll(s, p, r) + checkEvaluation(expr, Seq("100-200", "300-400", "500-600"), row1) + checkEvaluation(expr, Seq("100", "300", "500"), row2) + checkEvaluation(expr, Seq("200", "400", "600"), row3) + checkEvaluation(expr, Seq("100"), row4) + checkEvaluation(expr, Seq(), row5) + checkEvaluation(expr, null, row6) + checkEvaluation(expr, null, row7) + checkEvaluation(expr, null, row8) + + val expr1 = new RegExpExtractAll(s, p) + checkEvaluation(expr1, Seq("100", "300", "500"), row2) + + val nonNullExpr = RegExpExtractAll(Literal("100-200,300-400,500-600"), + Literal("(\\d+)-(\\d+)"), Literal(1)) + checkEvaluation(nonNullExpr, Seq("100", "300", "500"), row2) + + // invalid group index + val row9 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 3) + val row10 = create_row("100-200,300-400,500-600", "(\\d+).*", 2) + val row11 = create_row("100-200,300-400,500-600", "\\d+", 1) + val row12 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", -1) + val row13 = create_row("100-200,300-400,500-600", "\\d+", -1) + + checkExceptionInExpression[IllegalArgumentException]( + expr, row9, "Regex group count is 2, but the specified group index is 3") + checkExceptionInExpression[IllegalArgumentException]( + expr, row10, "Regex group count is 1, but the specified group index is 2") + checkExceptionInExpression[IllegalArgumentException]( + expr, row11, "Regex group count is 0, but the specified group index is 1") + checkExceptionInExpression[IllegalArgumentException]( + expr, row12, "The specified group index cannot be less than zero") + checkExceptionInExpression[IllegalArgumentException]( + expr, row13, "The specified group index cannot be less than zero") + } + test("SPLIT") { val s1 = 'a.string.at(0) val s2 = 'b.string.at(1) 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 fe0057c3d588b..653fe5bf7c9b3 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 @@ -2478,6 +2478,8 @@ object functions { /** * Extract a specific group matched by a Java regex, from the specified string column. * If the regex did not match, or the specified group did not match, an empty string is returned. + * if the specified group index exceeds the group count of regex, an IllegalArgumentException + * will be thrown. * * @group string_funcs * @since 1.5.0 @@ -2486,6 +2488,19 @@ object functions { RegExpExtract(e.expr, lit(exp).expr, lit(groupIdx).expr) } + /** + * Extract all specific groups matched by a Java regex, from the specified string column. + * If the regex did not match, or the specified group did not match, return an empty array. + * if the specified group index exceeds the group count of regex, an IllegalArgumentException + * will be thrown. + * + * @group string_funcs + * @since 3.1.0 + */ + def regexp_extract_all(e: Column, exp: String, groupIdx: Int): Column = withExpr { + RegExpExtractAll(e.expr, lit(exp).expr, lit(groupIdx).expr) + } + /** * Replace all substrings of the specified string value that match regexp with rep. * 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 edf2ede9e5a44..a212d8ce40642 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 @@ -214,6 +214,7 @@ | org.apache.spark.sql.catalyst.expressions.Randn | randn | SELECT randn() | struct | | org.apache.spark.sql.catalyst.expressions.Rank | rank | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.RegExpExtract | regexp_extract | SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1) | struct | +| org.apache.spark.sql.catalyst.expressions.RegExpExtractAll | regexp_extract_all | SELECT regexp_extract_all('100-200, 300-400', '(\\d+)-(\\d+)', 1) | struct> | | org.apache.spark.sql.catalyst.expressions.RegExpReplace | regexp_replace | SELECT regexp_replace('100-200', '(\\d+)', 'num') | struct | | org.apache.spark.sql.catalyst.expressions.Remainder | % | 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.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)> | diff --git a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql index 8a531be30d896..7128dee0a00d7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql @@ -4,8 +4,30 @@ SELECT regexp_extract('1a 2b 14m', '\\d+', 0); SELECT regexp_extract('1a 2b 14m', '\\d+', 1); SELECT regexp_extract('1a 2b 14m', '\\d+', 2); SELECT regexp_extract('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract('a b m', '(\\d+)?', 1); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)'); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1); + +-- regexp_extract_all +SELECT regexp_extract_all('1a 2b 14m', '\\d+'); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)'); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1); diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out index 64aa6053d8d70..2eef926f63e37 100644 --- a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 10 +-- Number of queries: 30 -- !query @@ -46,6 +46,22 @@ java.lang.IllegalArgumentException The specified group index cannot be less than zero +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output + + + -- !query SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)') -- !query schema @@ -78,10 +94,161 @@ struct a +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 2, but the specified group index is 3 + + -- !query SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1) -- !query schema struct<> -- !query output java.lang.IllegalArgumentException -The specified group index cannot be less than zero \ No newline at end of file +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 1 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 1 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 2 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct> +-- !query output +["1","","","2","","","14","",""] + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct> +-- !query output +["","","2","","","14","",""] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query schema +struct> +-- !query output +["1a","2b","14m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query schema +struct> +-- !query output +["a","b","m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 2, but the specified group index is 3 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["","2","14"] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index f904b53fe47eb..8d5166b5398cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -154,8 +154,25 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { Row("300", "100") :: Row("400", "100") :: Row("400-400", "100") :: Nil) } + test("string regex_extract_all") { + val df = Seq( + ("100-200,300-400", "(\\d+)-(\\d+)"), + ("101-201,301-401", "(\\d+)-(\\d+)"), + ("102-202,302-402", "(\\d+)")).toDF("a", "b") + + checkAnswer( + df.select( + regexp_extract_all($"a", "(\\d+)-(\\d+)", 1), + regexp_extract_all($"a", "(\\d+)-(\\d+)", 2)), + Row(Seq("100", "300"), Seq("200", "400")) :: + Row(Seq("101", "301"), Seq("201", "401")) :: + Row(Seq("102", "302"), Seq("202", "402")) :: Nil) + } + test("non-matching optional group") { val df = Seq(Tuple1("aaaac")).toDF("s") + + // regexp_extract checkAnswer( df.select(regexp_extract($"s", "(foo)", 1)), Row("") @@ -164,6 +181,16 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { df.select(regexp_extract($"s", "(a+)(b)?(c)", 2)), Row("") ) + + // regexp_extract_all + checkAnswer( + df.select(regexp_extract_all($"s", "(foo)", 1)), + Row(Seq()) + ) + checkAnswer( + df.select(regexp_extract_all($"s", "(a+)(b)?(c)", 2)), + Row(Seq("")) + ) } test("string ascii function") { From 3deb59d5c29fada8f08b0068fbaf3ee706cea312 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 3 Aug 2020 12:48:22 +0000 Subject: [PATCH 354/384] [SPARK-31709][SQL] Proper base path for database/table location when it is a relative path ### What changes were proposed in this pull request? Currently, the user home directory is used as the base path for the database and table locations when their locationa are specified with a relative paths, e.g. ```sql > set spark.sql.warehouse.dir; spark.sql.warehouse.dir file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/spark-warehouse/ spark-sql> create database loctest location 'loctestdbdir'; spark-sql> desc database loctest; Database Name loctest Comment Location file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/loctestdbdir Owner kentyao spark-sql> create table loctest(id int) location 'loctestdbdir'; spark-sql> desc formatted loctest; id int NULL # Detailed Table Information Database default Table loctest Owner kentyao Created Time Thu May 14 16:29:05 CST 2020 Last Access UNKNOWN Created By Spark 3.1.0-SNAPSHOT Type EXTERNAL Provider parquet Location file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/loctestdbdir Serde Library org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe InputFormat org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat OutputFormat org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat ``` The user home is not always warehouse-related, unchangeable in runtime, and shared both by database and table as the parent directory. Meanwhile, we use the table path as the parent directory for relative partition locations. The config `spark.sql.warehouse.dir` represents `the default location for managed databases and tables`. For databases, the case above seems not to follow its semantics, because it should use ` `spark.sql.warehouse.dir` as the base path instead. For tables, it seems to be right but here I suggest enriching the meaning that lets it also be the for external tables with relative paths for locations. With changes in this PR, The location of a database will be `warehouseDir/dbpath` when `dbpath` is relative. The location of a table will be `dbpath/tblpath` when `tblpath` is relative. ### Why are the changes needed? bugfix and improvement Firstly, the databases with relative locations should be created under the default location specified by `spark.sql.warehouse.dir`. Secondly, the external tables with relative paths may also follow this behavior for consistency. At last, the behavior for database, tables and partitions with relative paths to choose base paths should be the same. ### Does this PR introduce _any_ user-facing change? Yes, this PR changes the `createDatabase`, `alterDatabase`, `createTable` and `alterTable` APIs and related DDLs. If the LOCATION clause is followed by a relative path, the root path will be `spark.sql.warehouse.dir` for databases, and `spark.sql.warehouse.dir` / `dbPath` for tables. e.g. #### after ```sql spark-sql> desc database loctest; Database Name loctest Comment Location file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest Owner kentyao spark-sql> use loctest; spark-sql> create table loctest(id int) location 'loctest'; 20/05/14 18:18:02 WARN InMemoryFileIndex: The directory file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/loctest was not found. Was it deleted very recently? 20/05/14 18:18:02 WARN SessionState: METASTORE_FILTER_HOOK will be ignored, since hive.security.authorization.manager is set to instance of HiveAuthorizerFactory. 20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.internal.ss.authz.settings.applied.marker does not exist 20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist 20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist spark-sql> desc formatted loctest; id int NULL # Detailed Table Information Database loctest Table loctest Owner kentyao Created Time Thu May 14 18:18:03 CST 2020 Last Access UNKNOWN Created By Spark 3.1.0-SNAPSHOT Type EXTERNAL Provider parquet Location file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest/loctest Serde Library org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe InputFormat org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat OutputFormat org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat spark-sql> alter table loctest set location 'loctest2' > ; spark-sql> desc formatted loctest; id int NULL # Detailed Table Information Database loctest Table loctest Owner kentyao Created Time Thu May 14 18:18:03 CST 2020 Last Access UNKNOWN Created By Spark 3.1.0-SNAPSHOT Type EXTERNAL Provider parquet Location file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest/loctest2 Serde Library org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe InputFormat org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat OutputFormat org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat ``` ### How was this patch tested? Add unit tests. Closes #28527 from yaooqinn/SPARK-31709. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/SessionCatalog.scala | 32 ++++-- .../datasources/v2/V2SessionCatalog.scala | 11 ++- .../v2/V2SessionCatalogSuite.scala | 98 +++++++++++++++++-- 3 files changed, 123 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index ec0c34d4c7960..6fba3156c3919 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -219,12 +219,20 @@ class SessionCatalog( "you cannot create a database with this name.") } validateName(dbName) - val qualifiedPath = makeQualifiedPath(dbDefinition.locationUri) externalCatalog.createDatabase( - dbDefinition.copy(name = dbName, locationUri = qualifiedPath), + dbDefinition.copy(name = dbName, locationUri = makeQualifiedDBPath(dbDefinition.locationUri)), ignoreIfExists) } + private def makeQualifiedDBPath(locationUri: URI): URI = { + if (locationUri.isAbsolute) { + locationUri + } else { + val fullPath = new Path(conf.warehousePath, CatalogUtils.URIToString(locationUri)) + makeQualifiedPath(fullPath.toUri) + } + } + def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { val dbName = formatDatabaseName(db) if (dbName == DEFAULT_DATABASE) { @@ -241,7 +249,8 @@ class SessionCatalog( def alterDatabase(dbDefinition: CatalogDatabase): Unit = { val dbName = formatDatabaseName(dbDefinition.name) requireDbExists(dbName) - externalCatalog.alterDatabase(dbDefinition.copy(name = dbName)) + externalCatalog.alterDatabase(dbDefinition.copy( + name = dbName, locationUri = makeQualifiedDBPath(dbDefinition.locationUri))) } def getDatabaseMetadata(db: String): CatalogDatabase = { @@ -283,8 +292,7 @@ class SessionCatalog( * by users. */ def getDefaultDBPath(db: String): URI = { - val database = formatDatabaseName(db) - new Path(new Path(conf.warehousePath), database + ".db").toUri + CatalogUtils.stringToURI(formatDatabaseName(db) + ".db") } // ---------------------------------------------------------------------------- @@ -317,7 +325,7 @@ class SessionCatalog( && !tableDefinition.storage.locationUri.get.isAbsolute) { // make the location of the table qualified. val qualifiedTableLocation = - makeQualifiedPath(tableDefinition.storage.locationUri.get) + makeQualifiedTablePath(tableDefinition.storage.locationUri.get, db) tableDefinition.copy( storage = tableDefinition.storage.copy(locationUri = Some(qualifiedTableLocation)), identifier = tableIdentifier) @@ -350,6 +358,16 @@ class SessionCatalog( } } + private def makeQualifiedTablePath(locationUri: URI, database: String): URI = { + if (locationUri.isAbsolute) { + locationUri + } else { + val dbName = formatDatabaseName(database) + val dbLocation = makeQualifiedDBPath(getDatabaseMetadata(dbName).locationUri) + new Path(new Path(dbLocation), CatalogUtils.URIToString(locationUri)).toUri + } + } + /** * Alter the metadata of an existing metastore table identified by `tableDefinition`. * @@ -369,7 +387,7 @@ class SessionCatalog( && !tableDefinition.storage.locationUri.get.isAbsolute) { // make the location of the table qualified. val qualifiedTableLocation = - makeQualifiedPath(tableDefinition.storage.locationUri.get) + makeQualifiedTablePath(tableDefinition.storage.locationUri.get, db) tableDefinition.copy( storage = tableDefinition.storage.copy(locationUri = Some(qualifiedTableLocation)), identifier = tableIdentifier) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index df3f231f7d0ef..6dda1d4aaf37e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -126,11 +126,18 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) val schema = CatalogV2Util.applySchemaChanges(catalogTable.schema, changes) val comment = properties.get(TableCatalog.PROP_COMMENT) val owner = properties.getOrElse(TableCatalog.PROP_OWNER, catalogTable.owner) + val location = properties.get(TableCatalog.PROP_LOCATION).map(CatalogUtils.stringToURI) + val storage = if (location.isDefined) { + catalogTable.storage.copy(locationUri = location) + } else { + catalogTable.storage + } try { catalog.alterTable( - catalogTable - .copy(properties = properties, schema = schema, owner = owner, comment = comment)) + catalogTable.copy( + properties = properties, schema = schema, owner = owner, comment = comment, + storage = storage)) } catch { case _: NoSuchTableException => throw new NoSuchTableException(ident) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index c399a011f9073..c3bcf86c1ed27 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -17,17 +17,19 @@ package org.apache.spark.sql.execution.datasources.v2 +import java.net.URI import java.util import java.util.Collections import scala.collection.JavaConverters._ +import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -160,6 +162,36 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(catalog.tableExists(testIdent)) } + private def makeQualifiedPathWithWarehouse(path: String): URI = { + val p = new Path(spark.sessionState.conf.warehousePath, path) + val fs = p.getFileSystem(spark.sessionState.newHadoopConf()) + fs.makeQualified(p).toUri + + } + + test("createTable: location") { + val catalog = newCatalog() + val properties = new util.HashMap[String, String]() + assert(!catalog.tableExists(testIdent)) + + // default location + val t1 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t1.catalogTable.location === + spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) + catalog.dropTable(testIdent) + + // relative path + properties.put(TableCatalog.PROP_LOCATION, "relative/path") + val t2 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t2.catalogTable.location === makeQualifiedPathWithWarehouse("db.db/relative/path")) + catalog.dropTable(testIdent) + + // absolute path + properties.put(TableCatalog.PROP_LOCATION, "/absolute/path") + val t3 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t3.catalogTable.location.toString === "file:/absolute/path") + } + test("tableExists") { val catalog = newCatalog() @@ -640,6 +672,26 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(exc.message.contains("not found")) } + test("alterTable: location") { + val catalog = newCatalog() + assert(!catalog.tableExists(testIdent)) + + // default location + val t1 = catalog.createTable(testIdent, schema, Array.empty, emptyProps).asInstanceOf[V1Table] + assert(t1.catalogTable.location === + spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) + + // relative path + val t2 = catalog.alterTable(testIdent, + TableChange.setProperty(TableCatalog.PROP_LOCATION, "relative/path")).asInstanceOf[V1Table] + assert(t2.catalogTable.location === makeQualifiedPathWithWarehouse("db.db/relative/path")) + + // absolute path + val t3 = catalog.alterTable(testIdent, + TableChange.setProperty(TableCatalog.PROP_LOCATION, "/absolute/path")).asInstanceOf[V1Table] + assert(t3.catalogTable.location.toString === "file:/absolute/path") + } + test("dropTable") { val catalog = newCatalog() @@ -812,11 +864,15 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { test("createNamespace: basic behavior") { val catalog = newCatalog() - val expectedPath = sqlContext.sessionState.catalog.getDefaultDBPath(testNs(0)).toString + + val sessionCatalog = sqlContext.sessionState.catalog + val expectedPath = + new Path(spark.sessionState.conf.warehousePath, + sessionCatalog.getDefaultDBPath(testNs(0)).toString).toString catalog.createNamespace(testNs, Map("property" -> "value").asJava) - assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) + assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri) assert(catalog.namespaceExists(testNs) === true) val metadata = catalog.loadNamespaceMetadata(testNs).asScala @@ -842,6 +898,23 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { catalog.dropNamespace(testNs) } + test("createNamespace: relative location") { + val catalog = newCatalog() + val expectedPath = + new Path(spark.sessionState.conf.warehousePath, "a/b/c").toString + + catalog.createNamespace(testNs, Map("location" -> "a/b/c").asJava) + + assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri) + + assert(catalog.namespaceExists(testNs) === true) + val metadata = catalog.loadNamespaceMetadata(testNs).asScala + checkMetadata(metadata, Map.empty) + assert(expectedPath === metadata("location")) + + catalog.dropNamespace(testNs) + } + test("createNamespace: fail if namespace already exists") { val catalog = newCatalog() @@ -954,16 +1027,23 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { test("alterNamespace: update namespace location") { val catalog = newCatalog() - val initialPath = sqlContext.sessionState.catalog.getDefaultDBPath(testNs(0)).toString - val newPath = "file:/tmp/db.db" + val initialPath = + new Path(spark.sessionState.conf.warehousePath, + spark.sessionState.catalog.getDefaultDBPath(testNs(0)).toString).toString + val newAbsoluteUri = "file:/tmp/db.db" catalog.createNamespace(testNs, emptyProps) + assert(initialPath === spark.catalog.getDatabase(testNs(0)).locationUri) + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newAbsoluteUri)) + assert(newAbsoluteUri === spark.catalog.getDatabase(testNs(0)).locationUri) - assert(initialPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) - - catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newPath)) + val newAbsolutePath = "/tmp/newAbsolutePath" + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newAbsolutePath)) + assert("file:" + newAbsolutePath === spark.catalog.getDatabase(testNs(0)).locationUri) - assert(newPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) + val newRelativePath = new Path(spark.sessionState.conf.warehousePath, "relativeP").toString + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", "relativeP")) + assert(newRelativePath === spark.catalog.getDatabase(testNs(0)).locationUri) catalog.dropNamespace(testNs) } From 7f5326c082081af465deb81c368306e569325b53 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 3 Aug 2020 12:49:34 +0000 Subject: [PATCH 355/384] [SPARK-32492][SQL] Fulfill missing column meta information COLUMN_SIZE /DECIMAL_DIGITS/NUM_PREC_RADIX/ORDINAL_POSITION for thriftserver client tools ### What changes were proposed in this pull request? This PR fulfills some missing fields for SparkGetColumnsOperation including COLUMN_SIZE /DECIMAL_DIGITS/NUM_PREC_RADIX/ORDINAL_POSITION and improve the test coverage. ### Why are the changes needed? make jdbc tools happier ### Does this PR introduce _any_ user-facing change? yes, #### before ![image](https://user-images.githubusercontent.com/8326978/88911764-e78b2180-d290-11ea-8abb-96f137f9c3c4.png) #### after ![image](https://user-images.githubusercontent.com/8326978/88911709-d04c3400-d290-11ea-90ab-02bda3e628e9.png) ![image](https://user-images.githubusercontent.com/8326978/88912007-39cc4280-d291-11ea-96d6-1ef3abbbddec.png) ### How was this patch tested? add unit tests Closes #29303 from yaooqinn/SPARK-32492. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../SparkGetColumnsOperation.scala | 54 +++++-- .../ThriftServerWithSparkContextSuite.scala | 132 ++++++++++++++++++ 2 files changed, 178 insertions(+), 8 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index d42732f426681..069517acd68cc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.UUID import java.util.regex.Pattern import scala.collection.JavaConverters.seqAsJavaListConverter -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObject} import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType import org.apache.hive.service.cli._ @@ -34,7 +32,7 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.hive.thriftserver.ThriftserverShimUtils.toJavaSQLType -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ /** * Spark's own SparkGetColumnsOperation @@ -126,12 +124,52 @@ private[hive] class SparkGetColumnsOperation( HiveThriftServer2.eventManager.onStatementFinish(statementId) } + /** + * For boolean, numeric and datetime types, it returns the default size of its catalyst type + * For struct type, when its elements are fixed-size, the summation of all element sizes will be + * returned. + * For array, map, string, and binaries, the column size is variable, return null as unknown. + */ + private def getColumnSize(typ: DataType): Option[Int] = typ match { + case dt @ (BooleanType | _: NumericType | DateType | TimestampType) => Some(dt.defaultSize) + case StructType(fields) => + val sizeArr = fields.map(f => getColumnSize(f.dataType)) + if (sizeArr.contains(None)) { + None + } else { + Some(sizeArr.map(_.get).sum) + } + case other => None + } + + /** + * The number of fractional digits for this type. + * Null is returned for data types where this is not applicable. + * For boolean and integrals, the decimal digits is 0 + * For floating types, we follow the IEEE Standard for Floating-Point Arithmetic (IEEE 754) + * For timestamp values, we support microseconds + * For decimals, it returns the scale + */ + private def getDecimalDigits(typ: DataType) = typ match { + case BooleanType | _: IntegerType => Some(0) + case FloatType => Some(7) + case DoubleType => Some(15) + case d: DecimalType => Some(d.scale) + case TimestampType => Some(6) + case _ => None + } + + private def getNumPrecRadix(typ: DataType): Option[Int] = typ match { + case _: NumericType => Some(10) + case _ => None + } + private def addToRowSet( columnPattern: Pattern, dbName: String, tableName: String, schema: StructType): Unit = { - schema.foreach { column => + schema.zipWithIndex.foreach { case (column, pos) => if (columnPattern != null && !columnPattern.matcher(column.name).matches()) { } else { val rowData = Array[AnyRef]( @@ -141,17 +179,17 @@ private[hive] class SparkGetColumnsOperation( column.name, // COLUMN_NAME toJavaSQLType(column.dataType.sql).asInstanceOf[AnyRef], // DATA_TYPE column.dataType.sql, // TYPE_NAME - null, // COLUMN_SIZE + getColumnSize(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // COLUMN_SIZE null, // BUFFER_LENGTH, unused - null, // DECIMAL_DIGITS - null, // NUM_PREC_RADIX + getDecimalDigits(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // DECIMAL_DIGITS + getNumPrecRadix(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // NUM_PREC_RADIX (if (column.nullable) 1 else 0).asInstanceOf[AnyRef], // NULLABLE column.getComment().getOrElse(""), // REMARKS null, // COLUMN_DEF null, // SQL_DATA_TYPE null, // SQL_DATETIME_SUB null, // CHAR_OCTET_LENGTH - null, // ORDINAL_POSITION + pos.asInstanceOf[AnyRef], // ORDINAL_POSITION "YES", // IS_NULLABLE null, // SCOPE_CATALOG null, // SCOPE_SCHEMA diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 2bb9169693a15..196c093aafddf 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -19,9 +19,12 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.SQLException +import scala.collection.JavaConverters._ + import org.apache.hive.service.cli.HiveSQLException import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.types._ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { @@ -101,6 +104,135 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } + + test("check results from get columns operation from thrift server") { + val schemaName = "default" + val tableName = "spark_get_col_operation" + val schema = new StructType() + .add("c0", "boolean", nullable = false, "0") + .add("c1", "tinyint", nullable = true, "1") + .add("c2", "smallint", nullable = false, "2") + .add("c3", "int", nullable = true, "3") + .add("c4", "long", nullable = false, "4") + .add("c5", "float", nullable = true, "5") + .add("c6", "double", nullable = false, "6") + .add("c7", "decimal(38, 20)", nullable = true, "7") + .add("c8", "decimal(10, 2)", nullable = false, "8") + .add("c9", "string", nullable = true, "9") + .add("c10", "array", nullable = false, "10") + .add("c11", "array", nullable = true, "11") + .add("c12", "map", nullable = false, "12") + .add("c13", "date", nullable = true, "13") + .add("c14", "timestamp", nullable = false, "14") + .add("c15", "struct", nullable = true, "15") + .add("c16", "binary", nullable = false, "16") + + val ddl = + s""" + |CREATE TABLE $schemaName.$tableName ( + | ${schema.toDDL} + |) + |using parquet""".stripMargin + + withCLIServiceClient { client => + val sessionHandle = client.openSession(user, "") + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] + val opHandle = client.executeStatement(sessionHandle, ddl, confOverlay) + var status = client.getOperationStatus(opHandle) + while (!status.getState.isTerminal) { + Thread.sleep(10) + status = client.getOperationStatus(opHandle) + } + val getCol = client.getColumns(sessionHandle, "", schemaName, tableName, null) + val rowSet = client.fetchResults(getCol) + val columns = rowSet.toTRowSet.getColumns + + val catalogs = columns.get(0).getStringVal.getValues.asScala + assert(catalogs.forall(_.isEmpty), "catalog name mismatches") + + val schemas = columns.get(1).getStringVal.getValues.asScala + assert(schemas.forall(_ == schemaName), "schema name mismatches") + + val tableNames = columns.get(2).getStringVal.getValues.asScala + assert(tableNames.forall(_ == tableName), "table name mismatches") + + val columnNames = columns.get(3).getStringVal.getValues.asScala + columnNames.zipWithIndex.foreach { + case (v, i) => assert(v === "c" + i, "column name mismatches") + } + + val javaTypes = columns.get(4).getI32Val.getValues + import java.sql.Types._ + assert(javaTypes.get(0).intValue() === BOOLEAN) + assert(javaTypes.get(1).intValue() === TINYINT) + assert(javaTypes.get(2).intValue() === SMALLINT) + assert(javaTypes.get(3).intValue() === INTEGER) + assert(javaTypes.get(4).intValue() === BIGINT) + assert(javaTypes.get(5).intValue() === FLOAT) + assert(javaTypes.get(6).intValue() === DOUBLE) + assert(javaTypes.get(7).intValue() === DECIMAL) + assert(javaTypes.get(8).intValue() === DECIMAL) + assert(javaTypes.get(9).intValue() === VARCHAR) + assert(javaTypes.get(10).intValue() === ARRAY) + assert(javaTypes.get(11).intValue() === ARRAY) + assert(javaTypes.get(12).intValue() === JAVA_OBJECT) + assert(javaTypes.get(13).intValue() === DATE) + assert(javaTypes.get(14).intValue() === TIMESTAMP) + assert(javaTypes.get(15).intValue() === STRUCT) + assert(javaTypes.get(16).intValue() === BINARY) + + val typeNames = columns.get(5).getStringVal.getValues.asScala + typeNames.zip(schema).foreach { case (tName, f) => + assert(tName === f.dataType.sql) + } + + val colSize = columns.get(6).getI32Val.getValues.asScala + + colSize.zip(schema).foreach { case (size, f) => + f.dataType match { + case StringType | BinaryType | _: ArrayType | _: MapType => assert(size === 0) + case o => assert(size === o.defaultSize) + } + } + + val decimalDigits = columns.get(8).getI32Val.getValues.asScala + decimalDigits.zip(schema).foreach { case (dd, f) => + f.dataType match { + case BooleanType | _: IntegerType => assert(dd === 0) + case d: DecimalType => assert(dd === d.scale) + case FloatType => assert(dd === 7) + case DoubleType => assert(dd === 15) + case TimestampType => assert(dd === 6) + case _ => assert(dd === 0) // nulls + } + } + + val radixes = columns.get(9).getI32Val.getValues.asScala + radixes.zip(schema).foreach { case (radix, f) => + f.dataType match { + case _: NumericType => assert(radix === 10) + case _ => assert(radix === 0) // nulls + } + } + + val nullables = columns.get(10).getI32Val.getValues.asScala + assert(nullables.forall(_ === 1)) + + val comments = columns.get(11).getStringVal.getValues.asScala + comments.zip(schema).foreach { case (c, f) => assert(c === f.getComment().get) } + + val positions = columns.get(16).getI32Val.getValues.asScala + positions.zipWithIndex.foreach { case (pos, idx) => + assert(pos === idx, "the client columns disorder") + } + + val isNullables = columns.get(17).getStringVal.getValues.asScala + assert(isNullables.forall(_ === "YES")) + + val autoIncs = columns.get(22).getStringVal.getValues.asScala + assert(autoIncs.forall(_ === "NO")) + } + } } From c6109ba9181520359222fb032d989f266d3221d8 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 3 Aug 2020 13:00:07 +0000 Subject: [PATCH 356/384] [SPARK-32257][SQL] Reports explicit errors for invalid usage of SET/RESET command MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR modified the parser code to handle invalid usages of a SET/RESET command. For example; ``` SET spark.sql.ansi.enabled true ``` The above SQL command does not change the configuration value and it just tries to display the value of the configuration `spark.sql.ansi.enabled true`. This PR disallows using special characters including spaces in the configuration name and reports a user-friendly error instead. In the error message, it tells users a workaround to use quotes or a string literal if they still needs to specify a configuration with them.  Before this PR: ``` scala> sql("SET spark.sql.ansi.enabled true").show(1, -1) +---------------------------+-----------+ |key |value | +---------------------------+-----------+ |spark.sql.ansi.enabled true|| +---------------------------+-----------+ ``` After this PR: ``` scala> sql("SET spark.sql.ansi.enabled true") org.apache.spark.sql.catalyst.parser.ParseException: Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0) == SQL == SET spark.sql.ansi.enabled true ^^^ ``` ### Why are the changes needed? For better user-friendly errors. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests in `SparkSqlParserSuite`. Closes #29146 from maropu/SPARK-32257. Lead-authored-by: Takeshi Yamamuro Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../spark/internal/config/ConfigEntry.scala | 3 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 6 ++ .../spark/sql/execution/SparkSqlParser.scala | 49 ++++++++--- .../sql/execution/SparkSqlParserSuite.scala | 81 ++++++++++++++++++- .../spark/sql/internal/SQLConfSuite.scala | 6 +- 5 files changed, 129 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala index b98c7436f9906..a295ef06a6376 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -275,7 +275,8 @@ private[spark] object ConfigEntry { val UNDEFINED = "" - private val knownConfigs = new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() + private[spark] val knownConfigs = + new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() def registerEntry(entry: ConfigEntry[_]): Unit = { val existing = knownConfigs.putIfAbsent(entry.key, entry) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 39f94651a0cb5..6fce7819897a6 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -246,11 +246,17 @@ statement | SET TIME ZONE interval #setTimeZone | SET TIME ZONE timezone=(STRING | LOCAL) #setTimeZone | SET TIME ZONE .*? #setTimeZone + | SET configKey (EQ .*?)? #setQuotedConfiguration | SET .*? #setConfiguration + | RESET configKey #resetQuotedConfiguration | RESET .*? #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand ; +configKey + : quotedIdentifier + ; + unsupportedHiveNativeCommands : kw1=CREATE kw2=ROLE | kw1=DROP kw2=ROLE diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 012ae0a76043c..129312160b1b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -58,6 +58,9 @@ class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser(conf) { class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { import org.apache.spark.sql.catalyst.parser.ParserUtils._ + private val configKeyValueDef = """([a-zA-Z_\d\\.:]+)\s*=(.*)""".r + private val configKeyDef = """([a-zA-Z_\d\\.:]+)$""".r + /** * Create a [[SetCommand]] logical plan. * @@ -66,17 +69,28 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * character in the raw string. */ override def visitSetConfiguration(ctx: SetConfigurationContext): LogicalPlan = withOrigin(ctx) { - // Construct the command. - val raw = remainder(ctx.SET.getSymbol) - val keyValueSeparatorIndex = raw.indexOf('=') - if (keyValueSeparatorIndex >= 0) { - val key = raw.substring(0, keyValueSeparatorIndex).trim - val value = raw.substring(keyValueSeparatorIndex + 1).trim - SetCommand(Some(key -> Option(value))) - } else if (raw.nonEmpty) { - SetCommand(Some(raw.trim -> None)) + remainder(ctx.SET.getSymbol).trim match { + case configKeyValueDef(key, value) => + SetCommand(Some(key -> Option(value.trim))) + case configKeyDef(key) => + SetCommand(Some(key -> None)) + case s if s == "-v" => + SetCommand(Some("-v" -> None)) + case s if s.isEmpty => + SetCommand(None) + case _ => throw new ParseException("Expected format is 'SET', 'SET key', or " + + "'SET key=value'. If you want to include special characters in key, " + + "please use quotes, e.g., SET `ke y`=value.", ctx) + } + } + + override def visitSetQuotedConfiguration(ctx: SetQuotedConfigurationContext) + : LogicalPlan = withOrigin(ctx) { + val keyStr = ctx.configKey().getText + if (ctx.EQ() != null) { + SetCommand(Some(keyStr -> Option(remainder(ctx.EQ().getSymbol).trim))) } else { - SetCommand(None) + SetCommand(Some(keyStr -> None)) } } @@ -90,7 +104,20 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { */ override def visitResetConfiguration( ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { - ResetCommand(Option(remainder(ctx.RESET().getSymbol).trim).filter(_.nonEmpty)) + remainder(ctx.RESET.getSymbol).trim match { + case configKeyDef(key) => + ResetCommand(Some(key)) + case s if s.trim.isEmpty => + ResetCommand(None) + case _ => throw new ParseException("Expected format is 'RESET' or 'RESET key'. " + + "If you want to include special characters in key, " + + "please use quotes, e.g., RESET `ke y`.", ctx) + } + } + + override def visitResetQuotedConfiguration( + ctx: ResetQuotedConfigurationContext): LogicalPlan = withOrigin(ctx) { + ResetCommand(Some(ctx.configKey().getText)) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 1991f139e48c5..62712cf72eb59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.execution +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} @@ -25,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} -import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} /** @@ -61,6 +64,82 @@ class SparkSqlParserSuite extends AnalysisTest { private def intercept(sqlCommand: String, messages: String*): Unit = interceptParseException(parser.parsePlan)(sqlCommand, messages: _*) + test("Checks if SET/RESET can parse all the configurations") { + // Force to build static SQL configurations + StaticSQLConf + ConfigEntry.knownConfigs.values.asScala.foreach { config => + assertEqual(s"SET ${config.key}", SetCommand(Some(config.key -> None))) + if (config.defaultValue.isDefined && config.defaultValueString != null) { + assertEqual(s"SET ${config.key}=${config.defaultValueString}", + SetCommand(Some(config.key -> Some(config.defaultValueString)))) + } + assertEqual(s"RESET ${config.key}", ResetCommand(Some(config.key))) + } + } + + test("Report Error for invalid usage of SET command") { + assertEqual("SET", SetCommand(None)) + assertEqual("SET -v", SetCommand(Some("-v", None))) + assertEqual("SET spark.sql.key", SetCommand(Some("spark.sql.key" -> None))) + assertEqual("SET spark.sql.key ", SetCommand(Some("spark.sql.key" -> None))) + assertEqual("SET spark:sql:key=false", SetCommand(Some("spark:sql:key" -> Some("false")))) + assertEqual("SET spark:sql:key=", SetCommand(Some("spark:sql:key" -> Some("")))) + assertEqual("SET spark:sql:key= ", SetCommand(Some("spark:sql:key" -> Some("")))) + assertEqual("SET spark:sql:key=-1 ", SetCommand(Some("spark:sql:key" -> Some("-1")))) + assertEqual("SET spark:sql:key = -1", SetCommand(Some("spark:sql:key" -> Some("-1")))) + assertEqual("SET 1.2.key=value", SetCommand(Some("1.2.key" -> Some("value")))) + assertEqual("SET spark.sql.3=4", SetCommand(Some("spark.sql.3" -> Some("4")))) + assertEqual("SET 1:2:key=value", SetCommand(Some("1:2:key" -> Some("value")))) + assertEqual("SET spark:sql:3=4", SetCommand(Some("spark:sql:3" -> Some("4")))) + assertEqual("SET 5=6", SetCommand(Some("5" -> Some("6")))) + assertEqual("SET spark:sql:key = va l u e ", + SetCommand(Some("spark:sql:key" -> Some("va l u e")))) + assertEqual("SET `spark.sql. key`=value", + SetCommand(Some("spark.sql. key" -> Some("value")))) + assertEqual("SET `spark.sql. key`= v a lu e ", + SetCommand(Some("spark.sql. key" -> Some("v a lu e")))) + assertEqual("SET `spark.sql. key`= -1", + SetCommand(Some("spark.sql. key" -> Some("-1")))) + + val expectedErrMsg = "Expected format is 'SET', 'SET key', or " + + "'SET key=value'. If you want to include special characters in key, " + + "please use quotes, e.g., SET `ke y`=value." + intercept("SET spark.sql.key value", expectedErrMsg) + intercept("SET spark.sql.key 'value'", expectedErrMsg) + intercept("SET spark.sql.key \"value\" ", expectedErrMsg) + intercept("SET spark.sql.key value1 value2", expectedErrMsg) + intercept("SET spark. sql.key=value", expectedErrMsg) + intercept("SET spark :sql:key=value", expectedErrMsg) + intercept("SET spark . sql.key=value", expectedErrMsg) + intercept("SET spark.sql. key=value", expectedErrMsg) + intercept("SET spark.sql :key=value", expectedErrMsg) + intercept("SET spark.sql . key=value", expectedErrMsg) + } + + test("Report Error for invalid usage of RESET command") { + assertEqual("RESET", ResetCommand(None)) + assertEqual("RESET spark.sql.key", ResetCommand(Some("spark.sql.key"))) + assertEqual("RESET spark.sql.key ", ResetCommand(Some("spark.sql.key"))) + assertEqual("RESET 1.2.key ", ResetCommand(Some("1.2.key"))) + assertEqual("RESET spark.sql.3", ResetCommand(Some("spark.sql.3"))) + assertEqual("RESET 1:2:key ", ResetCommand(Some("1:2:key"))) + assertEqual("RESET spark:sql:3", ResetCommand(Some("spark:sql:3"))) + assertEqual("RESET `spark.sql. key`", ResetCommand(Some("spark.sql. key"))) + + val expectedErrMsg = "Expected format is 'RESET' or 'RESET key'. " + + "If you want to include special characters in key, " + + "please use quotes, e.g., RESET `ke y`." + intercept("RESET spark.sql.key1 key2", expectedErrMsg) + intercept("RESET spark. sql.key1 key2", expectedErrMsg) + intercept("RESET spark.sql.key1 key2 key3", expectedErrMsg) + intercept("RESET spark: sql:key", expectedErrMsg) + intercept("RESET spark .sql.key", expectedErrMsg) + intercept("RESET spark : sql:key", expectedErrMsg) + intercept("RESET spark.sql: key", expectedErrMsg) + intercept("RESET spark.sql .key", expectedErrMsg) + intercept("RESET spark.sql : key", expectedErrMsg) + } + test("refresh resource") { assertEqual("REFRESH prefix_path", RefreshResource("prefix_path")) assertEqual("REFRESH /", RefreshResource("/")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 565a8351d9c06..5701cc917258b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -115,7 +115,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") assert(spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) === 10) } finally { - sql(s"set ${SQLConf.SHUFFLE_PARTITIONS}=$original") + sql(s"set ${SQLConf.SHUFFLE_PARTITIONS.key}=$original") } } @@ -149,7 +149,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES) === Some("org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation")) } finally { - sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") + sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=$original") } } @@ -165,7 +165,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 0) } finally { - sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS}=$original") + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}=$original") } } From bc7885901dd99de21ecbf269d72fa37a393b2ffc Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 3 Aug 2020 08:50:34 -0700 Subject: [PATCH 357/384] [SPARK-32310][ML][PYSPARK] ML params default value parity in feature and tuning ### What changes were proposed in this pull request? set params default values in trait Params for feature and tuning in both Scala and Python. ### Why are the changes needed? Make ML has the same default param values between estimator and its corresponding transformer, and also between Scala and Python. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing and modified tests Closes #29153 from huaxingao/default2. Authored-by: Huaxin Gao Signed-off-by: Huaxin Gao --- .../org/apache/spark/ml/feature/Imputer.scala | 4 +- .../spark/ml/feature/MinMaxScaler.scala | 4 +- .../spark/ml/feature/OneHotEncoder.scala | 5 +- .../ml/feature/QuantileDiscretizer.scala | 4 +- .../apache/spark/ml/feature/RFormula.scala | 6 +- .../spark/ml/feature/RobustScaler.scala | 8 +- .../apache/spark/ml/feature/Selector.scala | 8 +- .../spark/ml/feature/StringIndexer.scala | 6 +- .../spark/ml/feature/VectorIndexer.scala | 6 +- .../spark/ml/feature/VectorSlicer.scala | 6 +- .../apache/spark/ml/feature/Word2Vec.scala | 9 +- .../org/apache/spark/ml/tree/treeParams.scala | 16 +-- .../spark/ml/tuning/CrossValidator.scala | 4 +- .../spark/ml/util/DefaultReadWriteTest.scala | 3 + python/pyspark/ml/classification.py | 56 +++++--- python/pyspark/ml/clustering.py | 30 +++-- python/pyspark/ml/feature.py | 120 +++++++++++++++--- python/pyspark/ml/fpm.py | 9 +- python/pyspark/ml/recommendation.py | 20 ++- python/pyspark/ml/regression.py | 60 ++++++--- python/pyspark/ml/tests/test_param.py | 8 +- python/pyspark/ml/tuning.py | 17 ++- 22 files changed, 274 insertions(+), 135 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index 64f1722f5fcb8..ad1010da5c104 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -64,6 +64,8 @@ private[feature] trait ImputerParams extends Params with HasInputCol with HasInp /** @group getParam */ def getMissingValue: Double = $(missingValue) + setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN) + /** Returns the input and output column names corresponding in pair. */ private[feature] def getInOutCols(): (Array[String], Array[String]) = { if (isSet(inputCol)) { @@ -144,8 +146,6 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) @Since("3.0.0") def setRelativeError(value: Double): this.type = set(relativeError, value) - setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN) - override def fit(dataset: Dataset[_]): ImputerModel = { transformSchema(dataset.schema, logging = true) val spark = dataset.sparkSession diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 90187c331e835..22c4ca9cddf4b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -58,6 +58,8 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H /** @group getParam */ def getMax: Double = $(max) + setDefault(min -> 0.0, max -> 1.0) + /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { require($(min) < $(max), s"The specified min(${$(min)}) is larger or equal to max(${$(max)})") @@ -93,8 +95,6 @@ class MinMaxScaler @Since("1.5.0") (@Since("1.5.0") override val uid: String) @Since("1.5.0") def this() = this(Identifiable.randomUID("minMaxScal")) - setDefault(min -> 0.0, max -> 1.0) - /** @group setParam */ @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 0ef092f6be463..5a500fefb57ec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -53,8 +53,6 @@ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid "during fitting, invalid data will result in an error.", ParamValidators.inArray(OneHotEncoder.supportedHandleInvalids)) - setDefault(handleInvalid, OneHotEncoder.ERROR_INVALID) - /** * Whether to drop the last category in the encoded vector (default: true) * @group param @@ -62,12 +60,13 @@ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid @Since("2.3.0") final val dropLast: BooleanParam = new BooleanParam(this, "dropLast", "whether to drop the last category") - setDefault(dropLast -> true) /** @group getParam */ @Since("2.3.0") def getDropLast: Boolean = $(dropLast) + setDefault(handleInvalid -> OneHotEncoder.ERROR_INVALID, dropLast -> true) + /** Returns the input and output column names corresponding in pair. */ private[feature] def getInOutCols(): (Array[String], Array[String]) = { if (isSet(inputCol)) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 4eedfc4dc0efa..b93c9b1fcd204 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -46,7 +46,6 @@ private[feature] trait QuantileDiscretizerBase extends Params val numBuckets = new IntParam(this, "numBuckets", "Number of buckets (quantiles, or " + "categories) into which data points are grouped. Must be >= 2.", ParamValidators.gtEq(2)) - setDefault(numBuckets -> 2) /** @group getParam */ def getNumBuckets: Int = getOrDefault(numBuckets) @@ -82,7 +81,8 @@ private[feature] trait QuantileDiscretizerBase extends Params "how to handle invalid entries. Options are skip (filter out rows with invalid values), " + "error (throw an error), or keep (keep invalid values in a special additional bucket).", ParamValidators.inArray(Bucketizer.supportedHandleInvalids)) - setDefault(handleInvalid, Bucketizer.ERROR_INVALID) + + setDefault(handleInvalid -> Bucketizer.ERROR_INVALID, numBuckets -> 2) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 7ccfafa4ac813..b8da020017f12 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -60,7 +60,6 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with @Since("2.1.0") val forceIndexLabel: BooleanParam = new BooleanParam(this, "forceIndexLabel", "Force to index label whether it is numeric or string") - setDefault(forceIndexLabel -> false) /** @group getParam */ @Since("2.1.0") @@ -80,7 +79,6 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with "type. Options are 'skip' (filter out rows with invalid data), error (throw an error), " + "or 'keep' (put invalid data in a special additional bucket, at index numLabels).", ParamValidators.inArray(StringIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, StringIndexer.ERROR_INVALID) /** * Param for how to order categories of a string FEATURE column used by `StringIndexer`. @@ -113,12 +111,14 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with "The default value is 'frequencyDesc'. When the ordering is set to 'alphabetDesc', " + "RFormula drops the same category as R when encoding strings.", ParamValidators.inArray(StringIndexer.supportedStringOrderType)) - setDefault(stringIndexerOrderType, StringIndexer.frequencyDesc) /** @group getParam */ @Since("2.3.0") def getStringIndexerOrderType: String = $(stringIndexerOrderType) + setDefault(forceIndexLabel -> false, handleInvalid -> StringIndexer.ERROR_INVALID, + stringIndexerOrderType -> StringIndexer.frequencyDesc) + protected def hasLabelCol(schema: StructType): Boolean = { schema.map(_.name).contains($(labelCol)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala index 72ab3dbc31016..e8f325ec58432 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala @@ -50,8 +50,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getLower: Double = $(lower) - setDefault(lower -> 0.25) - /** * Upper quantile to calculate quantile range, shared by all features * Default: 0.75 @@ -64,8 +62,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getUpper: Double = $(upper) - setDefault(upper -> 0.75) - /** * Whether to center the data with median before scaling. * It will build a dense output, so take care when applying to sparse input. @@ -78,8 +74,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getWithCentering: Boolean = $(withCentering) - setDefault(withCentering -> false) - /** * Whether to scale the data to quantile range. * Default: true @@ -91,7 +85,7 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getWithScaling: Boolean = $(withScaling) - setDefault(withScaling -> true) + setDefault(withScaling -> true, lower -> 0.25, upper -> 0.75, withCentering -> false) /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala index 627133968d149..46052a89fdf1a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala @@ -50,7 +50,6 @@ private[feature] trait SelectorParams extends Params "Number of features that selector will select, ordered by ascending p-value. If the" + " number of features is < numTopFeatures, then this will select all features.", ParamValidators.gtEq(1)) - setDefault(numTopFeatures -> 50) /** @group getParam */ @Since("3.1.0") @@ -66,7 +65,6 @@ private[feature] trait SelectorParams extends Params final val percentile = new DoubleParam(this, "percentile", "Percentile of features that selector will select, ordered by ascending p-value.", ParamValidators.inRange(0, 1)) - setDefault(percentile -> 0.1) /** @group getParam */ @Since("3.1.0") @@ -81,7 +79,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fpr = new DoubleParam(this, "fpr", "The higest p-value for features to be kept.", ParamValidators.inRange(0, 1)) - setDefault(fpr -> 0.05) /** @group getParam */ @Since("3.1.0") @@ -96,7 +93,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fdr = new DoubleParam(this, "fdr", "The upper bound of the expected false discovery rate.", ParamValidators.inRange(0, 1)) - setDefault(fdr -> 0.05) /** @group getParam */ def getFdr: Double = $(fdr) @@ -110,7 +106,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fwe = new DoubleParam(this, "fwe", "The upper bound of the expected family-wise error rate.", ParamValidators.inRange(0, 1)) - setDefault(fwe -> 0.05) /** @group getParam */ def getFwe: Double = $(fwe) @@ -125,12 +120,13 @@ private[feature] trait SelectorParams extends Params "The selector type. Supported options: numTopFeatures, percentile, fpr, fdr, fwe", ParamValidators.inArray(Array("numTopFeatures", "percentile", "fpr", "fdr", "fwe"))) - setDefault(selectorType -> "numTopFeatures") /** @group getParam */ @Since("3.1.0") def getSelectorType: String = $(selectorType) + setDefault(numTopFeatures -> 50, percentile -> 0.1, fpr -> 0.05, fdr -> 0.05, fwe -> 0.05, + selectorType -> "numTopFeatures") } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index be32f44287b6a..ab51fe6e78bd7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -56,8 +56,6 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi "or 'keep' (put invalid data in a special additional bucket, at index numLabels).", ParamValidators.inArray(StringIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, StringIndexer.ERROR_INVALID) - /** * Param for how to order labels of string column. The first label after ordering is assigned * an index of 0. @@ -80,6 +78,9 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi s"Supported options: ${StringIndexer.supportedStringOrderType.mkString(", ")}.", ParamValidators.inArray(StringIndexer.supportedStringOrderType)) + setDefault(handleInvalid -> StringIndexer.ERROR_INVALID, + stringOrderType -> StringIndexer.frequencyDesc) + /** @group getParam */ @Since("2.3.0") def getStringOrderType: String = $(stringOrderType) @@ -155,7 +156,6 @@ class StringIndexer @Since("1.4.0") ( /** @group setParam */ @Since("2.3.0") def setStringOrderType(value: String): this.type = set(stringOrderType, value) - setDefault(stringOrderType, StringIndexer.frequencyDesc) /** @group setParam */ @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 866074fb1453e..b7cf4392cd177 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -60,8 +60,6 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu "number of categories of the feature).", ParamValidators.inArray(VectorIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, VectorIndexer.ERROR_INVALID) - /** * Threshold for the number of values a categorical feature can take. * If a feature is found to have {@literal >} maxCategories values, then it is declared @@ -75,10 +73,10 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu " If a feature is found to have > maxCategories values, then it is declared continuous.", ParamValidators.gtEq(2)) - setDefault(maxCategories -> 20) - /** @group getParam */ def getMaxCategories: Int = $(maxCategories) + + setDefault(maxCategories -> 20, handleInvalid -> VectorIndexer.ERROR_INVALID) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index be91844ba39e6..3590b9118f3b8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -57,8 +57,6 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri "An array of indices to select features from a vector column." + " There can be no overlap with names.", VectorSlicer.validIndices) - setDefault(indices -> Array.emptyIntArray) - /** @group getParam */ @Since("1.5.0") def getIndices: Array[Int] = $(indices) @@ -79,8 +77,6 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri "An array of feature names to select features from a vector column." + " There can be no overlap with indices.", VectorSlicer.validNames) - setDefault(names -> Array.empty[String]) - /** @group getParam */ @Since("1.5.0") def getNames: Array[String] = $(names) @@ -97,6 +93,8 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) + setDefault(indices -> Array.emptyIntArray, names -> Array.empty[String]) + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { // Validity checks diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index db2665fa2e4a3..01db39f9e3921 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -47,7 +47,6 @@ private[feature] trait Word2VecBase extends Params final val vectorSize = new IntParam( this, "vectorSize", "the dimension of codes after transforming from words (> 0)", ParamValidators.gt(0)) - setDefault(vectorSize -> 100) /** @group getParam */ def getVectorSize: Int = $(vectorSize) @@ -60,7 +59,6 @@ private[feature] trait Word2VecBase extends Params final val windowSize = new IntParam( this, "windowSize", "the window size (context words from [-window, window]) (> 0)", ParamValidators.gt(0)) - setDefault(windowSize -> 5) /** @group expertGetParam */ def getWindowSize: Int = $(windowSize) @@ -73,7 +71,6 @@ private[feature] trait Word2VecBase extends Params final val numPartitions = new IntParam( this, "numPartitions", "number of partitions for sentences of words (> 0)", ParamValidators.gt(0)) - setDefault(numPartitions -> 1) /** @group getParam */ def getNumPartitions: Int = $(numPartitions) @@ -86,7 +83,6 @@ private[feature] trait Word2VecBase extends Params */ final val minCount = new IntParam(this, "minCount", "the minimum number of times a token must " + "appear to be included in the word2vec model's vocabulary (>= 0)", ParamValidators.gtEq(0)) - setDefault(minCount -> 5) /** @group getParam */ def getMinCount: Int = $(minCount) @@ -101,13 +97,12 @@ private[feature] trait Word2VecBase extends Params final val maxSentenceLength = new IntParam(this, "maxSentenceLength", "Maximum length " + "(in words) of each sentence in the input data. Any sentence longer than this threshold will " + "be divided into chunks up to the size (> 0)", ParamValidators.gt(0)) - setDefault(maxSentenceLength -> 1000) /** @group getParam */ def getMaxSentenceLength: Int = $(maxSentenceLength) - setDefault(stepSize -> 0.025) - setDefault(maxIter -> 1) + setDefault(vectorSize -> 100, windowSize -> 5, numPartitions -> 1, minCount -> 5, + maxSentenceLength -> 1000, stepSize -> 0.025, maxIter -> 1) /** * Validate and transform the input schema. diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 7e2c287f146fb..19ea8ae4775d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -333,8 +333,6 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { "Fraction of the training data used for learning each decision tree, in range (0, 1].", ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) - setDefault(subsamplingRate -> 1.0) - /** @group getParam */ final def getSubsamplingRate: Double = $(subsamplingRate) @@ -386,10 +384,10 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { || Try(value.toInt).filter(_ > 0).isSuccess || Try(value.toDouble).filter(_ > 0).filter(_ <= 1.0).isSuccess) - setDefault(featureSubsetStrategy -> "auto") - /** @group getParam */ final def getFeatureSubsetStrategy: String = $(featureSubsetStrategy).toLowerCase(Locale.ROOT) + + setDefault(subsamplingRate -> 1.0, featureSubsetStrategy -> "auto") } /** @@ -448,8 +446,6 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { new IntParam(this, "numTrees", "Number of trees to train (at least 1)", ParamValidators.gtEq(1)) - setDefault(numTrees -> 20) - /** @group getParam */ final def getNumTrees: Int = $(numTrees) @@ -461,11 +457,11 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { final val bootstrap: BooleanParam = new BooleanParam(this, "bootstrap", "Whether bootstrap samples are used when building trees.") - setDefault(bootstrap -> true) - /** @group getParam */ @Since("3.0.0") final def getBootstrap: Boolean = $(bootstrap) + + setDefault(numTrees -> 20, bootstrap -> true) } private[ml] trait RandomForestClassifierParams @@ -518,9 +514,7 @@ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasS "(a.k.a. learning rate) in interval (0, 1] for shrinking the contribution of each estimator.", ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) - setDefault(maxIter -> 20, stepSize -> 0.1, validationTol -> 0.01) - - setDefault(featureSubsetStrategy -> "all") + setDefault(maxIter -> 20, stepSize -> 0.1, validationTol -> 0.01, featureSubsetStrategy -> "all") /** (private[ml]) Create a BoostingStrategy instance to use with the old API. */ private[ml] def getOldBoostingStrategy( diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index e99c55b0cdd85..275d3c5510f7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -55,8 +55,6 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { /** @group getParam */ def getNumFolds: Int = $(numFolds) - setDefault(numFolds -> 3) - /** * Param for the column name of user specified fold number. Once this is specified, * `CrossValidator` won't do random k-fold split. Note that this column should be @@ -68,7 +66,7 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { def getFoldCol: String = $(foldCol) - setDefault(foldCol, "") + setDefault(foldCol -> "", numFolds -> 3) } /** diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala index 4d9e664850c12..dd0139b94f098 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala @@ -63,6 +63,9 @@ trait DefaultReadWriteTest extends TempDirectory { self: Suite => (instance.getOrDefault(p), newInstance.getOrDefault(p)) match { case (Array(values), Array(newValues)) => assert(values === newValues, s"Values do not match on param ${p.name}.") + case (value: Double, newValue: Double) => + assert(value.isNaN && newValue.isNaN || value == newValue, + s"Values do not match on param ${p.name}.") case (value, newValue) => assert(value === newValue, s"Values do not match on param ${p.name}.") } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 3bc9dc0628aa8..a223a9be4d8ea 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -515,8 +515,8 @@ class _LinearSVCParams(_ClassifierParams, HasRegParam, HasMaxIter, HasFitInterce " all predictions 0.0 and -Inf will make all predictions 1.0.", typeConverter=TypeConverters.toFloat) - def __init__(self): - super(_LinearSVCParams, self).__init__() + def __init__(self, *args): + super(_LinearSVCParams, self).__init__(*args) self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, standardization=True, threshold=0.0, aggregationDepth=2, blockSize=1) @@ -589,6 +589,8 @@ class LinearSVC(_JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadabl True >>> model.intercept == model2.intercept True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 2.2.0 """ @@ -822,8 +824,8 @@ class _LogisticRegressionParams(_ProbabilisticClassifierParams, HasRegParam, "classes for multinomial regression.", typeConverter=TypeConverters.toVector) - def __init__(self): - super(_LogisticRegressionParams, self).__init__() + def __init__(self, *args): + super(_LogisticRegressionParams, self).__init__(*args) self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto", blockSize=1) @@ -1020,6 +1022,8 @@ class LogisticRegression(_JavaProbabilisticClassifier, _LogisticRegressionParams True >>> model2 LogisticRegressionModel: uid=..., numClasses=2, numFeatures=2 + >>> blorModel.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.3.0 """ @@ -1315,8 +1319,8 @@ class _DecisionTreeClassifierParams(_DecisionTreeParams, _TreeClassifierParams): Params for :py:class:`DecisionTreeClassifier` and :py:class:`DecisionTreeClassificationModel`. """ - def __init__(self): - super(_DecisionTreeClassifierParams, self).__init__() + def __init__(self, *args): + super(_DecisionTreeClassifierParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", leafCol="", minWeightFractionPerNode=0.0) @@ -1386,7 +1390,8 @@ class DecisionTreeClassifier(_JavaProbabilisticClassifier, _DecisionTreeClassifi >>> model2 = DecisionTreeClassificationModel.load(model_path) >>> model.featureImportances == model2.featureImportances True - + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> df3 = spark.createDataFrame([ ... (1.0, 0.2, Vectors.dense(1.0)), ... (1.0, 0.8, Vectors.dense(1.0)), @@ -1552,8 +1557,8 @@ class _RandomForestClassifierParams(_RandomForestParams, _TreeClassifierParams): Params for :py:class:`RandomForestClassifier` and :py:class:`RandomForestClassificationModel`. """ - def __init__(self): - super(_RandomForestClassifierParams, self).__init__() + def __init__(self, *args): + super(_RandomForestClassifierParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", numTrees=20, featureSubsetStrategy="auto", @@ -1630,6 +1635,8 @@ class RandomForestClassifier(_JavaProbabilisticClassifier, _RandomForestClassifi >>> model2 = RandomForestClassificationModel.load(model_path) >>> model.featureImportances == model2.featureImportances True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.4.0 """ @@ -1895,8 +1902,8 @@ class _GBTClassifierParams(_GBTParams, _HasVarianceImpurity): "Supported options: " + ", ".join(supportedLossTypes), typeConverter=TypeConverters.toString) - def __init__(self): - super(_GBTClassifierParams, self).__init__() + def __init__(self, *args): + super(_GBTClassifierParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, lossType="logistic", maxIter=20, stepSize=0.1, subsamplingRate=1.0, @@ -1994,6 +2001,8 @@ class GBTClassifier(_JavaProbabilisticClassifier, _GBTClassifierParams, True >>> model.treeWeights == model2.treeWeights True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.trees [DecisionTreeRegressionModel...depth=..., DecisionTreeRegressionModel...] >>> validation = spark.createDataFrame([(0.0, Vectors.dense(-1.0),)], @@ -2227,8 +2236,8 @@ class _NaiveBayesParams(_PredictorParams, HasWeightCol): "and gaussian.", typeConverter=TypeConverters.toString) - def __init__(self): - super(_NaiveBayesParams, self).__init__() + def __init__(self, *args): + super(_NaiveBayesParams, self).__init__(*args) self._setDefault(smoothing=1.0, modelType="multinomial") @since("1.5.0") @@ -2314,6 +2323,8 @@ class NaiveBayes(_JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, True >>> model.theta == model2.theta True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> nb = nb.setThresholds([0.01, 10.00]) >>> model3 = nb.fit(df) >>> result = model3.transform(test0).head() @@ -2440,8 +2451,8 @@ class _MultilayerPerceptronParams(_ProbabilisticClassifierParams, HasSeed, HasMa initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", typeConverter=TypeConverters.toVector) - def __init__(self): - super(_MultilayerPerceptronParams, self).__init__() + def __init__(self, *args): + super(_MultilayerPerceptronParams, self).__init__(*args) self._setDefault(maxIter=100, tol=1E-6, blockSize=128, stepSize=0.03, solver="l-bfgs") @since("1.6.0") @@ -2523,6 +2534,8 @@ class MultilayerPerceptronClassifier(_JavaProbabilisticClassifier, _MultilayerPe True >>> model.weights == model2.weights True + >>> model.transform(testDF).take(1) == model2.transform(testDF).take(1) + True >>> mlp2 = mlp2.setInitialWeights(list(range(0, 12))) >>> model3 = mlp2.fit(df) >>> model3.weights != model2.weights @@ -2742,6 +2755,8 @@ class OneVsRest(Estimator, _OneVsRestParams, HasParallelism, JavaMLReadable, Jav >>> model2 = OneVsRestModel.load(model_path) >>> model2.transform(test0).head().newPrediction 0.0 + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.transform(test2).columns ['features', 'rawPrediction', 'newPrediction'] @@ -3167,6 +3182,17 @@ class FMClassifier(_JavaProbabilisticClassifier, _FactorizationMachinesParams, J DenseVector([14.8232]) >>> model.factors DenseMatrix(1, 2, [0.0163, -0.0051], 1) + >>> model_path = temp_path + "/fm_model" + >>> model.save(model_path) + >>> model2 = FMClassificationModel.load(model_path) + >>> model2.intercept + -7.316665276826291 + >>> model2.linear + DenseVector([14.8232]) + >>> model2.factors + DenseMatrix(1, 2, [0.0163, -0.0051], 1) + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 3.0.0 """ diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 6ca413d696368..2d70f876849f8 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -109,8 +109,8 @@ class _GaussianMixtureParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionC k = Param(Params._dummy(), "k", "Number of independent Gaussians in the mixture model. " + "Must be > 1.", typeConverter=TypeConverters.toInt) - def __init__(self): - super(_GaussianMixtureParams, self).__init__() + def __init__(self, *args): + super(_GaussianMixtureParams, self).__init__(*args) self._setDefault(k=2, tol=0.01, maxIter=100, aggregationDepth=2, blockSize=1) @since("2.0.0") @@ -325,6 +325,8 @@ class GaussianMixture(JavaEstimator, _GaussianMixtureParams, JavaMLWritable, Jav Row(mean=DenseVector([0.825, 0.8675])) >>> model2.gaussiansDF.select("cov").head() Row(cov=DenseMatrix(2, 2, [0.0056, -0.0051, -0.0051, 0.0046], False)) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True >>> gm2.setWeightCol("weight") GaussianMixture... @@ -503,8 +505,8 @@ class _KMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionCol, HasTo initSteps = Param(Params._dummy(), "initSteps", "The number of steps for k-means|| " + "initialization mode. Must be > 0.", typeConverter=TypeConverters.toInt) - def __init__(self): - super(_KMeansParams, self).__init__() + def __init__(self, *args): + super(_KMeansParams, self).__init__(*args) self._setDefault(k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, distanceMeasure="euclidean") @@ -637,6 +639,8 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): array([ True, True], dtype=bool) >>> model.clusterCenters()[1] == model2.clusterCenters()[1] array([ True, True], dtype=bool) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -760,8 +764,8 @@ class _BisectingKMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionC "proportion of points (if < 1.0) of a divisible cluster.", typeConverter=TypeConverters.toFloat) - def __init__(self): - super(_BisectingKMeansParams, self).__init__() + def __init__(self, *args): + super(_BisectingKMeansParams, self).__init__(*args) self._setDefault(maxIter=20, k=4, minDivisibleClusterSize=1.0) @since("2.0.0") @@ -914,6 +918,8 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav array([ True, True], dtype=bool) >>> model.clusterCenters()[1] == model2.clusterCenters()[1] array([ True, True], dtype=bool) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -1072,8 +1078,8 @@ class _LDAParams(HasMaxIter, HasFeaturesCol, HasSeed, HasCheckpointInterval): " partition is lost, so set this bit with care.", TypeConverters.toBoolean) - def __init__(self): - super(_LDAParams, self).__init__() + def __init__(self, *args): + super(_LDAParams, self).__init__(*args) self._setDefault(maxIter=20, checkpointInterval=10, k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51, subsamplingRate=0.05, optimizeDocConcentration=True, @@ -1389,6 +1395,8 @@ class LDA(JavaEstimator, _LDAParams, JavaMLReadable, JavaMLWritable): >>> local_model_path = temp_path + "/lda_local_model" >>> localModel.save(local_model_path) >>> sameLocalModel = LocalLDAModel.load(local_model_path) + >>> model.transform(df).take(1) == sameLocalModel.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -1600,8 +1608,8 @@ class _PowerIterationClusteringParams(HasMaxIter, HasWeightCol): "Name of the input column for destination vertex IDs.", typeConverter=TypeConverters.toString) - def __init__(self): - super(_PowerIterationClusteringParams, self).__init__() + def __init__(self, *args): + super(_PowerIterationClusteringParams, self).__init__(*args) self._setDefault(k=2, maxIter=20, initMode="random", srcCol="src", dstCol="dst") @since("2.4.0") @@ -1677,6 +1685,8 @@ class PowerIterationClustering(_PowerIterationClusteringParams, JavaParams, Java 2 >>> pic2.getMaxIter() 40 + >>> pic2.assignClusters(df).take(6) == assignments.take(6) + True .. versionadded:: 2.4.0 """ diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index a319dace6869a..2220293d54ba4 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -92,6 +92,8 @@ class Binarizer(JavaTransformer, HasThreshold, HasThresholds, HasInputCol, HasOu >>> loadedBinarizer = Binarizer.load(binarizerPath) >>> loadedBinarizer.getThreshold() == binarizer.getThreshold() True + >>> loadedBinarizer.transform(df).take(1) == binarizer.transform(df).take(1) + True >>> df2 = spark.createDataFrame([(0.5, 0.3)], ["values1", "values2"]) >>> binarizer2 = Binarizer(thresholds=[0.0, 1.0]) >>> binarizer2.setInputCols(["values1", "values2"]).setOutputCols(["output1", "output2"]) @@ -195,6 +197,10 @@ class _LSHParams(HasInputCol, HasOutputCol): "and decreasing it improves the running performance.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_LSHParams, self).__init__(*args) + self._setDefault(numHashTables=1) + def getNumHashTables(self): """ Gets the value of numHashTables or its default value. @@ -390,7 +396,6 @@ def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, super(BucketedRandomProjectionLSH, self).__init__() self._java_obj = \ self._new_java_obj("org.apache.spark.ml.feature.BucketedRandomProjectionLSH", self.uid) - self._setDefault(numHashTables=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -477,6 +482,8 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu >>> loadedBucketizer = Bucketizer.load(bucketizerPath) >>> loadedBucketizer.getSplits() == bucketizer.getSplits() True + >>> loadedBucketizer.transform(df).take(1) == bucketizer.transform(df).take(1) + True >>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect() >>> len(bucketed) 4 @@ -733,6 +740,8 @@ class CountVectorizer(JavaEstimator, _CountVectorizerParams, JavaMLReadable, Jav >>> loadedModel = CountVectorizerModel.load(modelPath) >>> loadedModel.vocabulary == model.vocabulary True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True >>> fromVocabModel = CountVectorizerModel.from_vocabulary(["a", "b", "c"], ... inputCol="raw", outputCol="vectors") >>> fromVocabModel.transform(df).show(truncate=False) @@ -920,6 +929,8 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit >>> dctPath = temp_path + "/dct" >>> dct.save(dctPath) >>> loadedDtc = DCT.load(dctPath) + >>> loadedDtc.transform(df1).take(1) == dct.transform(df1).take(1) + True >>> loadedDtc.getInverse() False @@ -1003,6 +1014,8 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada >>> loadedEp = ElementwiseProduct.load(elementwiseProductPath) >>> loadedEp.getScalingVec() == ep.getScalingVec() True + >>> loadedEp.transform(df).take(1) == ep.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -1201,6 +1214,8 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures, Java >>> loadedHashingTF = HashingTF.load(hashingTFPath) >>> loadedHashingTF.getNumFeatures() == hashingTF.getNumFeatures() True + >>> loadedHashingTF.transform(df).take(1) == hashingTF.transform(df).take(1) + True >>> hashingTF.indexOf("b") 5 @@ -1292,6 +1307,10 @@ def getMinDocFreq(self): """ return self.getOrDefault(self.minDocFreq) + def __init__(self, *args): + super(_IDFParams, self).__init__(*args) + self._setDefault(minDocFreq=0) + @inherit_doc class IDF(JavaEstimator, _IDFParams, JavaMLReadable, JavaMLWritable): @@ -1345,7 +1364,6 @@ def __init__(self, minDocFreq=0, inputCol=None, outputCol=None): """ super(IDF, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.IDF", self.uid) - self._setDefault(minDocFreq=0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1445,6 +1463,10 @@ class _ImputerParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCols, Has "The placeholder for the missing values. All occurrences of missingValue " "will be imputed.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_ImputerParams, self).__init__(*args) + self._setDefault(strategy="mean", missingValue=float("nan"), relativeError=0.001) + @since("2.2.0") def getStrategy(self): """ @@ -1582,7 +1604,6 @@ def __init__(self, strategy="mean", missingValue=float("nan"), inputCols=None, """ super(Imputer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Imputer", self.uid) - self._setDefault(strategy="mean", missingValue=float("nan"), relativeError=0.001) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1811,6 +1832,8 @@ class MaxAbsScaler(JavaEstimator, _MaxAbsScalerParams, JavaMLReadable, JavaMLWri >>> loadedModel = MaxAbsScalerModel.load(modelPath) >>> loadedModel.maxAbs == model.maxAbs True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -1951,7 +1974,6 @@ def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1): """ super(MinHashLSH, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinHashLSH", self.uid) - self._setDefault(numHashTables=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2002,6 +2024,10 @@ class _MinMaxScalerParams(HasInputCol, HasOutputCol): max = Param(Params._dummy(), "max", "Upper bound of the output feature range", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_MinMaxScalerParams, self).__init__(*args) + self._setDefault(min=0.0, max=1.0) + @since("1.6.0") def getMin(self): """ @@ -2065,6 +2091,8 @@ class MinMaxScaler(JavaEstimator, _MinMaxScalerParams, JavaMLReadable, JavaMLWri True >>> loadedModel.originalMax == model.originalMax True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -2076,7 +2104,6 @@ def __init__(self, min=0.0, max=1.0, inputCol=None, outputCol=None): """ super(MinMaxScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinMaxScaler", self.uid) - self._setDefault(min=0.0, max=1.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2209,6 +2236,8 @@ class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWr >>> loadedNGram = NGram.load(ngramPath) >>> loadedNGram.getN() == ngram.getN() True + >>> loadedNGram.transform(df).take(1) == ngram.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -2289,6 +2318,8 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav >>> loadedNormalizer = Normalizer.load(normalizerPath) >>> loadedNormalizer.getP() == normalizer.getP() True + >>> loadedNormalizer.transform(df).take(1) == normalizer.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -2362,6 +2393,10 @@ class _OneHotEncoderParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCol dropLast = Param(Params._dummy(), "dropLast", "whether to drop the last category", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_OneHotEncoderParams, self).__init__(*args) + self._setDefault(handleInvalid="error", dropLast=True) + @since("2.3.0") def getDropLast(self): """ @@ -2422,6 +2457,8 @@ class OneHotEncoder(JavaEstimator, _OneHotEncoderParams, JavaMLReadable, JavaMLW >>> loadedModel = OneHotEncoderModel.load(modelPath) >>> loadedModel.categorySizes == model.categorySizes True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.3.0 """ @@ -2436,7 +2473,6 @@ def __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropL super(OneHotEncoder, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.feature.OneHotEncoder", self.uid) - self._setDefault(handleInvalid="error", dropLast=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2583,6 +2619,8 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead >>> loadedPx = PolynomialExpansion.load(polyExpansionPath) >>> loadedPx.getDegree() == px.getDegree() True + >>> loadedPx.transform(df).take(1) == px.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -2879,6 +2917,11 @@ class _RobustScalerParams(HasInputCol, HasOutputCol, HasRelativeError): withScaling = Param(Params._dummy(), "withScaling", "Whether to scale the data to " "quantile range", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_RobustScalerParams, self).__init__(*args) + self._setDefault(lower=0.25, upper=0.75, withCentering=False, withScaling=True, + relativeError=0.001) + @since("3.0.0") def getLower(self): """ @@ -2954,6 +2997,8 @@ class RobustScaler(JavaEstimator, _RobustScalerParams, JavaMLReadable, JavaMLWri True >>> loadedModel.range == model.range True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.0.0 """ @@ -2967,8 +3012,6 @@ def __init__(self, lower=0.25, upper=0.75, withCentering=False, withScaling=True """ super(RobustScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RobustScaler", self.uid) - self._setDefault(lower=0.25, upper=0.75, withCentering=False, withScaling=True, - relativeError=0.001) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -3113,6 +3156,8 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, True >>> loadedReTokenizer.getGaps() == reTokenizer.getGaps() True + >>> loadedReTokenizer.transform(df).take(1) == reTokenizer.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -3237,6 +3282,8 @@ class SQLTransformer(JavaTransformer, JavaMLReadable, JavaMLWritable): >>> loadedSqlTrans = SQLTransformer.load(sqlTransformerPath) >>> loadedSqlTrans.getStatement() == sqlTrans.getStatement() True + >>> loadedSqlTrans.transform(df).take(1) == sqlTrans.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -3291,6 +3338,10 @@ class _StandardScalerParams(HasInputCol, HasOutputCol): withStd = Param(Params._dummy(), "withStd", "Scale to unit standard deviation", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_StandardScalerParams, self).__init__(*args) + self._setDefault(withMean=False, withStd=True) + @since("1.4.0") def getWithMean(self): """ @@ -3348,6 +3399,8 @@ class StandardScaler(JavaEstimator, _StandardScalerParams, JavaMLReadable, JavaM True >>> loadedModel.mean == model.mean True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -3359,7 +3412,6 @@ def __init__(self, withMean=False, withStd=True, inputCol=None, outputCol=None): """ super(StandardScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.StandardScaler", self.uid) - self._setDefault(withMean=False, withStd=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -3513,6 +3565,8 @@ class StringIndexer(JavaEstimator, _StringIndexerParams, JavaMLReadable, JavaMLW >>> loadedInverter = IndexToString.load(indexToStringPath) >>> loadedInverter.getLabels() == inverter.getLabels() True + >>> loadedModel.transform(stringIndDf).take(1) == model.transform(stringIndDf).take(1) + True >>> stringIndexer.getStringOrderType() 'frequencyDesc' >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed", handleInvalid="error", @@ -3798,6 +3852,8 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, True >>> loadedRemover.getCaseSensitive() == remover.getCaseSensitive() True + >>> loadedRemover.transform(df).take(1) == remover.transform(df).take(1) + True >>> df2 = spark.createDataFrame([(["a", "b", "c"], ["a", "b"])], ["text1", "text2"]) >>> remover2 = StopWordsRemover(stopWords=["b"]) >>> remover2.setInputCols(["text1", "text2"]).setOutputCols(["words1", "words2"]) @@ -4109,6 +4165,10 @@ class _VectorIndexerParams(HasInputCol, HasOutputCol, HasHandleInvalid): "of categories of the feature).", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_VectorIndexerParams, self).__init__(*args) + self._setDefault(maxCategories=20, handleInvalid="error") + @since("1.4.0") def getMaxCategories(self): """ @@ -4189,6 +4249,8 @@ class VectorIndexer(JavaEstimator, _VectorIndexerParams, JavaMLReadable, JavaMLW True >>> loadedModel.categoryMaps == model.categoryMaps True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True >>> dfWithInvalid = spark.createDataFrame([(Vectors.dense([3.0, 1.0]),)], ["a"]) >>> indexer.getHandleInvalid() 'error' @@ -4209,7 +4271,6 @@ def __init__(self, maxCategories=20, inputCol=None, outputCol=None, handleInvali """ super(VectorIndexer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.VectorIndexer", self.uid) - self._setDefault(maxCategories=20, handleInvalid="error") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -4332,6 +4393,8 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J True >>> loadedVs.getNames() == vs.getNames() True + >>> loadedVs.transform(df).take(1) == vs.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -4431,6 +4494,11 @@ class _Word2VecParams(HasStepSize, HasMaxIter, HasSeed, HasInputCol, HasOutputCo "be divided into chunks up to the size.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_Word2VecParams, self).__init__(*args) + self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + windowSize=5, maxSentenceLength=1000) + @since("1.4.0") def getVectorSize(self): """ @@ -4524,6 +4592,8 @@ class Word2Vec(JavaEstimator, _Word2VecParams, JavaMLReadable, JavaMLWritable): True >>> loadedModel.getVectors().first().vector == model.getVectors().first().vector True + >>> loadedModel.transform(doc).take(1) == model.transform(doc).take(1) + True .. versionadded:: 1.4.0 """ @@ -4537,8 +4607,6 @@ def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, """ super(Word2Vec, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid) - self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - windowSize=5, maxSentenceLength=1000) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -4730,6 +4798,8 @@ class PCA(JavaEstimator, _PCAParams, JavaMLReadable, JavaMLWritable): True >>> loadedModel.explainedVariance == model.explainedVariance True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -4846,6 +4916,11 @@ class _RFormulaParams(HasFeaturesCol, HasLabelCol, HasHandleInvalid): "additional bucket, at index numLabels).", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_RFormulaParams, self).__init__(*args) + self._setDefault(forceIndexLabel=False, stringIndexerOrderType="frequencyDesc", + handleInvalid="error") + @since("1.5.0") def getFormula(self): """ @@ -4948,8 +5023,6 @@ def __init__(self, formula=None, featuresCol="features", labelCol="label", """ super(RFormula, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RFormula", self.uid) - self._setDefault(forceIndexLabel=False, stringIndexerOrderType="frequencyDesc", - handleInvalid="error") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5058,6 +5131,11 @@ class _SelectorParams(HasFeaturesCol, HasOutputCol, HasLabelCol): fwe = Param(Params._dummy(), "fwe", "The upper bound of the expected family-wise error rate.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_SelectorParams, self).__init__(*args) + self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, + fpr=0.05, fdr=0.05, fwe=0.05) + @since("2.1.0") def getSelectorType(self): """ @@ -5257,6 +5335,8 @@ class ANOVASelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = ANOVASelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ @@ -5272,8 +5352,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(ANOVASelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.ANOVASelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5356,6 +5434,8 @@ class ChiSqSelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = ChiSqSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -5371,8 +5451,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(ChiSqSelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.ChiSqSelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5459,6 +5537,8 @@ class FValueSelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = FValueSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ @@ -5474,8 +5554,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(FValueSelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.FValueSelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5652,6 +5730,8 @@ class VarianceThresholdSelector(JavaEstimator, _VarianceThresholdSelectorParams, >>> loadedModel = VarianceThresholdSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index a1a8a4e3e3ac4..37d3b6eec02d0 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -55,8 +55,8 @@ class _FPGrowthParams(HasPredictionCol): "but will affect the association rules generation.", typeConverter=TypeConverters.toFloat) - def __init__(self): - super(_FPGrowthParams, self).__init__() + def __init__(self, *args): + super(_FPGrowthParams, self).__init__(*args) self._setDefault(minSupport=0.3, minConfidence=0.8, itemsCol="items", predictionCol="prediction") @@ -197,6 +197,11 @@ class FPGrowth(JavaEstimator, _FPGrowthParams, JavaMLWritable, JavaMLReadable): >>> new_data = spark.createDataFrame([(["t", "s"], )], ["items"]) >>> sorted(fpm.transform(new_data).first().newPrediction) ['x', 'y', 'z'] + >>> model_path = temp_path + "/fpm_model" + >>> fpm.save(model_path) + >>> model2 = FPGrowthModel.load(model_path) + >>> fpm.transform(data).take(1) == model2.transform(data).take(1) + True .. versionadded:: 2.2.0 """ diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 99d80aa867bda..62b856046234a 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -46,6 +46,10 @@ class _ALSModelParams(HasPredictionCol, HasBlockSize): "Supported values: 'nan', 'drop'.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_ALSModelParams, self).__init__(*args) + self._setDefault(blockSize=4096) + @since("1.4.0") def getUserCol(self): """ @@ -99,6 +103,14 @@ class _ALSParams(_ALSModelParams, HasMaxIter, HasRegParam, HasCheckpointInterval "StorageLevel for ALS model factors.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_ALSParams, self).__init__(*args) + self._setDefault(rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, + implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", + ratingCol="rating", nonnegative=False, checkpointInterval=10, + intermediateStorageLevel="MEMORY_AND_DISK", + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + @since("1.4.0") def getRank(self): """ @@ -275,6 +287,8 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): True >>> sorted(model.itemFactors.collect()) == sorted(model2.itemFactors.collect()) True + >>> model.transform(test).take(1) == model2.transform(test).take(1) + True .. versionadded:: 1.4.0 """ @@ -294,12 +308,6 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB """ super(ALS, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.recommendation.ALS", self.uid) - self._setDefault(rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, - implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", - ratingCol="rating", nonnegative=False, checkpointInterval=10, - intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", - blockSize=4096) kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 6fe6486c5a04a..4a8d1530b8a6f 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -104,8 +104,8 @@ class _LinearRegressionParams(_PredictorParams, HasRegParam, HasElasticNetParam, "robustness. Must be > 1.0. Only valid when loss is huber", typeConverter=TypeConverters.toFloat) - def __init__(self): - super(_LinearRegressionParams, self).__init__() + def __init__(self, *args): + super(_LinearRegressionParams, self).__init__(*args) self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35, blockSize=1) @@ -190,6 +190,8 @@ class LinearRegression(_JavaRegressor, _LinearRegressionParams, JavaMLWritable, True >>> model.intercept == model2.intercept True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.numFeatures 1 >>> model.write().format("pmml").save(model_path + "_2") @@ -622,8 +624,8 @@ class _IsotonicRegressionParams(HasFeaturesCol, HasLabelCol, HasPredictionCol, H "The index of the feature if featuresCol is a vector column, no effect otherwise.", typeConverter=TypeConverters.toInt) - def __init__(self): - super(_IsotonicRegressionParams, self).__init__() + def __init__(self, *args): + super(_IsotonicRegressionParams, self).__init__(*args) self._setDefault(isotonic=True, featureIndex=0) def getIsotonic(self): @@ -675,6 +677,8 @@ class IsotonicRegression(JavaEstimator, _IsotonicRegressionParams, HasWeightCol, True >>> model.predictions == model2.predictions True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.6.0 """ @@ -814,8 +818,8 @@ class _DecisionTreeRegressorParams(_DecisionTreeParams, _TreeRegressorParams, Ha .. versionadded:: 3.0.0 """ - def __init__(self): - super(_DecisionTreeRegressorParams, self).__init__() + def __init__(self, *args): + super(_DecisionTreeRegressorParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", leafCol="", minWeightFractionPerNode=0.0) @@ -876,7 +880,8 @@ class DecisionTreeRegressor(_JavaRegressor, _DecisionTreeRegressorParams, JavaML True >>> model.transform(test1).head().variance 0.0 - + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> df3 = spark.createDataFrame([ ... (1.0, 0.2, Vectors.dense(1.0)), ... (1.0, 0.8, Vectors.dense(1.0)), @@ -1060,8 +1065,8 @@ class _RandomForestRegressorParams(_RandomForestParams, _TreeRegressorParams): .. versionadded:: 3.0.0 """ - def __init__(self): - super(_RandomForestRegressorParams, self).__init__() + def __init__(self, *args): + super(_RandomForestRegressorParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, numTrees=20, @@ -1127,6 +1132,8 @@ class RandomForestRegressor(_JavaRegressor, _RandomForestRegressorParams, JavaML >>> model2 = RandomForestRegressionModel.load(model_path) >>> model.featureImportances == model2.featureImportances True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.4.0 """ @@ -1319,8 +1326,8 @@ class _GBTRegressorParams(_GBTParams, _TreeRegressorParams): "Supported options: " + ", ".join(supportedLossTypes), typeConverter=TypeConverters.toString) - def __init__(self): - super(_GBTRegressorParams, self).__init__() + def __init__(self, *args): + super(_GBTRegressorParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, @@ -1390,6 +1397,8 @@ class GBTRegressor(_JavaRegressor, _GBTRegressorParams, JavaMLWritable, JavaMLRe True >>> model.treeWeights == model2.treeWeights True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.trees [DecisionTreeRegressionModel...depth=..., DecisionTreeRegressionModel...] >>> validation = spark.createDataFrame([(0.0, Vectors.dense(-1.0))], @@ -1642,8 +1651,8 @@ class _AFTSurvivalRegressionParams(_PredictorParams, HasMaxIter, HasTol, HasFitI "corresponding quantileProbabilities if it is set.", typeConverter=TypeConverters.toString) - def __init__(self): - super(_AFTSurvivalRegressionParams, self).__init__() + def __init__(self, *args): + super(_AFTSurvivalRegressionParams, self).__init__(*args) self._setDefault(censorCol="censor", quantileProbabilities=[0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99], maxIter=100, tol=1E-6, blockSize=1) @@ -1722,6 +1731,8 @@ class AFTSurvivalRegression(_JavaRegressor, _AFTSurvivalRegressionParams, True >>> model.scale == model2.scale True + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -1906,8 +1917,8 @@ class _GeneralizedLinearRegressionParams(_PredictorParams, HasFitIntercept, HasM "or empty, we treat all instance offsets as 0.0", typeConverter=TypeConverters.toString) - def __init__(self): - super(_GeneralizedLinearRegressionParams, self).__init__() + def __init__(self, *args): + super(_GeneralizedLinearRegressionParams, self).__init__(*args) self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls", variancePower=0.0, aggregationDepth=2) @@ -2025,6 +2036,8 @@ class GeneralizedLinearRegression(_JavaRegressor, _GeneralizedLinearRegressionPa True >>> model.coefficients[0] == model2.coefficients[0] True + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -2391,7 +2404,7 @@ def __repr__(self): class _FactorizationMachinesParams(_PredictorParams, HasMaxIter, HasStepSize, HasTol, - HasSolver, HasSeed, HasFitIntercept, HasRegParam): + HasSolver, HasSeed, HasFitIntercept, HasRegParam, HasWeightCol): """ Params for :py:class:`FMRegressor`, :py:class:`FMRegressionModel`, :py:class:`FMClassifier` and :py:class:`FMClassifierModel`. @@ -2416,8 +2429,8 @@ class _FactorizationMachinesParams(_PredictorParams, HasMaxIter, HasStepSize, Ha solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) - def __init__(self): - super(_FactorizationMachinesParams, self).__init__() + def __init__(self, *args): + super(_FactorizationMachinesParams, self).__init__(*args) self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, tol=1e-6, solver="adamW") @@ -2495,6 +2508,17 @@ class FMRegressor(_JavaRegressor, _FactorizationMachinesParams, JavaMLWritable, DenseVector([0.9978]) >>> model.factors DenseMatrix(1, 2, [0.0173, 0.0021], 1) + >>> model_path = temp_path + "/fm_model" + >>> model.save(model_path) + >>> model2 = FMRegressionModel.load(model_path) + >>> model2.intercept + -0.0032501766849261557 + >>> model2.linear + DenseVector([0.9978]) + >>> model2.factors + DenseMatrix(1, 2, [0.0173, 0.0021], 1) + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 3.0.0 """ diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index e1abd59a2d7b2..44731568b6d1f 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -359,16 +359,14 @@ def test_java_params(self): and issubclass(cls, JavaParams) and not inspect.isabstract(cls) \ and not re.match("_?Java", name) and name != '_LSH' \ and name != '_Selector': - # NOTE: disable check_params_exist until there is parity with Scala API - - check_params(self, cls(), check_params_exist=False) + check_params(self, cls(), check_params_exist=True) # Additional classes that need explicit construction from pyspark.ml.feature import CountVectorizerModel, StringIndexerModel check_params(self, CountVectorizerModel.from_vocabulary(['a'], 'input'), - check_params_exist=False) + check_params_exist=True) check_params(self, StringIndexerModel.from_labels(['a', 'b'], 'input'), - check_params_exist=False) + check_params_exist=True) if __name__ == "__main__": diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 7f3d942e2e456..d7800e0c9020e 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -206,6 +206,10 @@ class _CrossValidatorParams(_ValidatorParams): "with range [0, numFolds) and Spark will throw exception on out-of-range " + "fold numbers.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_CrossValidatorParams, self).__init__(*args) + self._setDefault(numFolds=3, foldCol="") + @since("1.4.0") def getNumFolds(self): """ @@ -262,6 +266,8 @@ class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollec [0.5, ... >>> evaluator.evaluate(cvModel.transform(dataset)) 0.8333... + >>> evaluator.evaluate(cvModelRead.transform(dataset)) + 0.8333... .. versionadded:: 1.4.0 """ @@ -274,7 +280,7 @@ def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numF seed=None, parallelism=1, collectSubModels=False, foldCol="") """ super(CrossValidator, self).__init__() - self._setDefault(numFolds=3, parallelism=1, foldCol="") + self._setDefault(parallelism=1) kwargs = self._input_kwargs self._set(**kwargs) @@ -600,6 +606,10 @@ class _TrainValidationSplitParams(_ValidatorParams): trainRatio = Param(Params._dummy(), "trainRatio", "Param for ratio between train and\ validation data. Must be between 0 and 1.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_TrainValidationSplitParams, self).__init__(*args) + self._setDefault(trainRatio=0.75) + @since("2.0.0") def getTrainRatio(self): """ @@ -645,8 +655,11 @@ class TrainValidationSplit(Estimator, _TrainValidationSplitParams, HasParallelis [0.5, ... >>> evaluator.evaluate(tvsModel.transform(dataset)) 0.833... + >>> evaluator.evaluate(tvsModelRead.transform(dataset)) + 0.833... .. versionadded:: 2.0.0 + """ @keyword_only @@ -657,7 +670,7 @@ def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, trai parallelism=1, collectSubModels=False, seed=None) """ super(TrainValidationSplit, self).__init__() - self._setDefault(trainRatio=0.75, parallelism=1) + self._setDefault(parallelism=1) kwargs = self._input_kwargs self._set(**kwargs) From f3b10f526b2e700680b7c56bfdaffbcf0eb0f269 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 3 Aug 2020 16:05:54 +0000 Subject: [PATCH 358/384] [SPARK-32290][SQL][FOLLOWUP] Add version for the SQL config `spark.sql.optimizeNullAwareAntiJoin` ### What changes were proposed in this pull request? Add the version `3.1.0` for the SQL config `spark.sql.optimizeNullAwareAntiJoin`. ### Why are the changes needed? To inform users when the config was added, for example on the page http://spark.apache.org/docs/latest/configuration.html. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By compiling and running `./dev/scalastyle`. Closes #29335 from MaxGekk/leanken-SPARK-32290-followup. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6b407e65a9a72..bae41114caf1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2686,6 +2686,7 @@ object SQLConf { "optimized from O(M*N) calculation into O(M) calculation " + "using Hash lookup instead of Looping lookup." + "Only support for singleColumn NAAJ for now.") + .version("3.1.0") .booleanConf .createWithDefault(true) From 9bbe8c7418137e03df8bfbd80e7c569192d3711a Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 4 Aug 2020 11:23:28 +0900 Subject: [PATCH 359/384] [MINOR][SQL] Fix versions in the SQL migration guide for Spark 3.1 ### What changes were proposed in this pull request? Change _To restore the behavior before Spark **3.0**_ to _To restore the behavior before Spark **3.1**_ in the SQL migration guide while telling about the behaviour before new version 3.1. ### Why are the changes needed? To have correct info in the SQL migration guide. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? N/A Closes #29336 from MaxGekk/fix-version-in-sql-migration. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- docs/sql-migration-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 7c9c0a726c881..709bde3206e3d 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -24,9 +24,9 @@ license: | ## Upgrading from Spark SQL 3.0 to 3.1 - - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.integerGroupingId` to `true`. + - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.integerGroupingId` to `true`. - - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. + - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.1, you can set `spark.sql.ui.explainMode` to `extended`. - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. From 7deb67c28f948cca4e768317ade6d68d2534408f Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 4 Aug 2020 12:45:06 +0900 Subject: [PATCH 360/384] [SPARK-32160][CORE][PYSPARK][FOLLOWUP] Change the config name to switch allow/disallow SparkContext in executors ### What changes were proposed in this pull request? This is a follow-up of #29278. This PR changes the config name to switch allow/disallow `SparkContext` in executors as per the comment https://github.com/apache/spark/pull/29278#pullrequestreview-460256338. ### Why are the changes needed? The config name `spark.executor.allowSparkContext` is more reasonable. ### Does this PR introduce _any_ user-facing change? Yes, the config name is changed. ### How was this patch tested? Updated tests. Closes #29340 from ueshin/issues/SPARK-32160/change_config_name. Authored-by: Takuya UESHIN Signed-off-by: HyukjinKwon --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- .../main/scala/org/apache/spark/internal/config/package.scala | 4 ++-- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 2 +- docs/core-migration-guide.md | 2 +- python/pyspark/context.py | 2 +- python/pyspark/tests/test_context.py | 2 +- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 4 ++-- .../scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala | 4 ++-- 8 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9ecf316beeaa1..78f509c670839 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -83,7 +83,7 @@ class SparkContext(config: SparkConf) extends Logging { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() - if (!config.get(ALLOW_SPARK_CONTEXT_IN_EXECUTORS)) { + if (!config.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) { // In order to prevent SparkContext from being created in executors. SparkContext.assertOnDriver() } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index fdc9253ce9b02..200cde0a2d3ed 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1909,8 +1909,8 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val ALLOW_SPARK_CONTEXT_IN_EXECUTORS = - ConfigBuilder("spark.driver.allowSparkContextInExecutors") + private[spark] val EXECUTOR_ALLOW_SPARK_CONTEXT = + ConfigBuilder("spark.executor.allowSparkContext") .doc("If set to true, SparkContext can be created in executors.") .version("3.0.1") .booleanConf diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 1f7aa8eec8942..ebdf2f59a2770 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -952,7 +952,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc.range(0, 1).foreach { _ => new SparkContext(new SparkConf().setAppName("test").setMaster("local") - .set(ALLOW_SPARK_CONTEXT_IN_EXECUTORS, true)).stop() + .set(EXECUTOR_ALLOW_SPARK_CONTEXT, true)).stop() } } } diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index b2a08502d0d6f..11d3e0019617f 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -24,7 +24,7 @@ license: | ## Upgrading from Core 3.0 to 3.1 -- In Spark 3.0 and below, `SparkContext` can be created in executors. Since Spark 3.1, an exception will be thrown when creating `SparkContext` in executors. You can allow it by setting the configuration `spark.driver.allowSparkContextInExecutors` when creating `SparkContext` in executors. +- In Spark 3.0 and below, `SparkContext` can be created in executors. Since Spark 3.1, an exception will be thrown when creating `SparkContext` in executors. You can allow it by setting the configuration `spark.executor.allowSparkContext` when creating `SparkContext` in executors. ## Upgrading from Core 2.4 to 3.0 diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 0816657692826..55a5657b64055 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -118,7 +118,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ValueError:... """ if (conf is None or - conf.get("spark.driver.allowSparkContextInExecutors", "false").lower() != "true"): + conf.get("spark.executor.allowSparkContext", "false").lower() != "true"): # In order to prevent SparkContext from being created in executors. SparkContext._assert_on_driver() diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 64fe3837e7697..f398cec344725 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -279,7 +279,7 @@ def test_allow_to_create_spark_context_in_executors(self): # SPARK-32160: SparkContext can be created in executors if the config is set. def create_spark_context(): - conf = SparkConf().set("spark.driver.allowSparkContextInExecutors", "true") + conf = SparkConf().set("spark.executor.allowSparkContext", "true") with SparkContext(conf=conf): pass diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 306c3235b0bc0..e5d53f5fd4c65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -29,7 +29,7 @@ import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.ALLOW_SPARK_CONTEXT_IN_EXECUTORS +import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog @@ -904,7 +904,7 @@ object SparkSession extends Logging { val sparkConf = new SparkConf() options.foreach { case (k, v) => sparkConf.set(k, v) } - if (!sparkConf.get(ALLOW_SPARK_CONTEXT_IN_EXECUTORS)) { + if (!sparkConf.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) { assertOnDriver() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index cc261a9ed3598..9da32d02aa723 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} -import org.apache.spark.internal.config.ALLOW_SPARK_CONTEXT_IN_EXECUTORS +import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf._ @@ -277,7 +277,7 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { session.range(1).foreach { v => SparkSession.builder.master("local") - .config(ALLOW_SPARK_CONTEXT_IN_EXECUTORS.key, true).getOrCreate().stop() + .config(EXECUTOR_ALLOW_SPARK_CONTEXT.key, true).getOrCreate().stop() () } } From 1597d8fcd4c68e723eb3152335298c7d05155643 Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Tue, 4 Aug 2020 04:41:19 +0000 Subject: [PATCH 361/384] [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT ### What changes were proposed in this pull request? This PR is related to https://github.com/apache/spark/pull/26656. https://github.com/apache/spark/pull/26656 only support use FILTER clause on aggregate expression without DISTINCT. This PR will enhance this feature when one or more DISTINCT aggregate expressions which allows the use of the FILTER clause. Such as: ``` select sum(distinct id) filter (where sex = 'man') from student; select class_id, sum(distinct id) filter (where sex = 'man') from student group by class_id; select count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student; select class_id, count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student group by class_id; select sum(distinct id), sum(distinct id) filter (where sex = 'man') from student; select class_id, sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id; select class_id, count(id), count(id) filter (where class_id = 1), sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id; ``` ### Why are the changes needed? Spark SQL only support use FILTER clause on aggregate expression without DISTINCT. This PR support Filter expression allows simultaneous use of DISTINCT ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Exists and new UT Closes #29291 from beliefer/support-distinct-with-filter. Lead-authored-by: gengjiaan Co-authored-by: beliefer Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 12 +- .../optimizer/RewriteDistinctAggregates.scala | 151 ++++++-- .../analysis/AnalysisErrorSuite.scala | 5 - .../sql-tests/inputs/group-by-filter.sql | 46 ++- .../inputs/postgreSQL/aggregates_part3.sql | 7 +- .../inputs/postgreSQL/groupingsets.sql | 5 +- .../sql-tests/results/group-by-filter.sql.out | 363 +++++++++++++++++- .../postgreSQL/aggregates_part3.sql.out | 16 +- .../results/postgreSQL/groupingsets.sql.out | 21 +- 9 files changed, 554 insertions(+), 72 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e58b0ae64784d..477863a1b86d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1973,15 +1973,9 @@ class Analyzer( } // We get an aggregate function, we need to wrap it in an AggregateExpression. case agg: AggregateFunction => - // TODO: SPARK-30276 Support Filter expression allows simultaneous use of DISTINCT - if (filter.isDefined) { - if (isDistinct) { - failAnalysis("DISTINCT and FILTER cannot be used in aggregate functions " + - "at the same time") - } else if (!filter.get.deterministic) { - failAnalysis("FILTER expression is non-deterministic, " + - "it cannot be used in aggregate functions") - } + if (filter.isDefined && !filter.get.deterministic) { + failAnalysis("FILTER expression is non-deterministic, " + + "it cannot be used in aggregate functions") } AggregateExpression(agg, Complete, isDistinct, filter) // This function is not an aggregate function, just return the resolved one. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index 15aa02ff677de..af3a8fe684bb3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, Complete} +import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types.IntegerType @@ -81,10 +81,10 @@ import org.apache.spark.sql.types.IntegerType * COUNT(DISTINCT cat1) as cat1_cnt, * COUNT(DISTINCT cat2) as cat2_cnt, * SUM(value) FILTER (WHERE id > 1) AS total - * FROM - * data - * GROUP BY - * key + * FROM + * data + * GROUP BY + * key * }}} * * This translates to the following (pseudo) logical plan: @@ -93,7 +93,7 @@ import org.apache.spark.sql.types.IntegerType * key = ['key] * functions = [COUNT(DISTINCT 'cat1), * COUNT(DISTINCT 'cat2), - * sum('value) with FILTER('id > 1)] + * sum('value) FILTER (WHERE 'id > 1)] * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) * LocalTableScan [...] * }}} @@ -108,7 +108,7 @@ import org.apache.spark.sql.types.IntegerType * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) * Aggregate( * key = ['key, 'cat1, 'cat2, 'gid] - * functions = [sum('value) with FILTER('id > 1)] + * functions = [sum('value) FILTER (WHERE 'id > 1)] * output = ['key, 'cat1, 'cat2, 'gid, 'total]) * Expand( * projections = [('key, null, null, 0, cast('value as bigint), 'id), @@ -118,6 +118,49 @@ import org.apache.spark.sql.types.IntegerType * LocalTableScan [...] * }}} * + * Third example: aggregate function with distinct and filter clauses (in sql): + * {{{ + * SELECT + * COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt, + * COUNT(DISTINCT cat2) FILTER (WHERE id > 2) as cat2_cnt, + * SUM(value) FILTER (WHERE id > 3) AS total + * FROM + * data + * GROUP BY + * key + * }}} + * + * This translates to the following (pseudo) logical plan: + * {{{ + * Aggregate( + * key = ['key] + * functions = [COUNT(DISTINCT 'cat1) FILTER (WHERE 'id > 1), + * COUNT(DISTINCT 'cat2) FILTER (WHERE 'id > 2), + * sum('value) FILTER (WHERE 'id > 3)] + * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) + * LocalTableScan [...] + * }}} + * + * This rule rewrites this logical plan to the following (pseudo) logical plan: + * {{{ + * Aggregate( + * key = ['key] + * functions = [count(if (('gid = 1) and 'max_cond1) 'cat1 else null), + * count(if (('gid = 2) and 'max_cond2) 'cat2 else null), + * first(if (('gid = 0)) 'total else null) ignore nulls] + * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) + * Aggregate( + * key = ['key, 'cat1, 'cat2, 'gid] + * functions = [max('cond1), max('cond2), sum('value) FILTER (WHERE 'id > 3)] + * output = ['key, 'cat1, 'cat2, 'gid, 'max_cond1, 'max_cond2, 'total]) + * Expand( + * projections = [('key, null, null, 0, null, null, cast('value as bigint), 'id), + * ('key, 'cat1, null, 1, 'id > 1, null, null, null), + * ('key, null, 'cat2, 2, null, 'id > 2, null, null)] + * output = ['key, 'cat1, 'cat2, 'gid, 'cond1, 'cond2, 'value, 'id]) + * LocalTableScan [...] + * }}} + * * The rule does the following things here: * 1. Expand the data. There are three aggregation groups in this query: * i. the non-distinct group; @@ -126,15 +169,24 @@ import org.apache.spark.sql.types.IntegerType * An expand operator is inserted to expand the child data for each group. The expand will null * out all unused columns for the given group; this must be done in order to ensure correctness * later on. Groups can by identified by a group id (gid) column added by the expand operator. + * If distinct group exists filter clause, the expand will calculate the filter and output it's + * result (e.g. cond1) which will be used to calculate the global conditions (e.g. max_cond1) + * equivalent to filter clauses. * 2. De-duplicate the distinct paths and aggregate the non-aggregate path. The group by clause of * this aggregate consists of the original group by clause, all the requested distinct columns * and the group id. Both de-duplication of distinct column and the aggregation of the * non-distinct group take advantage of the fact that we group by the group id (gid) and that we - * have nulled out all non-relevant columns the given group. + * have nulled out all non-relevant columns the given group. If distinct group exists filter + * clause, we will use max to aggregate the results (e.g. cond1) of the filter output in the + * previous step. These aggregate will output the global conditions (e.g. max_cond1) equivalent + * to filter clauses. * 3. Aggregating the distinct groups and combining this with the results of the non-distinct - * aggregation. In this step we use the group id to filter the inputs for the aggregate - * functions. The result of the non-distinct group are 'aggregated' by using the first operator, - * it might be more elegant to use the native UDAF merge mechanism for this in the future. + * aggregation. In this step we use the group id and the global condition to filter the inputs + * for the aggregate functions. If the global condition (e.g. max_cond1) is true, it means at + * least one row of a distinct value satisfies the filter. This distinct value should be included + * in the aggregate function. The result of the non-distinct group are 'aggregated' by using + * the first operator, it might be more elegant to use the native UDAF merge mechanism for this + * in the future. * * This rule duplicates the input data by two or more times (# distinct groups + an optional * non-distinct group). This will put quite a bit of memory pressure of the used aggregate and @@ -144,28 +196,24 @@ import org.apache.spark.sql.types.IntegerType */ object RewriteDistinctAggregates extends Rule[LogicalPlan] { - private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = { - val distinctAggs = exprs.flatMap { _.collect { - case ae: AggregateExpression if ae.isDistinct => ae - }} - // We need at least two distinct aggregates for this rule because aggregation - // strategy can handle a single distinct group. + private def mayNeedtoRewrite(a: Aggregate): Boolean = { + val aggExpressions = collectAggregateExprs(a) + val distinctAggs = aggExpressions.filter(_.isDistinct) + // We need at least two distinct aggregates or the single distinct aggregate group exists filter + // clause for this rule because aggregation strategy can handle a single distinct aggregate + // group without filter clause. // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a). - distinctAggs.size > 1 + distinctAggs.size > 1 || distinctAggs.exists(_.filter.isDefined) } def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a) + case a: Aggregate if mayNeedtoRewrite(a) => rewrite(a) } def rewrite(a: Aggregate): Aggregate = { - // Collect all aggregate expressions. - val aggExpressions = a.aggregateExpressions.flatMap { e => - e.collect { - case ae: AggregateExpression => ae - } - } + val aggExpressions = collectAggregateExprs(a) + val distinctAggs = aggExpressions.filter(_.isDistinct) // Extract distinct aggregate expressions. val distinctAggGroups = aggExpressions.filter(_.isDistinct).groupBy { e => @@ -184,8 +232,8 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { } } - // Aggregation strategy can handle queries with a single distinct group. - if (distinctAggGroups.size > 1) { + // Aggregation strategy can handle queries with a single distinct group without filter clause. + if (distinctAggGroups.size > 1 || distinctAggs.exists(_.filter.isDefined)) { // Create the attributes for the grouping id and the group by clause. val gid = AttributeReference("gid", IntegerType, nullable = false)() val groupByMap = a.groupingExpressions.collect { @@ -195,7 +243,13 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val groupByAttrs = groupByMap.map(_._2) // Functions used to modify aggregate functions and their inputs. - def evalWithinGroup(id: Literal, e: Expression) = If(EqualTo(gid, id), e, nullify(e)) + def evalWithinGroup(id: Literal, e: Expression, condition: Option[Expression]) = + if (condition.isDefined) { + If(And(EqualTo(gid, id), condition.get), e, nullify(e)) + } else { + If(EqualTo(gid, id), e, nullify(e)) + } + def patchAggregateFunctionChildren( af: AggregateFunction)( attrs: Expression => Option[Expression]): AggregateFunction = { @@ -207,13 +261,28 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair) val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2) + // Setup all the filters in distinct aggregate. + val (distinctAggFilters, distinctAggFilterAttrs, maxConds) = distinctAggs.collect { + case AggregateExpression(_, _, _, filter, _) if filter.isDefined => + val (e, attr) = expressionAttributePair(filter.get) + val aggregateExp = Max(attr).toAggregateExpression() + (e, attr, Alias(aggregateExp, attr.name)()) + }.unzip3 // Setup expand & aggregate operators for distinct aggregate expressions. val distinctAggChildAttrLookup = distinctAggChildAttrMap.toMap + val distinctAggFilterAttrLookup = distinctAggFilters.zip(maxConds.map(_.toAttribute)).toMap val distinctAggOperatorMap = distinctAggGroups.toSeq.zipWithIndex.map { case ((group, expressions), i) => val id = Literal(i + 1) + // Expand projection for filter + val filters = expressions.filter(_.filter.isDefined).map(_.filter.get) + val filterProjection = distinctAggFilters.map { + case e if filters.contains(e) => e + case e => nullify(e) + } + // Expand projection val projection = distinctAggChildren.map { case e if group.contains(e) => e @@ -224,12 +293,17 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val operators = expressions.map { e => val af = e.aggregateFunction val naf = patchAggregateFunctionChildren(af) { x => - distinctAggChildAttrLookup.get(x).map(evalWithinGroup(id, _)) + val condition = if (e.filter.isDefined) { + e.filter.map(distinctAggFilterAttrLookup.get(_)).get + } else { + None + } + distinctAggChildAttrLookup.get(x).map(evalWithinGroup(id, _, condition)) } - (e, e.copy(aggregateFunction = naf, isDistinct = false)) + (e, e.copy(aggregateFunction = naf, isDistinct = false, filter = None)) } - (projection, operators) + (projection ++ filterProjection, operators) } // Setup expand for the 'regular' aggregate expressions. @@ -257,7 +331,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { // Select the result of the first aggregate in the last aggregate. val result = AggregateExpression( - aggregate.First(evalWithinGroup(regularGroupId, operator.toAttribute), true), + aggregate.First(evalWithinGroup(regularGroupId, operator.toAttribute, None), true), mode = Complete, isDistinct = false) @@ -280,6 +354,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { Seq(a.groupingExpressions ++ distinctAggChildren.map(nullify) ++ Seq(regularGroupId) ++ + distinctAggFilters.map(nullify) ++ regularAggChildren) } else { Seq.empty[Seq[Expression]] @@ -297,7 +372,8 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { // Construct the expand operator. val expand = Expand( regularAggProjection ++ distinctAggProjections, - groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ regularAggChildAttrMap.map(_._2), + groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ distinctAggFilterAttrs ++ + regularAggChildAttrMap.map(_._2), a.child) // Construct the first aggregate operator. This de-duplicates all the children of @@ -305,7 +381,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val firstAggregateGroupBy = groupByAttrs ++ distinctAggChildAttrs :+ gid val firstAggregate = Aggregate( firstAggregateGroupBy, - firstAggregateGroupBy ++ regularAggOperatorMap.map(_._2), + firstAggregateGroupBy ++ maxConds ++ regularAggOperatorMap.map(_._2), expand) // Construct the second aggregate @@ -331,6 +407,13 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { } } + private def collectAggregateExprs(a: Aggregate): Seq[AggregateExpression] = { + // Collect all aggregate expressions. + a.aggregateExpressions.flatMap { _.collect { + case ae: AggregateExpression => ae + }} + } + private def nullify(e: Expression) = Literal.create(null, e.dataType) private def expressionAttributePair(e: Expression) = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 166ffec44a60d..a99f7e2be6e7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -207,11 +207,6 @@ class AnalysisErrorSuite extends AnalysisTest { "FILTER (WHERE c > 1)"), "FILTER predicate specified, but aggregate is not an aggregate function" :: Nil) - errorTest( - "DISTINCT aggregate function with filter predicate", - CatalystSqlParser.parsePlan("SELECT count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"), - "DISTINCT and FILTER cannot be used in aggregate functions at the same time" :: Nil) - errorTest( "non-deterministic filter predicate in aggregate functions", CatalystSqlParser.parsePlan("SELECT count(a) FILTER (WHERE rand(int(c)) > 1) FROM TaBlE2"), diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql index 4c1816e93b083..24d303621faea 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql @@ -36,8 +36,13 @@ SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp; SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp; SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp; SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp; +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp; +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData; +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp; -- Aggregate with filter and non-empty GroupBy expressions. SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a; @@ -47,8 +52,11 @@ SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_date("2003-01-01")) FROM emp GROUP BY dept_id; SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_timestamp("2003-01-01 00:00:00")) FROM emp GROUP BY dept_id; SELECT dept_id, SUM(salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2003-01-01") FROM emp GROUP BY dept_id; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id; +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b; -- Aggregate with filter and grouped by literals. SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1; @@ -61,13 +69,24 @@ select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id; select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id; +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id; -- Aggregate with filter and grouped by literals (hash aggregate), here the input table is filtered using WHERE. SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1; @@ -81,9 +100,8 @@ SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1; SELECT a + 1 + 1, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY a + 1; -- Aggregate with filter, foldable input and multiple distinct groups. --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) --- FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; -- Check analysis exceptions SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql index 746b677234832..657ea59ec8f11 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql @@ -241,10 +241,9 @@ select sum(1/ten) filter (where ten > 0) from tenk1; -- select ten, sum(distinct four) filter (where four::text ~ '123') from onek a -- group by ten; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select ten, sum(distinct four) filter (where four > 10) from onek a --- group by ten --- having exists (select 1 from onek b where sum(distinct a.four) = b.four); +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four); -- [SPARK-28682] ANSI SQL: Collation Support -- select max(foo COLLATE "C") filter (where (bar collate "POSIX") > '0') diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql index fc54d179f742c..45617c53166aa 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql @@ -336,9 +336,8 @@ order by 2,1; -- order by 2,1; -- FILTER queries --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select ten, sum(distinct four) filter (where string(four) like '123') from onek a --- group by rollup(ten); +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten); -- More rescan tests -- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) 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 d41d25280146b..c349d9d84c226 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 37 +-- Number of queries: 68 -- !query @@ -94,6 +94,62 @@ struct +-- !query output +2 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp +-- !query schema +struct +-- !query output +8 2 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp +-- !query schema +struct +-- !query output +2 2 + + +-- !query +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp +-- !query schema +struct +-- !query output +2450.0 8 2 + + +-- !query +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp +-- !query schema +struct +-- !query output +8 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp +-- !query schema +struct +-- !query output +0 + + -- !query SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a -- !query schema @@ -177,6 +233,68 @@ struct "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 +100 400.0 +20 300.0 +30 400.0 +70 150.0 +NULL NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 300.0 +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL 400.0 NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct DATE '2001-01-01')):double,sum(DISTINCT salary) FILTER (WHERE (date_format(CAST(hiredate AS TIMESTAMP), yyyy-MM-dd HH:mm:ss) > 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 300.0 +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL NULL NULL + + +-- !query +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01)):double> +-- !query output +10 3 300.0 300.0 +100 2 400.0 400.0 +20 1 300.0 300.0 +30 1 400.0 400.0 +70 1 150.0 150.0 +NULL 1 400.0 NULL + + +-- !query +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b +-- !query schema +struct +-- !query output +1 1 +2 1 +NULL 0 + + -- !query SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1 -- !query schema @@ -261,6 +379,240 @@ struct 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 0 300.0 +30 0 400.0 +70 1 150.0 +NULL 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 2 0 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double> +-- !query output +10 2 0 400.0 +100 2 2 800.0 +20 1 0 300.0 +30 1 0 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 2 0 400.0 NULL +100 2 2 800.0 800.0 +20 1 1 300.0 300.0 +30 1 1 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 1 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 2 0 400.0 NULL +100 2 2 800.0 800.0 +20 1 0 300.0 300.0 +30 1 0 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 0 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate):bigint,sum(salary):double> +-- !query output +10 0 2 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 0 1 400.0 +100 2 1 800.0 +20 1 0 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double> +-- !query output +10 0 1 400.0 +100 2 1 NULL +20 1 0 300.0 +30 1 1 NULL +70 1 1 150.0 +NULL 1 0 NULL + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 0 1 400.0 NULL +100 2 1 NULL 800.0 +20 1 0 300.0 300.0 +30 1 1 NULL 400.0 +70 1 1 150.0 150.0 +NULL 1 0 NULL 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT emp_name):bigint,sum(salary):double> +-- !query output +10 0 2 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT emp_name) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 0 1 400.0 +100 2 1 800.0 +20 1 0 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate):bigint,sum(salary):double> +-- !query output +10 NULL 2 400.0 +100 1500 2 800.0 +20 320 1 300.0 +30 430 1 400.0 +70 870 1 150.0 +NULL NULL 1 400.0 + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 NULL 1 400.0 +100 1500 1 800.0 +20 320 0 300.0 +30 430 1 400.0 +70 870 1 150.0 +NULL NULL 0 400.0 + + +-- !query +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query schema +struct 200)):double,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double> +-- !query output +10 NULL 1 400.0 +100 750.0 1 NULL +20 320.0 0 300.0 +30 430.0 1 NULL +70 870.0 1 150.0 +NULL NULL 0 NULL + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id +-- !query schema +struct 0)):bigint,sum(salary):double> +-- !query output +10 2 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 1 0 400.0 +100 1 1 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + -- !query SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1 -- !query schema @@ -309,6 +661,15 @@ struct<((a + 1) + 1):int,count(b) FILTER (WHERE (b > 0)):bigint> NULL 1 +-- !query +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct 0)):bigint,count(DISTINCT b, c) FILTER (WHERE ((b > 0) AND (c > 2))):bigint> +-- !query output +1 1 + + -- !query SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out index 69f96b02782e3..e1f735e5fe1dc 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 4 +-- Number of queries: 5 -- !query @@ -27,6 +27,20 @@ struct 0)):d 2828.9682539682954 +-- !query +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query schema +struct 10)):bigint> +-- !query output +0 NULL +2 NULL +4 NULL +6 NULL +8 NULL + + -- !query select (select count(*) from (values (1)) t0(inner_c)) diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out index 7312c20876296..2619634d7d569 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 54 +-- Number of queries: 55 -- !query @@ -443,6 +443,25 @@ struct NULL 1 +-- !query +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten) +-- !query schema +struct +-- !query output +0 NULL +1 NULL +2 NULL +3 NULL +4 NULL +5 NULL +6 NULL +7 NULL +8 NULL +9 NULL +NULL NULL + + -- !query select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col) -- !query schema From 005ef3a5b8715b38874888f2768c463b60c704f8 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 4 Aug 2020 14:51:25 +0900 Subject: [PATCH 362/384] [SPARK-32468][SS][TESTS][FOLLOWUP] Provide "default.api.timeout.ms" as well when specifying "request.timeout.ms" on replacing "default.api.timeout.ms" ### What changes were proposed in this pull request? This patch is a follow-up to fill the gap in #29272 which missed to also provide `default.api.timeout.ms` as well. #29272 unintentionally changed the behavior on Kafka side timeout which is incompatible with the test timeout. (`default.api.timeout.ms` gets default value which is 60 seconds, longer than test timeout.) ### Why are the changes needed? We realized the PR for SPARK-32468 (#29272) doesn't work as we expect. See https://github.com/apache/spark/pull/29272#issuecomment-668333483 for more details. ### Does this PR introduce _any_ user-facing change? No, as it only touches the tests. ### How was this patch tested? Will trigger builds from Jenkins or Github Action multiple time and confirm. Closes #29343 from HeartSaVioR/SPARK-32468-FOLLOWUP. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../spark/sql/kafka010/KafkaContinuousSourceSuite.scala | 1 + .../spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala | 1 + .../spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala | 4 ++++ 3 files changed, 6 insertions(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index 099d6ff13051b..14dcbeef0d9a3 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -194,6 +194,7 @@ class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala index 5b634e4d50641..54ce1717acc71 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala @@ -224,6 +224,7 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with KafkaM .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", "failOnDataLoss.*") .option("startingOffsets", "earliest") .option("failOnDataLoss", "false") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 517d153ca3c91..63659989dec1b 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -364,6 +364,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") @@ -401,6 +402,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("startingOffsets", "earliest") .option("subscribePattern", s"$topicPrefix-.*") @@ -590,6 +592,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("subscribe", topic) // If a topic is deleted and we try to poll data starting from offset 0, // the Kafka consumer will just block until timeout and return an empty result. @@ -1861,6 +1864,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest { .option("subscribePattern", "stress.*") .option("failOnDataLoss", "false") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .as[(String, String)] From 7fec6e0c16409235a40ee7bb1cc7e0eae7751d69 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 4 Aug 2020 17:49:52 +0900 Subject: [PATCH 363/384] [SPARK-32524][SQL][TESTS] CachedBatchSerializerSuite should clean up InMemoryRelation.ser ### What changes were proposed in this pull request? This PR aims to clean up `InMemoryRelation.ser` in `CachedBatchSerializerSuite`. ### Why are the changes needed? SPARK-32274 makes SQL cache serialization pluggable. ``` [SPARK-32274][SQL] Make SQL cache serialization pluggable ``` This causes UT failures. ``` $ build/sbt "sql/testOnly *.CachedBatchSerializerSuite *.CachedTableSuite" ... [info] Cause: java.lang.IllegalStateException: This does not work. This is only for testing [info] at org.apache.spark.sql.execution.columnar.TestSingleIntColumnarCachedBatchSerializer.convertInternalRowToCachedBatch(CachedBatchSerializerSuite.scala:49) ... [info] *** 30 TESTS FAILED *** [error] Failed: Total 51, Failed 30, Errors 0, Passed 21 [error] Failed tests: [error] org.apache.spark.sql.CachedTableSuite [error] (sql/test:testOnly) sbt.TestsFailedException: Tests unsuccessful ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually. ``` $ build/sbt "sql/testOnly *.CachedBatchSerializerSuite *.CachedTableSuite" [info] Tests: succeeded 51, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. [info] Passed: Total 51, Failed 0, Errors 0, Passed 51 ``` Closes #29346 from dongjoon-hyun/SPARK-32524-3. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- .../sql/execution/columnar/InMemoryRelation.scala | 3 +++ .../columnar/CachedBatchSerializerSuite.scala | 11 +++++++++++ 2 files changed, 14 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index be3dc5934e84f..07411c0d3803c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -277,6 +277,9 @@ object InMemoryRelation { ser.get } + /* Visible for testing */ + private[columnar] def clearSerializer(): Unit = synchronized { ser = None } + def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match { case gen: WholeStageCodegenExec => gen.child match { case c2r: ColumnarToRowTransition => c2r.child match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala index 72eba7f6e6907..099a1aa996c11 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} +import org.apache.spark.sql.execution.columnar.InMemoryRelation.clearSerializer import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.SharedSparkSession @@ -120,6 +121,16 @@ class CachedBatchSerializerSuite extends QueryTest with SharedSparkSession { classOf[TestSingleIntColumnarCachedBatchSerializer].getName) } + protected override def beforeAll(): Unit = { + super.beforeAll() + clearSerializer() + } + + protected override def afterAll(): Unit = { + clearSerializer() + super.afterAll() + } + test("Columnar Cache Plugin") { withTempPath { workDir => val workDirPath = workDir.getAbsolutePath From 6d690680576ba58c35e6fbc86d37b45fef1c50d9 Mon Sep 17 00:00:00 2001 From: "fqaiser94@gmail.com" Date: Tue, 4 Aug 2020 12:11:04 +0000 Subject: [PATCH 364/384] [SPARK-32521][SQL] Bug-fix: WithFields Expression should not be foldable ### What changes were proposed in this pull request? Make WithFields Expression not foldable. ### Why are the changes needed? The following query currently fails on master brach: ``` sql("SELECT named_struct('a', 1, 'b', 2) a") .select($"a".withField("c", lit(3)).as("a")) .show(false) // java.lang.UnsupportedOperationException: Cannot evaluate expression: with_fields(named_struct(a, 1, b, 2), c, 3) ``` This happens because the Catalyst optimizer sees that the WithFields Expression is foldable and tries to statically evaluate the WithFields Expression (via the ConstantFolding rule), however it cannot do so because WithFields Expression is Unevaluable. ### Does this PR introduce _any_ user-facing change? Yes, queries like the one shared above will now succeed. That said, this bug was introduced in Spark 3.1.0 which has yet to be released. ### How was this patch tested? A new unit test was added. Closes #29338 from fqaiser94/SPARK-32521. Lead-authored-by: fqaiser94@gmail.com Co-authored-by: fqaiser94 Signed-off-by: Wenchen Fan --- .../expressions/complexTypeCreator.scala | 2 -- .../spark/sql/ColumnExpressionSuite.scala | 32 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 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 cf7cc3a5e16ff..563ce7133a3dc 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 @@ -563,8 +563,6 @@ case class WithFields( override def dataType: StructType = evalExpr.dataType.asInstanceOf[StructType] - override def foldable: Boolean = structExpr.foldable && valExprs.forall(_.foldable) - override def nullable: Boolean = structExpr.nullable override def prettyName: String = "with_fields" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 761632e76b165..36c10874cc183 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -1420,4 +1420,36 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { }.getMessage should include("No such struct field b in a, B") } } + + test("withField user-facing examples") { + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".withField("c", lit(3))), + Row(Row(1, 2, 3))) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".withField("b", lit(3))), + Row(Row(1, 3))) + + checkAnswer( + sql("SELECT CAST(NULL AS struct) struct_col") + .select($"struct_col".withField("c", lit(3))), + Row(null)) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col") + .select($"struct_col".withField("b", lit(100))), + Row(Row(1, 100, 100))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".withField("a.c", lit(3))), + Row(Row(Row(1, 2, 3)))) + + intercept[AnalysisException] { + sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") + .select($"struct_col".withField("a.c", lit(3))) + }.getMessage should include("Ambiguous reference to fields") + } } From 171b7d5d7114c516be722274dbc433a0897b62c0 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 4 Aug 2020 21:11:00 +0800 Subject: [PATCH 365/384] [SPARK-23431][CORE] Expose stage level peak executor metrics via REST API ### What changes were proposed in this pull request? Note that this PR is forked from #23340 originally written by edwinalu. This PR proposes to expose the peak executor metrics at the stage level via the REST APIs: * `/applications//stages/`: peak values of executor metrics for **each stage** * `/applications//stages//< stage_attempt_id >`: peak values of executor metrics for **each executor** for the stage, followed by peak values of executor metrics for the stage ### Why are the changes needed? The stage level peak executor metrics can help better understand your application's resource utilization. ### Does this PR introduce _any_ user-facing change? 1. For the `/applications//stages/` API, you will see the following new info for **each stage**: ```JSON "peakExecutorMetrics" : { "JVMHeapMemory" : 213367864, "JVMOffHeapMemory" : 189011656, "OnHeapExecutionMemory" : 0, "OffHeapExecutionMemory" : 0, "OnHeapStorageMemory" : 2133349, "OffHeapStorageMemory" : 0, "OnHeapUnifiedMemory" : 2133349, "OffHeapUnifiedMemory" : 0, "DirectPoolMemory" : 282024, "MappedPoolMemory" : 0, "ProcessTreeJVMVMemory" : 0, "ProcessTreeJVMRSSMemory" : 0, "ProcessTreePythonVMemory" : 0, "ProcessTreePythonRSSMemory" : 0, "ProcessTreeOtherVMemory" : 0, "ProcessTreeOtherRSSMemory" : 0, "MinorGCCount" : 13, "MinorGCTime" : 115, "MajorGCCount" : 4, "MajorGCTime" : 339 } ``` 2. For the `/applications//stages//` API, you will see the following new info for **each executor** under `executorSummary`: ```JSON "peakMemoryMetrics" : { "JVMHeapMemory" : 0, "JVMOffHeapMemory" : 0, "OnHeapExecutionMemory" : 0, "OffHeapExecutionMemory" : 0, "OnHeapStorageMemory" : 0, "OffHeapStorageMemory" : 0, "OnHeapUnifiedMemory" : 0, "OffHeapUnifiedMemory" : 0, "DirectPoolMemory" : 0, "MappedPoolMemory" : 0, "ProcessTreeJVMVMemory" : 0, "ProcessTreeJVMRSSMemory" : 0, "ProcessTreePythonVMemory" : 0, "ProcessTreePythonRSSMemory" : 0, "ProcessTreeOtherVMemory" : 0, "ProcessTreeOtherRSSMemory" : 0, "MinorGCCount" : 0, "MinorGCTime" : 0, "MajorGCCount" : 0, "MajorGCTime" : 0 } ``` , and the following at the stage level: ```JSON "peakExecutorMetrics" : { "JVMHeapMemory" : 213367864, "JVMOffHeapMemory" : 189011656, "OnHeapExecutionMemory" : 0, "OffHeapExecutionMemory" : 0, "OnHeapStorageMemory" : 2133349, "OffHeapStorageMemory" : 0, "OnHeapUnifiedMemory" : 2133349, "OffHeapUnifiedMemory" : 0, "DirectPoolMemory" : 282024, "MappedPoolMemory" : 0, "ProcessTreeJVMVMemory" : 0, "ProcessTreeJVMRSSMemory" : 0, "ProcessTreePythonVMemory" : 0, "ProcessTreePythonRSSMemory" : 0, "ProcessTreeOtherVMemory" : 0, "ProcessTreeOtherRSSMemory" : 0, "MinorGCCount" : 13, "MinorGCTime" : 115, "MajorGCCount" : 4, "MajorGCTime" : 339 } ``` ### How was this patch tested? Added tests. Closes #29020 from imback82/metrics. Lead-authored-by: Terry Kim Co-authored-by: edwinalu Signed-off-by: Gengliang Wang --- .../spark/status/AppStatusListener.scala | 43 +- .../apache/spark/status/AppStatusStore.scala | 3 +- .../org/apache/spark/status/LiveEntity.scala | 10 +- .../org/apache/spark/status/api/v1/api.scala | 10 +- .../org/apache/spark/ui/jobs/JobPage.scala | 3 +- .../application_list_json_expectation.json | 15 + .../completed_app_list_json_expectation.json | 15 + .../limit_app_list_json_expectation.json | 30 +- .../minDate_app_list_json_expectation.json | 45 +- .../minEndDate_app_list_json_expectation.json | 15 + ...ge_list_with_peak_metrics_expectation.json | 204 ++++ .../stage_with_peak_metrics_expectation.json | 998 ++++++++++++++++++ .../spark-events/app-20200706201101-0003 | 124 +++ .../deploy/history/HistoryServerSuite.scala | 2 + .../spark/status/AppStatusListenerSuite.scala | 90 +- .../org/apache/spark/ui/StagePageSuite.scala | 3 +- dev/.rat-excludes | 1 + 17 files changed, 1558 insertions(+), 53 deletions(-) create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json create mode 100644 core/src/test/resources/spark-events/app-20200706201101-0003 diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index f7b0e9b62fc29..7ae9117137caa 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.CPUS_PER_TASK import org.apache.spark.internal.config.Status._ @@ -868,13 +868,17 @@ private[spark] class AppStatusListener( // check if there is a new peak value for any of the executor level memory metrics // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed // for the live UI. - event.executorUpdates.foreach { case (_, peakUpdates) => + event.executorUpdates.foreach { case (key, peakUpdates) => liveExecutors.get(event.execId).foreach { exec => if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(peakUpdates)) { - maybeUpdate(exec, now) + update(exec, now) } } + + // Update stage level peak executor metrics. + updateStageLevelPeakExecutorMetrics(key._1, key._2, event.execId, peakUpdates, now) } + // Flush updates if necessary. Executor heartbeat is an event that happens periodically. Flush // here to ensure the staleness of Spark UI doesn't last more than // `max(heartbeat interval, liveUpdateMinFlushPeriod)`. @@ -885,17 +889,38 @@ private[spark] class AppStatusListener( } } - override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = { + override def onStageExecutorMetrics(event: SparkListenerStageExecutorMetrics): Unit = { val now = System.nanoTime() // check if there is a new peak value for any of the executor level memory metrics, // while reading from the log. SparkListenerStageExecutorMetrics are only processed // when reading logs. - liveExecutors.get(executorMetrics.execId).orElse( - deadExecutors.get(executorMetrics.execId)).foreach { exec => - if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) { - update(exec, now) - } + liveExecutors.get(event.execId).orElse( + deadExecutors.get(event.execId)).foreach { exec => + if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(event.executorMetrics)) { + update(exec, now) + } + } + + // Update stage level peak executor metrics. + updateStageLevelPeakExecutorMetrics( + event.stageId, event.stageAttemptId, event.execId, event.executorMetrics, now) + } + + private def updateStageLevelPeakExecutorMetrics( + stageId: Int, + stageAttemptId: Int, + executorId: String, + executorMetrics: ExecutorMetrics, + now: Long): Unit = { + Option(liveStages.get((stageId, stageAttemptId))).foreach { stage => + if (stage.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics)) { + update(stage, now) + } + val esummary = stage.executorSummary(executorId) + if (esummary.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics)) { + update(esummary, now) + } } } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 0a8d188dc1553..5c6543fe28a18 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -506,7 +506,8 @@ private[spark] class AppStatusStore( tasks = Some(tasks), executorSummary = Some(executorSummary(stage.stageId, stage.attemptId)), killedTasksSummary = stage.killedTasksSummary, - resourceProfileId = stage.resourceProfileId) + resourceProfileId = stage.resourceProfileId, + peakExecutorMetrics = stage.peakExecutorMetrics) } def rdd(rddId: Int): v1.RDDStorageInfo = { diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 81478214994b0..0fadd330a01ad 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -365,6 +365,8 @@ private class LiveExecutorStageSummary( var metrics = createMetrics(default = 0L) + val peakExecutorMetrics = new ExecutorMetrics() + override protected def doUpdate(): Any = { val info = new v1.ExecutorStageSummary( taskTime, @@ -381,7 +383,8 @@ private class LiveExecutorStageSummary( metrics.shuffleWriteMetrics.recordsWritten, metrics.memoryBytesSpilled, metrics.diskBytesSpilled, - isBlacklisted) + isBlacklisted, + Some(peakExecutorMetrics).filter(_.isSet)) new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info) } @@ -420,6 +423,8 @@ private class LiveStage extends LiveEntity { var blackListedExecutors = new HashSet[String]() + val peakExecutorMetrics = new ExecutorMetrics() + // Used for cleanup of tasks after they reach the configured limit. Not written to the store. @volatile var cleaning = false var savedTasks = new AtomicInteger(0) @@ -484,7 +489,8 @@ private class LiveStage extends LiveEntity { tasks = None, executorSummary = None, killedTasksSummary = killedSummary, - resourceProfileId = info.resourceProfileId) + resourceProfileId = info.resourceProfileId, + Some(peakExecutorMetrics).filter(_.isSet)) } override protected def doUpdate(): Any = { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index e89e29101a126..d207a6023f7f9 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -82,7 +82,10 @@ class ExecutorStageSummary private[spark]( val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, val diskBytesSpilled : Long, - val isBlacklistedForStage: Boolean) + val isBlacklistedForStage: Boolean, + @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) + @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) + val peakMemoryMetrics: Option[ExecutorMetrics]) class ExecutorSummary private[spark]( val id: String, @@ -259,7 +262,10 @@ class StageData private[spark]( val tasks: Option[Map[Long, TaskData]], val executorSummary: Option[Map[String, ExecutorStageSummary]], val killedTasksSummary: Map[String, Int], - val resourceProfileId: Int) + val resourceProfileId: Int, + @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) + @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) + val peakExecutorMetrics: Option[ExecutorMetrics]) class TaskData private[spark]( val taskId: Long, diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index bba5e3dda6c47..df239d6d0e187 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -255,7 +255,8 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP tasks = None, executorSummary = None, killedTasksSummary = Map(), - ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) + ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID, + peakExecutorMetrics = None) } } diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index d2b3d1b069204..06015ec46e44d 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index d2b3d1b069204..06015ec46e44d 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index 82489e94a84c8..8e6be68b4670c 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { @@ -28,19 +43,4 @@ "lastUpdatedEpoch" : 0, "endTimeEpoch" : 1562101355974 } ] -}, { - "id" : "application_1553914137147_0018", - "name" : "LargeBlocks", - "attempts" : [ { - "startTime" : "2019-04-08T20:39:44.286GMT", - "endTime" : "2019-04-08T20:40:46.454GMT", - "lastUpdated" : "", - "duration" : 62168, - "sparkUser" : "systest", - "completed" : true, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1554755984286, - "endTimeEpoch" : 1554756046454, - "lastUpdatedEpoch" : 0 - } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index ac2bb0e29b2fb..35d71f9d4409b 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { @@ -14,22 +29,20 @@ "lastUpdatedEpoch" : 0 } ] }, { - "id": "application_1555004656427_0144", - "name": "Spark shell", - "attempts": [ - { - "startTime": "2019-07-02T21:02:17.180GMT", - "endTime": "2019-07-02T21:02:35.974GMT", - "lastUpdated": "", - "duration": 18794, - "sparkUser": "tgraves", - "completed": true, - "appSparkVersion": "3.0.0-SNAPSHOT", - "startTimeEpoch": 1562101337180, - "lastUpdatedEpoch": 0, - "endTimeEpoch": 1562101355974 - } - ] + "id" : "application_1555004656427_0144", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2019-07-02T21:02:17.180GMT", + "endTime" : "2019-07-02T21:02:35.974GMT", + "lastUpdated" : "", + "duration" : 18794, + "sparkUser" : "tgraves", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "endTimeEpoch" : 1562101355974, + "startTimeEpoch" : 1562101337180, + "lastUpdatedEpoch" : 0 + } ] }, { "id" : "application_1553914137147_0018", "name" : "LargeBlocks", diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json index 156167606ff20..c6530b14bf271 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json new file mode 100644 index 0000000000000..d455b97fdafa4 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json @@ -0,0 +1,204 @@ +[ { + "status" : "COMPLETE", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:21.040GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT", + "completionTime" : "2020-07-07T03:11:23.044GMT", + "executorDeserializeTime" : 3905, + "executorDeserializeCpuTime" : 979900000, + "executorRunTime" : 25579, + "executorCpuTime" : 8810338000, + "resultSize" : 33883, + "jvmGcTime" : 1010, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 384640, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "foreach at :26", + "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)", + "schedulingPool" : "default", + "rddIds" : [ 10, 8, 6, 7, 9 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } +}, { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:20.499GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:20.502GMT", + "completionTime" : "2020-07-07T03:11:20.930GMT", + "executorDeserializeTime" : 424, + "executorDeserializeCpuTime" : 63666000, + "executorRunTime" : 6105, + "executorCpuTime" : 426449000, + "resultSize" : 31546, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 16256, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266", + "description" : "broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)", + "details" : "org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", + "schedulingPool" : "default", + "rddIds" : [ 5, 3, 4 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + } +}, { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:18.860GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:19.253GMT", + "completionTime" : "2020-07-07T03:11:20.381GMT", + "executorDeserializeTime" : 15399, + "executorDeserializeCpuTime" : 3668889000, + "executorRunTime" : 1292, + "executorCpuTime" : 172863000, + "resultSize" : 22375, + "jvmGcTime" : 1744, + "resultSerializationTime" : 3, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266", + "description" : "broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)", + "details" : "org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", + "schedulingPool" : "default", + "rddIds" : [ 2, 0, 1 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 155100856, + "JVMOffHeapMemory" : 64239224, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 6964, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 6964, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 1852, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 7, + "MinorGCTime" : 33, + "MajorGCCount" : 3, + "MajorGCTime" : 110 + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json new file mode 100644 index 0000000000000..373510d23058e --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json @@ -0,0 +1,998 @@ +{ + "status" : "COMPLETE", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:21.040GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT", + "completionTime" : "2020-07-07T03:11:23.044GMT", + "executorDeserializeTime" : 3905, + "executorDeserializeCpuTime" : 979900000, + "executorRunTime" : 25579, + "executorCpuTime" : 8810338000, + "resultSize" : 33883, + "jvmGcTime" : 1010, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 384640, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "foreach at :26", + "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)", + "schedulingPool" : "default", + "rddIds" : [ 10, 8, 6, 7, 9 ], + "accumulatorUpdates" : [ ], + "tasks" : { + "42" : { + "taskId" : 42, + "index" : 10, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.120GMT", + "duration" : 1923, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 229, + "executorDeserializeCpuTime" : 58152000, + "executorRunTime" : 1624, + "executorCpuTime" : 508230000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 70, + "gettingResultTime" : 0 + }, + "37" : { + "taskId" : 37, + "index" : 5, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.100GMT", + "duration" : 1915, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 256, + "executorDeserializeCpuTime" : 60890000, + "executorRunTime" : 1596, + "executorCpuTime" : 507192000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 + }, + "46" : { + "taskId" : 46, + "index" : 14, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.132GMT", + "duration" : 1905, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 218, + "executorDeserializeCpuTime" : 51464000, + "executorRunTime" : 1618, + "executorCpuTime" : 490927000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 69, + "gettingResultTime" : 0 + }, + "38" : { + "taskId" : 38, + "index" : 6, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.104GMT", + "duration" : 1835, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 255, + "executorDeserializeCpuTime" : 60358000, + "executorRunTime" : 1498, + "executorCpuTime" : 414110000, + "resultSize" : 2158, + "jvmGcTime" : 62, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + }, + "33" : { + "taskId" : 33, + "index" : 1, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.082GMT", + "duration" : 1943, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 267, + "executorDeserializeCpuTime" : 54442000, + "executorRunTime" : 1597, + "executorCpuTime" : 519178000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 79, + "gettingResultTime" : 0 + }, + "41" : { + "taskId" : 41, + "index" : 9, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.116GMT", + "duration" : 1916, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 240, + "executorDeserializeCpuTime" : 55787000, + "executorRunTime" : 1614, + "executorCpuTime" : 489923000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 62, + "gettingResultTime" : 0 + }, + "32" : { + "taskId" : 32, + "index" : 0, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.077GMT", + "duration" : 1960, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 271, + "executorDeserializeCpuTime" : 56827000, + "executorRunTime" : 1619, + "executorCpuTime" : 496683000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 70, + "gettingResultTime" : 0 + }, + "34" : { + "taskId" : 34, + "index" : 2, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.087GMT", + "duration" : 1939, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 265, + "executorDeserializeCpuTime" : 69492000, + "executorRunTime" : 1606, + "executorCpuTime" : 508433000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 68, + "gettingResultTime" : 0 + }, + "45" : { + "taskId" : 45, + "index" : 13, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.129GMT", + "duration" : 1895, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 221, + "executorDeserializeCpuTime" : 54222000, + "executorRunTime" : 1595, + "executorCpuTime" : 495138000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 79, + "gettingResultTime" : 0 + }, + "44" : { + "taskId" : 44, + "index" : 12, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.126GMT", + "duration" : 1917, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 222, + "executorDeserializeCpuTime" : 51988000, + "executorRunTime" : 1624, + "executorCpuTime" : 498187000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + }, + "39" : { + "taskId" : 39, + "index" : 7, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.109GMT", + "duration" : 1915, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 254, + "executorDeserializeCpuTime" : 64380000, + "executorRunTime" : 1596, + "executorCpuTime" : 539451000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 65, + "gettingResultTime" : 0 + }, + "35" : { + "taskId" : 35, + "index" : 3, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.091GMT", + "duration" : 1925, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 263, + "executorDeserializeCpuTime" : 62944000, + "executorRunTime" : 1598, + "executorCpuTime" : 502908000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 64, + "gettingResultTime" : 0 + }, + "43" : { + "taskId" : 43, + "index" : 11, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.123GMT", + "duration" : 1906, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 225, + "executorDeserializeCpuTime" : 48849000, + "executorRunTime" : 1609, + "executorCpuTime" : 502120000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 72, + "gettingResultTime" : 0 + }, + "40" : { + "taskId" : 40, + "index" : 8, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.112GMT", + "duration" : 1904, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 246, + "executorDeserializeCpuTime" : 69760000, + "executorRunTime" : 1595, + "executorCpuTime" : 510597000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 + }, + "36" : { + "taskId" : 36, + "index" : 4, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.095GMT", + "duration" : 1920, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 260, + "executorDeserializeCpuTime" : 112849000, + "executorRunTime" : 1596, + "executorCpuTime" : 503010000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 64, + "gettingResultTime" : 0 + }, + "47" : { + "taskId" : 47, + "index" : 15, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.136GMT", + "duration" : 1878, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 213, + "executorDeserializeCpuTime" : 47496000, + "executorRunTime" : 1594, + "executorCpuTime" : 1324251000, + "resultSize" : 2115, + "jvmGcTime" : 52, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + } + }, + "executorSummary" : { + "0" : { + "taskTime" : 30596, + "failedTasks" : 0, + "succeededTasks" : 16, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false, + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + } + }, + "driver" : { + "taskTime" : 0, + "failedTasks" : 0, + "succeededTasks" : 0, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false, + "peakMemoryMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } + } + }, + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } +} diff --git a/core/src/test/resources/spark-events/app-20200706201101-0003 b/core/src/test/resources/spark-events/app-20200706201101-0003 new file mode 100644 index 0000000000000..b2923ca0f001f --- /dev/null +++ b/core/src/test/resources/spark-events/app-20200706201101-0003 @@ -0,0 +1,124 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.1.0-SNAPSHOT"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":64415},"Maximum Memory":384093388,"Timestamp":1594091461118,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre","Java Version":"1.8.0_231 (Oracle Corporation)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","yarn.timeline-service.timeline-client.number-of-async-entities-to-merge":"10","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.jobhistory.loadedjob.tasks.max":"-1","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","hadoop.security.groups.cache.background.reload.threads":"3","yarn.resourcemanager.webapp.cross-origin.enabled":"false","fs.AbstractFileSystem.ftp.impl":"org.apache.hadoop.fs.ftp.FtpFs","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","mapreduce.input.fileinputformat.split.minsize":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","yarn.sharedcache.checksum.algo.impl":"org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl","yarn.nodemanager.amrmproxy.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor","yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size":"10485760","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","mapreduce.task.profile.maps":"0-2","yarn.scheduler.include-port-in-node-name":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","yarn.resourcemanager.node-removal-untracked.timeout-ms":"60000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"0.0.0.0:19890","yarn.node-labels.fs-store.impl.class":"org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore","yarn.nodemanager.collector-service.address":"${yarn.nodemanager.hostname}:8048","fs.trash.checkpoint.interval":"0","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","yarn.resourcemanager.node-ip-cache.expiry-interval-secs":"-1","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","yarn.resourcemanager.placement-constraints.handler":"disabled","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"1000","yarn.resourcemanager.system-metrics-publisher.enabled":"false","yarn.resourcemanager.placement-constraints.algorithm.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm.DefaultPlacementAlgorithm","yarn.sharedcache.webapp.address":"0.0.0.0:8788","yarn.resourcemanager.delegation.token.renew-interval":"*********(redacted)","yarn.sharedcache.nm.uploader.replication.factor":"10","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","mapreduce.reduce.skip.proc-count.auto-incr":"true","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","yarn.nodemanager.resource.memory.enforced":"true","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","yarn.acl.reservation-enable":"false","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","yarn.app.mapreduce.am.hard-kill-timeout-ms":"10000","fs.s3a.etag.checksum.enabled":"false","yarn.nodemanager.container-metrics.enable":"true","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","yarn.scheduler.configuration.leveldb-store.path":"${hadoop.tmp.dir}/yarn/system/confstore","yarn.sharedcache.store.in-memory.initial-delay-mins":"10","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","mapreduce.map.skip.proc-count.auto-incr":"true","fs.s3a.committer.name":"file","mapreduce.task.profile.reduces":"0-2","hadoop.zk.num-retries":"1000","yarn.webapp.xfs-filter.enabled":"true","seq.io.sort.mb":"100","yarn.scheduler.configuration.max.version":"100","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"${yarn.resourcemanager.hostname}:8030","yarn.node-labels.enabled":"false","yarn.resourcemanager.webapp.ui-actions.enabled":"true","mapreduce.task.timeout":"600000","yarn.sharedcache.client-server.thread-count":"50","hadoop.security.groups.shell.command.timeout":"0s","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.nodemanager.elastic-memory-control.oom-handler":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.DefaultOOMHandler","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"file:///","yarn.minicluster.use-rpc":"false","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","mapreduce.jobhistory.address":"0.0.0.0:10020","yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.is.minicluster":"false","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.abfss.impl":"org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","yarn.resourcemanager.epoch.range":"0","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","fs.AbstractFileSystem.wasbs.impl":"org.apache.hadoop.fs.azure.Wasbs","yarn.timeline-service.entity-group-fs-store.cache-store-class":"org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","yarn.resourcemanager.node-labels.provider.fetch-interval-ms":"1800000","yarn.router.webapp.https.address":"0.0.0.0:8091","yarn.nodemanager.webapp.cross-origin.enabled":"false","fs.wasb.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem","yarn.resourcemanager.auto-update.containers":"false","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","yarn.scheduler.configuration.zk-store.parent-path":"/confstore","yarn.nodemanager.default-container-executor.log-dirs.permissions":"710","yarn.app.attempt.diagnostics.limit.kc":"64","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"-1","fs.AbstractFileSystem.abfs.impl":"org.apache.hadoop.fs.azurebfs.Abfs","yarn.timeline-service.writer.flush-interval-seconds":"60","fs.s3a.fast.upload.active.blocks":"4","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","yarn.resourcemanager.fs.state-store.num-retries":"0","yarn.resourcemanager.nodemanager-connect-retries":"10","yarn.nodemanager.log-aggregation.num-log-files-per-app":"30","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","fs.s3a.committer.magic.enabled":"false","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","yarn.system-metrics-publisher.enabled":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","mapreduce.task.io.sort.factor":"10","yarn.nodemanager.amrmproxy.client.thread-count":"25","ha.failover-controller.new-active.rpc-timeout.ms":"60000","yarn.nodemanager.container-localizer.java.opts":"-Xmx256m","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","yarn.nodemanager.windows-container.memory-limit.enabled":"false","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.minicluster.fixed.ports":"false","hadoop.tags.system":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.cluster.max-application-priority":"0","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.caller.context.signature.max.size":"40","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","yarn.scheduler.configuration.store.max-logs":"1000","yarn.nodemanager.node-attributes.provider.fetch-interval-ms":"600000","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","hadoop.http.cross-origin.enabled":"false","hadoop.zk.acl":"world:anyone:rwcda","mapreduce.map.sort.spill.percent":"0.80","yarn.timeline-service.entity-group-fs-store.scan-interval-seconds":"60","yarn.node-attribute.fs-store.impl.class":"org.apache.hadoop.yarn.server.resourcemanager.nodelabels.FileSystemNodeAttributeStore","fs.s3a.retry.interval":"500ms","yarn.timeline-service.client.best-effort":"false","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","fs.AbstractFileSystem.swebhdfs.impl":"org.apache.hadoop.fs.SWebHdfs","yarn.nodemanager.elastic-memory-control.timeout-sec":"5","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","yarn.timeline-service.reader.webapp.address":"${yarn.timeline-service.webapp.address}","yarn.resourcemanager.placement-constraints.algorithm.pool-size":"1","yarn.timeline-service.hbase.coprocessor.jar.hdfs.location":"/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler","yarn.app.mapreduce.am.command-opts":"-Xmx1024m","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","io.mapfile.bloom.error.rate":"0.005","fs.client.resolve.topology.enabled":"false","yarn.nodemanager.runtime.linux.allowed-runtimes":"default","yarn.sharedcache.store.class":"org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","fs.s3a.metadatastore.impl":"org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore","io.skip.checksum.errors":"false","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","yarn.timeline-service.webapp.xfs-filter.xframe-options":"SAMEORIGIN","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","hadoop.registry.zk.session.timeout.ms":"60000","yarn.federation.cache-ttl.secs":"300","mapreduce.jvm.system-properties-to-log":"os.name,os.version,java.home,java.runtime.version,java.vendor,java.version,java.vm.name,java.class.path,java.io.tmpdir,user.dir,user.name","yarn.resourcemanager.opportunistic-container-allocation.nodes-used":"10","yarn.timeline-service.entity-group-fs-store.active-dir":"/tmp/entity-file-history/active","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","fs.AbstractFileSystem.abfss.impl":"org.apache.hadoop.fs.azurebfs.Abfss","yarn.sharedcache.uploader.server.address":"0.0.0.0:8046","yarn.resourcemanager.delegation-token.max-conf-size-bytes":"*********(redacted)","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","yarn.minicluster.yarn.nodemanager.resource.memory-mb":"4096","mapreduce.job.emit-timeline-data":"false","yarn.nodemanager.resource.system-reserved-memory-mb":"-1","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","yarn.resourcemanager.admin.client.thread-count":"1","yarn.dispatcher.drain-events.timeout":"300000","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","yarn.router.clientrm.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.clientrm.DefaultClientRequestInterceptor","yarn.resourcemanager.zk-appid-node.split-index":"0","ftp.blocksize":"67108864","yarn.nodemanager.runtime.linux.sandbox-mode.local-dirs.permissions":"read","yarn.router.rmadmin.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.rmadmin.DefaultRMAdminRequestInterceptor","yarn.nodemanager.log-container-debug-info.enabled":"true","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms":"20","yarn.nodemanager.delete.debug-delay-sec":"0","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","yarn.nodemanager.container-retry-minimum-interval-ms":"1000","hadoop.security.groups.cache.secs":"300","yarn.federation.enabled":"false","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","yarn.nodemanager.amrmproxy.ha.enable":"false","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","yarn.client.application-client-protocol.poll-interval-ms":"200","yarn.scheduler.configuration.leveldb-store.compaction-interval-secs":"86400","yarn.timeline-service.writer.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl","ha.zookeeper.parent-znode":"/hadoop-ha","yarn.nodemanager.log-aggregation.policy.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","yarn.resourcemanager.placement-constraints.scheduler.pool-size":"1","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"${yarn.resourcemanager.hostname}:8032","ipc.client.ping":"true","mapreduce.task.local-fs.write-limit.bytes":"-1","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","yarn.nodemanager.resource-plugins.gpu.docker-plugin":"nvidia-docker-v1","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","yarn.scheduler.configuration.store.class":"file","yarn.resourcemanager.nm-container-queuing.queue-limit-stdev":"1.0f","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","ipc.client.bind.wildcard.addr":"false","yarn.resourcemanager.webapp.rest-csrf.enabled":"false","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","yarn.nodemanager.runtime.linux.docker.allowed-container-networks":"host,none,bridge","yarn.nodemanager.node-labels.resync-interval-ms":"120000","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","mapreduce.jobhistory.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","yarn.rm.system-metrics-publisher.emit-container-events":"false","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","fs.s3a.s3guard.ddb.throttle.retry.interval":"100ms","fs.AbstractFileSystem.wasb.impl":"org.apache.hadoop.fs.azure.Wasb","mapreduce.client.submit.file.replication":"10","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","yarn.nodemanager.runtime.linux.docker.userremapping-gid-threshold":"1","yarn.nodemanager.webapp.rest-csrf.enabled":"false","fs.ftp.host.port":"21","ipc.ping.interval":"60000","yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size":"10","yarn.resourcemanager.admin.address":"${yarn.resourcemanager.hostname}:8033","file.client-write-packet-size":"65536","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"true","hadoop.security.key.default.bitlength":"128","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","yarn.nodemanager.log.deletion-threads-count":"4","yarn.webapp.filter-entity-list-by-user":"false","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"100","yarn.nodemanager.localizer.client.thread-count":"5","io.erasurecode.codec.rs.rawcoders":"rs_native,rs_java","io.erasurecode.codec.rs-legacy.rawcoders":"rs-legacy_java","yarn.sharedcache.admin.address":"0.0.0.0:8047","yarn.resourcemanager.placement-constraints.algorithm.iterator":"SERIAL","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec","mapreduce.job.cache.limit.max-resources-mb":"0","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"5000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","fs.s3a.multiobjectdelete.enable":"true","yarn.sharedcache.cleaner.resource-sleep-ms":"0","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"false","io.file.buffer.size":"65536","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","hadoop.security.sensitive-config-keys":"*********(redacted)","fs.s3a.s3guard.ddb.max.retries":"9","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","yarn.intermediate-data-encryption.enable":"false","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.nodemanager.container.stderr.pattern":"{*stderr*,*STDERR*}","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/tmp/hadoop-yarn/staging","mapreduce.reduce.shuffle.read.timeout":"180000","hadoop.http.cross-origin.max-age":"1800","io.erasurecode.codec.xor.rawcoders":"xor_native,xor_java","fs.s3a.connection.establish.timeout":"5000","mapreduce.job.running.map.limit":"0","yarn.minicluster.control-resource-monitoring":"false","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","yarn.federation.state-store.class":"org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","adl.http.timeout":"-1","yarn.resourcemanager.placement-constraints.retry-attempts":"3","hadoop.caller.context.enabled":"false","yarn.nodemanager.vmem-pmem-ratio":"2.1","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","hadoop.zk.timeout-ms":"10000","fs.s3a.s3guard.cli.prune.age":"86400000","yarn.nodemanager.resource.pcores-vcores-multiplier":"1.0","yarn.nodemanager.runtime.linux.sandbox-mode":"disabled","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3a.committer.threads":"8","hadoop.zk.retry-interval-ms":"1000","hadoop.security.crypto.buffer.size":"8192","yarn.nodemanager.node-labels.provider.fetch-interval-ms":"600000","mapreduce.jobhistory.recovery.store.leveldb.path":"${hadoop.tmp.dir}/mapred/history/recoverystore","yarn.client.failover-retries-on-socket-timeouts":"0","yarn.nodemanager.resource.memory.enabled":"false","fs.azure.authorization.caching.enable":"true","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","mapreduce.job.finish-when-all-reducers-done":"true","hadoop.registry.jaas.context":"Client","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","io.map.index.interval":"128","yarn.resourcemanager.nm-container-queuing.max-queue-wait-time-ms":"100","fs.abfs.impl":"org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem","mapreduce.job.counters.max":"120","mapreduce.jobhistory.webapp.rest-csrf.enabled":"false","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","yarn.timeline-service.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/timeline","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","hadoop.security.auth_to_local.mechanism":"hadoop","yarn.timeline-service.app-collector.linger-period.ms":"60000","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.reservation-system.planfollower.time-step":"1000","yarn.nodemanager.runtime.linux.docker.enable-userremapping.allowed":"true","yarn.webapp.api-service.enable":"false","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","fs.ftp.impl":"org.apache.hadoop.fs.ftp.FTPFileSystem","yarn.nodemanager.container.stderr.tail.bytes":"4096","hadoop.security.group.mapping.ldap.read.timeout.ms":"60000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","mapreduce.outputcommitter.factory.scheme.s3a":"org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory","hadoop.security.groups.cache.background.reload":"false","yarn.nodemanager.container-monitor.enabled":"true","yarn.nodemanager.elastic-memory-control.enabled":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","yarn.resourcemanager.rm.container-allocation.expiry-interval-ms":"600000","mapreduce.fileoutputcommitter.algorithm.version":"2","yarn.resourcemanager.work-preserving-recovery.enabled":"true","mapreduce.map.skip.maxrecords":"0","yarn.sharedcache.root-dir":"/sharedcache","fs.s3a.retry.throttle.limit":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","yarn.nodemanager.resource-monitor.interval-ms":"3000","yarn.nodemanager.resource-plugins.gpu.allowed-gpu-devices":"auto","mapreduce.job.sharedcache.mode":"disabled","yarn.nodemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.shuffle.listen.queue.size":"128","yarn.scheduler.configuration.mutation.acl-policy.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy","mapreduce.map.cpu.vcores":"1","yarn.log-aggregation.file-formats":"TFile","yarn.timeline-service.client.fd-retain-secs":"300","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","yarn.router.webapp.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.webapp.DefaultRequestInterceptorREST","yarn.nodemanager.resource.memory.cgroups.soft-limit-percentage":"90.0","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","yarn.nodemanager.disk-validator":"basic","yarn.app.mapreduce.client.job.max-retries":"3","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","hadoop.security.group.mapping.ldap.connection.timeout.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","yarn.nodemanager.container-diagnostics-maximum-size":"10000","hadoop.registry.zk.retry.interval.ms":"1000","yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","fs.s3a.assumed.role.session.duration":"30m","hadoop.security.group.mapping.ldap.conversion.rule":"none","hadoop.ssl.server.conf":"ssl-server.xml","fs.s3a.retry.throttle.interval":"1000ms","seq.io.sort.factor":"100","yarn.sharedcache.cleaner.initial-delay-mins":"10","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.nodemanager.runtime.linux.docker.capabilities":"CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE","yarn.acl.enable":"false","yarn.timeline-service.entity-group-fs-store.done-dir":"/tmp/entity-file-history/done/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","yarn.nodemanager.opportunistic-containers-use-pause-for-preemption":"false","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","yarn.timeline-service.reader.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","yarn.nodemanager.runtime.linux.docker.userremapping-uid-threshold":"1","yarn.resourcemanager.configuration.file-system-based-store":"/yarn/conf","mapreduce.job.cache.limit.max-single-resource-mb":"0","yarn.nodemanager.runtime.linux.docker.stop.grace-period":"10","yarn.resourcemanager.resource-profiles.source-file":"resource-profiles.json","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","yarn.nodemanager.container-localizer.log.level":"INFO","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","mapreduce.jobhistory.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","yarn.timeline-service.client.drain-entities.timeout.ms":"2000","yarn.nodemanager.resource-plugins.fpga.vendor-plugin.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.IntelFpgaOpenclPlugin","yarn.timeline-service.entity-group-fs-store.summary-store":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.reduce.cpu.vcores":"1","mapreduce.job.encrypted-intermediate-data.buffer.kb":"128","fs.client.resolve.remote.symlinks":"true","yarn.nodemanager.webapp.https.address":"0.0.0.0:8044","hadoop.http.cross-origin.allowed-origins":"*","mapreduce.job.encrypted-intermediate-data":"false","yarn.timeline-service.entity-group-fs-store.retain-seconds":"604800","yarn.resourcemanager.metrics.runtime.buckets":"60,300,1440","yarn.timeline-service.generic-application-history.max-applications":"10000","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","yarn.node-labels.configuration-type":"centralized","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","yarn.sharedcache.store.in-memory.staleness-period-mins":"10080","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","mapreduce.jobhistory.webapp.xfs-filter.xframe-options":"SAMEORIGIN","yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled":"false","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","yarn.timeline-service.reader.webapp.https.address":"${yarn.timeline-service.webapp.https.address}","fs.ftp.data.connection.mode":"ACTIVE_LOCAL_DATA_CONNECTION_MODE","mapreduce.job.local-fs.single-disk-limit.check.kill-limit-exceed":"true","yarn.scheduler.maximum-allocation-vcores":"4","hadoop.http.cross-origin.allowed-headers":"X-Requested-With,Content-Type,Accept,Origin","yarn.nodemanager.log-aggregation.compression-type":"none","yarn.timeline-service.version":"1.0f","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"0","ipc.maximum.response.length":"134217728","yarn.resourcemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","hadoop.caller.context.max.size":"128","yarn.nodemanager.runtime.linux.docker.host-pid-namespace.allowed":"false","yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed":"false","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","fs.s3a.max.total.tasks":"5","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.attempts.maximum":"20","hadoop.registry.zk.connection.timeout.ms":"15000","yarn.resourcemanager.delegation-token-renewer.thread-count":"*********(redacted)","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","yarn.resourcemanager.decommissioning-nodes-watcher.poll-interval-secs":"20","mapreduce.output.fileoutputformat.compress.type":"RECORD","yarn.resourcemanager.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.timeline-service.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.ifile.readahead.bytes":"4194304","yarn.sharedcache.app-checker.class":"org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","yarn.nodemanager.resource.detect-hardware-capabilities":"false","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","hadoop.security.group.mapping.ldap.search.group.hierarchy.levels":"0","yarn.resourcemanager.fs.state-store.retry-interval-ms":"1000","file.stream-buffer-size":"4096","yarn.resourcemanager.application-timeouts.monitor.interval-ms":"3000","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.map.speculative":"true","mapreduce.job.speculative.retry-after-speculate":"15000","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","yarn.app.mapreduce.am.log.level":"INFO","mapreduce.job.reduce.slowstart.completedmaps":"0.05","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","yarn.nodemanager.resource-plugins.fpga.allowed-fpga-devices":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","hadoop.http.logs.enabled":"true","fs.s3a.block.size":"32M","yarn.sharedcache.client-server.address":"0.0.0.0:8045","yarn.nodemanager.logaggregation.threadpool-size-max":"100","yarn.resourcemanager.hostname":"0.0.0.0","yarn.resourcemanager.delegation.key.update-interval":"86400000","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"-1","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","yarn.nodemanager.disk-health-checker.enable":"true","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","ipc.client.low-latency":"false","mapreduce.input.lineinputformat.linespermap":"1","yarn.router.interceptor.user.threadpool-size":"5","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.app-cache-size":"10","fs.s3a.socket.recv.buffer":"8192","yarn.resourcemanager.resource-tracker.address":"${yarn.resourcemanager.hostname}:8031","yarn.nodemanager.node-labels.provider.fetch-timeout-ms":"1200000","mapreduce.job.heap.memory-mb.ratio":"0.8","yarn.resourcemanager.leveldb-state-store.compaction-interval-secs":"3600","yarn.resourcemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","yarn.scheduler.configuration.fs.path":"file://${hadoop.tmp.dir}/yarn/system/schedconf","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"-1","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","yarn.nodemanager.container-metrics.period-ms":"-1","mapreduce.fileoutputcommitter.task.cleanup.enabled":"false","yarn.nodemanager.log.retain-seconds":"10800","yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds":"3600","yarn.resourcemanager.keytab":"/etc/krb5.keytab","hadoop.security.group.mapping.providers.combined":"true","mapreduce.reduce.merge.inmem.threshold":"1000","yarn.timeline-service.recovery.enabled":"false","fs.azure.saskey.usecontainersaskeyforallaccess":"true","yarn.sharedcache.nm.uploader.thread-count":"20","yarn.resourcemanager.nodemanager-graceful-decommission-timeout-secs":"3600","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","yarn.nodemanager.opportunistic-containers-max-queue-length":"0","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.http.cross-origin.allowed-methods":"GET,POST,HEAD","yarn.resourcemanager.opportunistic-container-allocation.enabled":"false","mapreduce.jobhistory.webapp.address":"0.0.0.0:19888","hadoop.system.tags":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.log-aggregation.file-controller.TFile.class":"org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"${yarn.resourcemanager.hostname}:8088","mapreduce.jobhistory.recovery.enable":"false","mapreduce.reduce.shuffle.parallelcopies":"5","fs.AbstractFileSystem.webhdfs.impl":"org.apache.hadoop.fs.WebHdfs","fs.trash.interval":"0","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","yarn.app.mapreduce.am.resource.mb":"1536","mapreduce.input.fileinputformat.list-status.num-threads":"1","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.resourcemanager.nm-container-queuing.min-queue-length":"5","yarn.nodemanager.resource.cpu-vcores":"-1","mapreduce.job.reduces":"1","fs.s3a.multipart.size":"100M","yarn.scheduler.minimum-allocation-vcores":"1","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","mapreduce.job.encrypted-intermediate-data-key-size-bits":"128","fs.s3a.metadatastore.authoritative":"false","yarn.nodemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","ha.health-monitor.sleep-after-disconnect.ms":"1000","yarn.app.mapreduce.shuffle.log.limit.kb":"0","hadoop.security.group.mapping":"org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback","yarn.client.application-client-protocol.poll-timeout-ms":"-1","mapreduce.jobhistory.jhist.format":"binary","yarn.resourcemanager.ha.enabled":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.jobhistory.intermediate-user-done-dir.permissions":"770","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","yarn.resourcemanager.reservation-system.enable":"false","mapreduce.map.output.compress":"false","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.nodemanager.runtime.linux.docker.default-container-network":"host","yarn.router.webapp.address":"0.0.0.0:8089","yarn.scheduler.maximum-allocation-mb":"8192","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.sharedcache.cleaner.period-mins":"1440","yarn.nodemanager.resource-plugins.gpu.docker-plugin.nvidia-docker-v1.endpoint":"http://localhost:3476/v1.0/docker/cli","yarn.app.mapreduce.am.container.log.limit.kb":"0","ipc.client.connect.retry.interval":"1000","yarn.timeline-service.http-cross-origin.enabled":"false","fs.wasbs.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem$Secure","yarn.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","mapreduce.jobhistory.cleaner.enable":"true","yarn.timeline-service.client.fd-flush-interval-secs":"10","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms":"1000","fs.s3a.committer.staging.tmp.path":"tmp/staging","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","yarn.resourcemanager.nm-container-queuing.min-queue-wait-time-ms":"10","mapreduce.job.end-notification.retry.attempts":"0","yarn.nodemanager.resource.count-logical-processors-as-cores":"false","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","yarn.resourcemanager.zk-max-znode-size.bytes":"1048576","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","yarn.nodemanager.localizer.cache.target-size-mb":"10240","fs.s3a.committer.staging.conflict-mode":"fail","mapreduce.client.libjars.wildcard":"true","fs.s3a.committer.staging.unique-filenames":"true","yarn.nodemanager.node-attributes.provider.fetch-timeout-ms":"1200000","fs.s3a.list.version":"2","ftp.client-write-packet-size":"65536","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","hadoop.security.key.default.cipher":"AES/CTR/NoPadding","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","mapreduce.job.local-fs.single-disk-limit.check.interval-ms":"5000","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","yarn.nodemanager.amrmproxy.address":"0.0.0.0:8049","ipc.server.listen.queue.size":"128","map.sort.class":"org.apache.hadoop.util.QuickSort","fs.viewfs.rename.strategy":"SAME_MOUNTPOINT","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","fs.s3a.assumed.role.credentials.provider":"org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider","yarn.nodemanager.vmem-check-enabled":"true","yarn.nodemanager.numa-awareness.enabled":"false","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","yarn.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.jobhistory.jobname.limit":"50","yarn.client.nodemanager-connect.retry-interval-ms":"10000","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME,HADOOP_HOME,PATH,LANG,TZ","yarn.sharedcache.nested-level":"3","yarn.timeline-service.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","fs.azure.user.agent.prefix":"unknown","yarn.resourcemanager.zk-delegation-token-node.split-index":"*********(redacted)","yarn.nodemanager.numa-awareness.read-topology":"false","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","yarn.registry.class":"org.apache.hadoop.registry.client.impl.FSRegistryOperationsService","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","yarn.scheduler.queue-placement-rules":"user-group","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"System Properties":{"java.io.tmpdir":"/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib","user.dir":"*********(redacted)","java.library.path":"*********(redacted)/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.231-b11","jetty.git.hash":"ab228fde9e55e9164c738d7fa121f8ac5acd51c9","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_231-b11","java.vm.info":"mixed mode","java.ext.dirs":"*********(redacted)/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.15.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","jdk.lang.Process.launchMechanism":"POSIX_SPAWN","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"terryk","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master spark://*********(redacted) --conf spark.eventLog.logStageExecutorMetrics=true --conf spark.eventLog.enabled=true --class org.apache.spark.repl.Main --name Spark shell spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre","java.version":"1.8.0_231","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"*********(redacted)/audience-annotations-0.5.0.jar":"System Classpath","*********(redacted)/zstd-jni-1.4.5-2.jar":"System Classpath","*********(redacted)/hadoop-yarn-common-3.2.0.jar":"System Classpath","*********(redacted)/apache-spark/common/unsafe/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerb-identity-1.0.1.jar":"System Classpath","*********(redacted)/jersey-hk2-2.30.jar":"System Classpath","*********(redacted)/apache-spark/mllib/target/jars/*":"System Classpath","*********(redacted)/slf4j-log4j12-1.7.30.jar":"System Classpath","*********(redacted)/kerby-pkix-1.0.1.jar":"System Classpath","*********(redacted)/metrics-jmx-4.1.1.jar":"System Classpath","*********(redacted)/xz-1.5.jar":"System Classpath","*********(redacted)/jakarta.validation-api-2.0.2.jar":"System Classpath","*********(redacted)/jetty-webapp-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/metrics-core-4.1.1.jar":"System Classpath","*********(redacted)/apache-spark/graphx/target/scala-2.12/classes/":"System Classpath","*********(redacted)/metrics-graphite-4.1.1.jar":"System Classpath","*********(redacted)/jersey-server-2.30.jar":"System Classpath","*********(redacted)/jetty-jndi-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-proxy-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/parquet-jackson-1.10.1.jar":"System Classpath","*********(redacted)/commons-crypto-1.0.0.jar":"System Classpath","*********(redacted)/xbean-asm7-shaded-4.15.jar":"System Classpath","*********(redacted)/spark-kvstore_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/netty-3.10.6.Final.jar":"System Classpath","*********(redacted)/jsr305-3.0.2.jar":"System Classpath","*********(redacted)/osgi-resource-locator-1.0.3.jar":"System Classpath","*********(redacted)/kerb-server-1.0.1.jar":"System Classpath","*********(redacted)/avro-1.8.2.jar":"System Classpath","*********(redacted)/jersey-media-jaxb-2.30.jar":"System Classpath","*********(redacted)/accessors-smart-1.2.jar":"System Classpath","*********(redacted)/spark-network-shuffle_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/protobuf-java-2.5.0.jar":"System Classpath","*********(redacted)/minlog-1.3.0.jar":"System Classpath","*********(redacted)/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","*********(redacted)/jetty-continuation-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/apache-spark/sql/hive/target/scala-2.12/classes/":"System Classpath","*********(redacted)/commons-net-3.6.jar":"System Classpath","*********(redacted)/apache-spark/core/target/jars/*":"System Classpath","*********(redacted)/istack-commons-runtime-3.0.8.jar":"System Classpath","*********(redacted)/jsp-api-2.1.jar":"System Classpath","*********(redacted)/hadoop-annotations-3.2.0.jar":"System Classpath","*********(redacted)/spark-launcher_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jetty-servlets-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/hadoop-hdfs-client-3.2.0.jar":"System Classpath","*********(redacted)/hive-storage-api-2.7.1.jar":"System Classpath","*********(redacted)/aopalliance-repackaged-2.6.1.jar":"System Classpath","*********(redacted)/kerby-xdr-1.0.1.jar":"System Classpath","*********(redacted)/orc-mapreduce-1.5.10.jar":"System Classpath","*********(redacted)/jackson-databind-2.10.0.jar":"System Classpath","*********(redacted)/spark-mllib_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","*********(redacted)/hk2-utils-2.6.1.jar":"System Classpath","*********(redacted)/jetty-security-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/hadoop-client-3.2.0.jar":"System Classpath","*********(redacted)/hadoop-auth-3.2.0.jar":"System Classpath","*********(redacted)/gson-2.2.4.jar":"System Classpath","*********(redacted)/htrace-core4-4.1.0-incubating.jar":"System Classpath","*********(redacted)/curator-client-2.12.0.jar":"System Classpath","*********(redacted)/kerby-util-1.0.1.jar":"System Classpath","*********(redacted)/jackson-jaxrs-base-2.9.5.jar":"System Classpath","*********(redacted)/jackson-module-paranamer-2.10.0.jar":"System Classpath","*********(redacted)/commons-beanutils-1.9.3.jar":"System Classpath","*********(redacted)/woodstox-core-5.0.3.jar":"System Classpath","*********(redacted)/arrow-memory-0.15.1.jar":"System Classpath","*********(redacted)/activation-1.1.1.jar":"System Classpath","*********(redacted)/okhttp-2.7.5.jar":"System Classpath","*********(redacted)/jackson-mapper-asl-1.9.13.jar":"System Classpath","*********(redacted)/core-1.1.2.jar":"System Classpath","*********(redacted)/jetty-plus-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/apache-spark/resource-managers/yarn/target/scala-2.12/classes/":"System Classpath","*********(redacted)/antlr4-runtime-4.7.1.jar":"System Classpath","*********(redacted)/commons-compress-1.8.1.jar":"System Classpath","*********(redacted)/spark-tags_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/orc-shims-1.5.10.jar":"System Classpath","*********(redacted)/opencsv-2.3.jar":"System Classpath","*********(redacted)/compress-lzf-1.0.3.jar":"System Classpath","*********(redacted)/jackson-core-asl-1.9.13.jar":"System Classpath","*********(redacted)/objenesis-2.5.1.jar":"System Classpath","*********(redacted)/apache-spark/conf/":"System Classpath","*********(redacted)/apache-spark/common/network-common/target/scala-2.12/classes/":"System Classpath","*********(redacted)/httpcore-4.4.4.jar":"System Classpath","*********(redacted)/jcip-annotations-1.0-1.jar":"System Classpath","*********(redacted)/apache-spark/launcher/target/scala-2.12/classes/":"System Classpath","*********(redacted)/apache-spark/resource-managers/mesos/target/scala-2.12/classes":"System Classpath","*********(redacted)/apache-spark/sql/hive-thriftserver/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jetty-util-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-server-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-xml-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","*********(redacted)/javax.servlet-api-3.1.0.jar":"System Classpath","*********(redacted)/jackson-core-2.10.0.jar":"System Classpath","*********(redacted)/jetty-client-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/netty-all-4.1.47.Final.jar":"System Classpath","*********(redacted)/arpack_combined_all-0.1.jar":"System Classpath","*********(redacted)/jersey-container-servlet-core-2.30.jar":"System Classpath","*********(redacted)/paranamer-2.8.jar":"System Classpath","*********(redacted)/spark-sql_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/hadoop-common-3.2.0.jar":"System Classpath","*********(redacted)/jersey-common-2.30.jar":"System Classpath","*********(redacted)/hk2-locator-2.6.1.jar":"System Classpath","*********(redacted)/okio-1.6.0.jar":"System Classpath","*********(redacted)/jetty-http-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","*********(redacted)/ivy-2.4.0.jar":"System Classpath","*********(redacted)/apache-spark/mllib/target/scala-2.12/classes/":"System Classpath","*********(redacted)/scala-library-2.12.10.jar":"System Classpath","*********(redacted)/scala-xml_2.12-1.2.0.jar":"System Classpath","*********(redacted)/apache-spark/streaming/target/scala-2.12/classes/":"System Classpath","*********(redacted)/spark-repl_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/apache-spark/common/tags/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerby-config-1.0.1.jar":"System Classpath","*********(redacted)/zookeeper-3.4.14.jar":"System Classpath","*********(redacted)/janino-3.1.2.jar":"System Classpath","*********(redacted)/breeze_2.12-1.0.jar":"System Classpath","*********(redacted)/hk2-api-2.6.1.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","*********(redacted)/flatbuffers-java-1.9.0.jar":"System Classpath","*********(redacted)/curator-framework-2.13.0.jar":"System Classpath","*********(redacted)/metrics-jvm-4.1.1.jar":"System Classpath","*********(redacted)/stax2-api-3.1.4.jar":"System Classpath","*********(redacted)/json-smart-2.3.jar":"System Classpath","*********(redacted)/parquet-hadoop-1.10.1.jar":"System Classpath","*********(redacted)/kerb-common-1.0.1.jar":"System Classpath","*********(redacted)/breeze-macros_2.12-1.0.jar":"System Classpath","*********(redacted)/scala-compiler-2.12.10.jar":"System Classpath","*********(redacted)/guava-14.0.1.jar":"System Classpath","*********(redacted)/jul-to-slf4j-1.7.30.jar":"System Classpath","*********(redacted)/spark-assembly_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-core_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-graphx_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","*********(redacted)/spark-unsafe_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jetty-io-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/JTransforms-3.1.jar":"System Classpath","*********(redacted)/commons-io-2.5.jar":"System Classpath","*********(redacted)/spark-catalyst_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spire_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/commons-cli-1.2.jar":"System Classpath","*********(redacted)/shims-0.7.45.jar":"System Classpath","*********(redacted)/spotbugs-annotations-3.1.9.jar":"System Classpath","*********(redacted)/kerb-simplekdc-1.0.1.jar":"System Classpath","*********(redacted)/commons-codec-1.11.jar":"System Classpath","*********(redacted)/jcl-over-slf4j-1.7.30.jar":"System Classpath","*********(redacted)/lz4-java-1.7.1.jar":"System Classpath","*********(redacted)/aopalliance-1.0.jar":"System Classpath","*********(redacted)/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","*********(redacted)/json4s-jackson_2.12-3.6.6.jar":"System Classpath","*********(redacted)/commons-math3-3.4.1.jar":"System Classpath","*********(redacted)/shapeless_2.12-2.3.3.jar":"System Classpath","*********(redacted)/spark-mllib-local_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/dnsjava-2.1.7.jar":"System Classpath","*********(redacted)/unused-1.0.0.jar":"System Classpath","*********(redacted)/log4j-1.2.17.jar":"System Classpath","*********(redacted)/kerb-client-1.0.1.jar":"System Classpath","*********(redacted)/commons-lang-2.6.jar":"System Classpath","*********(redacted)/apache-spark/repl/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerb-util-1.0.1.jar":"System Classpath","*********(redacted)/slf4j-api-1.7.30.jar":"System Classpath","*********(redacted)/jackson-annotations-2.10.0.jar":"System Classpath","*********(redacted)/kerb-core-1.0.1.jar":"System Classpath","*********(redacted)/apache-spark/sql/catalyst/target/scala-2.12/classes/":"System Classpath","*********(redacted)/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/apache-spark/examples/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jakarta.annotation-api-1.3.5.jar":"System Classpath","*********(redacted)/json4s-scalap_2.12-3.6.6.jar":"System Classpath","*********(redacted)/apache-spark/core/target/scala-2.12/classes/":"System Classpath","*********(redacted)/arrow-format-0.15.1.jar":"System Classpath","*********(redacted)/kryo-shaded-4.0.2.jar":"System Classpath","*********(redacted)/scala-reflect-2.12.10.jar":"System Classpath","*********(redacted)/kerb-admin-1.0.1.jar":"System Classpath","*********(redacted)/hadoop-yarn-client-3.2.0.jar":"System Classpath","*********(redacted)/nimbus-jose-jwt-4.41.1.jar":"System Classpath","*********(redacted)/apache-spark/common/kvstore/target/scala-2.12/classes/":"System Classpath","*********(redacted)/oro-2.0.8.jar":"System Classpath","*********(redacted)/jakarta.inject-2.6.1.jar":"System Classpath","*********(redacted)/chill-java-0.9.5.jar":"System Classpath","*********(redacted)/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","*********(redacted)/jersey-container-servlet-2.30.jar":"System Classpath","*********(redacted)/py4j-0.10.9.jar":"System Classpath","*********(redacted)/parquet-format-2.4.0.jar":"System Classpath","*********(redacted)/apache-spark/sql/core/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jline-2.14.6.jar":"System Classpath","*********(redacted)/JLargeArrays-1.5.jar":"System Classpath","*********(redacted)/kerby-asn1-1.0.1.jar":"System Classpath","*********(redacted)/jaxb-runtime-2.3.2.jar":"System Classpath","*********(redacted)/pmml-model-1.4.8.jar":"System Classpath","*********(redacted)/parquet-encoding-1.10.1.jar":"System Classpath","*********(redacted)/machinist_2.12-0.6.8.jar":"System Classpath","*********(redacted)/commons-compiler-3.1.2.jar":"System Classpath","*********(redacted)/kerb-crypto-1.0.1.jar":"System Classpath","*********(redacted)/aircompressor-0.10.jar":"System Classpath","*********(redacted)/leveldbjni-all-1.8.jar":"System Classpath","*********(redacted)/metrics-json-4.1.1.jar":"System Classpath","*********(redacted)/jackson-module-jaxb-annotations-2.9.5.jar":"System Classpath","*********(redacted)/jaxb-api-2.2.11.jar":"System Classpath","*********(redacted)/spire-util_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/jetty-servlet-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/spark-network-common_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-streaming_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/json4s-core_2.12-3.6.6.jar":"System Classpath","*********(redacted)/stream-2.9.6.jar":"System Classpath","*********(redacted)/spark-sketch_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/chill_2.12-0.9.5.jar":"System Classpath","*********(redacted)/commons-configuration2-2.1.1.jar":"System Classpath","*********(redacted)/univocity-parsers-2.8.3.jar":"System Classpath","*********(redacted)/parquet-common-1.10.1.jar":"System Classpath","*********(redacted)/threeten-extra-1.5.0.jar":"System Classpath","*********(redacted)/token-provider-1.0.1.jar":"System Classpath","*********(redacted)/commons-text-1.6.jar":"System Classpath","*********(redacted)/apache-spark/common/sketch/target/scala-2.12/classes/":"System Classpath","*********(redacted)/javax.inject-1.jar":"System Classpath","*********(redacted)/httpclient-4.5.2.jar":"System Classpath","*********(redacted)/avro-ipc-1.8.2.jar":"System Classpath","*********(redacted)/curator-recipes-2.13.0.jar":"System Classpath","*********(redacted)/commons-collections-3.2.2.jar":"System Classpath","*********(redacted)/jersey-client-2.30.jar":"System Classpath","*********(redacted)/snappy-java-1.1.7.5.jar":"System Classpath","*********(redacted)/algebra_2.12-2.0.0-M2.jar":"System Classpath","*********(redacted)/javassist-3.25.0-GA.jar":"System Classpath","*********(redacted)/arrow-vector-0.15.1.jar":"System Classpath","*********(redacted)/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","*********(redacted)/re2j-1.1.jar":"System Classpath","*********(redacted)/parquet-column-1.10.1.jar":"System Classpath","*********(redacted)/pyrolite-4.30.jar":"System Classpath","*********(redacted)/apache-spark/common/network-shuffle/target/scala-2.12/classes/":"System Classpath","*********(redacted)/apache-spark/common/network-yarn/target/scala-2.12/classes":"System Classpath","*********(redacted)/hadoop-yarn-api-3.2.0.jar":"System Classpath","*********(redacted)/macro-compat_2.12-1.1.1.jar":"System Classpath","*********(redacted)/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/commons-lang3-3.10.jar":"System Classpath","*********(redacted)/json4s-ast_2.12-3.6.6.jar":"System Classpath","*********(redacted)/RoaringBitmap-0.7.45.jar":"System Classpath","*********(redacted)/orc-core-1.5.10.jar":"System Classpath","*********(redacted)/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20200706201101-0003","Timestamp":1594091460235,"User":"terryk"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1594091463318,"Executor ID":"0","Executor Info":{"Host":"127.0.0.1","Total Cores":16,"Log Urls":{"stdout":"http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout","stderr":"http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"127.0.0.1","Port":64419},"Maximum Memory":384093388,"Timestamp":1594091463413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"foreach at :26","details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","physicalPlanDescription":"== Physical Plan ==\nDeserializeToObject (8)\n+- * BroadcastHashJoin Inner BuildLeft (7)\n :- BroadcastExchange (5)\n : +- * BroadcastHashJoin Inner BuildRight (4)\n : :- * LocalTableScan (1)\n : +- BroadcastExchange (3)\n : +- LocalTableScan (2)\n +- * LocalTableScan (6)\n\n\n(1) LocalTableScan [codegen id : 1]\nOutput [3]: [i1#10, j1#11, k1#12]\nArguments: [i1#10, j1#11, k1#12]\n\n(2) LocalTableScan\nOutput [3]: [i2#26, j2#27, k2#28]\nArguments: [i2#26, j2#27, k2#28]\n\n(3) BroadcastExchange\nInput [3]: [i2#26, j2#27, k2#28]\nArguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#23]\n\n(4) BroadcastHashJoin [codegen id : 1]\nLeft keys [1]: [i1#10]\nRight keys [1]: [i2#26]\nJoin condition: None\n\n(5) BroadcastExchange\nInput [6]: [i1#10, j1#11, k1#12, i2#26, j2#27, k2#28]\nArguments: HashedRelationBroadcastMode(List(cast(input[3, int, false] as bigint))), [id=#32]\n\n(6) LocalTableScan\nOutput [3]: [i3#42, j3#43, k3#44]\nArguments: [i3#42, j3#43, k3#44]\n\n(7) BroadcastHashJoin [codegen id : 2]\nLeft keys [1]: [i2#26]\nRight keys [1]: [i3#42]\nJoin condition: None\n\n(8) DeserializeToObject\nInput [9]: [i1#10, j1#11, k1#12, i2#26, j2#27, k2#28, i3#42, j3#43, k3#44]\nArguments: createexternalrow(i1#10, j1#11, k1#12.toString, i2#26, j2#27, k2#28.toString, i3#42, j3#43, k3#44.toString, StructField(i1,IntegerType,false), StructField(j1,IntegerType,false), StructField(k1,StringType,true), StructField(i2,IntegerType,false), StructField(j2,IntegerType,false), StructField(k2,StringType,true), StructField(i3,IntegerType,false), StructField(j3,IntegerType,false), StructField(k3,StringType,true)), obj#93: org.apache.spark.sql.Row\n\n","sparkPlanInfo":{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(i1#10, j1#11, k1#12.toString, i2#26, j2#27, k2#28.toString, i3#42, j3#43, k3#44.toString, StructField(i1,IntegerType,false), StructField(j1,IntegerType,false), StructField(k1,StringType,true), StructField(i2,IntegerType,false), StructField(j2,IntegerType,false), StructField(k2,StringType,true), StructField(i3,IntegerType,false), StructField(j3,IntegerType,false), StructField(k3,StringType,true)), obj#93: org.apache.spark.sql.Row","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [i2#26], [i3#42], Inner, BuildLeft","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(cast(input[3, int, false] as bigint))), [id=#32]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [i1#10], [i2#26], Inner, BuildRight","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i1#10, j1#11, k1#12]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":8,"metricType":"sum"}]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#23]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i2#26, j2#27, k2#28]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":13,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"data size","accumulatorId":9,"metricType":"size"},{"name":"time to collect","accumulatorId":10,"metricType":"timing"},{"name":"time to build","accumulatorId":11,"metricType":"timing"},{"name":"time to broadcast","accumulatorId":12,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":7,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":6,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"time to collect","accumulatorId":3,"metricType":"timing"},{"name":"time to build","accumulatorId":4,"metricType":"timing"},{"name":"time to broadcast","accumulatorId":5,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i3#42, j3#43, k3#44]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":0,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1594091478577} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1594091478844,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)","spark.executor.id":"driver","spark.jobGroup.id":"bb0234e5-4157-49a4-b40c-6d538d9f2ec8","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091478860,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)","spark.executor.id":"driver","spark.jobGroup.id":"bb0234e5-4157-49a4-b40c-6d538d9f2ec8","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1594091479253,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1594091479271,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1594091479271,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480364,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":109,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":1387,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8922000,"Value":8922000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":80,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":235295000,"Value":235295000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":962,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":235295000,"Executor Run Time":80,"Executor CPU Time":8922000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":218,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":2778,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8860000,"Value":17782000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":81,"Value":161,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":213308000,"Value":448603000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":1925,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":213308000,"Executor Run Time":81,"Executor CPU Time":8860000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":327,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":4165,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10463000,"Value":28245000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":240,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":228677000,"Value":677280000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":2887,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":228677000,"Executor Run Time":79,"Executor CPU Time":10463000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":436,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1390,"Value":5555,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8782000,"Value":37027000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":83,"Value":323,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":196368000,"Value":873648000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":3849,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":196368000,"Executor Run Time":83,"Executor CPU Time":8782000,"Peak Execution Memory":0,"Result Size":1390,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1594091479253,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":545,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1384,"Value":6939,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10820000,"Value":47847000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":77,"Value":400,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":221708000,"Value":1095356000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":4811,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":221708000,"Executor Run Time":77,"Executor CPU Time":10820000,"Peak Execution Memory":0,"Result Size":1384,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":654,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1406,"Value":8345,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":13213000,"Value":61060000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":78,"Value":478,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":260380000,"Value":1355736000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":5774,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":260380000,"Executor Run Time":78,"Executor CPU Time":13213000,"Peak Execution Memory":0,"Result Size":1406,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":763,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":9736,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":9913000,"Value":70973000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":76,"Value":554,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":210788000,"Value":1566524000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":6736,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":210788000,"Executor Run Time":76,"Executor CPU Time":9913000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":872,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":11127,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8784000,"Value":79757000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":633,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":235620000,"Value":1802144000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":964,"Value":7700,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":964,"Executor Deserialize CPU Time":235620000,"Executor Run Time":79,"Executor CPU Time":8784000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":981,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":12518,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":12053000,"Value":91810000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":86,"Value":719,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":215398000,"Value":2017542000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":8663,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":215398000,"Executor Run Time":86,"Executor CPU Time":12053000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1090,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1403,"Value":13921,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":9030000,"Value":100840000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":798,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":196266000,"Value":2213808000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":9625,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":196266000,"Executor Run Time":79,"Executor CPU Time":9030000,"Peak Execution Memory":0,"Result Size":1403,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1199,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1390,"Value":15311,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10087000,"Value":110927000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":877,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":195342000,"Value":2409150000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":10587,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":195342000,"Executor Run Time":79,"Executor CPU Time":10087000,"Peak Execution Memory":0,"Result Size":1390,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1308,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1411,"Value":16722,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":12920000,"Value":123847000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":86,"Value":963,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":236044000,"Value":2645194000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":961,"Value":11548,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":961,"Executor Deserialize CPU Time":236044000,"Executor Run Time":86,"Executor CPU Time":12920000,"Peak Execution Memory":0,"Result Size":1411,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1417,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1430,"Value":18152,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10478000,"Value":134325000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":77,"Value":1040,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":205925000,"Value":2851119000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":12510,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":205925000,"Executor Run Time":77,"Executor CPU Time":10478000,"Peak Execution Memory":0,"Result Size":1430,"JVM GC Time":109,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1526,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":19539,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8972000,"Value":143297000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":87,"Value":1127,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":206247000,"Value":3057366000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":13473,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":206247000,"Executor Run Time":87,"Executor CPU Time":8972000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1635,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1452,"Value":20991,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":20898000,"Value":164195000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":83,"Value":1210,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":389356000,"Value":3446722000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":14436,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":389356000,"Executor Run Time":83,"Executor CPU Time":20898000,"Peak Execution Memory":0,"Result Size":1452,"JVM GC Time":109,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1744,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1384,"Value":22375,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8668000,"Value":172863000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":82,"Value":1292,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":222167000,"Value":3668889000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":15399,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":222167000,"Executor Run Time":82,"Executor CPU Time":8668000,"Peak Execution Memory":0,"Result Size":1384,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091478860,"Completion Time":1594091480381,"Accumulables":[{"ID":17,"Name":"internal.metrics.executorRunTime","Value":1292,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Value":1744,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"number of output rows","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3668889000,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Value":22375,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Value":172863000,"Internal":true,"Count Failed Values":true},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Value":15399,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1594091480385,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[9,1048632],[10,1736],[11,37],[12,5]]} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1594091480498,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)","spark.executor.id":"driver","spark.jobGroup.id":"bac7289a-c1d8-4966-a6a3-d9f347e13a5d","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091480499,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)","spark.executor.id":"driver","spark.jobGroup.id":"bac7289a-c1d8-4966-a6a3-d9f347e13a5d","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1594091480502,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":8,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":9,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":10,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":11,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":12,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":13,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":14,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":15,"Attempt":0,"Launch Time":1594091480507,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480921,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":1016,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1966,"Value":1966,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3116000,"Value":3116000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":378,"Value":378,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4258000,"Value":4258000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":4258000,"Executor Run Time":378,"Executor CPU Time":3116000,"Peak Execution Memory":1016,"Result Size":1966,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480921,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":2032,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1951,"Value":3917,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2895000,"Value":6011000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":379,"Value":757,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2641000,"Value":6899000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":58,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":2641000,"Executor Run Time":379,"Executor CPU Time":2895000,"Peak Execution Memory":1016,"Result Size":1951,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480923,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"60","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"54","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":3048,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1957,"Value":5874,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4164000,"Value":10175000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":1137,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2012000,"Value":8911000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":86,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":2012000,"Executor Run Time":380,"Executor CPU Time":4164000,"Peak Execution Memory":1016,"Result Size":1957,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":9,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480923,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"80","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"72","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":4064,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1957,"Value":7831,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4285000,"Value":14460000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":1517,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2749000,"Value":11660000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":112,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":2749000,"Executor Run Time":380,"Executor CPU Time":4285000,"Peak Execution Memory":1016,"Result Size":1957,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480924,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"93","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"31","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":5080,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2007,"Value":9838,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":16921000,"Value":31381000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":1900,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3854000,"Value":15514000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":138,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":3854000,"Executor Run Time":383,"Executor CPU Time":16921000,"Peak Execution Memory":1016,"Result Size":2007,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":14,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480925,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"120","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"111","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":6096,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1955,"Value":11793,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4798000,"Value":36179000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":2282,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2959000,"Value":18473000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":162,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":2959000,"Executor Run Time":382,"Executor CPU Time":4798000,"Peak Execution Memory":1016,"Result Size":1955,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":11,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480926,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"140","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"132","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":7112,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2015,"Value":13808,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4850000,"Value":41029000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":2664,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4278000,"Value":22751000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":187,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":4278000,"Executor Run Time":382,"Executor CPU Time":4850000,"Peak Execution Memory":1016,"Result Size":2015,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"161","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"153","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"51","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":8128,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1998,"Value":15806,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4100000,"Value":45129000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":3047,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3622000,"Value":26373000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":215,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":3622000,"Executor Run Time":383,"Executor CPU Time":4100000,"Peak Execution Memory":1016,"Result Size":1998,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":10,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"182","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"57","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":9144,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1973,"Value":17779,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4350000,"Value":49479000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":3430,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3471000,"Value":29844000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":240,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":3471000,"Executor Run Time":383,"Executor CPU Time":4350000,"Peak Execution Memory":1016,"Result Size":1973,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":12,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"202","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":10160,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1966,"Value":19745,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4682000,"Value":54161000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":384,"Value":3814,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2652000,"Value":32496000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":23,"Value":263,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":23,"Executor Deserialize CPU Time":2652000,"Executor Run Time":384,"Executor CPU Time":4682000,"Peak Execution Memory":1016,"Result Size":1966,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":15,"Attempt":0,"Launch Time":1594091480507,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"223","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"210","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"70","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":11176,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2008,"Value":21753,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3954000,"Value":58115000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":4196,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4289000,"Value":36785000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":288,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":4289000,"Executor Run Time":382,"Executor CPU Time":3954000,"Peak Execution Memory":1016,"Result Size":2008,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"243","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"228","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"76","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":12192,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1978,"Value":23731,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":349926000,"Value":408041000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":385,"Value":4581,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":14543000,"Value":51328000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":27,"Value":315,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":27,"Executor Deserialize CPU Time":14543000,"Executor Run Time":385,"Executor CPU Time":349926000,"Peak Execution Memory":1016,"Result Size":1978,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"263","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"246","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"82","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":13208,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1965,"Value":25696,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4473000,"Value":412514000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":381,"Value":4962,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3906000,"Value":55234000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":27,"Value":342,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":27,"Executor Deserialize CPU Time":3906000,"Executor Run Time":381,"Executor CPU Time":4473000,"Peak Execution Memory":1016,"Result Size":1965,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":13,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480929,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"283","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"264","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":14224,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1955,"Value":27651,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":6459000,"Value":418973000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":381,"Value":5343,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3298000,"Value":58532000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":366,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":3298000,"Executor Run Time":381,"Executor CPU Time":6459000,"Peak Execution Memory":1016,"Result Size":1955,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":8,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480929,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"304","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"282","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"94","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":15240,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1954,"Value":29605,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4365000,"Value":423338000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":5723,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2411000,"Value":60943000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":394,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":2411000,"Executor Run Time":380,"Executor CPU Time":4365000,"Peak Execution Memory":1016,"Result Size":1954,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1594091480502,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480930,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"324","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":16256,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1941,"Value":31546,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3111000,"Value":426449000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":6105,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2723000,"Value":63666000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":30,"Value":424,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":30,"Executor Deserialize CPU Time":2723000,"Executor Run Time":382,"Executor CPU Time":3111000,"Peak Execution Memory":1016,"Result Size":1941,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091480499,"Completion Time":1594091480930,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Value":63666000,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"number of output rows","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":44,"Name":"internal.metrics.resultSize","Value":31546,"Internal":true,"Count Failed Values":true},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Value":16256,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Value":424,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"number of output rows","Value":"300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.executorCpuTime","Value":426449000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"duration","Value":"324","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":42,"Name":"internal.metrics.executorRunTime","Value":6105,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1594091480930,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[2,1048616],[3,2276],[4,13],[5,2]]} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1594091481039,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[2],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"19\",\"name\":\"foreach\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Submission Time":1594091481040,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"19\",\"name\":\"foreach\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1594091481077,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1594091481082,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1594091481087,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1594091481091,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1594091481095,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1594091481100,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1594091481104,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1594091481109,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1594091481112,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1594091481116,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1594091481120,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1594091481123,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1594091481126,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1594091481129,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1594091481132,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1594091481136,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1594091481104,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091482939,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1467","Value":"1467","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"375000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"6250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":24040,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.resultSerializationTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2158,"Value":2158,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":414110000,"Value":414110000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1498,"Value":1498,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":60358000,"Value":60358000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":255,"Value":255,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":255,"Executor Deserialize CPU Time":60358000,"Executor Run Time":1498,"Executor CPU Time":414110000,"Peak Execution Memory":24040,"Result Size":2158,"JVM GC Time":62,"Result Serialization Time":11,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1594091481136,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483014,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1563","Value":"3030","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"750000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"12500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":48080,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":52,"Value":114,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":4273,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":1324251000,"Value":1738361000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1594,"Value":3092,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":47496000,"Value":107854000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":213,"Value":468,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":213,"Executor Deserialize CPU Time":47496000,"Executor Run Time":1594,"Executor CPU Time":1324251000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":52,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1594091481100,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483015,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1563","Value":"4593","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1125000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"18750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":72120,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":176,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":6388,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":507192000,"Value":2245553000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":4688,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":60890000,"Value":168744000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":256,"Value":724,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":256,"Executor Deserialize CPU Time":60890000,"Executor Run Time":1596,"Executor CPU Time":507192000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1594091481095,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483015,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1564","Value":"6157","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1500000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"25000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":96160,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":238,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":8503,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":503010000,"Value":2748563000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":6284,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":112849000,"Value":281593000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":260,"Value":984,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":260,"Executor Deserialize CPU Time":112849000,"Executor Run Time":1596,"Executor CPU Time":503010000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1594091481091,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483016,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"7723","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1875000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"31250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":120200,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":300,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":10618,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":502908000,"Value":3251471000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1598,"Value":7882,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":62944000,"Value":344537000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":263,"Value":1247,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":263,"Executor Deserialize CPU Time":62944000,"Executor Run Time":1598,"Executor CPU Time":502908000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1594091481112,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483016,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1562","Value":"9285","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"2250000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"37500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":144240,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":362,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":12733,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":510597000,"Value":3762068000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1595,"Value":9477,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":69760000,"Value":414297000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":246,"Value":1493,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":246,"Executor Deserialize CPU Time":69760000,"Executor Run Time":1595,"Executor CPU Time":510597000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1594091481129,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483024,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1564","Value":"10849","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"2625000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"43750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":168280,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":424,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":14848,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":495138000,"Value":4257206000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1595,"Value":11072,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54222000,"Value":468519000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":221,"Value":1714,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":221,"Executor Deserialize CPU Time":54222000,"Executor Run Time":1595,"Executor CPU Time":495138000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1594091481109,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483024,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"12415","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"50000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":192320,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":486,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":16963,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":539451000,"Value":4796657000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":12668,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":64380000,"Value":532899000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":254,"Value":1968,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":254,"Executor Deserialize CPU Time":64380000,"Executor Run Time":1596,"Executor CPU Time":539451000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1594091481082,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483025,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"13981","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3375000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"56250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":216360,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":548,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":19078,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":519178000,"Value":5315835000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1597,"Value":14265,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54442000,"Value":587341000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":267,"Value":2235,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":267,"Executor Deserialize CPU Time":54442000,"Executor Run Time":1597,"Executor CPU Time":519178000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1594091481087,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483026,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1574","Value":"15555","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3750000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"62500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":240400,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":614,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":21193,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":508433000,"Value":5824268000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1606,"Value":15871,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":69492000,"Value":656833000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":265,"Value":2500,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":265,"Executor Deserialize CPU Time":69492000,"Executor Run Time":1606,"Executor CPU Time":508433000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1594091481123,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483029,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1578","Value":"17133","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4125000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"68750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":264440,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":680,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":23308,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":502120000,"Value":6326388000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1609,"Value":17480,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":48849000,"Value":705682000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":225,"Value":2725,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":225,"Executor Deserialize CPU Time":48849000,"Executor Run Time":1609,"Executor CPU Time":502120000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1594091481116,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483032,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1583","Value":"18716","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4500000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"75000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":288480,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":746,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":25423,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":489923000,"Value":6816311000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1614,"Value":19094,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":55787000,"Value":761469000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":240,"Value":2965,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":240,"Executor Deserialize CPU Time":55787000,"Executor Run Time":1614,"Executor CPU Time":489923000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1594091481132,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483037,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1587","Value":"20303","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4875000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"81250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":312520,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":812,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":27538,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":490927000,"Value":7307238000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1618,"Value":20712,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51464000,"Value":812933000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":218,"Value":3183,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":218,"Executor Deserialize CPU Time":51464000,"Executor Run Time":1618,"Executor CPU Time":490927000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1594091481077,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483037,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1587","Value":"21890","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"5250000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"87500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":336560,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":878,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":29653,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":496683000,"Value":7803921000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1619,"Value":22331,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":56827000,"Value":869760000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":271,"Value":3454,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":271,"Executor Deserialize CPU Time":56827000,"Executor Run Time":1619,"Executor CPU Time":496683000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1594091481120,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483043,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1593","Value":"23483","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"5625000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"93750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":360600,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":944,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":31768,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":508230000,"Value":8312151000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1624,"Value":23955,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":58152000,"Value":927912000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":229,"Value":3683,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":229,"Executor Deserialize CPU Time":58152000,"Executor Run Time":1624,"Executor CPU Time":508230000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1594091481126,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483043,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1593","Value":"25076","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"6000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"100000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":384640,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":1010,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":33883,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":498187000,"Value":8810338000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1624,"Value":25579,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51988000,"Value":979900000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":222,"Value":3905,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":222,"Executor Deserialize CPU Time":51988000,"Executor Run Time":1624,"Executor CPU Time":498187000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":213367864,"JVMOffHeapMemory":189011656,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":2133349,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":2133349,"OffHeapUnifiedMemory":0,"DirectPoolMemory":282024,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":13,"MinorGCTime":115,"MajorGCCount":4,"MajorGCTime":339}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Submission Time":1594091481040,"Completion Time":1594091483044,"Accumulables":[{"ID":68,"Name":"internal.metrics.executorCpuTime","Value":8810338000,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.resultSerializationTime","Value":11,"Internal":true,"Count Failed Values":true},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Value":384640,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Value":3905,"Internal":true,"Count Failed Values":true},{"ID":14,"Name":"number of output rows","Value":"100000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"internal.metrics.executorRunTime","Value":25579,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Value":1010,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"6000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":69,"Name":"internal.metrics.resultSize","Value":33883,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Value":979900000,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"duration","Value":"25076","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1594091483044,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1594091483045} +{"Event":"SparkListenerApplicationEnd","Timestamp":1594091824231} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index d30c1dfea9145..51e38f9cdcd2d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -178,6 +178,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "executor memory usage" -> "applications/app-20161116163331-0000/executors", "executor resource information" -> "applications/application_1555004656427_0144/executors", "multiple resource profiles" -> "applications/application_1578436911597_0052/environment", + "stage list with peak metrics" -> "applications/app-20200706201101-0003/stages", + "stage with peak metrics" -> "applications/app-20200706201101-0003/stages/2/0", "app environment" -> "applications/app-20161116163331-0000/environment", diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 24eb1685f577a..d5829c352be9b 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1523,14 +1523,32 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.metricToOffset.foreach { metric => - assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) - } + checkExecutorMetrics(metrics, actual) case _ => assert(false) } } } + + // check stage level executor metrics + val expectedStageValues = Map( + 0 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 80L, 50L, 20L, 50L, 10L, 100L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)), + Map( + "1" -> new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), + 1 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 70L, 25L, 40L, 60L, 30L, 70L, 60L, + 40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)), + Map( + "1" -> new ExecutorMetrics(Array(7000L, 70L, 20L, 30L, 60L, 30L, 70L, 55L, + 30L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)), + "2" -> new ExecutorMetrics(Array(5500L, 40L, 25L, 40L, 10L, 30L, 35L, 60L, + 40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) + checkStageExecutorMetrics(expectedStageValues) } test("stage executor metrics") { @@ -1573,14 +1591,74 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.metricToOffset.foreach { metric => - assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) - } + checkExecutorMetrics(metrics, actual) case _ => assert(false) } } } + + // check stage level executor metrics + val expectedStageValues = Map( + 0 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 50L, 10L, 100L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)), + Map( + "1" -> new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), + 1 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 60L, 30L, 80L, 60L, + 50L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)), + Map( + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, + 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) + checkStageExecutorMetrics(expectedStageValues) + } + + /** expected stage executor metrics */ + private case class StageExecutorMetrics( + peakExecutorMetrics: ExecutorMetrics, + executorMetrics: Map[String, ExecutorMetrics]) + + private def checkExecutorMetrics(expected: ExecutorMetrics, actual: ExecutorMetrics): Unit = { + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === expected.getMetricValue(metric._1)) + } + } + + /** check stage level peak executor metric values, and executor peak values for each stage */ + private def checkStageExecutorMetrics(expectedStageValues: Map[Int, StageExecutorMetrics]) = { + // check stage level peak executor metric values for each stage + for ((stageId, expectedMetrics) <- expectedStageValues) { + check[StageDataWrapper](Array(stageId, 0)) { stage => + stage.info.peakExecutorMetrics match { + case Some(actual) => + checkExecutorMetrics(expectedMetrics.peakExecutorMetrics, actual) + case None => + assert(false) + } + } + } + + // check peak executor metric values for each stage and executor + val stageExecSummaries = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.toSeq + stageExecSummaries.foreach { exec => + expectedStageValues.get(exec.stageId) match { + case Some(stageValue) => + (stageValue.executorMetrics.get(exec.executorId), exec.info.peakMemoryMetrics) match { + case (Some(expected), Some(actual)) => + checkExecutorMetrics(expected, actual) + case _ => + assert(false) + } + case None => + assert(false) + } + } } test("storage information on executor lost/down") { diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 3d52199b01327..48e0d218c0e5c 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -93,7 +93,8 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { tasks = None, executorSummary = None, killedTasksSummary = Map.empty, - ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, + peakExecutorMetrics = None ) val taskTable = new TaskPagedTable( stageData, diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 3889dc9ba3d3a..df1dd51a7c519 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -123,3 +123,4 @@ SessionHandler.java GangliaReporter.java application_1578436911597_0052 config.properties +app-20200706201101-0003 From 7eb6f45688a05cb426edba889f220b3ffc5d946d Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 4 Aug 2020 14:57:09 +0000 Subject: [PATCH 366/384] [SPARK-32499][SQL] Use `{}` in conversions maps and structs to strings ### What changes were proposed in this pull request? Change casting of map and struct values to strings by using the `{}` brackets instead of `[]`. The behavior is controlled by the SQL config `spark.sql.legacy.castComplexTypesToString.enabled`. When it is `true`, `CAST` wraps maps and structs by `[]` in casting to strings. Otherwise, if this is `false`, which is the default, maps and structs are wrapped by `{}`. ### Why are the changes needed? - To distinguish structs/maps from arrays. - To make `show`'s output consistent with Hive and conversions to Hive strings. - To display dataframe content in the same form by `spark-sql` and `show` - To be consistent with the `*.sql` tests ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? By existing test suite `CastSuite`. Closes #29308 from MaxGekk/show-struct-map. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + python/pyspark/ml/stat.py | 4 +- python/pyspark/sql/functions.py | 28 +++--- .../spark/sql/catalyst/expressions/Cast.scala | 20 +++-- .../apache/spark/sql/internal/SQLConf.scala | 9 ++ .../sql/catalyst/expressions/CastSuite.scala | 88 +++++++++++-------- .../resources/sql-tests/results/pivot.sql.out | 10 +-- .../sql-tests/results/udf/udf-pivot.sql.out | 8 +- .../org/apache/spark/sql/DataFrameSuite.scala | 6 +- .../org/apache/spark/sql/DatasetSuite.scala | 4 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 4 +- 11 files changed, 105 insertions(+), 78 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 709bde3206e3d..f646c09438ad1 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -32,6 +32,8 @@ license: | - In Spark 3.1, the Parquet, ORC, Avro and JSON datasources throw the exception `org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema` in read if they detect duplicate names in top-level columns as well in nested structures. The datasources take into account the SQL config `spark.sql.caseSensitive` while detecting column name duplicates. + - In Spark 3.1, structs and maps are wrapped by the `{}` brackets in casting them to strings. For instance, the `show()` action and the `CAST` expression use such brackets. In Spark 3.0 and earlier, the `[]` brackets are used for the same purpose. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.castComplexTypesToString.enabled` to `true`. + ## Upgrading from Spark SQL 3.0 to 3.0.1 - In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Since version 3.0.1, the timestamp type inference is disabled by default. Set the JSON option `inferTimestamp` to `true` to enable such type inference. diff --git a/python/pyspark/ml/stat.py b/python/pyspark/ml/stat.py index 70de8425613ec..517c984252768 100644 --- a/python/pyspark/ml/stat.py +++ b/python/pyspark/ml/stat.py @@ -222,14 +222,14 @@ class Summarizer(object): +-----------------------------------+ |aggregate_metrics(features, weight)| +-----------------------------------+ - |[[1.0,1.0,1.0], 1] | + |{[1.0,1.0,1.0], 1} | +-----------------------------------+ >>> df.select(summarizer.summary(df.features)).show(truncate=False) +--------------------------------+ |aggregate_metrics(features, 1.0)| +--------------------------------+ - |[[1.0,1.5,2.0], 2] | + |{[1.0,1.5,2.0], 2} | +--------------------------------+ >>> df.select(Summarizer.mean(df.features, df.weight)).show(truncate=False) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 02a6f0022e7ab..308642b136f75 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1996,7 +1996,7 @@ def map_from_arrays(col1, col2): +----------------+ | map| +----------------+ - |[2 -> a, 5 -> b]| + |{2 -> a, 5 -> b}| +----------------+ """ sc = SparkContext._active_spark_context @@ -2316,9 +2316,9 @@ def explode_outer(col): +---+----------+----+ | id| a_map| col| +---+----------+----+ - | 1|[x -> 1.0]| foo| - | 1|[x -> 1.0]| bar| - | 2| []|null| + | 1|{x -> 1.0}| foo| + | 1|{x -> 1.0}| bar| + | 2| {}|null| | 3| null|null| +---+----------+----+ """ @@ -2351,9 +2351,9 @@ def posexplode_outer(col): +---+----------+----+----+ | id| a_map| pos| col| +---+----------+----+----+ - | 1|[x -> 1.0]| 0| foo| - | 1|[x -> 1.0]| 1| bar| - | 2| []|null|null| + | 1|{x -> 1.0}| 0| foo| + | 1|{x -> 1.0}| 1| bar| + | 2| {}|null|null| | 3| null|null|null| +---+----------+----+----+ """ @@ -2750,7 +2750,7 @@ def map_entries(col): +----------------+ | entries| +----------------+ - |[[1, a], [2, b]]| + |[{1, a}, {2, b}]| +----------------+ """ sc = SparkContext._active_spark_context @@ -2770,7 +2770,7 @@ def map_from_entries(col): +----------------+ | map| +----------------+ - |[1 -> a, 2 -> b]| + |{1 -> a, 2 -> b}| +----------------+ """ sc = SparkContext._active_spark_context @@ -2822,7 +2822,7 @@ def map_concat(*cols): +------------------------+ |map3 | +------------------------+ - |[1 -> a, 2 -> b, 3 -> c]| + |{1 -> a, 2 -> b, 3 -> c}| +------------------------+ """ sc = SparkContext._active_spark_context @@ -3241,7 +3241,7 @@ def transform_keys(col, f): +-------------------------+ |data_upper | +-------------------------+ - |[BAR -> 2.0, FOO -> -2.0]| + |{BAR -> 2.0, FOO -> -2.0}| +-------------------------+ """ return _invoke_higher_order_function("TransformKeys", [col], [f]) @@ -3268,7 +3268,7 @@ def transform_values(col, f): +---------------------------------------+ |new_data | +---------------------------------------+ - |[OPS -> 34.0, IT -> 20.0, SALES -> 2.0]| + |{OPS -> 34.0, IT -> 20.0, SALES -> 2.0}| +---------------------------------------+ """ return _invoke_higher_order_function("TransformValues", [col], [f]) @@ -3294,7 +3294,7 @@ def map_filter(col, f): +--------------------------+ |data_filtered | +--------------------------+ - |[baz -> 32.0, foo -> 42.0]| + |{baz -> 32.0, foo -> 42.0}| +--------------------------+ """ return _invoke_higher_order_function("MapFilter", [col], [f]) @@ -3324,7 +3324,7 @@ def map_zip_with(col1, col2, f): +---------------------------+ |updated_data | +---------------------------+ - |[SALES -> 16.8, IT -> 48.0]| + |{SALES -> 16.8, IT -> 48.0}| +---------------------------+ """ return _invoke_higher_order_function("MapZipWith", [col1, col2], [f]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index e27c021556377..e93dc588819b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -297,6 +297,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) + // The brackets that are used in casting structs and maps to strings + private val (leftBracket, rightBracket) = + if (SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING)) ("[", "]") else ("{", "}") + // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { case CalendarIntervalType => @@ -330,7 +334,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case MapType(kt, vt, _) => buildCast[MapData](_, map => { val builder = new UTF8StringBuilder - builder.append("[") + builder.append(leftBracket) if (map.numElements > 0) { val keyArray = map.keyArray() val valueArray = map.valueArray() @@ -355,13 +359,13 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit i += 1 } } - builder.append("]") + builder.append(rightBracket) builder.build() }) case StructType(fields) => buildCast[InternalRow](_, row => { val builder = new UTF8StringBuilder - builder.append("[") + builder.append(leftBracket) if (row.numFields > 0) { val st = fields.map(_.dataType) val toUTF8StringFuncs = st.map(castToString) @@ -378,7 +382,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit i += 1 } } - builder.append("]") + builder.append(rightBracket) builder.build() }) case pudt: PythonUserDefinedType => castToString(pudt.sqlType) @@ -962,7 +966,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val getMapKeyArray = CodeGenerator.getValue(mapKeyArray, kt, loopIndex) val getMapValueArray = CodeGenerator.getValue(mapValueArray, vt, loopIndex) code""" - |$buffer.append("["); + |$buffer.append("$leftBracket"); |if ($map.numElements() > 0) { | $buffer.append($keyToStringFunc($getMapFirstKey)); | $buffer.append(" ->"); @@ -980,7 +984,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit | } | } |} - |$buffer.append("]"); + |$buffer.append("$rightBracket"); """.stripMargin } @@ -1015,9 +1019,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (classOf[UTF8StringBuilder].getName, buffer.code) :: Nil) code""" - |$buffer.append("["); + |$buffer.append("$leftBracket"); |$writeStructCode - |$buffer.append("]"); + |$buffer.append("$rightBracket"); """.stripMargin } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index bae41114caf1c..f54e0192b6df8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2690,6 +2690,15 @@ object SQLConf { .booleanConf .createWithDefault(true) + val LEGACY_COMPLEX_TYPES_TO_STRING = + buildConf("spark.sql.legacy.castComplexTypesToString.enabled") + .internal() + .doc("When true, maps and structs are wrapped by [] in casting to strings. " + + "Otherwise, if this is false, which is the default, maps and structs are wrapped by {}.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 4ab288a34cb08..b8b93d929d39d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -712,47 +712,59 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-22973 Cast map to string") { - val ret1 = cast(Literal.create(Map(1 -> "a", 2 -> "b", 3 -> "c")), StringType) - checkEvaluation(ret1, "[1 -> a, 2 -> b, 3 -> c]") - val ret2 = cast( - Literal.create(Map("1" -> "a".getBytes, "2" -> null, "3" -> "c".getBytes)), - StringType) - checkEvaluation(ret2, "[1 -> a, 2 ->, 3 -> c]") - val ret3 = cast( - Literal.create(Map( - 1 -> Date.valueOf("2014-12-03"), - 2 -> Date.valueOf("2014-12-04"), - 3 -> Date.valueOf("2014-12-05"))), - StringType) - checkEvaluation(ret3, "[1 -> 2014-12-03, 2 -> 2014-12-04, 3 -> 2014-12-05]") - val ret4 = cast( - Literal.create(Map( - 1 -> Timestamp.valueOf("2014-12-03 13:01:00"), - 2 -> Timestamp.valueOf("2014-12-04 15:05:00"))), - StringType) - checkEvaluation(ret4, "[1 -> 2014-12-03 13:01:00, 2 -> 2014-12-04 15:05:00]") - val ret5 = cast( - Literal.create(Map( - 1 -> Array(1, 2, 3), - 2 -> Array(4, 5, 6))), - StringType) - checkEvaluation(ret5, "[1 -> [1, 2, 3], 2 -> [4, 5, 6]]") + Seq( + "false" -> ("{", "}"), + "true" -> ("[", "]")).foreach { case (legacyBrackets, (lb, rb)) => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyBrackets) { + val ret1 = cast(Literal.create(Map(1 -> "a", 2 -> "b", 3 -> "c")), StringType) + checkEvaluation(ret1, s"${lb}1 -> a, 2 -> b, 3 -> c$rb") + val ret2 = cast( + Literal.create(Map("1" -> "a".getBytes, "2" -> null, "3" -> "c".getBytes)), + StringType) + checkEvaluation(ret2, s"${lb}1 -> a, 2 ->, 3 -> c$rb") + val ret3 = cast( + Literal.create(Map( + 1 -> Date.valueOf("2014-12-03"), + 2 -> Date.valueOf("2014-12-04"), + 3 -> Date.valueOf("2014-12-05"))), + StringType) + checkEvaluation(ret3, s"${lb}1 -> 2014-12-03, 2 -> 2014-12-04, 3 -> 2014-12-05$rb") + val ret4 = cast( + Literal.create(Map( + 1 -> Timestamp.valueOf("2014-12-03 13:01:00"), + 2 -> Timestamp.valueOf("2014-12-04 15:05:00"))), + StringType) + checkEvaluation(ret4, s"${lb}1 -> 2014-12-03 13:01:00, 2 -> 2014-12-04 15:05:00$rb") + val ret5 = cast( + Literal.create(Map( + 1 -> Array(1, 2, 3), + 2 -> Array(4, 5, 6))), + StringType) + checkEvaluation(ret5, s"${lb}1 -> [1, 2, 3], 2 -> [4, 5, 6]$rb") + } + } } test("SPARK-22981 Cast struct to string") { - val ret1 = cast(Literal.create((1, "a", 0.1)), StringType) - checkEvaluation(ret1, "[1, a, 0.1]") - val ret2 = cast(Literal.create(Tuple3[Int, String, String](1, null, "a")), StringType) - checkEvaluation(ret2, "[1,, a]") - val ret3 = cast(Literal.create( - (Date.valueOf("2014-12-03"), Timestamp.valueOf("2014-12-03 15:05:00"))), StringType) - checkEvaluation(ret3, "[2014-12-03, 2014-12-03 15:05:00]") - val ret4 = cast(Literal.create(((1, "a"), 5, 0.1)), StringType) - checkEvaluation(ret4, "[[1, a], 5, 0.1]") - val ret5 = cast(Literal.create((Seq(1, 2, 3), "a", 0.1)), StringType) - checkEvaluation(ret5, "[[1, 2, 3], a, 0.1]") - val ret6 = cast(Literal.create((1, Map(1 -> "a", 2 -> "b", 3 -> "c"))), StringType) - checkEvaluation(ret6, "[1, [1 -> a, 2 -> b, 3 -> c]]") + Seq( + "false" -> ("{", "}"), + "true" -> ("[", "]")).foreach { case (legacyBrackets, (lb, rb)) => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyBrackets) { + val ret1 = cast(Literal.create((1, "a", 0.1)), StringType) + checkEvaluation(ret1, s"${lb}1, a, 0.1$rb") + val ret2 = cast(Literal.create(Tuple3[Int, String, String](1, null, "a")), StringType) + checkEvaluation(ret2, s"${lb}1,, a$rb") + val ret3 = cast(Literal.create( + (Date.valueOf("2014-12-03"), Timestamp.valueOf("2014-12-03 15:05:00"))), StringType) + checkEvaluation(ret3, s"${lb}2014-12-03, 2014-12-03 15:05:00$rb") + val ret4 = cast(Literal.create(((1, "a"), 5, 0.1)), StringType) + checkEvaluation(ret4, s"$lb${lb}1, a$rb, 5, 0.1$rb") + val ret5 = cast(Literal.create((Seq(1, 2, 3), "a", 0.1)), StringType) + checkEvaluation(ret5, s"$lb[1, 2, 3], a, 0.1$rb") + val ret6 = cast(Literal.create((1, Map(1 -> "a", 2 -> "b", 3 -> "c"))), StringType) + checkEvaluation(ret6, s"${lb}1, ${lb}1 -> a, 2 -> b, 3 -> c$rb$rb") + } + } } test("up-cast") { diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index ac4e71e244bc0..bb0d452fa04a1 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -276,7 +276,7 @@ PIVOT ( FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) ) -- !query schema -struct +struct -- !query output 1 15000 NULL 2 NULL 30000 @@ -370,7 +370,7 @@ PIVOT ( FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) ) -- !query schema -struct,[2013, Java]:array> +struct,{2013, Java}:array> -- !query output 2012 [1,1] NULL 2013 NULL [2,2] @@ -404,7 +404,7 @@ PIVOT ( FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 @@ -421,7 +421,7 @@ PIVOT ( FOR s IN ((1, 'a'), (2, 'b')) ) -- !query schema -struct +struct -- !query output 2012 35000 NULL 2013 NULL 78000 @@ -438,7 +438,7 @@ PIVOT ( FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out index 087b4ed9302d8..414435e6b781d 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -276,7 +276,7 @@ PIVOT ( FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) ) -- !query schema -struct +struct -- !query output 1 15000 NULL 2 NULL 30000 @@ -370,7 +370,7 @@ PIVOT ( FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 @@ -387,7 +387,7 @@ PIVOT ( FOR s IN ((1, 'a'), (2, 'b')) ) -- !query schema -struct +struct -- !query output 2012 35000 NULL 2013 NULL 78000 @@ -404,7 +404,7 @@ PIVOT ( FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 0d0e91e2287e0..bcfc77545bbd2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1268,7 +1268,7 @@ class DataFrameSuite extends QueryTest s"""+----------------+ || a| |+----------------+ - ||[1 -> a, 2 -> b]| + ||{1 -> a, 2 -> b}| |+----------------+ |""".stripMargin) val df3 = Seq(((1, "a"), 0), ((2, "b"), 0)).toDF("a", "b") @@ -1276,8 +1276,8 @@ class DataFrameSuite extends QueryTest s"""+------+---+ || a| b| |+------+---+ - ||[1, a]| 0| - ||[2, b]| 0| + ||{1, a}| 0| + ||{2, b}| 0| |+------+---+ |""".stripMargin) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 06600c1e4b1d7..4923e8b556907 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1116,8 +1116,8 @@ class DatasetSuite extends QueryTest """+--------+ || f| |+--------+ - ||[foo, 1]| - ||[bar, 2]| + ||{foo, 1}| + ||{bar, 2}| |+--------+ |""".stripMargin diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index db85ae613eaa1..04af7d1a68682 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -550,7 +550,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { udf((d: LocalDate, i: Instant) => LocalDateInstantType(d, i))) checkAnswer(df.selectExpr(s"buildLocalDateInstantType(d, i) as di") .select('di.cast(StringType)), - Row(s"[$expectedDate, $expectedInstant]") :: Nil) + Row(s"{$expectedDate, $expectedInstant}") :: Nil) // test null cases spark.udf.register("buildLocalDateInstantType", @@ -580,7 +580,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { udf((t: Timestamp, i: Instant) => TimestampInstantType(t, i))) checkAnswer(df.selectExpr("buildTimestampInstantType(t, i) as ti") .select('ti.cast(StringType)), - Row(s"[$expectedTimestamp, $expectedInstant]")) + Row(s"{$expectedTimestamp, $expectedInstant}")) // test null cases spark.udf.register("buildTimestampInstantType", From 0660a0501d28c9a24cb537ebaee2d8f0a78fea17 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 4 Aug 2020 23:27:05 +0800 Subject: [PATCH 367/384] [SPARK-32525][DOCS] The layout of monitoring.html is broken ### What changes were proposed in this pull request? This PR fixes the layout of monitoring.html broken after SPARK-31566(#28354). The cause is there are 2 `` tags not closed in `monitoring.md`. ### Why are the changes needed? This is a bug. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Build docs and the following screenshots are before/after. * Before fixed ![broken-doc](https://user-images.githubusercontent.com/4736016/89257873-fba09b80-d661-11ea-90da-06cbc0783011.png) * After fixed. ![fixed-doc2](https://user-images.githubusercontent.com/4736016/89257910-0fe49880-d662-11ea-9a85-7a1ecb1d38d6.png) Of course, the table is still rendered correctly. ![fixed-doc1](https://user-images.githubusercontent.com/4736016/89257948-225ed200-d662-11ea-80fd-d9254b44d4a0.png) Closes #29345 from sarutak/fix-monitoring.md. Authored-by: Kousuke Saruta Signed-off-by: Gengliang Wang --- docs/monitoring.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/monitoring.md b/docs/monitoring.md index 2ab7b30a1dca9..247957d087fa3 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -573,6 +573,7 @@ can be identified by their `[attempt-id]`. In the API listed below, when running ?planDescription=[true (default) | false] enables/disables Physical planDescription on demand when Physical Plan size is high.
          ?offset=[offset]&length=[len] lists queries in the given range. + /applications/[app-id]/sql/[execution-id] @@ -581,6 +582,7 @@ can be identified by their `[attempt-id]`. In the API listed below, when running ?details=[true (default) | false] lists/hides metric details in addition to given query details.
          ?planDescription=[true (default) | false] enables/disables Physical planDescription on demand for the given query when Physical Plan size is high. + /applications/[app-id]/environment From 15b73339d9b4b387ea5c071490a04136dc378174 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 5 Aug 2020 11:14:14 +0900 Subject: [PATCH 368/384] [SPARK-32507][DOCS][PYTHON] Add main page for PySpark documentation ### What changes were proposed in this pull request? This PR proposes to write the main page of PySpark documentation. The base work is finished at https://github.com/apache/spark/pull/29188. ### Why are the changes needed? For better usability and readability in PySpark documentation. ### Does this PR introduce _any_ user-facing change? Yes, it creates a new main page as below: ![Screen Shot 2020-07-31 at 10 02 44 PM](https://user-images.githubusercontent.com/6477701/89037618-d2d68880-d379-11ea-9a44-562f2aa0e3fd.png) ### How was this patch tested? Manually built the PySpark documentation. ```bash cd python make clean html ``` Closes #29320 from HyukjinKwon/SPARK-32507. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- docs/img/pyspark-components.png | Bin 0 -> 32727 bytes docs/img/pyspark-components.pptx | Bin 0 -> 354673 bytes python/docs/source/index.rst | 36 +++++++++++++++++++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 docs/img/pyspark-components.png create mode 100644 docs/img/pyspark-components.pptx diff --git a/docs/img/pyspark-components.png b/docs/img/pyspark-components.png new file mode 100644 index 0000000000000000000000000000000000000000..a0979d3465a92a052b17194603d87c11a3c532d5 GIT binary patch literal 32727 zcmeFYRX|+Jwlxd{f(3_=;1VRbH|`P$?(XjH5+t|;x8T9up^?T34#Bl?cXz&K@AKb# z&e`w()8(OQ)>>6HYp$wUV~jZolb4l1LBd0Vf`USk{35Cd1qDk81qD5T@D_4qAWjJK ziPU2uA|fv-B0?hXXlH6+Z2|@LB`h%+UO8UleLwgnAD{^Z{W-_&7nY#ZZq8SDd{V?u z2_LWm0(3BN2#{&W)Ww~51M(D&=|bU<1NEUFU_W9fI8=RuOMLo9F?^W0%=a+Xyz}G; zdN^)5;W^n_f)ebk4eD}MhyM0XsDQc)9Pu7EOR)P1l?3VyjQraQx(4;%2xw?;LrwcY zR%P&E;?A>iPD9tXU1sR`2FPNmP}$MszE3!*U6Sq5RW%5o z(buTrf3l2_$NyyOHNRArRt^GmSwgK2&?QmBLDfL77n)Qn{opkw4l0KuXooV`RUHN#&V7)Mf3NbX(pka>3Aef6Ti)4Q3a(wlp?Dhk* zi3dxn0_!l&gZOBbxkiw6xp6GOC|DzR-i5+qf<+)^C}C@CO{{m+{=ERDb8xRktWYiU z3Jd59CTVFF?deUL&a7O+&AT(b>mjqjZPW8y7-KK9k(n_ZCR3;I4dZ%;CjRzX=7}12 zgMwshdIQfUy3F)r#*2trG7~4(*f(4}0PSm3B@O>I)_30D7!kzz+0aqC{ORpsI65$H zjUClwGZj|bq*LhE3r|@r-fbp1SV%)J4hEQ7*;t`;Y-VKeW1bL8FN=M_5jfzf`UnO7 z8XC`fkcST=Ukd-NYeXX}2M)?n6ROVF?5n88Wc$DYEYmlH%~&vuOE$`-KVp?=i4$x^ z2UamBdXPUgI$9DG$EyE35*XSJ%PQzif0!!d!#5PG2%K+eIuY%W*!-PVF`M4n`ZL%Q z^S)JNg3=AdV0zm}iqIv59g0*e!u^BP7>=;@j;KW^l z#_cw-r(Z(Q3$W|FG5Gw5Me`Jll288Jb65LF6ii)HKMRy={q zs8sal{#Y$4HJVv!H;LmYUZEBFOj3|2zeM|&=ekFmI|I0YfY^XWn6C!q!o^)0-*vw$ zh?V3<8KxKpue1{xlp{|RnKUSwkPkhCI&9#wJ~kf(2pF%}_*dIEpsOcnTx!PAe@xOyVo&l>Em7@M z#V=1QTN!3eNoKOUX}$IUmlKvvmd9ubG$_;{*3Fww8eWXf3@6!WS$d5{8}67jj#kVc zm$xy@4+g3GIF#!VzK@EgMbN&)rW-^`=m?Fi^J5*~F96hvx~%+BZMsk#WJb zGsY0jkjhY~{p?)tEaOz}O8C=`H;d=x_R$x>2S{2}02K3#@J&2PxD|&~zz^jOZ!{OS zQ}x#_T~Kr<#10mjw4T*mV|uW=`&_!7>256V2cDnZ4BYHps+>n%)1Ucl7w#mlt8D<* zPgX&LAa*D$bnGgg?BwEJW7g6Xa(`KWE0{{y*g&#CsenxQbhwBBE#cK49Y6R;<%Cg0 z;BsUPNFqNrP#sz{IozI~>b=*;4PvC$IVJ0&G;|U5`o!*NVeGQ0o}e~0TGT&5IT5$p z@Zl~+j9G=5*vQ8a*(l%PVtuA(Iea#BryoGIifM>Bh$W7D&#=^JuOHi;+bYHv5rtc+ z?QIm+7k5dqlRU_1LeG~GDj6wLCi^&uWRzGf9-kr!A1^+)a`<#OOCUF@JBmIVrS;n< z^)N|0g*RnAsfNlVbf=Sc9p$3!j^eSg7`j+OQJ~;^zRE9|U;PEMW~;22tfNQ?uoeNd zLv_jJ+B5WQxZ8;NLpn((TKr^4n28ajXNC?=1uTCET)s$uN~iGAIM&G5$WrYo<7`uO z$k#WKXwbNBI85g9X(*r6OK!2%Z+p2PMCn1X z!1rA?Z>zT72$88y*L19RoZY#$y9R5=QzcUk9xSf|w|=W$sy@$*+PE&FT0dlt4S+QC zb>x*#U3k16Mu@F_sh(pUDec}ID?xMA{g(#7oE7=d**$0k5H zYCeiwrj^pd$=m5>zN6NLzJeYFr2eGnAbX>|=aG29!-&qfshiT2;^a`LGRN)xc}vl6 z@lQj4@&*IArDj!XXF9udw(?G=WogtLJa`!l|}`5f!E^tZYl`0hR(XYOXgw?%m7qj93WB`y%~exN>> zxZSvtW0E`1qT!|YEVwH>1WbbW^P~S<`0)D>j*({x9Gl&Mo^>7{X8o6N6L7^CU367Z z7Em+s&++#?>rbaUtkw?B)>;LyUpR03Hb+tg_+Ab_gyDa5r}gAHvwiToy1^kn6sU^| zj^W22`hfqE@*?@9v}Ja`bIn+-ryE^)8~&jAIC7~qbd!A|N6A7-8AOL+Re~c(`j(37 z&1nS;loT)26a!Q!a-rA$XQ9cLHw6ffRGG&ug_{V1ge*ZGFstK8f^Q6iLlw}GCA<|f zp&DxA=LINa8)`E~#Vmz2<8#BpZoa&Pvcj!uo%Hm$PxkaQI{uoI_(IIeu{%{HONA}kf+>r0Dml?@O{(8jOijPcPMxI2(&e4Q~oq>&kiHskK zgoK3G(b$w*QB?e2%^`pBk(oO?+jBEAy1BVAxUn+WIhrvtb8&GoGO;kSu+T%Epm*}H zbvAIPw{;@_yOaN~A5jx0BS#B+XA3)9lGlC>4DDQ;`N+s#2l}6%zwI<}xA@9{c?6Qf(j3%4kgQ(%TT|)*HoYLzhC&iyu3^w9{ydOIya6C^7Sn9#owu! zm4~&3XBf4X6|`aIcD&?e>r**2>~qXDHgDNR9~=M$gFy1_KR$k`>mOj-#f*^t=jGR@ zWWQy{{-nMJSZ`W>>oA>!9NrKXXlWy{K+s#cI7{nS!4fWXaw%lmw#A^fE@-U z6hdZTlJ}3H1;_ng;Qz~fbiwQ|uu|^_ROhK-EZ(cP*!ra1{60qi&FepMM@!v&rE0-N z`m8$HH7v`b{S7qz4kSoG1cMJ3-VLU~MfP7O5ggCq)TW4L*7M(sNKWM%Qr*^yK|&z; zp_XlWS5sh+h~$Bb-T9N*2{#SrrGh?yzE%>pMlai9I=sNLnZ<%@0;6_2ogMaa(r`q! zMo#|$FHz<@OMcBs4<|Wn9Co&sSe$)X@IU4l28Y1?>Du7)Ki>DJ&w}HmlANj{|M7}m zz1RcsI_5*wKb;tdO|tvr6G7=emWZGsbO36YkNQ6sf+P~U^B2tM>3_JAq#EM-9)3Ep ze+>WW)ndec%YVF+AL@|hV@RCzkJTU;2eCLV$cX(P>*@;~WRlfrUorlBt&m7UESBu5 z#`~vV-a{ri7n1+a;eSCa{S3M zeRl))lS!>Hk=ZJl4nNFR+-tNw(q#2`RA;rjrepORwXVIIp^(1Ptd|+#c9D6al)2yw zsdx1$FIBAYhEvK>%=eznf%&%}+x!UrcrIoF)Ag> zl?JlhJ1~>R5AUnrn_afz3?--XFdQ63S9=T*%2e{t=3lxi9W`1HU|!gO&_>3<@0dze zD2bgoO2s|E^NGTC*rHwds`X@hp2do##ALQ4t5UA9B&*eeGXCw|lJw(_otoS-!!d9l z(`o+brl|rKgZAR>zaLnbuLtt)ZedssIsNO%J<~sP%(moO$AhT9C)j5r>B{d$rgqjB9+>cRndn<;0|-wh0y3i0spaI}jeSo}E#MJ# zH7j1Mik&x!;TxHY?H*yhSu#kt5z;Z#^}OYGIvepBQPr}+u?|H1_k9mJpKronQGk9e zY=H~`aT&PaiO2&6wNseK;nHu^S@HwB9H5!CP!^Ffny1az=UfM$!)&<;KA{V)`0`oOdxP{lWYeZ*z*EV=hufDpiR;fX|qou)(;7j^VPznOcekH_5k)nUDdXnXV zXTAL8y(=e>(m;5Fv@Irg$@#!L+5HojZ?WF{1HdvUy^;~b=%_C3=^+>6L!Gt&RhD0@ zLT143`3Mzp+SrOOpJ#(t<$Ead;KzO6=(`h*0+CstKu?GhT^FY#8#6t+TzF2AL!Pc9 z2u&Rif`mf28X^X|0Xq1fY?P4$-t(NS17c6vReWC}KWx9NXN6P8zN9vvKzPr;n|(}# zhgWAAl?JDi;rZA8)1a<0o)C80!NC>#nW{;5TrGDK>QTM2d+|^~%ZSRKth`gwLc}y} zHH%mRcDl+p-RcWL1Mpp!%@=sz>1+66p=rA%J`7L4iA*Qt)J79!|zKyVU6M~yj z;dkHjVBHMyx7ECh$R*})f_oH(fLRWaZ^t<4{{1Bgn^f}chWVjrC$m}ZbC+$8!jmsY z0PPC+mrViCQBZQs$LC~Z&fV?1>K=aZN?kn9B&tW zqxe!=# zPp_zmn}}VC&8MujCLNO-Dy+mt#z$;&`P6Dhh`qq}rAK7)11eUF)$#ZaLOLHVVYdd_ z8&eL5Jju+Xi&atE3EU46O|!bAFYT5Oo+XG+0!Z;F74?F!wapDA=0a7lS2?diX6fZ$ zt!)mXm_{_~SZI^zhm=%Cc!-3MK&Lv#aELk=LI>yYm& zTM_vR|4b#A9STRITw%@7o^LnIfQE!_Zr(R$JrvUNplx~Fw(ZDj@;K)mvm2zl0}laUB1KT^5*`x*qGl7ZXC^@pCp+(U7vra*z#cZ(?lh=G)Z>O`nFg=;) zPnS%qZH0S)*Zjay*kuSL^u463scU=a3)}7P5iA zraf)e2^Rk`(XRN8Pvo(%QkP|=S#Prg12sWiYh5!CVt1dV(ADKGCi9eT-6%5L`+D@G z2)J}L>EHiZJZAC9@|wOnQa&XGnuGR{*1mEwCuJ4R1~;e#!(fMn!=TsPBCLFWYqmmkE8&nlT<9c4uQjXo!w1 zz4yG)6=PnI6M_UEc2_qRMJ?8enga%pIY+0E;aK91z*HvPQ2ga8I z_xrs}THiZ{DVsUYB3wQN@`NDE!Jso+fi-SR4d%lb@CY{k5BaYMPs5EuFe8j#K7Voi z>_Uu~3_%t)svV@Q<;G84VZ=Uwuv6EI^h|aaDfxWcyxV6X3d+AYtD|_fW4&cTuFZ52 zl&`N2o$tf5)=uuH7|+n``#l-~)Y~p<=(<4%v&$NC^{Mb?E56&?r;w8oHl9C;QX)Lu zsu3J5DYhX5Ynn2^)W6~b5-CInKv!25BT0nWPm(VbBEOluTJ0V=1;0%PWckwV#xbo1 zx{ai&X~&x-@7tnR2DL!%4#_onKObvtr~4AW=j=e&ot5i<2v|?<_f+u>B@JvJ!h#A@ zX=J)U+EA1v3m?h6GDR(KuT6v+HsEttHrTKI+7X|B9x(A@ptx0mA;NTBTd{HxkX`}{!Q@C~L9 zW3hI9nqChrWC_ZXKm*mVL&KyHuyqAJFs4RUXCg+KW0pNd+VSTs8m}wx11dS$N0VdS zb%^U${>U=~+ba&r9WrS^PD5SM4xw_g1nl!wBC3xrAkNhkBHvjDy{`$>>OUP%-k@^e zc}^xbW_b;oyH|H5DBmAdIi>5*dY05V_J*(e_+nW$=a5SJNK#S-=_RYZq6+^*Yy`BB zr^Op^FYkbX$Xoaew??6GX*ws(3`;HY|oUlO+tX#x`C54B(AN&z_kSa zK8;^JMgJ`dy~Urx>=DOP*Ncq1pn$fG&)zAmoK|oCkxsYGax%)6X;OB$F;BRM7Fk#a z9}fVYQBk5i@_%}OD>eUxL=ms7IAD($V~=5{UT**7K8cdX#u}30T%yH$yI85ChUx-# zKROF}MX|kF^ka|8_n&dazDp(Q1js~p5r8$Sl3R{yne>Z1{uib$z=RLCgP{>ZuN1OdFpFEec=a04WZF)Gku z_@g|N-is&X7f(A`K*pH%^)S~d^#6%zUFEotuvQpKGib-=dr1wjYiw%~r5pMMxX(mK z<=mZ1z}@?uSqK+WoD`1@dGXmy5d8-!PfQ0E>GSx}IC&tubKb*Jn@`leb01CK3xTDs z-+8QlET(K94HAO)*~+5F`zS)LF6mpZy7bD(Ip5o1`-m{Oi|tJfIo5;|(x3VcBWdi+ z2RN<;j)DZzK0*O7OhsiA_4aQded&|au zEHTrvVnJYiC?-Vr!$z>f_>$iN!R+Y@u63cz-GDlMd6?pZF`n+_rIGV9yx^;A~-ou>R#6_(dqa&dEk_M(Uo@ji2Kb9%8t zVYJM44NeNJ7@*b_zT4PO= z2Cngf{r0#n@=$o$?3(kwq8In*9?X`X}6tkx??z^ z8W;JuSxij}z6HnotaW;6o8GHS6jeys9F6o%6lPdRB}6_|mrPjt-fh!PW?Qh{U|I1r zbDrd+SFj`RjJSL}V=rTb^IxZv&rzfR{0RiAiMmyv>d<0GqU~r8)?JkAR0<7?{4p!i z)Zeew_P@h%rQWj{Opeh=ekFv0b%KX>5)<|AF;Fm)h`Vn$BZn$Evf7@2Ruz>|yM0;% z?>124>mj%|U5T-E7!C{bNZH809}i7?fV5kN+-kbiPV;1j^aJHr5raJ{Alr(oS8|5t zMSnCNQ5)yp&3*2l+ME3CcB@MJMfrri-EVFz*AVRwx565Xxl|zZx#eFrHH9-&ziNZY(wOBk2E|j$zF@#NcYNR zkhq6=KDYq`L0PpA+Qbv(QiVm?y%#s&t<2pU&I9cL)SQEM-zVTW4BNfn$%$Hsb>EU-L^;Hi&e$ET;oiB_nVdNOsZH!r>4|(CH!c%p7Bd{D&d* zvU6#K|2ck8`b~&i-Z8qApv!7CRTIf~O~xMD#iO}}D7GAzE-@otV&8K==`mymL`|;S zExu9yv4)?Vzd=#%(@MDYyqqtM$3oFA(lBLVrjn93zm8E;W!W<^lZ zEI~Kxtfp5bDxD9aSjE?q@kJ4~z-}Ehind5K>_@J>rImt-zM!AL>c`cso#8e$z4vxF z+h(cXg2R-mI^YQ|8y$_uCP(Jgp2jZQ*HoQKPzsK!$9y|AO4B>|)b!GJUTo@}k=*o< zRQ0=VF&r?T(&4dSdL;#y(Y-&im}jRyZtM_0((!mZ@91-5Q9bLlaUUDd*LJ$-VSmeh zC0KAc+4&+u`9&cerbu-b;E7!c8O)t|4C*8XU1QaG44VBZJM$?#A;2u$=aRxlm{4UefxOkFz4a;t~L(fq${6}ht zx{I!d#+UI$38nI#uRt!Wq^aX8CrG4gkokSVpj`S40=BOg9>3D82&JV)4*pjcuQP`I zyoZf%jMi3(vrFTtS9t7*(Xz_|mm&kQ{*?mHB%I9kB#MHJ+IR~oi7ZWS)yqGRkDZE< zWy`eNmmV*#l5KUH7$&RX?w2m)58;NzvQvI9yymxxz^Mc&P7P^uiHo(BB?WLq=X4hAQqwhx_7a&RX7zYC%;r0R|HyX zD1jWXZj{DdSju%v9v?mSB9we7y8MR@3H=y0DEv)=2iA61^f@ zp256a@6R-O&mZC%vI8B(u-WwYP^0xAuwxSjBA(zms}WrG77DrXkBnXBt}(R`dWvRt zE9Y9msi>69ckseNA|!PrM78fSSHOjabyUp`IvV{0*$-S^&&|H;wFNS)IJ9B3Xg--ex0v)dY~+s+xv7I$8u3eiI<&swZoJ)K3fy; zONR9+>J7%#gdaD2YVDtSC#y7^Y9nk)mL0%PPx}>%lgY&GRuUy2O(3$sCMK}hMYg+O zBr2|f63M(|Xi6|WNOf79&A1G6sARV||9W~WyIZ=n;V8hzLYUrPw@#8xRUj%v%{;Bk4-H?cJE)Se4DOt|USiYYK;{bacOCGQ; z*3;deP^bZ~t`3gOvYgACoO!+Ge*xbJJoH39TNNGL*V`_0-+XdiROLwxv`%J&Q!>IK zd?4{E!;?~f8dKR{sL-W9;z%4;BM-a{Rr`6g>@6XYg81Wqn)$Yl+Y(kNJ`~Ph{<1!oQt-qd(%1lG+<+_3>P zc+sWo*7+8Wv@V9HvX%1-uz2Q;)X!3*i`2&}?)4k+&nyAZqwX|KcXE==LRuP!+epHh zkojCk1Y zHXEAVy}eW<0l#cWXp~Q*qKN4%uW0!s1A6Wu_i)WK!HWuK(D5)&fv6mJkSz6Y_|RSi z zE9R3+WGJ{rlFM?9JYG)Y3XSKq5i(Rl=%~Z|l;Qf%(n`01ZOX`xM{#W>HJNPpx%$5m z+Zd|I1*|`GiVMNileu57v>z9|!~=AhTNZLOh7v1RRXI ztWzu^8TRyD+(XEt(Obg0b{HJBh4(1@VLve!EIBoB1P1#djBo=;*OYoayEw+X7K}ap z;6N^cO|zTrGKI7rr*$0p;Yos?cEx+2nclUW(esjfEP+mAuYn! zRMlw+NaWpA`m&#g2E2I`&MtywQ@%=<6CkqoHB0}(4WuU7{^~qSE+CLPi+OBkU~&q; zu)?RXS>IIOs#!c?mIHfOWn#FoInE61=xoAdRL>X@y~;yhNUlMgVKe$4VGw@l2m)}^ zSWJr z#VH53ZbjR&?n`ACUeIjOD6{d6ax)@uLMEMTivT2a3>twy%WS!OfheA)Ao3WO2zJ;C zTidCq)zEmk2n5ce02p*H7XcCkZaOXCTsiK68iZ@XLBrFl>5DbvJjScTiDH)_CNpqA zVBpPI20M9ck>F&=B_|izYIfqn(e(VK-%b<>V{hE|h(ni>wFbcDj!>to*H0H;bo*0Q zcE^!ZJ@}T@z^r~+nD039B(xJnGK1`K=FQiIphEYp?S8_ud)b#V_3~O z(>|TLL?i+8} z>!gL?aa`cf#LC$(^Rh*52TJrRK1J-;tPZ8ITRj^0lxlJn4T(A{$oC}^YBrw5cUpdIh+nKCE|A{?{Sy~hNq?5En14K_KhGsE%c-Elo5P@5Uo)9Z2iFJ zqr6ufR1B%v_<Io@eYfA#a-`I#e1f?9iB$;&ju35)j;sg_dj7S z-4!Uq4gUW2XLldFE^-TZ#xk&&t>!90O-tBPttr+Dn9Yw;#qz{gkNfY9n>GEc+pcCD zN^BF&o~0LYdXsH9w(Cw$x+@1?vTYuCvDZwb$^Wif5wv{O&u8Mx|59L#{W}1zq5XRB zzE(*L8rGh=_F+S*=qGgvRcBMkjI3^YE|TL%hJ&{bu0i2laD?m2+ICpLd9`-ftOL@O zGjrcPmh?DH%ioeH&A1Hvp4kd)`|P%cPE2(UTx3LTSNHabypw0qjB>r$qtu7|w5lsp ziR9ey61P<|3k;uN3A)JsWtF`1YjMqJ68&FBVk`MWB#*Zu4A0!B=|Mv9e&GJhi1$-O4GoOCKDrNKjt?}F#F+pUW=IPJQt zOL;milC@K*Ra;vrZm&BYn$G0#f&qh(jXB;;j@!%7kl5#Q!(%S#hG6v6h%fz z7V@gV8UvatoAG`>g782>ZZ+XldVVv)YXU{8%xTzeJ>PXoOcoJ%*k_@|2(AJ#=+=Z?}iI5D_8 zlHpaEUGaIgNEUT?{y8tCPx@#VIZ8hB-2d)n=-vH!t$zHzr~J<$|Nct^)R&P$Vz}&qRes5Q zUV4_9Ehn~ zso&&@NZuMDtxB=w_3jK`N|w3sBKN(3S=ua--*1(N-DzB#p&&%l$|qFkPPi@W@uK%g zUy63Y39=8wn+5g6aD3jUYxVuaY|Am6eQe9^-(SUYg^rvT9=j&s^DIkxe=cXT&dt8{ zJ?NnTUsALa+@5^H@tQ9!S@6@F#xvd=C+)qvd$him^k$=rUP<$ZVX6hyFTe$&#~qUg(wqIEeeyw!zR?7L6mI{xh3Ct8`3I+ zDR`7HF6(DT(>XiPCYSQ8&sy%#MWhWlTjL`8cyMcI6m5(m-7*cm!JK9Mn5%6{?AaV5>-nhYk@4YC07mx zQMl0rC<}MRy~X1^I$eiE96C9x{hn-HIpM+JxbG+0S`sxzqI=-Kin%^Rs@i^FigN@A z8u~X7@}!1yXtgq0wwI;HaIqGq7hi|v!=TXs!-_O(3+7*nHoT`3S{XE&NIr}*Qs76{ zV)%G~YagH;lEg=C8$^bFe{2;Yc>`xTU2X=+nP%N>MyU-4fh6DnC2Jz|kZi6+>yhjB z4m6d($@#E87ThzH4(a95-I=|K_lKxcoJ@%5aUz{P?QTB-mf`xSkbwbm^?q~q$6 zr+Et;O#Y!aP$}CgD1J$eqm?{Y#aPez`AZlofpVF4ueRfX&UR_c3M(YnXImgs^D?^N zjdz8|!NKD;*Iic~?WR4mXv59R{YB_nEF%|3{)cz(xrr?m4lM+!x~(56AZkswN#;9F zTb`BJTzk8OXt&swPrRs2yi^7QDqi}N=rqk=Y^BnW5;@?WyIjH@J@5YHS5PEss(qKJ z`aI}U;JNsGpZvKcYGS+xvJroy`j;Q##$L@k(EJo*K$g~Mni|$)>sWbc5r53?0>q}5k%;NlHVKa^8 zzMiMiXsGBxk5&QWcI}}<(bo6pga z#F+H}Is0*OyW95XO@71V0KFa8$_m{6jEPceREn)dYt8!HVoh76*^Gn}lDbZi=`gF+ zO#Tb4;Y%Z!NpF$VN@cY9{_SL$*dbK?vtRqs`h_;_r+SwCho#b-Jh|mx&$gBP8`!v! zEms4$n9s}45EkWeU$H2ix4oE#TrswoOYI_or)$4yQ$ra2Ah`N9I41)aDppmg2O>oAA zBf|+&wlLPx#hGQXI27+ba;2^)>~Me(xOcBQiG0fUjwE!>>RGt$dnY+JJ=@D(gD!C1 zh$~M+G$%G3v~QyhlQ`P=!b2PGwWo-hHk-ye~38mn%9Ksv|j63L?}nW z+Dj50u-E41@@wTg%oU2-qtL*`NWhzi8&;{{!omuvl$+5Dxzy_GFa^ZL!d3`dG#&G~ zL9?Avt<<)7>-$`s%H|r$da745)e3~lX z+x;g;w5C(&{aMdc*v2x^lt~G7OB(i>MG9+WQ(svzY7n=0AnkjY2H@4iz-$@I>C+@N!*c^h)VIss}Y78O(=j zhtC}+T0YY_MH4ZO=0E=uBM$nH$21T(rxL}u*u522g++Eh7xt;R@#*`%IQ+OL@n@J z3;$%AHStIEakF%OwdapqODI02-<(@Jdde)6mUm(tnRhR8u51Yd#^jE?;t0<~#U3EK zSJ=b56o6O3e6h|0$%kW_IXxqZDEm&+mw7&)bJ1G&PMF$^k;e0r{b`02{-K7S=51OJ zDR+8x8CJ18mo2TdIPQ==M*%AB$ou zdP9IaWt!E0q=^J(>d#AbOI?LHiy_D9=hxa^LwSynj%|dx8o)21@y@pxU0ZFqS%@&G z_arCdIL}_w>d`u#qs0{P7BbUFO`g{ z$@Qtlrm++I`A@QR7JRpzYB}d+uc#hfH#EEt*VKp@T)Me+wM;@WBPTz8+o%g9sp;B{ zcYky{tJnTCD6{|7CL+VFT7*OuqM2}65)b<1sNqm-D^(e-r+`MV7d?foXJ0#r0yjk3_ycx{y`v;orAtYZuM;WPLH}B-b3Z zd!o_#d)M*O!VH+hmbY`^JA}9!tdw6tC}Wb=W)SntJI*pPNzHIiK&Mslvn6br{YFg3 ztlj^2!&FLx6uRU%!PV(1;CDk0 zur3!5QBW94GdpSSCY+_`xCtCAjp5(rZlZ#~x6s-m+{Xj~!>^MAh+`IG z9g`S!*3FvLDERpHT9Ge~#EZv1C%SjyG}>=eb{&>xL;lt0zWTMYsX_((4GoMkT0f6k zKCI-nVp%d>X5zYRb)tVsL^v2y*{VOHEIaoO2P7S;+cc5B05tq0XYbE<8i1*6NFw%8Me`E$J2?m*Z}+^@m(S{snx zOG)~R*>U?aDC(}H0+DJ_G^w3Jyi6)2>ydbJ!nJsu={tVhN}_|mzeZ%Y7%G zp5!WsYH6ffQ%>To8wvKii+nv2~|LLQsQU5$uKbEnx|{dzmJ z*Az=0kR8~*L~Xt?#QefH6ci%7baR>))n0eyw}Ya#MgP#N^fde#B9c499JL%8y@2(tdmpqkJY-1C zibUH3;wpIj+bU>`jzA;#jjBRhZ=(4}+3K5d3S`pE`Tb7lXDLoXptJB@{N)vJc~I2K zC(Iz?QcAH#VZ{;AO-1R;%w$ZaQsW4tx6=D=}7& zjp~Oa#5ET^zjXuL)ju+g{FAS2>EUI2M;y?JRK}>&%89^vIA!a0bJytJ>3uQ3SlN=~ zaX&n1W|+$|&~PX}VdkBynBs%erv!MU(NniV>1>&_ZE3NULEav7x!74e&c3JPJ)Ogn zzIb0=_5@&X|Me@De-|BAmZL%Yb7T(8XSvI^UY}q$fpzoR@}ta>6Jg8VT`dNV(f&R{ zYS3T*_JP2SjYx|xM#2S+(!Ujv#&@6F3WLP4Ytd?JIHXp#iwjo?oVXGS*>V+)gd~Y= zwdAm;_sKvtOB^|oxXyD8)3ODV&+yGM)Ce1InB$LdX~3J;|B|d~;x3O&LJls4LQKBm z_VY#R`cxpppq6}cJDP2LuRs)L-y!4zoa=~g&n_9xs9*b9R4^U6=r%n!;j8fuo}f-! zObCTASyOTs`SjBRTMVNBzV&UCpaSA9ct9CVKwG(g&=r7ii?6oK$@r6}AC7Q)I0Yl} z`PUe3z2=0{MG3oQQ{v+nIZIXWNmTN(%wusIBRl;N(eE{}_))?%!_ z^cX#_0hz2DPTh`ykKC!b_x}(mmgS@O*6| zjFB51zNxK=H79buVs5H9k{Y9rG>YInQc3+vt6W~&Yw~6i}mkUPugWaIFHMx zwtp9e0LHuJK5zQncBGABAZvl`Z&1$IC@TYq`!-TFyEA#E0WJ)rYtk^^2S#~zw9`WoUctu+R7X_n`3e(dtm)q(vJ|`e4cga;KhE#h@;0$4G@=~EN^Vs*2g5G-L zU@xKUl(VKc7%a2yzw^-h;vx0SI})yCW*=Z4=SxxfwF zQmSj1)Zpcc1T-J6;vz!e?lCr`ghALW61?qG(IHJB)}P7}(P?~x!d1RyC-4w7$#pyo zksscGTqZ3?vU59X?ZgU}knvoE5vdYI^m~4ERzwHj&{!fC))b;Bhe$XnSix^yO^mdM z%aIk2MM73fGd#BaxLK?p!fMOuExxJWoAT|a{o|d8nX4=#q;^KFnmM?>^&kIq?oe2U5x(>lVX;Wk#qdf zGB%qru-4ktiDFB!6x}C`_Div$6K4<28ph|6-Y;x8x$YJIsl+@r1nBr|$W*ORljcY{3(m(>DO64rmcn)2?R`rE_#KQ4&st76$lQOtbu^&U})~4|dgi@#qmq?Vw}+YSZa_{Vl6tt~A{=i}sO5dvhf?$1v1H#0sQ>o1? zGRXE4bmObzhgBw;LbGp~b8#WoKY0?Rw&dZ)XDs521fsiF;fhi20vt}bT(&bk&=eIm z9DrIxcZAP{@WJ;YEo7M*)c^8ESh*Yqa3dkb0aXn&vA zghH^@*mTW;%}{Em+pgSoU@Sk1IJGNdSu4x5RW7DaZ+a%9~^f6vdVou7})T_8*6+ zGyDchx!#telu*xCiCa-uGoMz((!RYu*@>p&T~Amv*i9DCK3_)0XImbZ`c%hhSo4?r zhz*2o*OtC4RG(!2ac+5!(Vv4+pQ{xtsyEs7fujnV%qX6FNTHb=J5&>B9_w+v!V~ax zf6U$a4pKu(QvOOd2S-NHL+R0Ff8^sq60sPUX44PM-64CuI3`=yBeGAtZL*DdWVdm< z!Cp_mtM)W6nXLwLj-Oosskhs9hgVa3bzdvksx|5U61Yp-5%TkPHE%I^AY4&?34#UR z`CLtS*|J5qq49p&rpDdl)V+Sq%NVD1EU;Sz4bB=L`K0mvinV|R$gLR;mozmbTLuNxX_azhZkkc3`Q+3 z`ISh+XH6!$EC}x(6ct9ZEd1~l+IMSLp%gj`M1qha3QvkuWqXlSuFWFmLf(AV$||ju z&gS`ZMC;eLu))3LYlr@gm7d$JKyppzmzRog1(C4gujyayT83P~)VJ+O`;cQNbo&3b zcjnPlc7Okm$UGL3xsZr(aHLF`?n34{Lm4w1QwPZ`Q<)lM%3LyZC^8*$8NxXsbEiWn zWS-~Yw=cQB-@5OAe``I@T91D$u5n-c+Sh0A&))C%OHw^YvNOR5&8K@C`qYy$qq3t8 z&X`KAlm0&*u8&m1XG)Ch5VQC;D*wqR+V*MQlC*^kFJ*Z_4wP<3td09rZWj}WIujd5 zXg{3Jg?>MfOCxFXp^Mf3{jTbh!S*uq7nu)J`>Sl43D$TvLo4s^5dLU;-1rDf1NX%F zRVry#YrRD$mY*KD6pKwWtIlW&;&rmWAv5byf)1lzLB#jVCI=gJFt#ZI5w zl2$2bA*qR+MQ|M^DvL44S|Ox0d68+oA?BL)S0Dx<6dDqfWxXQ_5*<_eEBGpYj{3r* zwIV3bsc)g0-47lzohR)zfQNn}VVg`Xof>i?W_>rO^QpGq?xPjMkT_=*4R$Sp(>Gs2 zT5KL(z5*{5qtg89P69LxJRkM798T=XWeA6o^FfoY z**}CT(JSKvoR70{$jOmX9EO+uC1*h0Glu@wT}$STe)l$ZntLjo=K{}B#3unSGj9n( z3w1{$O%;e_^-f@iY%FVas{5$kE*`-oUX-4Y$k^Z~S(9wJBfUOPe4f^dzONJ6E2>7_{yV$4Z#(kMDHD%_P^Ohl+tyFasW(T8Ip>0nm2Y@# zNz3Ea5^{`cUf(vf)_L+h^K!OUF;fD=UiK@7Yg(x!IAse}QZbZqr>>v@uVCe_L#R_J zM~|qG;HAOHlnDV8y&6A9z%B|q_p_An3;D?m`&j2AM_?x*7JyyCRU}pTtnyptu*cnb z-XpSY(cv%dw+#*~dDwM@TQF%$|j84UO#aji{pY3!b z-3uz}a~!Tpwf&l5m5`plI$krtM3~42bbe}8{H!{?Wb51-8!7!a&!(IbNi8e%2ebM_ zfdt|lK0vv=o%u~qygD8uTUcHUoB@MhoF`IEB7GH|fJmr_G zF;&?vJG1)2uS_GVM!el@BF+&A4CwKy8_qSyR3l zA#}~VcPfv{s5>iufqnmrS|U0jrkU9i#E8u5;A2YE?xxSEZ{p&9xS6p3w2Uhv&0lH= z)!M(1!1&>^3so#A#jZ19)UHD$%lMCHUv>NZ8i}F}9GN{=la3M&3sJqjZeZ@wEh?vh zr74jQVd1#PYf~pbH5n0gdgX2ClkO+1Jl8jec487;$|7a`2Rhv9=C-C~`YMO3w(nJc zN|JFZE;G&ip51SKGc<{kLnT5xxs)!IBXza|Y3>fDTAHSDc{L>}Dc)!&v%9ZucljxF z86rQd+gb`yrX{H2C^($GwuXOhPb!j#EATN~$0swfm*bq$n-j+o3ykN9FCDk%ip3M&Df~D)l?=0u zkH8B&AT(EJ6Zh%=Y6QDsa8!Dr&^Svu!{a5FsDY5V4gbIk;~xV0R#BwOX*Ni6dCh-p$9q%3*!D(h`a&x|El-4v2fNUdt5P(I;-Q_@PCWl+j!m7oIy zF1&!Yb1UP++5j1#FaLQrK3O3PF|njCD7_{E zALG&$0_AhU<+z|xE#bpFQb&0xN1#nJ1$+dgN4aBKq?;uyq}NCy*rLKkec$to#o~VY!w&bFn*I znkNVuTAzw;W&k0&4MlgZp-?eRY!(RTXFJ|pBj!F3(L~hgPdQy}dDc&QAWZt@nckOa z(L!Dt(%`GVz1V}ZAazN%Q4!C-_K*T8F6m9-hy$UhlZB_@raYDp|`}N%aD)9@zWm1?tKS2MlJ@-M+cwaa{@sCyt zE*IDh3j9|Jj|pT{{^Lf0vEUhLk3l;AKZ_hI1P z1}AW%<8}XSJ4Ff>TS;+PiI?Tr*692eqZd~KYD0DTilQiK4V`-n%+m;(>$!ffPVjEM zfML^%4CKoHXabxsp$6F6()35WfL>u87#q#gS24N=q8@B4xUc`w6697PIeLvlEAv0! zT`V0m^cxX%q5b0oPz0+q(AcIcv!B!bv*j`ab1c^xp`#@GveMt0;^A!$EA6nQ9ey{5 z!RCKFbZUy8i4F?>)qMaZl`7K(i7wt~xXw&_UYvmXPLsNNb(6YU)E6WM0Dt0|&Yasc z1=D1`m_!lmj9s!DgBq}zKJa_xLF2Hy)2DV4_63s$!(o!8+g_X34tBrRY2WQG>5T%+ zDcp-6lKo@{U4)|%G*!@>R=qD*e{od~UVT)?r_X1pC2TfSvKLV%r1iersiT9{we@y( z%c|fly>lK*emk=)OM?&F%SQ)u@v~z>%va&n&gEDL#xR7;`MYmx)B(3x zs6w!cLk}uPK-~R)csRL01SoB3diiupUk_CHs_39>)OGUiFc(|Qvcft&h&Ai#;|pcH z+day1Aer}w?Lv~=w`!(T5m|p2XQuMZCY9Pl2`AGs2EhcFC7Sv2JWH|LhMR$gsW5xZ zPhh*;6AJ_#2g-2@q~K^7eebmf}DR&gE&}h}(midZ}^LQoGzRu(BEzB*jmD`SQJX zw(NFknjg7sln3KZ;6;u!)7mcNvfZzVSUC$I>8}4As-B#z;QgEx_AUJya3F|Dx7SMM z6rNoiRgR$?ggr8_00P}Hz?&gs)XmWyD7{e*1Cw`gC#JQ^;;L6K2^*@k@B&%Az<-%f zRxw8e*^0)E-FWv$AtFR#Zb28+oiw>>i?Y;9b7q-N^<1&u{C?G@lFfu9A7xmBIdAr+ zuDbB1%B>Fd<;O}C*!5bK0Xh-hgll?vXjWqEw9al&Q16AxT(O6I12*WH)7r%d(_+iYRRDe= zX5qRLcr+#W0zOJeI^1TO|P<_$JCP(kEtgm0m4)xj$!FwfXEpQ z0*<6G23k#r1kjtSHm)PG{*_OdVbd3iYnnHGf(anTMcYz(GG;=E(E0L0IJ33$te#Ni z4*Sxuz=edjXTmDihRx%qqqbaapIdtL-;JN2w(w4=X4XI52+^R*7pNISdC}}{W+ocM-foyC#E!r<(X(GL=5(Kl#Z7ADe)j?nqd;wk5RHU}Du9 z8XD-cy_$?!e5rb?w})!Fcx_eu?6TC*G*b}nW{Loy3kp%uHN9aS!MKwZG-ThM0>t*AfQ8$6D`;5>OvuL zK;wlNzZs7SI%U~~)`v8Bzy;b6AG`(kx7hO=yO69i^3?$>(l~TS8hN&4D$$wB`S6sG z0^gXTSrGxrkA!do2(eQ!`k4X)+33X_OP#9hF)F|Eg+CI-fwLXmGItGbDR8ht+^^2# zF48-Lu6=mZp z!dI4u3JWij(+s*GJKoC^gh66^yM~CMXTkOg)hmnwmMF5S0T!!w=aBOTy6pjyU_X z#eO?dFhuuXU2|9i(|j^y<=ut)dl(>BxfY}q(+Y8LKoBDXIV^YG&HbE4Zh@&IYK_5n zPCRDF!?Ho=E>%V@i^)z-cb-Z6yCT$Sk3nCWESYU$LAEvg5G!)>r#!Ocb$QX;&RBs= ze0g0k!VAcpvwynwR>-F~%WQBa3}%^lLdy%s=EO66s4opXrz|t`LCk+Q4GeQxYz$|Z zW?^dv^q47h^aQVW_09z|tv7Oltncd|P&mauNa{@&wtj>apc6tmMu+I0FinPp#Qjg* zEb z#XwvwOvLyN3xxNy(b^a9GHRj3o3CNd9V$gH17VQz1|dBIk4U$NB}0kDyb^X#oDg6* zo_0K0u4%8fN8Hoe_m66wbS`d%q6><{ox78yQ`4~84*x=6loC)Z%8qI5^bH~PHJSYlFT zum-MyoF#VQ2FAGFvo(A!@4oD69;Eqz!;BaBoA%7vM$}E7?$mr;KA(iDQPO7J+@@i! z_E?tuy3Q`p=bKUl0NeJa5nW*fy-=QjoMe0hWiq8G8W&2)2&f@03b0Q}5{;N5c-<)p z9dOJkW!C$?I$hL>MzNyEkx(v9Up#8sGNMDVi7_M=x9vp+jA4?D|6MGMH70XHEEW?Ix8&jh|1@H zQnEjQuuyqTNN^CA89VqGH?{IZcpfq)!Bc(2ZZ(AYNzBt;n9%Iu^;6S*`mUplKSAd1 zEya$T+Y?%da*Wpp81EOva#0deBI-SD_4&?w$hTf=A+|T#ViZ?K1oKZY+)1!Bu@xIc zYYX{tQXdp`-2P&w&`BAi?_kC4775$P$q*AMyC=h)l9_7^J7(jm z9bCpCYrjgV2;iIH%iq}chkA@;YRO`qk-JNtENX_|m8Jg@JCS|eB!ajdy=c6&tJhis57H|wMUne3FKjs@h=}%Nw)1wx1`s6hye9Kte<4jo z#AoZ)Q1~Mk^-%z+Eks}>5>c~IZCSdj6|)fd-UIWu!*#ju!odh4FaGvqh2h^x#RsHy4?2;Un#z zhJ<2AoUdMG@y)dgE7CPWd$|G}7<>j#4crIqIkTsRXQ@XO@RYrUCUfw(We4E#$27u+ zZg)3YVcy6_g<^H@uDm6|maNIzw>}m8s{St8wElsX|MqMtZ21)jh9HBL0$~$mUhA_~ zLNNgXxt|X^<&OkIwAUk(E)L5oY|bU6@^!j%@0aM+`^b^%ojE7t@qQYaljjds009@v z(R*6^BSUWV{)DJpHbwXCeql*Pp55f-_8|yPBk)Wm`h-?}v>#o*TAJv&UPrhyAnMZH zW!}a@=D^dv71O1a_$cqCXP1~M_+1I3~w&v(FNzcqbswqb> zg2nP7#z`cEf0`7KgFN!kEx@=^q}F%ON%Lih(@)Ke&&OnDrP&1=1q&nj2YyPhIEh)I$V@AoLDkQa7`@jiWM0Zb0mrI2CaO97uk@EUQ7 zT~|g{E`6(01}oiU@8g__8>tmrrQzvna1nP0h#H@{H z4C}9V7ok{uI$bxOiTIhV>j~NpmeeO2aQJpmp_0r6BInTuPUhHrFwwXV@&#X+95tyO zcgSj~YRH92YS%Kh6giDi05u)9B`E}eQ4yLOLuV(1(uqx}0ace>I-~daTgAv0!0mT#g~O)D!N z$!D6@rwWlzBy6?Hz`Gz`F*%ACyt0(diVs)y#&ef`~Z`w4Q(a!8V#%21B_u6CbXB#8_UzAsfwibM)M}xnpYiC~w*AlszE! zu)Z!cmw9F8+W5PQBz1+F1`qNYFm#~ z-BQGKxw3g~4k*_jV?cGSEe;k~hZ@WCH8;)4`Z&K=2#|36@^x%_VsescwXa_Kxbq=L>r8Xg2ZFC%F`GkfJ06Pqp$Zclii0{x9AgxClg|a-5z- z*KkCS$9LvzmH-;=3__bHefazyGhL!(eH&k{1pq}PgV4Ed-H#wJnr^of$={!OnvZ#`pQ2b?FwDIEm;I(54&#AXom>{EZgS$6Nh1TC4xIcKx z4Hm8aDT?$23ciYOA#glmcYOn3dYk2K9$@3Ysi=g@h*KRMXc?25=s*tnINt)4DYaxx zm*>9MWQ0e~l`A7P%czm&{niFX3KItE^tePuLh< zrO;D>&p(+VrBN?+9?pw@028zFGU@Dkd99iK`XZq6B@UM&L!5suY#ygcPUdK5QH&Tg zTi%M^nw<;XWL-CsU0NzDy%!=qW~hXcrIy{0%?2aLw~Qo9P>#@rHodo$B%SKwM`12+ zsvyPT75>w)vaW%=$V?dTG-OS1U*c!Fwa zUo>*wbu}V#Iy`BV^SR%`iG&)jio^=5y9<}SqCxIf8eV6=_1$N(fvf-*f@q3! z$WTr}&))XUJgVIVWSM(zdePF=@iY3J*b=z}3D+u|e#xm4HGF8)Et%?=!C6QzjkKxI z;=dDoqtNwu{8QI_ROy2Pt zylxbE*2?6+4wtT$?is`IV3-eq#n{KMsaVv?c+a0=FuK0T%0^9sRu=BmpL*KG+ZzN&U)d z4vvS$SPj5EM~e%o_-)g#vDceK5P&6)$M)=hcWwsh+!62H)<)dN4vw7PTbl0XH^OBDThu&Ek7AGR5<#KW^S6Y6&SArh5pcwRwgnssWF*K^P(3~<|JycjLNwV)wrB=^>VtC+ zcG}SiOC4T#zfKOT;y=ykuqqC#;(tM`s2kl{8#m;2Jw}{rzROJ0Dx~^Do#Ukkc0WoU1O#lD@ literal 0 HcmV?d00001 diff --git a/docs/img/pyspark-components.pptx b/docs/img/pyspark-components.pptx new file mode 100644 index 0000000000000000000000000000000000000000..e0111a44e186e81aedb9139f53c99ae28422f6c9 GIT binary patch literal 354673 zcmeFYWlW@Bx8@CvySvl4H}3B4?(Xj1xVu~9?(Q@c?(Xi;I5adqboX;+p7))}Nharf zndDyym20nkSB1Khz1Hzka{m}OI&h*aaCN?I2-@nkg z+gP7#NZGBi;=Y4E!v|ed(X2>j=gRJgt=rnH5=t=L{b~`Gs-#pPJSpskJ9qf*Z?I82 z=D*>J*$ig>Ri|Apu?nyV{0d!w-%Ob)o5;EW-N=fVIDh%>8>d{Y>xHP~9M(oiW1%s5 zkP&RZ+u%=PFxK9&)id817Z(T0uMwcN2nzFK^-xUBKuDB*P&}U85JXdz#H8I|txs)? zkgfrh{B$Mm8nZCtW*VLym$Law(}t$aM9fzbRf(x7!Ko0T-97x=-%G25hCMR}elf3<=2m}q=J|;mDL)2E;uHg&(1NP}|Rg1RO4^-~a zem+o_cbt>s5ZI-dtSLS-XN?$F(HxsveRei&wpCiaUQREo$lYrWOEu0^zi?3%5~8Az zvP4Y5u^2r8ke3fEzi>^g!j-1U>k~xXZsx~tDMFs39?%F8swF2uMwEE;l@kOP%udd_05t`1`fLGnW|Jlf&)ld~<~XvI8eB zn0+;Xp!asWVRn2;zSZOY(u76t{6fIz_x5b3m;O}|g7^LK*Hp>SRL|>f4FLhpwhcT8 zUbmbf&9seokgGuyOb!Z<296A{qHXJ3xN<>r77k> zx&&J_6rC8?btK^16BN}zEm#G3gVC%T;ykrb^L`A&n^VLTL#M{O$`lK zqU}W{@X7_@iDYYqSD0A)mib*{<0AB0!l%;pg~Bh8=79~JM|iQsnP%XSD_W(cq08OX zyc>M{IioU`AFd40%E}Ejqg7JURM*|{29XU51`(C;JYbBC6&>yRULyd(%JJ}DOw6gj z3#OT0TuF#*WaehXe>q!oMpld0PKbx{fWe%L=V3h+MJy4`mOwsPjvWv$qR7P~iKo1?TuJ=GmJDjT;F&c|v|VU@ootGixG!|XYjm`@?qJ!Y z8?1FUt1}34bHR3kAC-t#dAchU`!-#$HH6<)oE}ek&o%%ym{0kDanh)fy+nE1EST7any_ES`)>o{AWe;6fok8q{v6PVI4| zCZBITxzu@LqL?RO+mNeX{Udc8E7KbtT1TG8$X?apC$$~fIHkW?(xJPl#b(*8^+aa5 zEu!%yZWc=mVihtN1w3O925hl`MrDhZ{WUYZl!&F%hnw{s7;;vzqww7akm$nRZ z+2pjsnC64_lpt>*tyL+!j3s7C#J6EixM>CpjcyuPZx>oR^-kMKs*=Mh##UnD1HpTK zVK9|9^CwBt^dFa=(c=ghM*ex}Su&BHbB_c~wHQK?p+aY4XwFoexV%G)@bTjrjX50>oza`@}C25FPS4v!mDtx#}Bl(Z+Grw(v`$MtPZ()CvbKfd~xSzo}emn)U?e zTd+~$#qXg9nr4<;A>ME81WN5SvF5bp4tlWmge=}QRq4M@D6@}&SB5)Hpm%JJW}7)w z$Slx;nD{C0k*$~9?w#DbVxmR4;D-K_MzXB1OszJHRJf`GI%tI|ya0|GL_0Roc$zsc;3 z|77-mGW$Q7{h!SKPiFsrIkOY}%d^|sIh#1i7&tkbI6Be)+u*N(*L<~IV~6z%@PZI} zcZGN4Zs;VGk^NCP(t_@TXqQs7EhavRb<*e2bAuDCqBsiURs?Jw zqM)hXMRz$KF(clYOjg>w2g8c;PTQG=cicZvqC6k_LmD!->N1+br|lfwfL~ZxIyyTlA+tmBWfo zIUqM@Z9BiWTE%#l-Okb#^Pc4gm<;gjM8Vu)gD23a=hxJ~5K$~B*1Zyrig=KZ-z=SH zU)7U{65T*&0T}2NwH4`weYgK|H7lUqTYS9C3W-(zUtJ8+8(J}>hY;6;5vwx zlz`Vv@Z~a;n)QoOf4&FW;!q}61AAbv$eTC?IkFPvNn9V>8OxWbNzUv}p*&kp51=SD ziz1QYJ+LyqT~zJ%%HYoNL=*<}=1N*}AaTlgB27f%Ll}k-U0UVTUBuVA zQo_mp>w=0}oBGGtmnLv>IIHWdpIAbDt%b>wwZbe7<1Q1eqYw7*OtDCe}27P%hdaa^bq6lV~pK6(R zABF8P_6bf!mE_6*ry!FdD@KMw%TzE}?rqAId1Aw@0KHsu&|2qsr=q-`A{?FYhAVA? z*3|QuykVu%Vm)wxtG&KiQneX9cky&BnbSzEhK?=r+l1pZGr;i{O?N0NZF<3N*vTS) za@_6_CC7rSN+t@LA;@$FtV*~{*j4W5MxVG!#1)c8TCIGOiqD-~st1Blno#M;@kPN__c?kN1K0DMPPZkEu(G`%H zGxnqb$PsgvtQR{y`s6Xt(J^yYd~n0;y}d3~Rsl%X-kg05Haq!DtshFeHv13+J9)_t zt`<8>%4@~+K_3UlcMcx7qr$IF+!=Q@f~hb^cR)vH%vp1XT4t|JU7@#(J6Xx9u9(5~ zhK@8KiVYHPU+sS);L;fE!qD+ZLpBN^7HIM}ku_#aTZX7IHD(N35?5epOblBNs)9_t z9(??7Pf-WhFXRsy3+rH-vqAH8^A)*T4)+QuLtpBVZ)PP8=vYdM-rMI2;TPIKYVl6@ z-^cJRVxPasmXd+vhX#roHX*p`t_(e;-AIHx#y!6R2mBz6(4}aeZxJ`

          %IN?3FEgH7u`4pGWzi1pJ@IwFa9z4r_1thN!p2C&(XyCf6v== zzoXGp6VQDOuna&6yBL8ZD8K9i!gI?!)7a2H{#HrF5(&B}LbPj2b@MePv zSz#{W$9P;h5i<)!0z*acTk5#W990CCe)vM}T|M5#d1OI3Dg5clpg_5;}vB|$~ z!XN#oc4e!++pcn=`UZ3fCB18rqiT+($ZlMy%ABe)dlGzS4-x)ZW0eVH*JCsUPdS0u zA{ZeT!{^7#$IIpY)VW7vQOeqcDw3gRY9R;~Z$i5$@^pFo5NVNt87UH;j@l^xFkkrX zlGj_2iHMd3iUz|pB(kwybpnyGFdsKls;tz%EqNYS%YDI0*|HN%b8?zutQu*kNgUwm zGwIo;xe)aBc_c*1Knqz0s~NGq%+iDyhx};Tp;C|aG(f!C-%z7r^@7JjY!3BAG*V-` zb(>{ob~xF^$`m(I1E2v~;nb|0fepDK#;qLEu**yJxvaK1K3DtJO+JZ5*J}K8ICrw^ zS`b4&8ad1YJ@T?y9Ccff=7^@VNAJFu(1M5qlvKdvG!eA^Sc6)$;^RIK zNr4(k>uQh61fkUMn1VD?Wq!uOZm>8ROV*^Mt)HP$S&rI6eu$un&ygU<5FW02y?zl+ zwC7?HrxY`SChffT^NHSsHo1#NVWLC^3AkKnBqR4btx$KAMZecsiK%IqIs~O2?FH4-J7-LMHA%$OY^UNS zn{KlG{XtY)E}ff%TdsSJs_y|$`A6I3mw8f6l?7!=*qMC@!c9ukKyoy zsFU6qt+I_pB5*~3rN;D9?H-hFoQ|z$`v->QD)=eEqStDPo4i81Nsm=lw~oyb%KNvjI9-DK}Vc`wsYyw!}`3b*_sBi zRPfU&#u+h@YJBETZ_u!Zly^{s2D8FH`2(_d5=CDvU51#&!SJfBoLhUL=J@J7Exm7B zdk%X%Qyulf(LMchuk_9@5PmOZvy`ZPUl1UmbEyApDO3F)mh!)4wg0uC|CQ9FVm^`@ z_%_No9`U7wCOTwvMm-i$oqpgpkm@Nfa}~+}c<=d0f@DKlk=_#y_tsQ&(XYw(h>)A8qv>fb1u=;zLJ#O-TRDKdn zqzB20PP@}MxAZ7SxO7aT^LbDax=DmHA z&t>#lDM~F91i{uIbrd((*RQ^@&|6+{dfn}EVRd~`LmHgGi!hL|S>PI&C1X&(+`U{5 zK>}ftLc)wGbNpEjw-*NF=ugrT?IRX7=wQo;4~&?B;}M;$Gq}U-1AawD?JCF6GNup` z*{#gNVznohqZqCtdYT68q9)z05ZyKm5W`>9XvC4TqXi>}Kq1e1;}0aMttM(MY^I?3qziiYB`uFY?DB;20>A>f@jJcIftE!`+{!C zs-v~0-^NvLNv%6fdBDqrLtV%UW6Ket8EB_qXh$h9ru$M88X9vx+KOLps2R@M${ zuspeX^9yGIoAj7&jcpoICr?y;u(@ zgzrBp7h1}muDTck@|ny%1CND5tr5G<<5(-e#-l~hTKigcl+i2oyTn*6e-wiiVP}Ms%VYs&SW=+~!=#A8JL)oT@hR zNivYaC^f)hGqM9J4aDe;_W*^#e^s^K17Kl~Huke5EE12 zAA|A+_)@w@#9IBNX?>_Rd>Bl1Wa=$y{ffQ`OMR^MNk^K5m=tl7=DFCj8JYZ;)WWE3 zwFXW~yKuV7^#K_5$;*q!KZ4x_t8ufBQiZj~RvhUvgV&SalKPs4xxe{Tt7!N(;T%3T zaR<6E^+)nWm{o>o=eJnhE!O3>;M~bZZ}Otw$p&s#&2S9`Y}!mCSl(UKnJWAr!fTgK;k~K= zfKB%K4U=CLhV_>bz7W&+D{^8zDF^&V5K+DZ2&k`v7GX*ffOMPZ9M)pkP?Y}S8*f}# zv=JV6^(=$iT5XfE>1X?|pV~0|0LH;0xP*JyNA6Q7CRbC@2^Z?^nNk(Ij>c^l;bT17 z&7W^=>CPjh#;`Ou!^e4yn?YV}8M8B_x9(q2(!GYQkKi_p+tJcRF2l!d8P_btWr~>` z@yviuI}1sijyZ9T5IpdybZ~qTGI3y6a#Wa{v4~7dpPoRud0#-_C4|~%ywE(sHqH^A zT(Rq;H^E$V74sj!g=+YYKfa31vj@JDX6)y4;9J`)Vh0yoT@NcQ7p|}Nu%(|k6 zTLzp<&ZVDKsOPN-kxC%S0LO%N{sbh0OC!LJrY29RH$~afC;RujZ9aOIEs68l>zm{) zmr{amvr~y{&-?d2CpWVB-i}`1*1PHJLeG=l{$45q(rkrA-tsVV{w&E+tJBZX)+J}V zr3MH%KV29(_i^-cyO#Cy$#VyQo-D&oUG5|}DQOk9qm#^o0ebWwIU1~oY*pMn+MiBno1JBCw$wGsTh#U&_^YiuoC09jy7vV^GUdF8KQr~_o(P5csdb`=E z(WzEtl@jzCa~zu7%q^g;?^@PT>KWaEXd^rxO`xp(@~8CaQozDze>nP-FaKHT`wvI| zZ|VI%9R0tA{$%`r6~C?zN00c&(f6H`RRxAIpA!cJMjnIUjx@nFOAL$5w6GO`K@2@V zy|xv5O(eQ28e(*YXsQwpXXmSuEk`kb9zz^Sl^~dnYD=<^N1DKcP|Pe}Ex)tj?_d}hu@ooQ3T*BE{Av{1?G^&x z%WZn|C>FFkqQUvuOfqi5z|F5tc?=o2+-81v;~wO9=@XX!!QJ#L>exE|a z--H#Uzqk|taC#)U@kV|*dz3-8!U>Z|DpK0;G(5$Y4y1{iJy3pk_Tjy9iI$HGA)HiX zUucOtp89wk8bU?_Pyd}&ODVccd<0bbCt8uYAts4>FK8S&Rsbbe%dq_?3a);s&;8rf zO4*U#^Bh*?Q>Kw zw7w<!bqk{tMO<3 zfl1wfAe{oWiYU`ney`M>W39p3!2r3HurgV@J0{l?@CNpIMIx`)x2N5SBaGA zCpCwZTq{^2yQ0h%%!z3<^guc`HT6hA*HrsE?>lQS#QEjAc&YuH5L{YpQK6}N^MH|w zPvkF|ZKWI1F0Pj=H*$dwy*TAy^-QInfeWGNEBZf+MOG=u7%USV9||34ecOWd>>8}G zgk)MD<{iHgx7^te@w`Q=S~Fy>(gj52^V7c^5{vq<&R#rycM|)o#L2?!GiNQ_TaRaV zmf7q))u8Ql{EJIA0IDyjJ6I$CdQf?YA(!(KA`?ci$egIJWA2PQ!sv@s)`c@>{SZZ^ z2#V(|BTS37`DC|7D?fH6<4c4glA}lec9GTX17-T@g*-b-WZ;*UlqzBh_SoeY4F73r z{}o4r6YOd!?*V@u*!>ok?`O?|RvCpJ^oz_|!fn*Ehz2})OzqKm=5~gWE0uM223Q@v zeyDKLCJ(5215(u0f>+)snm{Zg$I#+32()jEv$gBH1c#2k^`;D)E?9@POdRg;^s<6wt>JObAN} z#~C}II?m(X8;M2HFP_k{3fV{U^V=d3uI=XrDFpE!YV?tb#pw_aD?%zlo<4&;mqB1Gs|;gLAtf zu~i^%q(Gxl15g8=h=*$sndy-BsX-}0e~L_FqMc)Rg^|=~P3MXBk`Zhtg`Q{K)Q(VE zX;Yec{v_+jQhGp&-pTeb8N-MA<*2)YqtV?UMw1E4+ns)Um}0eN8@@9nL<50}F<-Ag z1Y^m;$IefCglsck&rJ%~Kf*I?;&Hj%%rm^P$Jgqj-mS68Nw;F6(}9NBf>cnlqDaH|CZPG*qeOwgG*LhzdT>rx_6tPE8MFBE=S%L-j-TI zV!X!QmK>#zkS_dAKCixtT0knHhkX684h+SfOS^)dwM3SK@9zVbcRzWAs_sp3ZxnNA;hYFDvw2K^rvK z8X@Lr11P;z7}f$;XoN=fIgOSaLOfrANf8c*T`Yf0?Gd_KA=UF#M9H%utZ9TadR?Mz zIxTJNU3Y7QoOxcJ{)7-(_p)QZi5Yq}#hSh7(zE^T4q}I%L#c#B&rODpt4+ioJ=wwm zV=j(IlKoE02`3%-$XIJeb~VTbP2oJaW1RhUIzxA|6|h8M7(_>I)Pnb;u?6a)1ENh+ zjO9AQV`%4^(nkkI+CcC1pKeWhB3&Wq+I_Ln35Or5AALjZSWPE;dn8I^iPJ`c{HW!-3A_M>&2s@s#xhgl>xA@1VCqkdsw#dP3e~XtXrc+F@b_Wm zJZV+Fu&Hfzp`(Z-Xtf@+-A6$eli8n^XhuQ-K^LV7y7YpX7Tb-G2Cs6DqKT7uGnZ2W zDoiUDuDfcWawMzJFJc{W2Gf91b-<=$&jTsIp3Nnja(hNQuGIU&cU%jx}<( zNcq9UCBAekh4V87SP9zKMa@QJsQDuq36kp0>5Af&Q74nTLbk`oZe!6D!HR0o31~iF zBS-?Np)87m_#iq6^JvPErZ_E^B#Ji$+O}iWp>yi6Q%}6Ztz65hY#{gSlowRB(})f@ zL88$_wJ^_+Nc(!Fl7zh^_1j-v7jWklck8J{zen{Y7@wi+21#u(9V=LRUJ|l+-4V@! zdhuMYRx;7NQa5PKWRc)|m8*qAgx6Ko{ftI+a-iSH?fBVsSKr@>F|L)1$s?EdJ;wvQ zNSG2KVy60&rl+b3nk=i7gpCEiM`oGbCaabgL^GUSV`zlvd}IC{09YgG&7B`x8O1+a zg0{1-6{K|#tr}-iZIBjzv%>yG64o-vHLZm0vNfId&MLPVwX6gVO-WCyWc{0%d5zaqwEq-R*@vj)G^B8l;_ zMr};JQ;>E|iL7gYQE77-)ud^Tjje*wiKIh4;3*(W+ zVUem$GtH8S%}I??JCf$+@kmoey(Uz|*UZ;PhoyH+FG5FEh;BxV<>^X!S`6;VH=36d z?d}!aN4wzi8}w7F0AK0MP)UIAbS}Z-F~o;h&h4q3L68G`H3{@pF6iM#yDI>HZ58GF z_>@s7{#ht*J|uUAJ)APp<=j_0R9sl^f?TidOj5dlpJR~f@%wZo@9d!X_XRWNs2zyz zxY3~wT%Z?20?e4H!1kyQ6Du$8=3Z2vdy&pU=y%_IYB>J`_14Y$~AvpnZhN&erj`R_~ipc;I zYvK2FU9H~6^g-Qx$2p%cWGgu+?Mrxtxw4|G$cif$R_GmeNj`9mEQ8!Hmw_x?UT`R` z(VLY_85W<<*@7%kzhi3GjMqa-FU?>P`Cx7q1Lr; zWkQ`2kZRT71rxIO2;bz1F0?R9onW0>H&N%RM2@I!v2pqmW|WOaqSk1nthZ zu8Tw^B89Y>ZJMfp8>zADTqBgq<7r{bThzj5yb}XMyUZ4eAsy%vqNhY-NEd7Buah6C zr^pBrGj;}|Ndr$-Y}96bJsda|HsJ>b)%7*43K>?cmGT-&$Q3Sbiw&`%1Iorc;jn9U z+8PV3T3jgTXVBycooIenw2W!PV0W;|S@)YPGea^SgKpgcE%@xY2mTc}(>Yo0Dmo+9 z+_qmXtPt%rh-ONClDNiDImrX4F0F-DBavE`jTV5UwJ>_-{m^gPmG{^Zd(RGi@R?umaNiQ^$oh~;y&qETBIiREt^Ss#AJNz*5!d#X z;mIr7dUp^2&DQsAt=9K?CD*}}J=_f{uHqB!fhtBHL9GJ{IEB$ZhI*PUl)?E+W9@JZ zE@7L;b3Lm2Ur4UAMF|(#&Tcdlj43Ee+bRR~)G6ma9&R*Mj48!DtUPGrIr_~lj~l&> z75YypAC!0x+MvVY3^7L;_?r?t0$1n+%k@`LhMIMzz2im#Ib6>MpKh5Uw?%_!rYec& zrQvxpE(Y!o4orj4uBKHa4FZ1}m03a>5>BRf1YKM8dqb zhzmUtvif&^lC`hO?_{fC2|DTqX>Zm*a!~Ne z8e*a0ED(*Ax}sGrHrjpBQI35M5=4qFKsd-ajp5F)ioo;WUqa=&PpeMha#vqcqsLR0 zq&hE2y6>L7$SqZ};45l;K+A(TD3N@oEb6M!=A}qdS(T$a_X+Nrp*#62qe5m58I}J; zm1Abe9R5n&lo`{SKXPL~`&VmV=Zc?S2Pk0AtTn8Sj%HV9j2$&-hd}h2jgfZE!YAq7 zZ_!x4-+P6mPjY+<_3Mk9i#+EOLn=}s#)ltx{qO_tS8b(f@iL^0$st+f72;E!k!$lm z2ri}%?7Sf-83%j%Ud~n3bNB_3d}-Sq-(ZLp8_{QQdJdef>3?@kozc-x8$K8jN<V z2>#T;e++?(Gs|B_{P!j@#R%ILsK#T&)D6@jw77d~O1v!f4b?m>UV`(`^oX`kN8SQotzh z7O`x`txXLmR~qYQ3cx-XF}c~P7&ZSSBmtWSUgI|-{+QhJF<>9Ql*LqlFRF$!wt9j^ z(EwgTo?V6Xi)hBk<6L0Mft|OFlYR!;3-Ie#yb56H9N%^lY@oK;6XO0gs=PN4oUVAz zH0^0nIWY4%Png>~CVoF2Db22Dlf|n_xwA4HHBCc3A;5<}%N`3nglBZ(n6~f`zl)x^ zSAf3th`5v4jY>0)M&BzmsWlaI4Sh{pst(a@cA|A-eI$3-)WFadjL#)r=qrd59F=e| zgm1Hze@wQ^M$lYna^b!8F(^XAwD*+tInpxkOD@Phm%>`)jhk03#k&In_B@P#wg@fH z9030XF|VY#YtK_t(5!dtaW>IqZefkogqn!R5Z4|iMc1@efR2}tVMQW` z*X3}A#}@ZeFn_Z_XPXe`GJjx1L22|6-N$3n1@$WK<`5oipRQTC?eZBD50R`E$?DYN z69=b*TQSughxXH1A(Sd>pCHo@t*2 z0Ns96XArc{1^{1ssH2gIfCS$H+`Le6UrFtXMEtmu2ki`skYN)0mWLpDLmGdc}Gn(OG4IZe>RiRSUDw2SZXgAQ+1 z5z{w5=n$t81~gDhdw087{pbm&MX}a-NB$dCARV?m#u=d#CO)XpDPh{`EQo!s>#LT8 zpW6K|n&&U0n?9>2qd8e7e4;BIu^u`r&t@yJ3ilVcHfs018uvy?UC;Vk$v*BWub3v6 zs?6>6v6J-#1!o(IvTFC&f9ACSr=vMJzd!$gak;)@7r~}@`CM|?Q)7?jw9Q?wL!HnT zuw$q2^JGB6Z<=G?ucu|;d!IdJ*{$L#F>T`|lEI=^&lS>qsvhtA^=s;s-)d8%)(p)- zl3{vPB8!MzT}?C8sv5e#UtRA4P528|gd5HgN3f)k$3c6Lk5QkH7UZ&r+sVzJV$eao zwOjK+hrs_0boeI+{!1)#=@KWw2i^C zS1(ZXwbJs5M~)pWd%V0n(AQGp;>hAqQRU8A(C=?=FLUa`2}jVhF^U?KV5rFKElU*Z z!}lPQc5x%f!2rs5TBzKiisv^)iT3Ul+DHA@Kw(+9rt{T9w>pCuks={iv0zkA(k2H% zMxTnH^$M|2qFZp!cj-c3d`2eaIL06tmn4_V)xUmrD? zj*AA1boyhkgE>Zyy_Dku)=~sx6``<$@c3htrTrp0%KE=>7s4zkHE>pC_~IQ_B%oOotr)&e|)5B zn87{Fc}N~M+$A#Eqi_czy&d8fXCq6esKTn}om0u-(Z(+`8;z~wkY5@QJ!=4Eo>^q- ztTS>?#c4U*nkC$d((``WHMD*Qhw_3stPq7O{ld z^YYsSW;F#&4n9p^w^>{x(_?EbssF8w>Wjoa=TWMW8w|DO$}D zz3sz)-wvo>+R(W-4L_+qs97|Amwk;9c&UAf?qr9SyLo`QVJmNxg-u8Z%fyS68mw^pl2uOPN>rl8~YBDS0UEOch5D#2l6duBuC zLF~OMsEdu}tg8m?Ubo;3Orm=1rJ~7azamrz7t?|7eo$c=A0q_>k^DN3_rU* z^}VPXL%QN=tVZc)Zp{%w!8D!=%+^?dq|+fSp`zKAm9(n5DWv97YyYB3v-CjE-_vGh-;Zc=2%a&!2j)ZKj~gb^0^reL^Z+ue5o?xxayS zxqA?Zjba-j7mp~jgnjdQVy1n10|G1VOL>`E2XFJ@%l0@o9u2^9=KDjXvyf18Q#&o>n0{c+&Z95xV|Wd|AxriFp^bNP)D{fk*e@ z#}h-Or>mp+DZ{jBZKx2W+;Q)EXe7#}fj2~#FW(6mop(yU7*ZV?pw33iYnExqEx?i| z3J!_jKCzRY1l%~0eYqn4!A=!C6Gwy;5|mCya{&z??9ei}Qv;e)6sV0N$RmdAMGkCL zPPlO5k`t_>oLa^yHH}mE@^?(~4ZyFCb2ID#`dxeK8XBo;F_3Ew%5_qKopFM&*%Li4CTx}iuQt~zmijQ%TlA*{>?hp$O9 z&PeFh`yr4&pt5IVKt;Upu*OY;Z9cl8y6>vY3=>@ZjmAp4g@?9NaZ@3v+qjY99n9{k?zCrIu8s|*#E)=x8 zevR>P%_dXcwgvVM>C!W33}L?e>IlQ64y{v{F1Rx{p!|CCc-hJ3AK5}9>x7gf|B@hg zhP{nXxuPty+&}ew5G8>PYG4Oq_`x)vlvsS00+!wK{<)5H@aTNse*zH~@oEGZxIX+r z(%b~{hWn_N646){l942UBp`%%xDv@&1(K2U@1PvXSPioNUo+K6`^u2@|C%X7+E;~i z{MSqs5_37?z9gswD4fW2>F++NISCR8(no7lH#*lLm{5*boYqKSk&QX?HX>ye>O&y=BxZudR?PR&5u|6+$NLPY7Glc04 z-NNMCZi2e2a}8NaWHwK~Bhicq%~7Z$h{Fq{B}b2w{aTFUrF#HQ=+4PP^j?b2d^FBi z*D!P*gv)1&uG_2b*ci?aS(@})g0p`}$>k4O^YS|?8G3v5raq?I;aw4sR(+`39s#mr}hah5r~Wi5u^EN6`ZZucWzg-vzmx%upnh& zaV}}0-K()aBXzb|?YX0)`XZ)6VbH~TxjE{Ge`~Ya*@gqlSS$A8u)T>s1ZcAktTV)7 z`KKT0IpiQ*gth0K1zfeRSn7!~O6{;`{@A!w1Km{&0?5yyjJ-*@zY5 z^Wyb0(ZjD2*>ql~FA86?PuVUck)m~C7eehz^+2)W6uQayGH!=!N0fmGEVbPY)gYsd zlv__5amd>@Qdogb|2AKB;0C4kEbyYsQ-G=R0hxBSIRcJ6H~79%xjaRF-NP3hrH!0W z7nwSq*s~17XZ-aAN3Ph*r_+k;QOV-hq}ffiw{uiq#y{=RUFQ;-wTW{;@$iTA(zJTZB!7bmG-{T8CL1YBd ziafEB#CMPSh&y4O^sTkDe&oe?w37Pf`5G9k@8N|L;9fVI3Mh%_8%94(hm~-gRCenVDvJ$$jzB zXs(aCeMZ|nG?ExqXxnVq6fb^`!PlD6(w0+alfXp2k%BP zW|et4jVe(rkm|923H_FQlEjV=l$Y2?(lB=K{q5eFR}S$lJfC|YohHg&I#>B3(xgw= zgjVD^O^WC}sR2KqQi;qIN@9N%vudJ}nCobzwQ)uYrN1oKscbDuDQvJJNm;*HF7*R~ zwb}456Ab8kv>5YA_v;xE>1RQt`=PY$(^N%|m&`E)DBjD__0TMpNK`ff>;>Pxm@$u7 z>Yg4F&02>v-}l1drdARn4ARuVp*UR-t7} z>R|kAxrs4REHzwPRGp2iu$!IVM9EIe6L&tdkF;*;uKZb3@Z&1GQC=ZD zL$lGl2P86u?~_@WKB%^^=6#KOM(_w>bNZ&Y=0eUYvLe=QVZ4MRo zaV2(65=RmeCH8|1h>lcCpqV!jt(>|N=oP^%^YhY*lv$@G`u8WvIfwb zmaA`*9V!J_4byz{EtB!c(~7}x(W`n*@dXmT2fXTu*hC%mpxxH1mY|OQKp(9N!fq3| zVhp~)&U>-V>y)HeB52(iQy#TTf^JW~u__ALwExAJ42@m%v>^$WteC?F+dHOQzD0rN zO1)5%j1BtMpf!zPM!IunOhF1ANE2-y!|= zeFy6#BsYf|B}0Pg<`A)L<^hI`ng#{53Mb3x2d4&&30PMi)@GX2;`lI8cM^CswmjL2 z?jgfEOh~^jL!IoycaG#ATdRfs_xJm#_eRiOODQmoY+5C?1WL;t_p!NMPTaw-X?>eb zSm5V2SCnT%sQ1;`Jm!}tat3_GwQhJ8{`Urhf5Q#9)4T&5f39{316v(Z`|~Ak-!{{~ z+2L;5OSr;@ToUXkIt{rORvePBl85;D-^@Aj@R`3VZ}DXUz6(2yDD9D|twYXVitkiD zIA~H(Oc7kXzg_RV5p1L~^ORRh7uKtPOI#?)v0mCKp6g86L8Z4%WBP#8;Ir3J_Z-0w zIAvCm?txR}#|dk=)?Iw;`CWZTz|nySKiG8Rzo7~HC+YrEeO#P5{#2hVuis7BP`PO? z$kVNu&704Eda&ppJy-`vAkB{+Y{GCtV}q>1$Js#I)sF*>q-7;d0ctXzn609|CR13l zXYF|B`xBsh-oMaIMe&$&lMc)}=PVb0jSTCzsME=~+|-pZE$#)QGE9p9L_Yy?cz7F0 zkTU}|wQJ=}$Ymgo0j8H*lnyC?kup#KdKAV-go<&q8B&W22`57-;n51NOfOA68VOkc4 z?YXGUXdwUQ%g9_9k}RuWw8lP#=Gko}Yq3#rA*>TtdyKm|mgvBS)*yv|^gR5o%^Miq zE~*XIs1^CX@~R0^xf0LrzMu<+M6(!s$v%i_QlUs(}M7|Fnk$% z!zx7~+Ym1ms!IU9!!ACDz{n0SOQ3FfXHsN3shB}Z(ZF+Dl*#e7kTqeS z->Y(HvQ_4Mv)K@i?0_3v&>%Bjr5SmLBV-oGAYF8EZo~?~HdvxpR@BPSQh&XdxzdL%4UD z$ZT)Q!#O-Zy*XSm?&E~L^`l*oE zeXLJ{p&5jWL|qK63uHQ_=*qr-;bl#$`bm;DpL8vg@_feR-mqjxdN{cL;4xr82#; ztzGto)PJ`&V4E-4-$2T(V4)^<*yYteD&^hGr@Qk}PzG!^= zM{=w>KvJj=dhA2z#I;)^V8Aea4jCyRGI~H}+XKjO~75$#UB;$mw!~n3gqo_S}dy zUgo2zywwZQrQ^q5ZBo&#sg(hrLKRNn*V3-Uoh7c@iAhFREEEBq8*pkmv6RV0f_S!e zM)}#7v>W))Of%3d-4t3PWwSqN>%%Jd8shwPHMi;HvYUFDmg#!aW?Y>0xp={+)U4wS zKI$E1bScsDUfU-7;6Ux6<8z+85m1|Lu=1K`)^*06@QyvUsq*912rlRBNcEwUbA{Fe z!iZ<6_K|tOd!_iFH4taXUSNL);mTZIu6$h`s9~1qJYS(yEUZsL6qiWD8}y%owcbgi zPMwJP%s#ZQEPr|&kWv}+_%k$>m*d3hb0%KngZfaJ>05LfvXk|_O9 z^j1)?Jnrh}{iq0GK9|J4TDfbij z3*NzM{eO4<{!@o6It5T8@hUf_H*q&eTI<0R-@d_$)$P+?|&mO!wD4ZtsUI z1A~}yN&9MsKZ+a8a$pYa;%JC&=Z3=LrTv~2~N zhnJBB55pW78mPfk$U)DT$jFf%QQ4&c(UKj_29P}wDithS8gW!9L>cNZWuh^%X;C2s zc@C;Y2_I!^W-Ba%=vLBGWCvzQxN&Y| zSR^dQ(&$t|+;~5Ta_#gUmN|}%!K>zNjIRy_e;9sA*+_0hce$~M+-IOFJga2{I}tBi|rO(tM3%IaDk0GKvL zWi5fN5$`EiOYa_d=nE)z>xz@_C6mj*u7pU2;jhke{oG7LQ0e{dVbvezn=G<)?xw2St?srX)F@%rUO zG}3g+;-hlE<^w2N60|!>OpXt)`G><(ksIMtx(}Lo2E>|I)et!(8rCe!65aMY_3{3! z9i&vOY14uH5H?VG0BT2;VH+-0o24VF&VbtFwJrxt>M!dzRmTi>G0nANr?P5TS=?xX z_U^81<)EFc0qQ%p|DIO3@zY&tuwkS<#j8TArBj>Yc9(FfSv?RMV1WM(bccBt5@e8> zGKQrUHQI6^VrJK6Bw-Zzb+x(H9Ft0TgGRU`^wKRqOiqOHhn;7a9x@MHcBh38T%lDO zP;t?TCMO2$vunh7n~hI84TJfWHSp#s*roYZ6sxoUN6sVWoj0p{l`vkb-ZI{ucYybN zQYMi7?(GVz1pB?zubC&&^e3g@S|{^>;OOQM{PB9dC9VZe{7VDbixEsc$0jBLwU? z%kkGw2P0;s65xFAJkaKs9$#!Gv!kK<09V8w-Jn2l=)4cUfNKTdG}Jij_!r=IJRz@q z>u|f?@Q(q_c>V801Mh}i&Dor-%GzZfL3^i!UiF=DoY#s_UWJ!r7sMsZ>5Gs2zAgD; zr59W(*nrwutv+r8Hy*z_(fqYTn+!lY>=LO}+9ejJP;lhese<9cUmM$i5xcCMd5@bW zzKy&OUYO31dUP}xRPme&t$7Sx@PpksC9Fm+4U2=aL?8ZS<6RjkRvW?R zX?uN`igwxhNOZ7c&QB3Hq6|06EU$Er^BRyUd(N#>`hZtopr(IEiGi~~af3n^xzlt) zcwYNGTTfr3{&`^gfcM9ibz_JuT77{iGcV(m@w#3+saKV@~IrfXwTC@Y;WZv z#@&~_>-h=Yv;cd~OU5i~7K&aTIYmRZRIZ-d3$b+%nT9=LlpQ}<)hV%;b~kjO1sOa{>1BixhnA_1ZcZB%mg9-xGHVx zZ0Hb*ZO8PKF>;H08->Vi7DD1f&B??XJkU+<7!MHg|Af zmrOziRVpxl(jed9+B@Nw1m3djG6siT>^1?}H`om)k(_Rx^3O`C2kGO`Z$h`kdH73t zh;Sa%?PORY+hoIZ1O<#aD+TcN)iKB>aQ|^-^E)d`vz>1;_6%PA` z=?3815vyeitSVbJwf`@qa9E-tO?3DGY!YM6pBrtDU;MhdN;<>&qh zS$Vc{kr4wqxtPI3+MFGB_nln^&mQ9!+gOY%82-bGSfEB>_u|TIkztGu<1iLF6I}^y zz*`Vsd>Y?Jca9&Q7Y36VI`TYB8Vn33`g}4UUtdnOu`C^(sjjRJ*iYDiBIbZzKR)1q z7Y5T+I#Xpv(;xp#O<;meSb$A{fx*K39?Nm-%MqH$dH&}Q>vfgdNN5?T&Xny-G)_jy z6pcQcf1UBjCE{2qS#)fdbe=5$)!|=j#lBlay8XC5x^>fKGNxe#*NSyRjCi-zBRLGk zlN_A;tm?*&I*RPp9cU_^8ykx8N4lmeZ}2QB zHXYY(Bh>i3|PjJZ0uaZSX(yZnYW@C;!YVoF#d&`~_wxKy60Z zaDR{&p33F;0mk*AEPCbe)jO5u;W{(!NhNIPmow(I`7M&u2hs8S!WLd6dde@=uyNOs zhFRGAre`)hFU=L0(Xpy%pZBNxYa6a)CdOZ-l7%;_r-qm3!L6h`G6Pag&&10O*)UiW zaaTG79!-o{CbFXARg!3PIs_V~z;+s+aJJ8X`D%U~B}7Yp*(#-f8+7yEv{nCO2y6b2 z!0SJ_^3VSOFRw4gn(lXL)3MsUW~VSxjancz%=z;_$P033Np!S^1LoMmDteA=g@i&) zn$l3XS|UT1LxM7bZppp!Agl!^yiS z$ZlduK_Nkc9SP3Jxt+{Z+SK#bvAXTxEy~g3rb?q$+H}upvuEOMk-6FLybY1N$Vv0B zFimm*8eNZqGk)D|Ck<_D%+Ol3VG%9U2?y1>?o0Y`ph*YJ5x*+ZanBn9+N`>AI&} zlO|#$7omyXaO6c76+leJ27L@{3@nnTO$#a#QSmsc>z^8cBeB3;he~B=!IolG7WbX< z=i};#3m%PAI{aX3q-Lrl)?eiP+PGJZe_yr&;Mz#b?eh9l_zPTBB9>4D1n+1aHMSw< z^Zme$u8Y}obLSp)pZdix?0#~^Ny;Hw2bdw+h7ljivbaG$>F#3ofk>CybAnCZWZibe@M>3ZXsa7H*bb&BFkKXQvHL zdF~YPZ42@ABJ}HwCUixJ6BCGI>q8C!*d$~v^(L#rTZbK@Mw$c+F|Wz`kR{{ib=YHv zYT@{fAZ(8x#rlqJSLDnM$y=qIiR`#nk=cIn<2(r!(3SU;PD!ayc(Y+R*r{bdOEp>D zCE5k3CKpFlrR#8a+5vVDf(uS1X8X_avXvzoC7*fAC^x5y0%#5!KJv59Ep#1v75TXJ znlO^4gIZhIyKml=eRL+_GO`73-vh28Oig4IVvb!e@ScQpa-1B~COR zt0W$uVtvsjg>Erw| zyUXWw=+(Z5s2NIpWc_i3dmzcY1kPH+j;)c|ZmoTwrUlDAY5h(5Zb&Ia6aF-%1&Z|i zbpT@&fQP2hrEy|CtW{s45 zgb}{5ckSAC*$3rQ^YFG}g-u~0c|#{v^y^ws1tsIj44yQ+^kwJ$Ahnp1T}aF~kEuuD z{@c;-x^TS7Ix$>O8(C?3In@y2?3~ip0;Y6R^$S|Nh#NAqAf!f{tT=SJha~EH-y9ig zM{KjA$~N_gUO}2@vS94&$)nt8AMtmk2qbk70yh4H!fe_;>f{;y&lp>JLOhj~)a*eF zXnN@Un9a}}rIdR`tyn|(LAG##>;ktR#cwHM5pN?uq6!?Ghbb8S9g;CEdq%qFu>!eOSrv*1rm!HF2r^)4->jAgL-h@g@*#Os-LBl!R6`E6Hk7o<0r8gR zwAy|QQ8AI@7%(CM zBZMuzIbg`DNi77jGYq?8gc(bVJt5q>u;vvyPcdMP*fd{HofVMDMYv!H* zv4aR1K`V-zkB4CbmjIf#me5<6mxo~@{q5J7o1cYYLjTjZFwY3X#QNK}GdEuk!vz1Q zZ(%Ns!u(wpc-~R&YioN=X)eqQD?29?siMGGS#s@*nNt*=W=FaPJ3>pfuS}s}E`PW| z`w*o*usO4%Z-WluK=EQva>F#?LTxztZDxs?YYA|sN#yP;jF8L{e-e9+!759#0!XKZ zCp{y!^l~XMKR8v-QsEOUKH_|JNQ!|rIfVR8ZddXBf6JQ*(Ov)-1&{&nI546{}`gLsbJWvfqFYCmO7aiW&C$u$_P6N@nRznR3 z0=V~r)}#sPP%$vl659ICuNtY~(z#bt3>}ZWc@Df+I%Z@I5=$wtWfJ1z&m{4^b4_pA zJY8S)jx-*`>*awkRN|IYY1fh5l`{%nMRyzOtHM&t-fkHWo%yYwJfDyn!?wYabfZems`4kRkET_!kDWW~vyx2) zL1wK^Gl`BOL9IF0X2wQ%P+h-t?4;0rgjbLyYfF@+NjWW?oO^L4CFl-!tbzZ;P zJ1d5OO}bYgGtTD@@2!+@*?=ddw=J{zt86T8yD@B?flOW<-Mut&!6$v|kl5CW7cYu^1hvs%;haP=QeGbbdnP80ESmA-1z{>Z>6GP3Cy-s1UP3_x%?Mv1D1_>qX% zuhP&vvJ#xG4{|c_#k2ASdB(QhG!--KCX;9M62*bO$2jDH2sJ8pM{CQPBu5_+3(7&m z!@hz6!oC`hl;Nl|?&xo?;`W%{>vOAj=*Qs2vXU#rT-NnsDKuD?6b>YyDyUPN)f5aQ zp=zjOo7oi%B%(U&uxM1@3cg|2frjIG?#ozP!*1?wCnX&PEGdx~=R+z6^#+T$>Pit_O6Fs8Y~J&NqTSDds`o}?Qa@@-3? zJ`$H5cf7qJSY;q@9JuYXx8F^UO(AobIXq=onj=kDt9o5Rbg}91rK{e5BJ<3<&fPUQr#y)8}%dP86?DOsf0y-rbm4qHZsi_j;=H5LTgBjw{cl{T%!&yP4dWBO(*v4;$Gqdlf|{8C2!Z{dFO>zEMQQoWU0F(= z6Q{7F6r^aMe+kt{8&c23q{%gP&+78t_gEvM0txNyFy%9Nsi`Wp>MJ;L$*)KJdPZ$poEx;Ca- zHuWH}1>LeLMHfy~I|pyBIsgvZC+e`m`GFWX*kf&djNe9sIVw-dhWJCA5r143a~)Ow zwpIL2kMz5G4xNJ%uc+}L{+G5=IH$y1#@Z*{;K?JV{73=a9gviSEXkp#>sHr zLr9}gpW8_zPky^r{@~5Qh zfu)?czp@sr!*fOaVlmf;_;<2K`Olv1|2UBRXHTZ~ZokR~|EH=@^rwtrx((~J*lbJ- z#_XZ^D*!+0b}01;F&}^ShjQG6h%LF)>ZB%kN)5^R@8XRGcb{I(b{|UB>H_g2|Hr;3 zPAK5f&|!P-(_o!0n5kC19N0q)WB)nt=DZ)CJWr?mMQKKWl7~MA-fgWTBoSr}rJ>8G zeimRGD&#_0=9C$eD?B0aLI=f>o96=^aZ(?Z861Y6Fp=E_;D`I$@9`lUE&J}cXGv(~ zqU@tL1C<5;4LiSbBDYw`)BNU4Fd`<3zart?15{Mt(;$ofc2}?m$Nn^z;E>%Vtq6~@@_08g?;A( zbYKsR3xtz|5rZNb@~rj3GG4l*+4`Lb&{k5kV`2vD|?D z2QLT`C3bg!=pc37v+glB0j6dS4zr?^FiKLA z4sv1RKys6KRAFMTKVI@9<#3(%vj6XHJ3uEr&Y?K4X^}$V(&#tKCEh7pVV3Qp# z6`LFwRb1sxHUV0vs`)b(4Xs)x_C%p;9mWnzBh6?|SYmTncugE)v!)l#z6z!FErody zmic&;;*zFA*Y*XGDiw;~$aC%BZmiR{oztRh8z?oiBpey{d- zH)s^x>>m4xFxP&!n!bCsWNfJ%(K;>eQo}LESkg7g+h}<4jfw6TX110925pb!UY6nq;%R{3BZU>dY$A15^P8y zdp^R>^RB5HVJ0l>JU&&Go%`UcKY5jevu1A+l0*s)z%8%fznR{-6p*YYI4dITk1?%8 znVZ$(bR?2$R2UcIxO9rdS0G7cncgTydoxO53fTejh-#OM949UiS5}c;Ym|uKN<4Psr*@2bk2<1#|*fl2l?dUR6p4l&!uMw-3zhvqb ziIkc-d?r$+dSekiIFjwmBZ09pM`dP=CFcW@AtPc8Hr#8-ZTZL^Pqxv zneH-c#h6ze>#M)#Z-I6Kz6M{YdgbG)@R&f@RXap3N<#8nyC{o)JQSB zS@o5-cZ(0mz%F?+1%kS2%+B=2xsIK^yp3x_s?tw*4RoQKc=|Ju44l)1*xyAmM$Ym> z3@K-C!C2F0X@@-t%)qU%3OE5j?O|w2ZsV?L&*mRz_8e}KF~qC|(X;fs<27c5C>Ij} z2mBW;ybR2pA^{xQRxWtbug}xxQ;AQ~QyfQgqb!qEG9+&SaTj7QY)?Ms;S47I1+9M4X6NRlyZeEt3?^A zWDa@l6?}e0Z&^h4#iKl3DTwb!>tn)xR}PM&gX%v)2>Xc^+KFqTJ?T7Si>j~#8_SpG zkhzo-lYYYE!(}+7QZh5ZDgNR!_^BzVo}>*kYD62&i;4z6Z^KFB5zo~A`nbR8B*c<&3yj9GY zbRRE_P?xVyh;cR>1tP!P+=VI%BxKC8I&OT_FeejB!MjHh0tl*>`s34Z=i2*-%mSK; z9X(8ug}Q=Wj;R(p`0dM$>{hxsa=-uu818uLz?loHhB(*2xC}e#KVY-U&&{Mla)Ygw-INHAgu> zT5~;kWf>isTB>nA1upK=o+$6IDlf#=phs@jhijlTk1kgauCChJ6MUfBPQ(tHAi5_t41NQP0uL#`=#$&ehiJzoLfU{Xc+%FImJwTT=`u_t+Mg zD&p;ENai^C@da}Se}eL=jV`(Lv|$V9zm(?BAoUEa^woLrh5&DT)zCh)44qXBTZXOl zS|R`%3k+j&?xH?YQCOM*3k9L45IwsHb>5orZkfoFM34&P17pC^MfkyA(?(Ak&MC78 zyCbSI76Fl>Au$l^r;+F$%1_%pnaX7@kFFwW#3nxj8$Mt9dvsY*a^bHrc}cEe!Z5k(2$J|w zCt2%#E=4E{2_AV%d-VQFugvE#Vu{3f62tLAI{0sA5)}Z<#p3B4v7Go;7kZG16qr^c zggR#Wm~Swft`l)GzC^zbZgM8dLImQMaX@y6GrkQGoMBSdBBWrnZk{ckN}$ccdr>dO z1)_-QK*8fhNzmlmiS(DC%nW?*tJicsU6@EcP~azrawAcmhMd;#9CaXIg7%m=DbfkC-E_X)8Ss0_KyKctDPhIo(ssqV6));_0~^khp_! zlA#Hz@zn4c>dNw7zWs6%iUr2QW3hgYBnB__Pn{o!>eNW#Fl&C3S41q6sDPwb@1rW0 z@Yh-87UeS}jX^l$n-voXn)Ig&KcG=GPz^slM#$9<9?Rj&m0JDvex7o-`IX;1@94KG zAk&9aQu_GYs|Z=uuiJGBAmn;A+$^J96#`zOkN(cxD9Z}%&_K@5Lr`Z(8>cWBLc!@i zQ}I7W6O#48KK*FkyzPXn`d(F^=h(>LHl2d4DalP04_-0SH@M$3uwTdEww#8|@>Sl5 zG&^xb&*G+pp1nQAQ<*ecJ$@*1BtLw_#OS7(*=;t3v-8iejCy==G> z`sZ@VP729~mr2K3h{5~P)k(INya_*w$%a_XQ5=)|6*EesQBgZvC|FUWf?S5YOXz)Y zLs_S*v{41MAH@n+6eWh=5sz(*TTq!$hH)GDt+L0hdrxj`K9=4di+5P>r=-2t#9r=~ zjvjr^-qG$%u$N3UI23fkTig4(6pa6Ii^m2$2d;jp$>e|Q7XMc%xPL77|5HQ!gW{5A zqaO9jc`_J#KpEJr30!=&2O78IT4nKADx{4^Cl!)`5{2lscF|K;#Ig>Ig&! zohLtzYskSfGhH;Cun&IyLe?EWl-nQ{U4Hm`!Uylk+Cj} z>57y?bnGj7%bvR>{!fE7(J(-iiYdwTUZ-@Y%HkAUH)1FFTAqc*bwnwWfDYRF*EU() zpp`j^hh$N)OScLn9+tuq8cF%|rn{yXt&Wb@zs$Y{Z~%-8U-#PiZ;c$x*Dq{s9citM z9Q6$K9Q6R%K?Yv|22fXx%mONUTE5XLV(Xg=PD#RzT18bDw44PeLcy2`BKx<)9%z)gsJPqH;N&L`8rF&ynRBn07 z+|!(BVh?y;n}p^N`I#|-$t>r;DZ}%yO2JBZY7N1ZpAYS-f&H3HY9%UeK~3ZXmDo%U zUi~X+HazzQn{-r#Sbb-pR^R|z==;VlwB3sS0&qb80ZuKh5yM3RpV}4NbNc16MP)CY zs<%=tCu6dMO&355cS zc?6Nu4w{*2!cYY+=5ZiFvvXXmp`e|s_hNC)x*E}Ca;TAgIL$`=FzDWPWarXB*hNqQ zK+yh$QsNxN?9k13n4+KFfQoNmC0wee(K+<$i=N^-f_zxp6m1}CY(TfQ%)&x+j#ImM zATXl31G{hQRHc5W*FakzO4GFB05U{KW4lRI=L6C2jKQf2_{}jSxc&vpx-!8Q?8^m=OU8HS7Uw0$J`<7B<6tFj z?DyKInuEg}xuWe*+u+(^62nNIKW9v_;-NRXY=*HCrYtc?xXy)g7a$`2ZR(S;;mJcwM7V7Xn10ps|4tO@8N0*FD8(WIFadg>7&%YstiZ z6RCVfW-ZE7kia!AR_B%Q`ShTdfF@h&`0Vd!#mfLv+5+M=qH7-52eoJ|yEG%j1r-qRWdhQ$inl?6P#fb6kFwJennj5r z?upOEK|tCA^xitIYlbX*ZV#DEgiu;qmP!Mj(lt5?xPj=VT@I|?be?|ENP|}es7kb> z$uX^y3jN0H6LE>_gI7UKlF~VjrCWlK~W=1XCp^513hUYCnFj&M?Fh3gYU-H zj(;?grHhg*7EHCZZ7{DL?8OB#B<}jg#s-D7Z_YqRa~lWE#q_p16msxx!YmNiXNN=f zh>(yFzmN#um@oi<9AHJ}bOb;WQhZ<1 z?N70)=90_U%uqWdIUi+HRCZF}fZ=0u^N&j}QHT(1fMaz0A%txKNj9G( zCaabmy1fD4 zdlIFDdA)(nXJoF>lWUs#NldCJ_kXq2V66Df(=T9@Gq1J`I40chvLERB$RVgd)6|WY zVSDzfAU#J_(Z6umY_FD<3jZbs>p-n6DN0yVdkMCIEmoJ-v{Zlw)ajX0L=E zi1Y%6*iOsA!`DUs)|ue1m~YWG0f%l&0cd@rtdPsXb2_*D%FcBV{Y@nYW`FNnS^=#-vQC0 zs6>87O3n~SFp|9wP1;ts1JvNpQrMhHdrh9aB-sHa?>a1|l?{hn9u6XGqGk1R^7y!C z-Q_-t!cZ_%aY9-#7DgEBaa3!ylSD?NiiGJAZ%Y%=-pcqWYCm zqHxgHxI>Mb*&~%&!1!`cPv;*?a=o zJRcE~O3sbJfa$aU9zw#WXue&zA~Y&Co{F13xQujlqt2OAMaNjlK5ca%>?*3b6mChj z!$o>tB?M9QRE%4FC#so0w&}is^v<+W1aha_@@|?a;|HCU#VE#A(59$xDdXs3$uRGe zk%Xe5S7FSrt4Aw{ZMSKNDZDwgs!@{O6HLSy4p4kV&~=atz2D^M74$c<^_6+(uraIdZ{2%BN{-jfs}$^uu)b zOEsV&mG(NY8k$kIbQi#=BjeV3X4T6*l@Zb5oirXnJJ8IppVskv<<+$V}vwgU-zAQcZas);d%Nq;Hy)p03K+%cd#^Cay>o6h$wI0{~BY3!2; zE8WB%bfAz}z-^qc-*k3sgQV7LH; z(fGG8ua~;R0IV9E4%w^}@d{FXtcni<@e(C^wAuQtrn<(?Y$qt(0|8Jy{aYTetMo9T zVPsAHW_bbqq%1P}DQKd^2Q<93<2r3zE8}{}( z7rR_!j@otrw^Dr{K%L@Sqj?U&!G6{WtGe@}7WH3?GYrr;B zqg2<#TS>lA65C0E&^?BM7m?CfT=1QGqk2v{zK#g-K%;MDd3(ahSJ&XD(U~3uNgp6a zLUKP;b$>$2ytH{LV~9Fg@#~qNz{RK#Q}X+pkE4@aB<2^N-EgdcMf#O8Rci4Z_dV8v zZKy8_*3y9zS!gcuxT>aj#_k)^0a>o1Y|~=xi6LvUkoLMB83~_~2Es;kG)EndT=`og z*YNY_h|qS;@phU}jQO$3sYyW_Ng2bw1PpRW3Nq#ij3VA8LNqpxe%n8R*&m6~#5Kba zaAt3AlqxdETmgn-hl8#X%n>KNH5*0`&Y>OCN#C4TiyJn$wN3v`%I24v)`Oj?LnDXO zx(#44f6GVu6PO*L63C3Im)1Xq=LJiS*(%t*+lRWiYFVH}#Gt)zHT{>%u$EebE)3nB_2 z;}UR;!$-?-;-g@|+K`|}d4y)Zc2Dx#6{bb($`Cd+T4U?hXGHVXJ+8CC&*@H2)e3=+ zo+7L_;<08K=^0jVZbZE2mt=6_w=?! zHmW*wzYtm!@Ylug#V_uaTjan=L%6UjB`H<+R$B4a?Odt?VEbO;WaZq9TtNVDVFOtS zD+MH!{oHzLUUi}6K)vXs!G*{fb+@R&sKtaTb3aPkdLjZ4ifs*{K*?D3z;Sd2-kUsS zK`h~ixQLBtaiF(xsEToUFEVSgjZRY6k`r;AHM6wC3Phs5QFe2j`!M2g<4`sHMRB$< z9;w*PW61)jf^W_fq}A9}8CGV<7C^u3amlLujmGFq^D@1@2g|q=7VGUjD{EIJd@ zUC?ifeewZOMK)|&dzryt9jTGHo2VT27g1&wyD=W)TdF$Py5|#Ux)qX{Nb{$}c@3$E5}IVo#-~^n|bLSS;co7D6=uc(BB*fyoVnoQXn+&8 zF%m0mLF}aDk;K6&naa~im93daYemKiYchm9){HiBkLK|5s5TUR@cR#QdHpVvdW_;a zCb;S9h);|O9zcE5Z|)3kQjfn!w)%hx5Ymne0Cnt(b$*clcrYWuT_PIkyr-`S)5ACX zRh=D8INrW1f$kH5D8H7mvR|WZyrOF_yF>7^u(JdlKEk=zROE{SQR4O8Zsp> za}3)sMu%w3FOGI##o`al;0`t>jKlE}%zkJX3@m!yA1^n?rxyq7VxbUN%Zye8E_f8= zRo=&V9lg1b9JrtH{q}aIpY(Q!?oppUz8q1y_t}?#NTTjMu3} zPSdZ;j7lM-7u%`tNLUM^`4QJ2-oGggQxlcjdUS0XaGvCJmz!+iWz`trN0=Blb0~({ zeKy4NN~GJx>LH}=3_q;d0CBk6=xYXA%4+l*ax~(Kq#h`aX{ZvF5nWVhPag6}hUC8@ z&kiEiVsl}%^oF@6lA{Nbx&iY&*Jo=TRFFUh%&@{Ih|~lQzAlisGI%O%{NjHca5dT0 zO0(8^k`oB@?t44hw13ICV_gpz)R52A)vcA}N%SCQH)u4*Z2RMjD)-jZVyW2U`RHAq&DPdl4 ze~E!KT+1bOJX#4PLss~`k4|qf`LU|uyURKw2J|8Bbxc}VMC^-74LzU*zg#9l0N{%; z3to!+lex6KIy1C`nL7aINYXdR!y)CPDIs>k02Gc=hyx%qM^k>8_uFe9qM96ZUYi6QfoU4Xw%yu zFsIXKnNG&1@Wk5Jrp?E4?p^6!s%%yed&9njT|6V&2h983#M@g(e{F~0ra2s@>@lgQ z^(;ao^RC0u%)J)z=+E;lJMA@h(_vCEGmXs-Si#ok;EGvRhaRa?!ACcnd*}R#UlBni zHhW_&IDpzP!P!6EiY*MuL2lKuy<1G{3_nn)4RYaa<%)+{Q7!vR;caJL(^_5xAS-HR z@NFoEmO^`dH=?QmBY zzbl$q!0&MjkPI&+1?$K`A`H;iAO=qSF%QpEH&$<0r>1I>I4D_{AdS9 z&l|=;YWFeA$CqHwOzI1|tNDvUcJ%j3^@b$da6H@NH}pF@pQVl4G_=4WfvwEfkE82} zsXdKLkN2#c<@DE>$()xAzsL{5Xqj7}<{?nf@8&F-N^q5ha+yk*m7}6$K5E|iW=4gZ z>}pmb8=cwDYH4S1HuE;AOEx2VRs;h6czFZIQ#JikCTlt(@l!n~!$$Fz$ zSAxRlVXPTllSyYlk;&!>-fWu&lK%-Y2eo>i0y&8zMdL@Dd6+0{13WTGrwk$~r# z>M}2#Gm$}57Ria&*4D3lppnf2SDAcM9Ca}+MT?Yn4tgkrJ=xPWzSn4Xag8QDFSZzK z8i@e)CvIBxC+r^eCup33ZMF{~MCSsdKhkX*pGMl$tCD|IUCn=@=lB{B_6bsAOFs9% z(vhGsOtyGG&TXr`e5eh_Qd>@eS3K@GEpKht47mw7A|pPkqP>25YJ?b!OCd-u@BfKd zt#qu)i6m_S#X61MjM-r$xFEFQk^lpj#cr0pNe#x7oL0#TN96lQ^vNizL(P+T zdrsM}V8(o)M-LIIk#Bc|;;vV-3;0?0Y?Tj}he}9+*I{G3JNrjkmZ@VS!|QtTum1~C zK(4zspb^gy8XH2q63_2J%henNOsCvhU|N1ELZpC*^0 zvkoikxmcH^#ffr8WJJWwK^kM^a`rIQ#V#33)#yLbC}q2$Z7L?4zv4no%WbbizF3Wv zx$OHLhLJL7PqltNHp68y+~w!u0ueW{@o8IR z9S6Cqgc}PT<`?etkx(8C@Xe;?KDGo)593N-(*Zil;mNc}qL|>(2$;*0VGkhJ7lG5k ziYF&E*Z$b}m`k7ax6HK8-4^m6)ngfr#;a>hdwre5mt_0AHCXF)@9HmssVBRyS`^wb z750@_<~ZU%gXSl-43uH!eu%w~D%2wF5Uc2rIW6MUTZIJGc)>e{xc!?0WJ@}-*u}HL z@8WWdCoI$6F}LCr=*2hrk(KmcOReY(RpCqN;{=-nfc8(1>N(kHwCjtDBWS`|&55+K zbq~1@L4x-~{P1Cx6<+W6`WJkdYyPYq#L*5j3`#O~ix){okfC3%WVN0d=xCu#i2HM= zH|hgcKktZ!i^@Z8o(->$1BB5^qA!Lpc792`BtE)rjCRgF7(QRr8W^_RU;K04y(5_A z$By8ndI6$t&=7$Zjt|K0?t%sBzdUX~ri*&FvO>{OgR-<07viIFNcir*`|L+?Di~c)mCi4(IHXLRsaEW9v}TyK<|Z8BW;9)(Zwf&l zaFR3ETQQRx2xV46c+%+GiL=#t%gU6D55$(OqcTFXZR#`2QA@B}b{{!wfoCF4w~(rg zPtCdPGVXhEQ?UVEL1}47nys=sH-pw;g|)01AAvKP+kVdyL$fN$(Xxj{O2N$Hnm7Lp zaxN{c`rl4(vTBTE@X@2dFaK6xB`3<$tQAkv^*hz^FenCRqjpb00lNQaUGZ^p7{I%c zEtqwPP7c3W%k`nn_a#p7R8B#u{_6MLheBnOXnpy-Z#tM)5+=#hQw=U_P~g&Nfmn8! z;_-Scz0wd$56S)V{L$r3a&A$ZO9@-4^+A$Amn)a+Ta^wfd7^!UihXGgTOlZaoa%G#; zE}W^WSItwN-j1nZGsq-t0`Rkgs;P(WXl=v%WXxS0SH3yH1VSa}?WJwxTnL%9tgQY` zrHaYuy5NA!z6SxrM!NEwl1-tfN2yAWn&7H&=Nh|lM$*- z^3kt=?a%Ml2G79$hGkSb+d(x4(2n(dh0C#dEIwT4! zucRSO+HP!nkl`@mH3dwwoUD_9B^=1mAL1Oqs-c9Ii< z@oa?!T%|3u=TPYiUNbEaB{)(m|B&r@9wNoAm?U}b*(+Li!6CM8_n)a}x~@;1neg)7H6 z*P$p$rbIlf(45?T2etXUfcdraqg!2!F_xbjGqQ=ajrEsTez@;aZA)*t#pLqONZOly(eUjxVtP>bCg@7dW}CM8!nE| zRH~P?7U!Co!&-7;K~|ZL{X}T@?Z|$Nf!=jPdDTnR8Ms5~Ji$&E2(QV@`BYUM3-htu-K5~031xft+D*CHU}gv^ zZ|`gutlQZiQG?Z}qjZ854JY_{u4@(rQD$9b+GetWIYK$(09HdM$EaBKNg4U{^EGu_ z;clAwmBI0*U?pqmc+4?kt*4Tg97odWnWH6SIuRLmTXdW5ViqQxNzMc7gRidnt1uY| zG)pOGRnedky+F|Jn{UNdAFLdW7;Ba$aUz9z_+Tf(@&ZFUQo^-meQs?vOVIRq#>9nubguM6C@Wn_oul3uXNNXK3cQ z@+(|1TwYzWll;)>vnPMZBy^4DXbh(USqsChHjltkOjGg8jALukdTw-#e{9Ey3w;pM zedf0pl!h&BoXtKA^>_QqHkb5ZVS1~I?NCxN1d5p5NtPnL zAA(;s2IocvYwy8v0-L6S_rDg}=R8-+^k#e1QCioX+6%U#A9a!5c5a@bt*AQ7$@Cae zvZYlKyz!L;(7!jCjsHxgzPXR%@QV#U;};X)8E90PnP@moE9@k5=R@R;;_(r^o$wOM z&bAYRJKm{HQfTw3(RkJXb0vfgl}zI4)b^v|2qd+7rtD18#8?ilmFn^5Ky0Zu)q^tL zSS;k{B^y7*86hVfb4wlh`N9fIU1V1I62sfGeS+Wq6v$1jcI#xZq-PS6qgZofulCa9 zeG7iTQfwqlv+!wN@A&sMIuv*>08V@m-IMg?FP{-z!V@d3)}{8FDjs{Mo#Lm3SyL-q z%M72@9vHj)=p;gzx!4tPb&P_7M2F=+M#nF9`r8_HAy%`x_fY@R`0%+OZ>rFx$=-0_ zIkap|SD!SAwC5AOk>w7YA!K5Zt%bKM7gb2A*w1siq* z#dP{pN9zd0+VpFF$9{w9pa1L#jwde3&D-`hx>|#7>#>5(OBGc9wzQ&;@kEFoFk%-% zSYN3T4$7H4%FKN@O^Ad>e+yU%(qsQ*lSZIWSpsQ!5FRyXk7IYMe-M^8c@%~w!c5%j zY}UFqD24_FWpc#e#Hsl9c#D0tl}Etah9}MfwKN$M*}#(`Ab?qcAD7_QNVvOjX3&}} z^mj{zSdR27vMX8AklM0miyR|rBIlR!s$l%0{pc{F9}=;)IIu*)dy)o{OiYQr@&f~z zn{~Czx>F(qgx^Cx6ri60Z8}*(ZJ+MW(y6P3^Z+42K89)>+KW}Os6BnJ-Acgnd5vym zQd?~V6rPL46;5tv~0R1EvZuGwkCi~lqEK(U*t z%M(tYvtI-P^7PO7t#fbKb`t6?`)*qT=%Faa z;vXB_+o8Drm$GOYNX;mLp)ESZF1Q^Vpy2BQDcsI+M$z=+gaZf4`uP3bgMz8u> zMyLEKw5-|QVBY=p@A^#}*a2EPoFPFR2T#SJ@%KE0rXe++4|nR(Ik2}ida(!TKSJ-X zO)Ri@NK%X>h=e+fE^LvE2ApJ78hol|aveX(wzNt;QuTbV#{uPFu0p$pvWfB4lJ=Rq z_dcYyjv%o$#v!S%pY)mgQ%ZbDS?c<2&x6a{q~u(+K(?*C_tF`io}nQ;3IdlV=Z=J-=MxWHH)~)L9@$9TkbfNG8!TyCkJr1|$4<5l!2dTOu z@wrM`KKxB)D0v0fjTiPeo(z$nKw4i4hjkcgNVy`)hkb1aNgQ?Wyp?1C$c5#nBt7nq z_Jy#*KRH^IT^;lP+DococnHz3iv-XfZ~g)woV1G5oA8>p3C!EU}=Xx8%ySZc0o*Mf{ac_<42oCJG#(jBqHoxT^=xM z!ieZu2D5orDuXT)!?!5pC@fW4l4(q}du|g2Szm9EKq{Bj7fE!rGOT~tbf0BsM%;T9 zw6*~yo&&6MBC<_OLE45!%2_w!bD^B86I#b2=4bPDx%75D+W*k!S!Ft^m~&lJn7R*Vs2A!C5sjxarhq z^%{kxlLJpHa}oimW$6GL5iYD5DqlFoyg?$3{D_FiZNUaQp=xI7hls-w%yCUwpj@`b zD)Ko*fu$h&S0RAk_r4h5GI(RoB*f7BP5J|(?3iEqz%;l0oyZP1#GUoOYagPwRF zs$~sCM6P!h#p6<&g#WTll7KGZ;Wr48vHV#-EW) z$Gwn3S6E0^l&BjTBZleD-?a!dfqKwoF=RN(BbQClj}*^`w3bMB*_6j;*RpL%l=6=ICtCA zo+ReE0qAp;9z_r#3Df+Q8Qy(5NGu(LMhA~oN+|-WSq$4}D~Z=Ahzu19E3fx30?ws~ zhQ*mj<49bDHE=n^AVd4tMHM*T?|8?BDmj2m{QIt_Brr0>qdMVy4im`)V07h zOhlMaSD&x1@RKA`&}+FNe*gAB3RbWRtV}f7kVKXRK6{!JN*jHZJ!u2CoRlacqNQ`p z;8Gr!)%|2{RG(QISNk1(epn(oZ(sGTNOoL}&jiOcqF3wc*^3R>p&vOw+=|)W;(WQ6 zPP$tEOPq0dc8_vDkASm}Fcca~{o~BK>3t~{De%LK%c9tHp`fjj_=AOi-7^MNi6T@k zxQm9mL){qFrHcQBU-o*apz+VVcZD&}hmvmXd7cu&7rv?v`4s?7JIU4xF1pAW@?DvL z(O}F!w8fxBG4-vP5ArMSoNcp_AG^34R+m$lA+%fKbd0P_LZcN0Ay5J7=MWC0dOTAz zwu@(!2bCE8kt|$mkZMsI^OOl!kDTob{t2qhO(s>bDA{gCPkhB&;n=G<@0+T4Mj6#l z^}JuyMdqvBlQW-)-Y6vSGHT@fA67~&ApuS#8P%;L=ZQ6TDOWz#A zLT5HLBX#+eM?;*$50TVWXNs(Cj0)?#VkBxcIs`M|*4G19AuVV1MQ)3vPn)5f_~vpo zr_WyGs=GiGI!fk7iamxMS4D!YL7g2WbUdC3&rx7`l_Niljv#4uHDwRU+jG~MG#I># z>HKeD#^5gXM<#v>T`K6WaMV5^#4Z+P-0H`mY=y#zL-#FOaAx-)6}N>H-)r@-g1cY9`dNk9!WQ2I)3Mv$7q@P^BNd1oy-9>QVjY9Nb~ z`rBIL`y7l~kgHVGgKVF=$(Ekh;tWZpy(Bsgi@+Rq=ZaX_(3;FUZLpTS(#6UMc*1Ms z@M~h#-b1i82R;hI`@KL3-~qzmn7Z6D=GGK`ipt>cy|)tNNnQ^8WGe?qta^7YXb?>< zS{g9FE!;v^{IXPtW>-=u0%lU0eT>03V^9`Z_!vC$!wCMKK?2gP8#8Rjc08dyNVkgn z&@pQBdDq}i^zY`c>Kr*bx5$iBM;X?L$T#8uuUh4*X;0Co0%{U8#eyk8T~6I!x`*-Zq8my8;Ln6l&X z$>f>&eKl63jZcSuh&CD^0|gx|yye>TGc&~5JkAab@sdRnTzt55ML!Ms6bE_+yu)k$mODg5Nuski`ay9@-0 z|J?p!oHHQp0ST~EaWQKeZq!zmp46OA_6YEho2gazCAlj-WXaqNDYJ$sZDx zZALpQ5*(y+mLIAg6APm5L*9|1JQRo{skkPYV zy8Z>ARi4J5#r*F25kgESGgwR$558N``()w+ZmZpN_E<%7X7PuJFH|A{NoR9qJF)zY zTBK<-a>xL0qrG0abuBZ2&77Px*DOg?2n_48%GHHK5VzK0OtNCmsANhfjp+sH{K}07N z(j{K3j!NW$+}x>-5=yD=(VjAvh*MmA<}%tG8dBX9Ntt!&_u3dYwShqP zmI(YSER`LsA>q`1T2_yqn0@_@*`PiwHx+qI672oImIL2F44*`nkc$oXnIM+o%hy_`}q9Y#3W zV-6E=pJ#%A+;LGLBLMFa1t0L-0=mCnQiz_{+EfPlt)R}VY$Sfb@D&WlI7~990?1P` z=chi%#5Gx1Mr?0G@ckVj(~!M>DeBX*h~Ix-wAE61XREvH0k;-eI0FF>Rz22gZEK() znZ6YtpyLjlQNT1t7hyvf>Zc#A2cdt|j3OY+m`AMyo|wPQX}_SG8|rk&eDVs|t>7mD za@!VZJC>E+ppjuD;cJsd2>_H3g!#^$_VE6+8XVq-y^2S`tUhB{@i?q!ppVqe!t-a) zWGy116_BWVGXn|PeM}1ZF{pV9?@kdQvoSndC(*-BqpG&W6g1i9+Gv{$wMs+pZSyjP z+T^m6vE<)-wg~)9-6%j(#KnI8eq25ochqTiiMYJ8mdpB3CmQK~LSB^RgjIlLqWr@f zfo}b~{U()yD$4P4>iy?CFF8!(0}MLRNMjQn5&3v_um|5v#9Yah1!v)`Sn)=srRqmwL+udpp;0~)g73=IHRzfEmG+_atx8C9D3(`^{%_`g(MzXyh6#n$hel}rrg zkTtEkV3|q^PckW_rio08t0e8hJY)pJ_IuXBA`o)v{K@1*K;vW}M?Bg^XkbgC%$dVj z1F7u9E0ICsbv%GF2Q?*XZ^2U(%WfU%7gr|yfHSaTm#s>xGVMH8>fw&1WCJJFqI$a~ z>5XVyXZ;b5c!Q{gada)4Aje1TYZ&ZB?n^Q&T^b!a#vB&OQij<|=ORrd%$@bBI-X{| zac#W{u%?CR{zh?Hyy?jXRn!WmPp&Ev@+*p}cW6g#N0KW<9evT40oE<3YKJnz5918? zuxxF6$2p6t1h|SvaO6DZPJpe{8wWd{+j4eWcU|w5g^0Nt)(i9KP^sXo*X#+dYH8Iz zBLO2aPlRZ7^P8*?<9SLT1T`>@7<{ue@kIeegLA~hUYnDf6&rw+&{ke(2zj26pi5?$ zut84y$_YEvx}k1+B|Np@$OPd5K8v#Sm~vw+d-uvmn4f;0aN?v#nV6F88I-wk7Hr4! z{>ppwF*D<)BX_0YLMkVLJw)%_agfJo@>>4AIXH80%tYJ?ii3+wXFHBNeA4z?6jm9_ zI+I|^#yR1hI#Q#LdxA`A8~S&fY}4kwKRqJG*4Ekwv!3No8#M-T2Dm z3Yq5`KG|Q^nId;@^o%UT4?_Zd8`fVyNAp$08E$+_of5?5Y~?!(*Xh(?)!3~bjpIB8 zR+6+a#u|FdXkHdhzY{Oq%krnMNGREnz@n9t>f2~h22h`xrsQ3M6^*UbbT@lnATV!Z zf$@t^ULI4>U?*`iY653bx3Orn2;yZ^$Yxk-Bms zs)g}p?2C_JHN~qk?@)KN61|UZeI4xx%*(6mR=$tmW^ze|Cx&yCt>}8HS0S;;JiN=% z!{y+7g5THvCDxcDEBnCmH+v>AX*_(fH%<6*&!To@ojwm;CI1;qouQ4+4yfFwG|Yv0I@w5>BB}(@(~ZwO)GXu5>li+ z17^}pRJ3i4wL4C{fpgdWpt?0P_X4?!b_6p8{ur1`3D7=iF`Q@ZcgKDBfxuN;I||v_ zJ$h;mv5WV_i9tywd6*}irq+kb#?37oN|En|mI?>;eYV;#Z@a;1`*0T3jy}p?r>fK$ z%Xgfv(FUhdJ(##%Ae^qYcj>{1iu~KkA&|A84-`&`ZS#xW;rpkT_o)omWo+w+(40f} z*d`VE)E+&${GIMMqM3uth+Fv=rTf*(7T>+2cp7ZA2PBmAPNBNfMFbHtcH^P`yf# z{G@Gmcq6m7Nhk8h5zYYi>L{0yN_;g%eESb6c%JgfV43)HeJ2&4WN7^6(Yh?+auVb( z@G~@K^S}b^q+%2-1PG*oOcVsSfBtuR&x%8)C-s*}|KUqbi&YFA(W*lo+qbP z2V23*-hEl?hB-5{vSK%@)xwMioRLjz-rPY*khsSLgKNLtH1yr$4tuhp+s>Oz|sb#FK+lETdGkloUW4tiIGktk#Ny~4JY_lH2JW-umK)S-jMTo9^oQ< zPaAwX^kv+dlD1LZ3DpabyPb-Ym-zO`@#IBXxP&w=m*tf&wN--7pQ683*TWEfzjSqt zsIvqQB2>u88`@SxlVaSJZEoms+K+>b;?p9Ed&V8v#sosT(AOa-_BPJW7vf;1ay(a3 z@oLa7&zdXNl{E*)yx!?2oflHlGfGG2l z@Hf;1rvAeXYNsA*3uxgS1bI$gkKaqoSCxuv;Xh`}A_W3Lo456?1w_@XzO1Fagi1ZT&XIT?`l>v5<0LKlA>rBv?XawQwSagEFM z^bD(jEUAuW=P`^l@3EToz4k{9g%cRu=L>p!5fx>6oMqrgLt$z=_H`$$n=ujSpD?@; z3x5)rzD{V8n9N@jS~TG*p5jvtR4t$K|M>^Rcr; zey>DZ@18fhoM4F3Rtl@2=q#6M5X@kun{)Pvs7RSsU^tnIADB@v+ADx?-rr}>Cy)^% zas0;jB8+~CBabymgl{qNs+^KwLE#kM4YC;!uvF>62ZnB9!EJ*@AUFK{);yyuTZxpQPp<9AlWRnn3wuYmHDW8vxc+9BhHl4*YCcPc;}^*U@XAa#%qQ~n zxf#V~_&Bwf5D1Lm*^IvWQ6tgek8SS5_;b00FCkMdjr8Ld(|&orbGUo@l{q~^sXxQG_R*vKfe@SxHLx=1C6Y%$Y!2&%Vp%2kL3_XSm~*9 zSIC8wTF9rGEbH8xzqd(SjF0xlLX6RpDstu?mX?&AUx+23FB(BzSIXd6%7MfT@_-Yl zbuKy5u;S%AOf091w}39d z_$dMS&-gD3jH81x*=yoUWK8|>`ej`~mwD}fMaIPRMYfs(eBQ|if-Ox(S@Maan(_rnvirc_D6{{Hs;i< zG2LR+Wlaxm&?i&AA;Y3)LGm;uWn~8SqeY=iNEe<&yj7)E%gRfvlZ72yH&=3MQ3pWU zUzvUZomyK1(TxOk1`T>2)XfH&p=+FG>o$yVW$bddEbJ*_jAc(dK9)zzT+MM;_|#ER z(IT|ZN~x&^t~Xki9BR@*YsTy9x7Jldwf$pkZTX`Gua7y@o)J|E2dni&OKcXdRouwV zPGxuAKww#*-*VoNFI+OB(Q-GH`jJ;~c)21=a59AXcV~OqIFlUg@%n$D2i;N!Yo?=Fdo}(eW z+mxcXOUG@{jqj=|)`Bw=`BUDSP{?8EGj#lQEG3xGk5wbb{uw{vG_IiYURx_;RLKXi z(Hv{otYu@!Fujj4ZI6}1vOqK@t%5LzN8|iphauhTLsB_}E%wfre3%*6ccjJYsj`He zWon%OTQ63R=PDK3mjc<@2t|~*RT(X;zUYh9SNc4mx zfbXp-hdk==qO)baQ~ayATIZ4R_|Nb)&An0M1Pnvf5uOFqn!~l}m8zxLzEh$vZ)Uhq zL{>OA(>r51k@|k+v2i%I@OE!B*8Lv&+MQ_FGiWlY4Di}-GZvJ?ug zbUNyrut4cwg$kn%E1)C~F}y_#Oi<^$Zb~|HC5*rile~e)v~Qct=7YWYwm4oBCG`hr zOVQ)uzk74`*gc0IS|%-y7<6eJ<5LFrKB`LH(H~U*x^gp0Kfc4Pu?*V`=HO_OwnUZb z=-=*3Ww`wK2|L+KlIM-&_ZjXzm6hv>ba|{lyUwRtY2(ZJF^Az|68<^rXGGW#_Pmst zbg#=J?cQ6~Tw^5y%7q-2!)I3}GDOeos#&PGPw^Ai^)J8X9YRscCjvtf1C{VpS#l=~ zk-J{T%9O5;o?P>oEb|BR3~5O;D%`w^{;Kms3Q5<2q9+Po>dh!X%i7HXmPbM=4ovkN@xRzf; zVFK!?)X72&9duea6{%-hJZgl;SPlfU!j{Z3g=-UR;jg098Lmr5-A`r7Im8TRdbw04 z4{rNWZ$p|fpn|G9dB$ni+SAmG;do=&0|Yi%}i8Tb~$}C_1~zJY~>P4(j`BP zL+pGIKNM8Cf(oMrX=t8|5Fn{lXWbT_v|3Qx50TVwH`IO4zdfvNvm3rZ9TmY#wN&7( z6YaE1&c0!K86-djuo`UHy+9`Lu>7%x0 z4)KFlV6bG}ssIb$sy;D#)bUX@^hbF6d!)*p!Da`cc~XX5sWIN&RrJPf6w`fmw(Ig} z^1`SN3h6hofHv}b3QtFV6vEBZ*#3#?dV7zSeT>Y-PHfngVNu(q$Div;RlDEnbTsTp zk&Rz=uU*E2NI!=%_;n3S+D7CC%IH2_S0I9BV4J2w|eVdyYApqr1UX7Ycq;Pt$ zXG_m98wp#u^Uy=*zm&E&b{zN9dpqNfVSj8yS8F+$!tL;uUcL^J&B|{NTC2&I(6u1R zzZPcik}ts56ydRp9OBs)XG?t|9sm3daZ(M5#La!#+dp8>9=#AcYb_IT8 zl(qi3@weDAIg3nqJ)HNi{&^J~e@^i+Y%kRF<0|x<$Y6zl-Y+B3WQ;Vp`AjZv`}9MY zww;XVDEJ>_gHFjXjwU@3tv03_-^=BbFC*s}lq3BVdQQmC;tN2+R=Gc{{EK@R8HB8$ ze3O$Q^ih5j?TF`{zMa;ul}m3F#AG&2{?Srz3Gtd{|ok^F_H_@hDoipZW!S@mgqZa0vd3;H?uW#6jM zy1G`sL^i?h7=KS@vxw=>Zo3gHJp%1k6+KX??o0w6|5tqPQ}@kG0@v*n z5rGX2A`w4&w!f715x-rZE{4v2(HOCyyRT(DL)pwKO@=Th_!uS@`;-$5;gJCC_@jyh zAj$yg^d22>ZGu6w0tu8>KEHFavLzZK5&+8EhfcZyLt3-!P7FG;pAXA&I@$Z2-!Qu1 z9=)L!X`Ju70BHeK9+0^{XWfZ0ns?7B@k(=ankT#$PeC?eIhb##G?9j}1(~bxnXh89 zfK6BUI3@5!hQC9k=82%>4^T0(UzbzmAf@%zynNgs#Vqi8lN9vB-4S}vk!%CF!7xmO&BH-xj zQl(r@+6VRaZS@>H+kIBVO_nhu3JOhV%9Znyw2|imv7Vl8{28cWKDovHzRm1G@k*T< zESz8t4?EWY`?>C{$=r63P_xtb`{X*Bh(P+>F*N@#a|xYLexh^$nC%RNLUBi&%(E^U z9LoEVT--2QeU)@Jag;vnozb!YW*ThVKq}?>!3LIQ>wZ%jmTMO|Kn({d^?vCgNccQl z4baK{5Fr$cqy`4VOK0_ztx_rT+&JQ@E2P*6mK$f-G;jjM=qF-b8zjG|NJ%|&Skhgd z1?pBy1J6*yW6aW`iV~)Ii(6L40gxdAJdvs>qZ7>b@aQz?@u+&o#w>N(kP5pqix(g+ z9eiSjPQyUDr77XrLw3r`;~4{v*j<_R@rz6HvyMMKa<0=Kd}a{i+A^~)d6wA3;lVUk z>aK-bm8+aEdvftgM4-`wNKzKs zkY^x5#@jm%%sGSrTCb#RDphO?4X|fGpbdpW`w|qP1UNGD;3B-+I*jn;b`5wowz}MK z96hO;u$>@3VX7Z0389jpo=}O{+G*GIoRhhxMq@id4#o_il;#V+@^R$Mvy3n}{ruxT zCr}8XqddrlfT4`711q6I)LU*rxk_Ic-mA32D|n=P!GVr|owFT@qRdBYo)vDrab5R< z=UGm~Syt=ep`=FwD{1U{RWK8KG247GhERcwGUA-n$_5!k&oxx0L4}I~e&wSDm7st{ z%LdL<_6Ti=6+{7W@q_UIQIQqnLEu%$;uUVUQaEHHs@|k9^T!xrGrn9CqiE!Zw$=HR$B>XYa7$X5stAUvU#fbj8rjIyz4*LnubON!j zJOJrB64!cnde}^{A}AD=_BN)tMx$H78inv%z({+AwD|*?{5)CLlW26m2uyq$l7V}# zs!o)0kK-t2?o_N-kB3lW9^(&qbWUgDO{he9P2)I>kW2&ln(&6K`>Cjfb8)?@r0`t| zAdMQ*^~#?GJOh0aL?y+LA9@CGnFwWn0L%OnfL29?@ax(T;wsFuj5R3{8uO?Oiyw?#SA{CE@JuG2<0O*jP#qjkk%3YPJ)h_ zjgow*YSLz%=@>nkBLvJ#!GnMV8O|JXR-u0vFmctSC7%zyb5(^KVxuM*?nE0gOI-;! zuJf;CswgTzp7oIBI+3&706_^k2LtxstB>p1gPJf8Sq8g!lt}GP5*jO+&Y@s&aG}G9 zPu)G!Tv<5{O846~@Ta(*bY5p07Cz&Tq>DXsa;{Hf?$EC_khdw&bTl<8f|p&4bni~jV{>JAp|%sF$^M%yrLhK%0!6Yn3jf}rJQ3Yuy7J4&6-Wd{!Uxg-!r~ku z@g+Duhp1>l9t2^-)}L+Aln(sRMISvkPGlYpmWm@gKx|iU9*e?a4Q8PaywZmU1DI-sZOp0S^5r?Kfs&A-qn92C5==xXlMOvJUQfB( z>FcU6<)EA!w%n4&O5LYEyugCa>~kj;RUJtY6Rh5YaO>06;YtwpD>(8(LVcOu*6DjkCfn3axkmRtbZ<4uD z(k(EHvjA7(g;KnPx8`2RzY%}7kVfM%LCwTh^FimWqLXsHIGT6gs}e@!O#3T@e^y_1SS*+yEf3s$+P1r{mmfooIjgE`q7RhSY7HxSiBk?b%h zqk8T8wb0K>X4|_>6B5@-LV9NegDhFmLBM99b6OAuvsHe&o407^0etEBto#fMBeWPH zUM81F^zQ2d?&KAw*g-%l6b5;*8Ps44B1hgtj8lRy^8+Ddgnv8j1CU6^BxXFeqgav5 z5o+fMr=;#}SgJH7J+PQE6#qR$|0GcH)CSWA>f|np;3vY1U@p&zfuT9}A#_@^~pACjZoV@D*CnJ6*P!r9fZJtZCC-J*ybN^T3!qa)SpR7P;P2zJ{6`(tP z`Q?@F;;G01(3Ze^0Q0?aGBEx8{IKBQ0vW-o58K(%T>v~WGvo89E#%w*m`u;Z5IYo# zW6k(5vM6Er|Q=g|vH zZy0N5GrSrzKRVBBnRgS6v~z&KCVH87rxH9iEewP}-WQB3MJFI{@tedPF%VeBoScJ@ z!4fzo_L-q3%BzD#DV!9lSuesEAMdfRJCV{`JA-AKQ}u_WkEW>6z2Cf@D>zpxMt1LH zHMG8J3-A^i43I#TszHkSZQ}M~HCx7#K^JgM$7paN)^pLy0s7(}lqh!l~!# zKbS_=al~)nbfZS0&oAt7HJt7S4GXduKd9us$bHBNaK6&FehT zMgY;_3{imtunfe*_4!ocW-_c_ek64*&0rl3TF+hSpta24wY#e+_u2yip<7rOc*A;9 z3v>)Fb?0#qjg&JPW#182CzZJ8y^!vC^;)0D`{s2|+>{l8gtBhw?y`P%zHN?7v@WS_ zPq2h@&Nd@D-LG2Qh5TGNzX)&A)n_8plL{`z9TRt)M3G4fUb71_cybx)Rfu~H~mo!l%fRw$bC`6nb z`KA%TIUF6cB}BTDGwG?gS^lbF0+or}2zd9Sn@r!*S7IerY zps@Qft`MJcZhp}NDY9uTA(h<-*WDId`3=CU#jY8(Nl0(LT*Fejfe)4o51B-R^tsxs ze~)Kw+0v3Ynh3p;iI7asn zL>7>BDvK~gP7MhOxX`qY!}IVM14`OyG3FLBzXrP62b@JvVDr<(D+31@#21c`)za;r z`8-!-M!TO|$$^WW_UkXd%W-9G&C{>bQ4}?s26k0Q2s$|h1aOvWqlp!zRtmcW-du1~ zn4jbw?nNi)ZxIae1Au@aGf@~omurOW_N4T{AwMEe+6*hBK>Lya8pM;UH3Neat6B#u zY15tO(%*)gB-h!s z+WZ-K-(Cp3d-*%K9CriVby!K`q~Ilmxx+Vrg|wqKn8GzOF{GJ*=Yb%e06^Nq zb|4(ezXKEM_0VED^aQQcV}xQ?IVF>R$Bgv|9f7B0e@(VQ#%BySvJ z8OOGUU*WbN-a3IFS>>>WDkg6owddbN2So64c~m56DRLPf&%7J}%c;Q9#}OeBab(K} zBO@ST2*Z^b3Xn<8x5@J`Tp*tX7zzuS2wBm8T_cke770i#pJ-Ora9it|mSw8*>)wc@ z(IU>=_bJhfy4YUOHm4S+K#0a8S3oj1KyblqJQ1i6MSfrz91pVqmIQWr&JQU^%6BDv z99i#{pD9g+m#xVmD)2@|fP`Hl9Si2hN-g)wajC*-p=mhmv_iw2jw0X<#;M$VbAk~n zfNlj$UNUka&I@gil#GK)dK2(3z|XKZjrtEcP-(267>p^SS*lymZIUy1jcZUdVwlQ8 zzGV%5d&-q&(8dwXD&jACrEJMTxqUk1buUCQfNoTTk?Q~e-O>X#uYV{E#7v%1+SmjD z14qb2VxxnQpbO7xlHD=f^;D1zb~sGpI?g|beZmn(rzsKSzUfIKqLvS5LZGSSbX4{r z?IB9tv27sXi~^Mq@}MD+>8~Q7zGbsIyt!N{O%cPg+&yK>H&)olLe5tr>U|+gm8l2o=qx^vmlA8 z`3;+okS__)CLYje1q!gil@LbYM1FV$mS&nT`a*)y zYZe(tF~l=no|#h;R0s&RD$YBb9zJV6M8$SM1>2#ZhmspC+A4c6%ospVs6*d6EPTZ=pdRe|IM;O zNKYaGU=3Ahe;Znwz2g~Eza%HJOT*zI_05u$2n+!bv5+hKe}T#cw5OHs6PB{^?amAH zIe>S|T5+DjVsiZ!xRtjbCDJOH_bX}QfSU?OJzeJEu+0SVyTLNDKN89!>0g?oh%)?K zrpwoeE(O!}e{5V#Cs78`GO}5bSl|Gock>62!_bfDk-0jw*d)rHr8i1$K~EvtU%)<8XtNCqLZ}3!DtSt^}XR14)vFhI_Xib;PD?B zTYvyajrbP4_Mz`0FrU{nY!XrewuT^0EK<=ymS~PkGNzjBo@f4nW^|lO$ML>yyq(($_ns3P0pmmhX zVPqExf+R2%vD8Otlq5WW+@*V+ZUrF#02$_?CQEVA&u%V)Ho!I50&jblEVBu5ocJvS zL?hS-5mpE^@UjfnGl}I>?gBWhSZ~b;ON*d-Rf(Ov-2kA#k(mU4JoP+nX}I~=oCh5u zs!{OJ@VM+7kO8^$4}iz?=FQ2n#Y;E|W<6w#hPWo;uT9j4s#vTnIJ^{6_@*(_@jkh< z2_sOoHZZ7OS32mFS9{%4kt;25v`H?QJaZ8k= z6N*ELCtdXJ9VD_If#ByjXDX$>PN``O-v~y8P`?Sn$Sef4&D#rzyZ+}yX|@vwl^yO1 z3VshD?`*A|E7xMGnIIdf0yQKisKQ&OZDFR^9k1#+{YatqllXVF*T7PYcBV=@!JD82 z01Z&|-tXaq@nHn+ou(RaAw6UFLEaOg5}jFSK>_>8goB_j;S(6W^dZSaLp_?g% z9oNMU+ggDjadJQ@6C~eXJcJO!_=#(rc`s%%>f0#|>wuaM5;WMGBAeIyXNIODa z;)vp3!sVgn4FENW5gQ2*9m|aVi zS{(<0IYl(P%QC-RBJ@~e7LdH)TnRU$DN(W1VtuXQ$Z&2!r&4UeDiaUPhh&Im=A0usmQ$KfRLS#985G zu>iSHm+kN?R57hcg3%iV5F&pxSC_U7)5<&#(D*c1@da9x)x{aSo0NJ^l#St`+YMc# z+}>vj!GVxGLBl{=20Z-dDz9sN=MyWB8b6y6*Ug5 zGwEl1z}$0CB1C_L<4Q)0A^O5ApbHb4VA(3OK!}FN2NGTal8K(EX2^QZ&-p-*Y${l} zvYU6-y3oz zdU&I@7(P4V$^nhVpoYDwj(*cbovjJXVVz#wqG zcbW?*F(?4mg99KC$XTgx#_T0b9kxgiRqdL{()5<;-dUQIT@V# zXih0G&RJW;|0o%BXBNtxP-NnCzh#qrDZ?IxNo4j1QWgqDC|rsIg?S9g7xg2-NdK(L z9cDp{C{Xg+tp|n9RI*ORmeK6^wZpk|U0Wd5;0K&-t_K+JHb@im1mYCLa=}+u<9LCC z-Ge@tc|P#}?WlGyK?OC0kkt}1yRmD}us-O@avw@=#_h-GQwRGYvPr6C2yFi{N>{^s zAUK(IQI&S>1|3=D-gZL>!MZ}_HwR;ia-ijno4&A$#}qRx&jlKBE$p@y8)!qA{KrxE zjvg|HaP7ERJQFeuI)0j>_L(U=bVn~i9x61^ILL8tyqGBo1Hf&pzH$(G zFjx6%b_=aNsZdz!>s3ML#Tfg##WO?~zkYcJ*8_ zlsA41ibgsD5?O$?_7OWdlEU71c|Mp^1Z03uT3G)ip${dp@D?&Nd_dffDJD=}5QV}) zJ}R^-sZVHCHaF4Ha5|O%)B$fK%u&CX8;>0zf*A$EX4?ZA9Wy?9()_^Io6pSy-3TL1 zHyCp(6=wcM=y%X?d1V7;gTO2V0+vcW(J#*v@g)3zHwH!oZb8kjdDy}Sd7*R|pw31L zB{kYZa%kUL5NX(cVZ)Glhk-FlIIc8F8{Ml3zcwOB3s9>m zm?4qq=wN5MLUvA5J**%P*Zr7 zpbs(vEKV9MCh;e78cTNYu(^yxO!X7u06sC<{cZ(zM9_n8Ye|^E%GCs9*e`*Vc1%!z zOAWWX^c7}kd1NX$OzS6iz_5*%S%09Ix8qZO)0kK;53#=&#?#1Tzi{0E`lfA&`okI@ z6g2&H`a>d4m^2#s8G~f_bP|=&=?{+eA>;IaR1gq?nPPuzKFCR-Ag!Bigfm3b$?zY` zSN$OY5SEr^p_obv=$0=rByJX#o)L>4!9I~dW*Lj={QVSuNN#-SxiHg7hr2!~&T)B= z(U7omT^nO!MjO?@IF~HP>}H%m?@p+XKH(ZA<7$H64P?O@CUr-2(uTPM3LcVPQ-tlE zj(iowzk&oRxTgWd)k_lI8y129bvu!{eR>=-mEcuF!5 z9TZXYU5I7oH3+Hyet$vFE0o8p>QmE4j=;87mMNDr9N zGKrtv^@5(zQm+I)8_?y4cxY^Nd^~8=OstT+OX*|X3zZv8gkqXyqd&F|$ej_TT&zrP ziRw_6BPeAkwa~x-#Q4eH7FFBbcVAJyuZHHc8sQ6kJ{fc$KO{Awm(;>!P*5CMQ3F8a z4_ino13_iYFu*=Ml=N7#o|CpEmW^dWh9syLx-^`$I{|2rs~)C`CpR1zZY2W-yptZs z@m=nc&~HdHoDA~{nAlfrH>_+zK5f)r2;ZkDTpy{(Lsm(rS`=z=fEfhd{Y(OszM`6K0YH?R1lZ)zpl}!y9SoL{ zj|YWB?Fy}71qZ+_O5L^wR+_y|GN*pNrXCL0)-h95H9A9g3`3a;_g?0&MQ>Xbh7G;| zcs?6svL+%9Jt|xDQxXA#_z3bmXe9R~_=;cw-RM^X8LV4^CAGaf?4}`a(WaMW5Oti! zLC+cN3Nf+C66^2Uc?DY@p6_x_{)|%L5YU#jndc>|;h;jt=*a>JGC+d#5D-iST|9@{ zqaQ#!&Eld#P|kpW0ybRB_}xep$C%-mu-F{*z3Cz?WZ2@{YN<`Y3vN4^>;P;+-dcQk z2AfdGJgKmYD3U({F0D?y+XL*M0~QXfUuIWM6y6Czfq<0h>J3Ri7*pLea2~xIgWknM zTU&GJzk~43ss2*6BBZJA`{{y)V_%#%&qzvm!!Qwe=L~_^emj>33L_`7x+~Z=+@`Dm zq?e>UAXf;6Na_YdMxRkkgz*>DESaqenJRJMPiENJKblZx7>EMnlN%7TDdFr1b@Tm#u z{&yKr!!%}$>`S|Wg6$E}zISiVUxc9g$!2{>OM=uK;FItR(7~+@X|LBSQ1FtD*lIoq z0}$<#Nj-0fcjC6~#42`$WV>dBLfHI(;hVfdQ!J(-@w~7v|4fD5DTQI2Z1kweg>0!3 z?}xK*h@Gn6llc0k1CFTG*A8(4=cb-Q_=!+Jk~!9~@=DwHlX{*8-L!N6Pm@f(JApBE zz8k4tVR_=1FJc;7pmnzV46$OD#$W{8s34oYf6 zh1nK#`DJW;xS6(a(nkVX0p6!kOr*4UOYykm1||#+2ubxp5MB1|m=-a1 zz~&4iIH+AO%{z&1^$_gJXp=+aQYD4%tstvW*+kP-wB-DilI=K1;I;*0#l3osF$ZoO z<3aghm>v8~V?2-LdI(GL1RIuh5uU<~s0ENL>70DQQoSuYbS*#(CksUWW1<5^WeO`1 z!Z_SIIbNB|RYyg~>$U&GB&Ncn&MHt*Um^;dalUg7zlOJ=#kP*);>fJ%2Jwt~q{#25 zVTGw+0m>CsQf-7ig%Vi7-hf*|PkmUCB}LZJ;(^~*?z{uZZ^I;=pp3NIo@Y88o@(9! zN?1BFhXR;>aVZ>44>8PbW(++4S%#k9F-kyq7h5SD48t-U1fXziXk~%!AbU;$?CuTk z$Q=^IEsh?}-AR&JpIj<(a)b--t`L?Q<=RLyx_c5GApUytSay%&HtI3ZdZec|1uGD$wJ=J!)&caSucYjU}bSb)=tYpX4f)>=}${Cr=lVgM)_2 z$W~-fH-AR$7uK@SAdANY2(gjfefJ88K^YkT7a*Fi-_<@Fg7@#9C&A-~&^Qs7N~S3a z(e!Mgnr#O$QqeoJhoQ!aqBCbV1a_|C?6CQ@Q})bj{?a%`7(cB%D-Xx=c;d&X2~Sk+ z{Y&UTjOG&nhU|)}oms-}drx#wC!~|2KzxXn^dE30oDrf9%8L={Y^oskzq8blQJAp> zuZt1L$Bh9&^P#eM8^7ZYKOYcLJdzX8 z^j7A!p@lp4#Mcucuy^Q*05k?8PFA_!Jm~ExiXpM&OunmQp_M*nA&V|YVqzf(kp*N4 zZoUFht}F+^?@VprhxHr?(N)@*;|hV#vL5kZV?cL0Bf8UOz51^%ANaXsObd?W2f~DD zlUS4&c1$rX$~q0=c>(DxO$AiKI3@j3)Ab;8__y~H;SHb?T?>E&D+LmmiG3Ct zURE^Zf)joP6tLf6=A6V$*LtcySW=xdCraAoD3(V&j8RA!x*eowqvaFpDOfFZ} z>XMImL=FC0!YNRf)R>kH2e8&{;$;u`gu%s4VMu43rK4GGE1A$$X^PS99+2?^!TnR4 zJd@bAFINq?r`dYSVs{MAB=v1`gsADrsfRY2>eXVIG7&)VliU|NF0wG-E6yH7nNZBf<*k*aELh;AxZH zLdUqgFbs-?)TtH7IdEuc#A_+RFyOb_J94;4M3b0mD@0;&WfF*3PDdXC0{F7v&8LI< z01N+;%v(ws-nXzPfbkSB@C4g{uyBrM2 z0yG=5td~o=tw+Dugr}-P(^sbeI;{2u-A75I@FDw^lM@^wAX)eO%PgqD7^V+o}hp#C_1?cUx8= zc5p&ychC5E?i5eBaoeXeyzwlNnJ-l^$Gz3Wq8$`CFh@fPtsau8yoek+A z$Q3tl%1-(8*mR6-0{#{jvkF=r?{TmN3|~MUfm!1DL=w+3WD5bKX!^AVr5AYgt2jXY zJA0KgY!%1`T-7&yccebL3F4%K_gAPTuyQ28S`s)cHwg7z?%E{-=%SZm$;wp$7-bMD zwWRBbrnsL#Ja%k>U_10q*r4lyO*W~f*zSiy3tvsqS{sa&X*(3c_B*XbL*Nh~6qENs zJ#GMqQY1%6Dgqt{;u)o2BU{c5k#Q^d0eleF!C<~_j31u`VCJ$L0(e`F;}Vgru>!S* z|HhJoS{i$uC9oS5=L@Y+{gi>!oM&vq3Osd^zd>WfeA#chv`!=*^*-i>44k}(`Fg4W`jF6no$8 zANFVf!>`+Th)SuTqyg@dcK<6p6c#C~G`R^Z#FVI1?AS3w8)hjxRMtrWfR%8_C$`Wb zvO&o9h)jx9V0w+%@Pnc;Zej#$uwu$+)U+|Sq6W|aNh2N^W&#d8Bc>hgnb&ULvtd6!N`eMi(UJ|CI&|4s$khZ1yVRJ&>-%lapE{;40o{9+9tl%93E2H6b9E)f>TAX?aUbZZl0~2 zYNFxaA>UE^;rjHK7L_BM!Q1{6VJkJF-C-AME-Wd&1kHJHVJRt3#kvhFehHb^yb~U% z5ok-%DF>*`iaxqj(%1s-SHUH~A_-!VO_os7)=9s1F!mxSNBSqtrdmj}VM!&lHIL2U zA@KluJdb9En6$)f8^BFV;pi`Qkm~YW16@iy3=v3xYtoMcQ8FUHzknN?L%}4bq?d#> z(OCdDn~W0{8XYZGi`r?YLAVbrQmCh-JQZxRH3eLu5@=RpCLIyiLn_zy)$4}ib4k*J zVwhc3X~YU``?)?)Rd*OIM4}qOm;)PIY}9t zg-|0u2GO7?sb<3~#RPSxjw|Ypz1Kb6K^*9v4uSQO{CIL zChnml!%VBI+YlAjxp**!D>33L%P>s(Uv;*~I4>&ZKaTpTAzn>|`i@he;mSK`rt&mo zSCb#N27bgrqHhGYC1^xjYV8w8t-}`X`sqK2z|vBvj5t2EN$!DLnkh914dK~HuhIt3NLhf^Bgouw?s(2}V|2RSB>l%+&1B zA>$LVf(9W8U?^&%JQK`e=(aFXj(qB!wF;3!r*r%zzlneZ?Izzf4egIJ(_zc>EF;MZi>uPV09G2B}3rN+P6}O%w-E z3}7wlLkiO|Bh%k~iimPD3n*jDGMSAUj|hocNC>Fsa1vFP1H-ucE;hU{Ie~w;QH3d^ z!T-c)>1KgJD-no7N`zn!Q&>@jdxHA{BNm{-hSAazki3e31Is7YNr1J{wM>T>F;+m~ z&qqAQQWW#LDr1;tsgSJHGkID4p(R<#hZ1WF(3@|ZglK8(r^{JiFK8dsyHO6M$9Iqr zyG2pmNh2v&Oh$3vxK@rYQ7d|*x=eq}ur0>deSDyt9Vk1ANof|kUR&G4@eCEzV~Ab< zO*2oCrnRb5RF6`7G)rqx=^M&y(R|jhs!=1k>4Pd*j4uNy5@{WOmy59Lb1>?nkhu)x zH3}!m;UCn-l;9%blM(y*J~B~ewV}=$bAVVQ>gh)DMHSfMahHi7p$1t+72S+TkDN=! z0xG)Q5^E-8R7O|X(qq2oT$3y(7ky=&Y9G2%#5R-{P*d$1WwX!64N=uXjn93+Y{IwP zltiQ0JqW%#$%$G+i7E%JbR~BSEEaBgk;6Dt?Vd6+}FTuWVieqFArT>>d}_LD~z z^u={zg)a}l4xx%VrX&?HTwZXhD7E3uc#C?H{6Nl8?F2zMV^gbx3at0E(> zh=ZO%#+^m1%PX_WMnc7%HI}5nQZRNS>0{oEkF4+j2D%XO;Aj=jM@9{Q|Gt2(L0Ak3 z9;kYZDg6KG$<$74*=>2vCj7h9U+ws18M5$bVSHl7U%?a?fGQli5lQ4!P%n@tcYb;@@UV**4gN`XD(Q^gp(!J9<6Hh`^r$7V)=Zb!*OkPs1j#uO)+BEb zS_YEw=`5yHiBg$egQ%cke+$4YQ;JI*55S}VMSDWu6T(rT4Xof(J*;ViYFRkkDVF7|dKyu%u8%9+`zupnl zjgm_r?04b>NQ2Qx#Y+SPs3WGKQPAeJA#LP|U4hmF1>=C-z^Ev``YYgw2)dTxT<|Pq zgk5A_?a?KFOLC&_IqJgv7R-6WNhcjdt5$n*1Fn1+EuJT-0KIJ_1}I+(S^X>30o(*z zD_z=^05HkcLs&1vfPx2@{3H{thd#gARFP_RP$mvHQD^)LcLhb*0+dWs;%l?8us4S* zN^tjag$%p}ga3k8+um;m_Z4Wf!B~T6%+og$NZ|{vZSSm-S|df?1Dl4$F32z6U>{=u z>Y}`}O%#>(u)&)gZIM1ZPzj_!D!cLVTGyx_@MD2Td2SykXSuV1>JU2-BVgFAU}~}Z z!RNslZ>W1yh=>Jw#HH7|{<}iTgL!kM156-|VyUEQgDEnmvJn)x`A1cA4=b5}bICys zac)6;%Z5b6XFZTXlBOrZ?!w7*7+&gV0jn5(Eri0`AbEjQK3c=l(_v0&c7;eK;zkwG z29oD75iyzL6>o+nn8UY>y2(mj3=KE<#q|q0ZJ>Z9VPV2U8;isNDy#sgHKAWY77I&3 zl#SFl`lSIFh>I|~E5BL4UG}s9jE!>E61dgktst5h1GfiX_1;=HV5SuQpOwh;_SyB2 zUo5=-%g@zSX3XZ!j72`h5v!3uvm4hryHQ{sO{v5aWdzy}g9%kOtC@`az#%M4jfohz zI*5A%*szA%0?a(!n60SU>%UoX7ggOZgVrUqyDAyd?=Cx*Bp7T3gu1Vpe00<>AHUEyJiRrwOAJ^HW}T&7jm;;MNpn*i0T@USjI zWFS|Hr7GoCx=7$fT7iR9p}gH3pRt4+>jqSpe*&_~|APl#8xl zS7RKjBJMq5UySB4|m~zF}nx4 zB`9hJ?qe4M^}B5QslYdrav7mOWJ(K*g4}9qXZMndc8H^&m70bK`7Ge|7R(8?z8 zSf3L}8ea;3ieG594NRFx$SMh`xKI#qB7xy0u`JgmjhIs8gP>vXkb*e=$hG1mBR4p? z!W?PshpUYf1QzRH4mBo+M00swz@29SRERWslx#r>&jBk2cwgZRP9$CfXxjchC3!s- z(}qEU^zl$Zyr3|^&V(%>LF4^6uJ#w6kf516ASB+>gf31eHUP{YM!ObC3<#U{LlAvH zLfYsoNLhtx&jp#_NAHeUnX844=dZkhQXZ;~SY^}J+FJ_@(244=RXo%6ccO27cRH4s&v`-brwa5EWUG*V~egBioJLc~{!C#~Ou1X0Ecn(30=5U8-R^ zR;a?EwMNPcQ0f$?{U*nu(QM_*$;zd7lJJOu5|;Z=ghT{GI*~Nrp7xK?`Io13DqDbLe4o69l3WC1n5L) z0RdYjbk-B$!t9uBGc5GwR+BVK2kS2RU!8#2iQ&@{kgdUuGhCJOF3J8y9MX?H9;6RD z6N0I%P8XHIH!Oy(7D>VqRU!#QYel9{ZUsn!pb`)WpI=1){vV-)p>gC`w;RZy-{yIZ z0mU+0Q&05a0)K#()c=VfPxN-^B{nCKj}c_I&&asxC!CzIfW(kJ;`>ZM5yfB|pb$RG zMJTGo{(jlbMW4$AC5#>jX3#Ew4F+^XjkjnoqEL+RHcEhkcVYqo^KvD4_M$J3dFlV$ zxC*FG58p_*34BWj2evI0WNvXEhL??nVpVudRm&xAM zd4Ba1!k6@(20b3}BiK)eKQ-$ItX?3zOz^$oJK5*a-a5XG`wD!6F1%oJ!HVhf+F*XN zWnI#8(sI)1o!J*~9i4PU?24y=3>XfLc;ewbRv2K1As0Tl<-li4Iu>fs01A2{N!oMlV1?gqRz*}rY`&FVQ96(4EKDv?K!TOL0DHI&iM<1rZjR&LcrZjWIi zC!XFUSI)I>5>R%bbzzqLM>1L!Z;eWnp6a-exaYQ`hGof%1A|oaxKaj0V0B-p*D%Fr z0MvUJ+ysK*sQusK0_*5*L?D59zN9fKw!U0R9NeEKS>^qpsM6i!C*V>d`ohxG2vUP% z4h9jqY-V2)2|+^wP6mdu56J{oxi_HMA0e_m)xmL9;YH187+Xy&jfp^mD#ONXzD>V~eo12b3zDBpA;ILPBDZ5a=#r&>l^I6ivi2E>3;Wp|Fx=Hx_fF6Ve>it8osMwit#OP8VbeE16h$<<2@rW~O;G2=Aj<%g zZ2;S%b$!cbuDaxC0FtEq&BjSra^ErTq)dB~iTPaL`lonO$bJqMQ)UY_J>jo!7lJ*# zUU9uaILup<8T?192n;=BV}V7?aiH#s2Kt2kF6iY zEyzm34ivF=dwajZk=LjfP3jO*`X1*kZXq)k;@GSu+m=w=SHFzks+qgpswKwiv$%#z zQM=g|7ZA>(xQ1VzzW&{P@3a8A)NU-i=e)dnItvz6vIKA16|Lpk_R|EkKmxR?^(g#n zo?r#dLxH@9g#A_VMv~qrY>)iF4bw_^>Vg3k9_rzeZaUE*mzy_mrn6dg2gVjfF0g`x zg3J;CZ8MPnaxltV)RF?&cqkh_@T+IL+zF%AY)Hvov~tp`!yFA$I*$~e9j8_2{E`3@ ziYe%jf&xfE@JS#@f7oq1cb4Ph2e+~aMkkk*szu0eF~kiX!B4t<1`I_w;)<1i-Ky(g z1}Kr!l^$Y<7fd|ux(D6wX+iclRt1!0R!N_I>RVjhqb{lUAD{6QNDb@?9iPx^1>giH zjbIB>tJ$aBjNBmo5hn#Ih2AX{jwVJ~5tuPNUS9?fK42y~IqlLyd^LIeLbhWK;&M9e z4~bZ&`f8$ELO3se*a%JFfd}vO3|99v0xNs;3_Bjg2v$Gj3^RWqVsXAeR!u z{PzebAxut`%v${A@h~Olv4~#|qG>{&7T81z?LZ7bGrlYKXFzTmkO0&exzvuj3tHlg zE%{<^pxHIa#gQce2cBJ_yCAX!lspiq85dYAYy9A3TLH$oQRral_j*Hj>`w1GVg;yP z$X4CeP)5i3f$6nMu0dP%4u*s5wglUM6urhRaZ~@%Y(t_%2#A-W72b#cCa;JQA5jKR z<{-;(fRf)T`70~ly%kjSRLSIx@Kj(qF7D=VR+Cc)${qC&1^!UIa;fG(?DR`K;om6% zp!+OD|G7}eZp=VeN=D#@0B`Zy*N?H|q&@?!=pS zVU2sTL%jC6aT8p)ai&%m>E8+T%EDbZ!F@Qu%{amBIKiy=z})!2urH3!k^tw)e`U#g z<;i*F$#!4K2k+#AH}XL1`5C(+s;1E9ZMaPp+@3%(mEVC~`R84E{_Y@_HN zI&5yx$sQkpm8(1cka(A0E7W&=uuUXsn7Vmp zYWEZm5SI{(60YgQ@p$0Wd!IU)Lh~{K0NcLBfmoO?+WbU z1-?QHVuC9}LMqO|ft~*Z*wSqkU1pF#JQ@dlXw&FgHBiuVH1qulrY#YMzFu22wmCYu zkm4(%!M+COxG~W#PITjqT=M8APqcZUZ4_?D2b~tq+`?`1!)TkM4vMkc+E(uZ33;&t zRwhpCL6NvK=U>sNRfj37yC9c}i`2e22P=@^v_;!aMIstbj7F9nMa>RD7SZi$f+wC% zY+pg{0~9avO_Hg>TcW6yI*uK9eBnfrQf`*y1GeSRk5}B;~U3 zM#BZgvC+gjjGb(hqnIu2JIQyb=ebICbH%9!WdK@{>ZuaSkpLASfB+5YB^#N|CKQ74 z!ikwBCR8Dv(OXPzLqw`mIGc~fgthEyE38UlOi4>#e9>Zxg)@2tB3m{bvS9gHjXNf4 zP3ulvrHN9}{Z)droZUYmyo-6-q9L*bWQ$@cDDkPdsF!gzx-a80rYlrIY1$OEmDUzP z+m+W^A-%13BK_Hm8W@^Isw&6J)sK>09=i!+JR8r{fY+Vthk156#_qdC6hqtagsXzD z+=xFfr||^r1UzlPE?jNqp=(A##>YbG{9w==%j4~KBY1r%9W_O~1H6lY_A$3Sk@qw` zcGrBN7MqB@tk4?5F^V|7b82I5^ykMj1$%*y_9Z7s4rg!!!v;(?0u%@sGVTKOYzl+K z2uF6v;GY73LBnAHVniacik!|vES?Y;y5#UWBC%sB(=^gVPX!57PvIj&uz@~+5H)@y za07Axue-fy362uSR@GG~Y59bVe0=nPMUeVa$zo*bsY~UmONuy3ybOrb1H#O$ ze-KC`6zHOM47KDl=4Tz|2ry!7@!yF#IyY6J#%7t9n__#5tB8lJVTa!LUhQ%vs(!)y zj;BxMNzhrFRn%Yg%vGU!%SG`ARFLsdtySG!XO$DMUrun^xm0`X{a7GbzJUwa z>7(@&Oo(J305*XBU2F?*-)b2e)fh?lI~vQK2-&5couKS2p}jxO$Nvra){Fhw2B3rX z*fVMRw)7k$iXqCy^)qKIf5ALLJtMF`5b7Zb}#Mn$%>7h>I?TmMU zstXdB@!%yH@V_kd8p`%Y0Sclrr<>tvIt~RYdf2P^qh-5t$&+U3+ zZ+C~26yrJ67$AUf3kex_R!|~mXD@;OG+rJd8&4F3+Z;GVxPL9wqNPhAmiZ~qENT2o*J5Xiv_56pm{T+nRn(w?0{7X4@&!bDp+*uKzN59m#3X17 zLyal9Ak_$#Ijq*946d%Wk_}=rH!6=2tJHge;MtX_S!16h0urbU<|7FOF@RGzuQ{H8-lC)rpWl#&vT_i2N!xDyt8AQV^`w;3ZGO{hVGmjVSnwn2|a zD`%Zj-EPNxGvZIos4-V@bq@19pA7OA6r3g@WVUTX@uf{@i9s12!U|fNG)2IW`xjg9 zj!Li1i-Kf6IKr5i4~)GU@-)PZ^;BL7t_*722qu%zwr*406l&EYU2pf?g*bcu1YE8b z&I;R}5>>=_O}3eEAb0!l#x(QnmJ49r$7((ew(;&K^U2f>jzZKgu>k}i5`vv_XzySkS?{0G zo8D#E87DIpT2dN~G(H~^`h;zkO&jG_=R8+Q<&HZ+7d$dG$gAUu2b3fw9b<4;?C%G9 zfq;lGgYxLi+qRFqIu0P|`_2N_#}zu=;K&wmlp37oaVmJNI?p6=y5!PT-f&^1ehTwL zmS~-)@sL2EvR?iq))$sJ?gWj!)~*Ck(MTdXI$RjG{{hl(W^s;7H^cfZeCLLei@sx5 zJ@`6M@Av8Aki-tn6)OoJM5*F8Nf(b=1ud$T7fc14fG>{pG(nBX;a1(B@odz%(kOtg z3<(CqZhkr>v87PW=P!^S2KG6K7ic9wH_u1aSYZ#Rj1fdwx*UR%&29*gq7uq3P_(n( z;kQ1R57j1J{2B+v{Zn6{Q=(vppGGqX6q{4f4?&UDTJ$$$b%>JJL#P>XK0G{M$B*#Z zjbsEKBY^-Z>1K9N3;`XB)4t)Awg-)K4n_zF#?2_ed`LlgBS{~KH$HvmUE@+n&&s_H z^v9&oJ?{SVvf#mSknPrc@w=EZ1yccG5K~p<`k?)H!?!yM+xaCm%(A;+M+nZ@nC3WL zioM$I72$Z^vI+s2Z6C+b<`Y=UB!Tptrr7FJr`8C)({v5#5dX- zdM4*<$BE=fAnda%8k?DraaJX?QW*u%C{CgZy`!GZj3m0!gr_785BIHWs4j}xoI^x| zj0%X5!ql>bpww1vTD3o^G>(=6bmV~$HMb({WgiINj0&e5bWfqaZx!GyHR?UQ`RY=_0fgA)iKyGAJ^%HZ#50pqRTVK7rNH+bSo6lIi$w<|s;zNijzLu{7@ zU{x-sfUDqP!_5SuwAWo6%f{Lishax&0eG11h$CCmc@#&78H}8 z5x%g`ikUTo4detoa){3Gp2ZxkVvZYpQhh8|JWOL|n>gtRgw?@kEmz@D*E?nyyIuy| zByt@9!|I}b+TF4b6s&2OO0k4mCXiHOXgd6)K`wU?=>H16%HZbl*nTC=tjC9;!So6V z$RAXn(}T=u_8=CsRII=){9^l&zU`+y(1dV1a1&tAOgRKmGzkhmyq%aPp2KLsU&?TY zd9Vfuq>Q=X5Z(I3y}H74m9fFkK!*TNfdRDe2xNg+5W(;aa6B0JAlXz%47dpAr;(zg z0bGW_PB0=Akw!QGrRd0nphp1^i^u1jHQ=FxgLMoZK;hvwEOpemB9_Sn30~|8Tx4%* zOn?+&Bn~)o!GMNa8B#pJwoO%K^NDA|6|2{heFvyijyk!-3@s|Y%i$ucW5GQOet@Vw zf^d=-LgDeRp|*^s)LatODBW}jfNf8VkQa*JX6Il&F;CGV$HX*5XOPjX_FE{?L#^Ux z!du6JRn-y<>&w^%QS#)}oJ9?Ou6H&cL(K!9CnV8-V?!Dzh!a`NM74`9I%cB@fXa~+ zSouZ-5W?ItDza!09wS~0uP_+ulZ-8gLaKH7u2>P>PWm1_!|G^pKLo?<8s-11Wt#kx z*ICLppz?@L#ygK}cN(kH6p%c_dI$q$)EUqs2fRBNBiz`*eaoFhRB@-c_^DF5c5l#g zB6QeLK%D%MQKzT@M_=-kUO_`?rHL&a6&qzD964X23~3Pog%g{K@oG}w6*Rg978UoT zJq7oGy6eGGwK`3SNr<}cg3f)YP_Il$0{eVUlNmC#(8P|^p1p`eF{BXa@7zT0eIE56 zWXg7lggR*uj4sHlIPiahtHpktFz!kALXkj8*H{x`yd*VwSv4$f2N9tG{hAOStS zmQRy`Lx2M(Ks{v&vUGYzH0j-?gHB|Rb+u21kBi(q4nKLNM>wym=}#c7<}x~_NhbkD zebUSC;HP0B6~>D{|0dgVgiV{Qpvg|TvaIH1$+JwNI!YiE5eiLt41!+K{-K7#PhDj| zs|O+g?J)Yoq0%R#;@w93%7b9zr$`@qfp3k(jE#a{zD0vN{S#DQGmoaF$^SGJ^&b8lqvDX zWc%A@LAD|^ZOMZU5AXui#c!8o)Ab*oP{JCgz~rRd870Nxgau}iXu*Z4CJOR@kV>Sy zvqwa8EI%v3f8yMcz?=^OW&>+^T$MNBfCtTU=VefBlS6tv(c4%5%6$mV`kuoKb z!Zlzvs_8z$M+@~0DnDj8uotk?E%Y2PsB3zJF(hBJaItq_VwBs+~Q}W_60fTGbZW56oyNjaIQVN|;4b<`^;tUTl@9JBMwu z(GsW{lyB5f8KasB`0LDi0=yI*8Y@3;)%Dkzn$0x$OIXeiMQwCJ%9=3dn4J3;yXGiK z2Z(&790=hAD9rLv;#ccA01E5x)m=N;p6BG->z-dC>1?bz2~>M>vIZCfVw76YniH#V z(`odmx!E*`vXUU)c7`VATBluLssrR)sLMWze;;{R6$ZMtFXgT+`fiz=f4vBrG&g4l#Zk;N8H-Ml=fNFBzha;3W}*+lXHwTk3Xn zkeTvUDEFP2np&M1gBO}~wb!zT#NjSu6;|UuuC8Y2E z<7ai*D2~PH(vNi;=3%Xo+X?m|tC%-Oo=}2&@DK?^a%0U>VzVTp&kC(j2G7v&f%F}IH8mN(GM%7O~0_lXH< z8DapT(IC@X^*}IHZJ2d23*q{RhAV;O)wBYXzEPT;9Rr}$tC#|!tqUI)(S8$!f|24Q3oq?!uchMg8Ou~HO{L((EFf)EAs zYbj#|KJ-{8O~2xhRFlZ-panl!z6Ka{l&T5Ch!RQ-18e{dc8c1*3c+!aX(BF*fWSkg z;>@_VGtl7kTcCzB?>k@7PC{fu@WbM(A)`#i8mw$&t6$Y)YSD$cCLTjvk}4)4PR0%$ zZcgZgA@2!acQkyQAPL1u)eIXZ=PGIjUJlS#O7WC>9D!RKBqxU!`!Z_!9&&{F5k; zQhzN&M0#Uf!S=DrdHlkBA`0zT$XO$2^SuO5mNhHqtO7XVAlD__T@F6AKTZn}BYxbg zZXONp@hoyG|AG5WbLFDaIh?Dqib#PGYe0L|Y85oaDv{x+%zJGu(IFEg#R$D%KD|J0QEq_SqV`k* zIS3!5GYVKBdL$cvUir)A8sAg30MyHS!R3iUi`1S;g7JPhZ#r9jmv)0P)Hi~xbRh{0fLphfV5QqBYIcZskaRG}a0S7Q=&auI01DxeW=?c@wmX;9s5s2dXXwfo zlw%Ez4o?OU1|s_L3mz4>>iRSUd+4Ja25OF*ilXQRA|TxtWQum;CxSZT2{#Nalj19e zWGLcZG$cPIgP9xbVZ8+)A3sZoL*aH<-sKVpC@G#m{71$G3A7b`FHC~U{0n7z$=+dU zk`GTG^msIW&LA-I3fx-=>@%v2)5swqLTnO<`%?lBl!91J4qV^~EEf({Hz zEgEU(6eVD$L+WQQAuX|bPpAW=_unUdSTB;_8LH2Y2V`g^zL zzfd>mW8+{Nx0XsdH*6H#w9rX=L=(*Zgf7uQ*E^f9tKB+vD)5hz~02}0cedvvpWLE8!qTTdQOiZPkodzmn0!D4kX>jA>%RN>{4UKzu=I$3|1vF z2vI@Q5H_H2gG4=sC&IS)$c>SPkFt7zT)2>cIP`zMnUkVvF#97}J;&eb&=6TnHFEy; z8GFJx;jY~zo@#&yJTZ!l#7Dwt_Uy)9ZN=QmBWCgD&j5tVjkr+RV!JwfH7dfzM_-|+Y2euUA zoPXJn)#g{fd6hDg#$Bpx$t?my6(g2q1)D1-`ek=ZaPl??kvr`{$?NFD5>+Ns4YBtm z8i#evs{4QDO5T$4lS86IZHX61^Ko!8o}Nj>5{-KuKQgB#@l7|{yXWKbJ7bM%X+^JZ z<+R=>sO**~b-Ni7h)P05R&!eN_$4EzQz6=g=2;wQ0id85Cfik-eXrl; zfrk6fTcMhRl9^;#X62Q;}we2ecA2cY6ST^>)1X2|%3?L<&` z4;qQ972lH22*L(UZWS0C3`f5yz;kn03&2z?nrZZA{Ah%sLJtS#AW%yO2*1GnEqa~+ z0D+zW08mQ<1QY-W2nYb;CjU?X00000000000000a0001RaC9$bZ*FvRFGyuWs~#y7Usp6ID(XEwdQC z^iE0S!L6lRWnn@vR#(FqQ|9V$H?wEBE5@KKFshzqk<>Ju0aD!-o=zQvON)8;8{9T3 ztZMKx9!#bDZRnk<$DL)LYAcM-vXP9(HnDP0`GC=%m;qX&Xvsi#W_$Y7+gWf&Gz zjA2i>ccx57>!S($E%w3LLI~dD&AU^%s!HF0yr$UH#nnkrB`;gy3&HL*yz8nXaG`R_ zsqA53p=b}wgSU`8UXl;}9^7~;(E;7bhET8V&aUD0MmEl!74JV^xL1r}U7Uoyfb|E< zTp*D!U6HZKN^U303o=T%0I3C43=gCulEed?rQIF+5-9pc)-~YpR7qyFR7)4LQ7uD6lq`B$`N= z;Mk*?%VA`n!q20cY`@3gPh)UJ%W$LGhbo}|hSMy92c&K0wup-albAY~*+rWV*DD2F zjh&(4USq}-mu%n__>eZoEGs*uG4ok}%gSSkS!uqV+c+@>UkXForQ;6F8o zecZn$l$(v1>xd315V65+oH?>W#W6f8-1vBHS&jqm5O}EAwG#$RDsi$(mX+czijG(0 zP-N@kYS7gp9-vA|OvBcwK^%bGsJvH2Ay6<&B~WmJxk>B|h23@;f~k4pLg1-V@{yNP zKn%rDl6Ac;0i;ZX#PIv437CXH$@(A@>!>Oei$VwJK4l)S^M65rRxDhPLBv;vFl>*ieIO(hv+w}e^XK&|9 zjJy>ox{8QiAGCHHq8l6Ys|VCE`n43^S@8+vM^I_f0oe4V@OWTGv^Pu+ubUfGOiCjV zK|;rXU0(z}_}<>t+fQ1^@0XQ-ePHhtFqNc1C`FyqqZvO8eZJ73uMtDkpC2-ba00%h zuLGZl@o;Hz8t{FEw|y^2Pg7N67&s-n)7L9sLH3=?48IsZ2H=s~qHVs7=~(MxOmN>K zv&V>9&e>s3UEAD~_iw8kS||2vS4!FK;*DM)iDJH1%C~JKCFjT9hkAwxhTc+7kEgR3 ze&z=)Qh783>emOqV>5k-WFa^4Nt|KuY6yPLQ6LiRU}~>bb&pa{ne%(#vC(<<$dXGU%JJ~Uc~!%c3~_~h+8^28 z%XW*wgPd)%*TC89rIZ$@qSfic$WmZ%R76z-x2B(*sh^2>^kIh60@4=k?-M+WOE|J- z-RIop0KGx|Wd?Ss{o7un;4gZ8i5#J4UXqv?HD#b%X=tgb`fN1zfgmgm+F|RQgIrK$ ze4BfYRLz9aIDLg5QN5kIs^L`Pr7FE!k|Eojs$VKieiS&`8ogC%nd)nv()!x6krcz) zh32d+O&yKpUSV8u!xw2Tx@A}@?M$C|QE3LXl4!5UeMDVM(|S8(65UrGD;nfnpFT?c z+Q;TC{lv(NT3ERC2$0jd>vL0wPrO)FbDQPh607Htr@*-UpCWut;3 zl^HqNRqXyxsNpcz5wR9Y9k^6PXr{ zeAm#}@ROF}TAqU^O$Xw)f~eo|n`&LNG2TwYx0V5xlEV+TG?%KbdCj%) z_E#*gRkxspQf)5Iq=|-z_buVHf(>{t!coFGH074{O=VaM3jq)1aX;jyfz&G7H{M|pj9C*U?{FB(Ag5)PC8jW`gde=g=4 zF7WeZV2L~9*iwZ$k!(2pe zA{8wuN8+`O1}|L22o6_bZZ!{`nI@($2S|jQVkI3zEFH_M{K?&wX@g7;De9tH7iX+ld zcG{GuJE1G67FAoXIVg@8E(hxsp7cmdY1 zY1(GSVEuK|9-S4>j-6VA^st~Ng*d!eN0=}C0kVXMBgR=tY*>Ie8PEsgO3W9CL;@00 zO-BU!qTD9ik?-%4n31^w%*evUWS2@k!<>4 zuA8*10NS0T;UFv`yFG8>sNQe+--xpJZTCBqkwb9F{_%9lL4j#U|L{C#40Tdwe+&>p z8_Kwn47gw*Hk0-jMP!1WHO(Xe58k7WogL zYB>l6-G+3{#DNDHoBbYIVuT>Z1R02w9XSSq7E(Ky$gb$Tba4Tl6A27^U*zXi( zeOpGRydJD?d|`4X*NTWk#sT|h{JZ?RdHmS^k)jw7O`2>s^WzK3<*BDSgsH)fXoQu1Ld(ziC#Z<`lp%VD#>nEG$hiFshYb zsW<>>A3s(K>)gFw08_OtYV)j1Qpln&yYV#~dyqdkIK?MCa!b7>F8CX+rU`zbLmuZ` z(aU%yT}wos1GD9B#0ah)-vqMOum)8Q*QgY$Vd4vBH9OkkCGB7og0Hi6QK-+^u3rJH<>LeT%J4v_T-!B6bzl- zf-U9y4cqUnrwyWzQ!O+5VFfne9?r*RMwOxy3IwnF*X~|6JyMQt4rfdk282?D(?X3# zW<#=;2Na0;n7mkKE5#;7@G7(z=z$iF78|Kgv3L)vhaMW8MS34m6OU(L2saxV8wWE% zA`tJ@LNsO@<;3?kmJ}Htt2ns+KHCu$JWz8Zr!-X`OTrtX@qA@su zz^;Z8%#w6->=md%Owt4Lp26(uk%EnIPa$tm$D<7}?vTLr^rtBn#n(P%hU`gQAE|ac z4G2{h61#UcrkactBf^HpSuth5trU)ff`W$2VvnO)arlHFN@0J8EBr`fnJSO1*==dnuhbs*bARFXy$R^qu_U{SaE;KwL7Uq&MWbh6wYvFgoj-wYCs<8IG5$x zp$q>ao6S7VTi9eZ64pEemTHVmzb*`eD-Tv9u%88AlA#>xmj{pKfo6bCxWodv>mFwU zyv9L5PwO$RA@-d9U*+*2ZQ*I3Srj0d8rOVUm1B-l9uL_YemZYf7L7U+aG0xg zE0fWjaZWOhL4VAeZ>Cp4j}z-pBrPk^T3E1ynYEg>FfryC-l)EedF91u@CtMpj4?lt776T4m@LP`SbP3D4m5GiCo*;i zk~4udY>ECK%;-!h`h)Nzrgm@o+#;AsI0nGCS@iHjtuh$&Dr;hO*U3>2UCQNt@?i|< za=4L4`d3PRQr(Cp=zF3PDmEylBq@X*@-hb`HxbI{H9JIk3AQGiaiQn8#`dAm0LN~q z0Co7Q;)Fh$Q@%fv!1Rron{(M<8@2Jru|8=R4+kdl)zDcJ@f`!6erv)FkEe{?ALsNx z*}euE#fMOWwtf0#9IrFhXTwEd$Yf-fVsR#oN0`}m28x83;zeMIJXagULhUte?4@f5mf7dnPsc=7t%1fm@!hs>y7s?O6W)?mPW zx!|b!TL3&M0FDs@`}x%(Qp<3trKM%#8AN)8rIkAUz9%)Z)|bLQ|Ar2zoOf2g`91NM zlw#U1-9%25jjGJ{;r)=PSTfXlWiTBT+Rz!=NW9088pB>o|3QjHiaIt{o`2WQ5E#n= zAIkF#aCP4a zEUMf^EurpmmLC^SDpI9HW3L%teUa4I$I=eN`U=&aieR77^j(bt|2vyP%jE+m{C$`B zp8n#HED!eKz5kQM?6`jPWMNJi%lDnKP?qogrQc~MMpqt?(EgRUvo4}J-y<5m7r#_h zYj1X{kB6}tzGP+^{j%q%)K;>!^gjrD!_rSD0bU%t$Yd}EaqjWY^RH#7D468i?R+EZ zOPaQ>;+~jBKupJQZf38@fci`S^YX_s5cLo|s$QoVI^aH9sV$U5elkONS}3n31# zqNjVE#0LVN!JiZ7X-c`Q7y|c^O_bWBf%Wojp)o}*BKcER_ z-N~L7W+&u~uWW)T8sbQ5!~2V=ZA{mDes3sR8@o_R-sU*GLi=OsmNfOtME>orZL5$o zRk>G!td6j<;q)?_L1qMlg}!n&7%AZ?a`;f;r@zo<5H+V?V`WCj$_!?{Yk54DCpAO9ph0WL;`w($Od|+= zOoWRXd!V&AOJTTIDDlB!a0o13CdhwaG1*Y#`1c1k=7)#+;?YfQ|F_EF4<3wiP)|n} zSynvz1rD7dxNzO{Xd308m-d|O2`i$EoQ-^5KFuvH1V~y4>h5hdwNZ^ZU(cH*O+>* zN?JXRoXdJ~$0ZObr5Jn#H{ABuj+b*3X9NrQUg!tECAAayd2BMBRfWQKPw&5TGD6# zfV#PwGL`sA{=mA8j`s)pAKq)k!Bb%ob7soWv%lNGDGOmPn z*YS^?2s%>$;yTZy3kR_a2TBIb%Bs5DKL)vfj5ROqt(r`ev}z9n2WnXlZoGhn_Dad$ z!@IY~3b@_Go11&<9sRU<$SM1^c^K~ucEJg({0%Zw&P*15>8?SeGU^>SGDsx&t}@VP zbY0e%keP8RI(v-5(%sTV_-Yjx8c=|~ISnHu7DxgpJNi;0gH6O)wkuM_r?n8JO%fQC zCD6ECmEp`RFjcaFRaWh!*E_Z282+pr{^h{Ah^-&JBV8uSo~T?yY)mKYyi}UC@mnjc zZnA)UGWe4zJt{-UHSc8*^Tk4VN(Py~%-_3J<|6r+p7ORI?Q>@{X{#*eH+tCjqV2-N zWq+aD7+y?(qPaHR_QO|!2Sx}TzoZ;4V{Zcldhn@T8NVJl*b5x_SCJi&7cQ$idUUAqX zyJe>6`n)H;X^q9gf=0#dWlFBlQ-Ik5+3xtSGoLc2atYQxOZ(T%vQoUN;Ymg;1Ld6W z+m|6gCI0iE;uZO=j%%9k2MI*eI2O*PyPP3vH(-A)gcjwdNQ#h9OSeV0zS-3THMUEE zoV_{RrH%{2hm~dEX}!x7{BZkg@x)`UoUMyy1x(} zW%;O^ok$!$&Fh^u8&%Aay$8o5k2VQ1*-4E76+pt`<)iM`Wp7hMEtI*h+V0%eTWlPE z%RBm21`U5wg-+jBiwuDUjMB@&WiQmfkyLE+&pq?OO9^V}E@J-;T+s(~VV0eJ)+>Vc zBYs+*ou!=}8lX1taO$F9ljJ4j8H5&gN-r=4$)d zZ>NsZT)_&Kqk-FYAt?p%R>5FbacgxQrO7|Y15Us-+tP(K zL8QVD-55uv@LcPf-dD(8e2#lBazbh9GF>Y zJ~GfwDy0UpC6_@dvs#&i;rRKVMF6Y zt_2p>etBmvvXLDUu!VbY4QeK{sN!BGh97hpUo*3RKk<6ehDP%UArP&89c{cVq8C*2D`*W8S*FC{dPcZkh~q32Aor8PO*UMZFnuQaq7 z%ZgJD>joA)foBNhA_gY|I{fQT#*TK<@Y;*hR1G!1@-Mwh6KBxb%`WDe*;7d3TX|g! z*!A5^sQX(LBHRT?UlYxhkJMRFiiW6CcI|mIsUu+7@r(2rbZRb5+vxzo>T8WR1ppZmIOLISVETBV{V~He=%u zvaQ=Q#aoS_E6S!sdZ2fDKXd#960}Nx?abBfqpCCOcm(@mcb}ZA9$N*Ly7~4s5ZU$n z$YlkJt@Tb>6@8TcewecOMUx0-?=EzTPPl1&V3U*QU&vG=uy#`^<$Zi8Q|@b6Lo!w)vs$8k`?{vL0-X{}3TFK(*d9eR3kX?s3^DRH}7 z(|dU?!K&(NyeNx~V^+}0;a*R)_*t_QZ+g0dy`%jzlxR8!C6P))|A8+=vCm`-Z+zRSYG~{*Y#%N*r3kZa?I@??@T##vR0n z>m~%fFw|4NA6^eY3I_4J5QHLO3$1v!k6v9uB8B4oag2(f)GEr0eW*v0vmwFBCR_Ch zvES~T7s*3o!PMIP83;_#V7@l<8;2%Q*bO*dCj!1tRO3YCfx@k=FpHhm42!8jVHmDB zqkC@$U1G1CV=&>-x%6_YIgs%P&Z54G7TV=mYxe8!0j0rMj&+tA3sZM)O0#H(N!t(| zgI@XE+!B^(@um=|X)m#K^{g|unV-c^g-Dxq*X+E4tUmDp4yOt+9;kMH%ViYHV3du# zSBZ&eQI4Ciep-5akzD2zreAE9DWcQ-EmyzpSapXXvyK-3QtIZ%W|#Ir;WuzsPw5yp zHP+nWN*;ru>%w-S`|P8Z6by(wBsC^JE!XQ6-T z@Ptu@^s;iYcz-UK(BQ*ss|19Dy5+XF9W)6GK}YY*F6=UY6Q-KNJ2Kiy$7zL_KNjWYRFRgY;Q zY~)2h_(dB%or};0wVPCf9b9Y`sH4%FznmMdZzx%X-Nn&sYs>WtBP`p1Ih9NH6Vw1q zO&(OSJrNr2m{yEqX6wvjZn6q&-d#3I!4xma2EoWYslxJB^tz?4ZvjEUUmAt##R=>` z>swkbS2xz+|I|;tX&Yv1M?&4txou;)LCo*1y70 z%W9~5*vWSNQF^SG+sAX^Jf-`2qmknqM`f0%`(v&C64{35OOofM>_0ZVao|%DQN=Xr zLjTRr7Q01{JGS%h>MeWu%*0P|(h8eBSB*$(zsfaJ$ez-sZwlpmyM>L{`hl&^;{dl= zbSNPOw*e(R#izRi7`yVM4GTqTRYE{}pg~z*-EuOu=e{3g?tl}S|u11n<-MJ zTL)o!dbQ`HE8l7;Z%2oWZr6{nmcM~g*g$HYwD1ry?ET;I)_58trel|m5Vp%ebct?R z(lv*mxov}66g0VDnG3~8cWmfrkxypf_Xs4&;lS*=Pox_wtWT;(YqD!O46FIo&g>@< z#DS_2#W;Ajf*C3-eienBmPGK-BO@f#3VFk(hx_*0sPniQiKHZ~aHO(W{e@tSPeHqq203LdujSD7-9kwYk za^XdQi)1ELS&Up#X-aHIlOvi#SBWMY82>s*#TacJZ${Q75C{a&2Ia7d#0(Aq-Y}4v zmh0FAKShaCenElR6Zez#8a3D?bb{)Pd<(4dywV+f+KOi}WihGeDoijg5mha&_PwKG zSTKe=YJt|+ZLI(t>{KiS>|`?^$2*2^UJ6L(7*yt(sKRSdC9Ok3RNvPxqb^F#7jTtV zLZxkBc|hTZkwwgKSU9CZ-%^kPI7~m`I#r z5_Q1di}gk#7F81$H^tZmMgd`6bRi9s8XRok{MZKx?P4s*Tl(<)!|r8>2OtZQuS)+= z`wJZM4Mp+`&PSS=(U*`EH8T4gTY}5a)~K)T?%~-c(6@H|VZ{K90SSra9NxNvBK43dS?2+*g9F9FKS&|{$nyZK^z=ClF@!%WnAWl%r3f}h^YBo_qLuL3k{e8$kTU@39TDmftY>&b`8rh>c zn{-#RQD^`Um3RRLBjme~pY}%q#mvH~Q>ktf7sV&G?zbgK5_R$shS+bki`e@(&`x(t zIA(ND{9?vtSx%Od6zjceGv;h=4a8`;ENsy2Cn+vN#OERviZT-zCJ7TtmqVMb;n0<_ zjNrU0I922r1wPhwD69ZYGxpYZzbO=RCxN}J{zSeVAsz!dX1*jdm^bB##GsClLP>%h zIoC*}L{Jk>j3T?vMMfmlHVZ^POZ74&vh<$ofM3a)BdCmxrNv&?-yfh5R{PZdq#%=r zAK9skp=Zb~0Ayc=^ni^H$4eyYzBo_4_a?f`f1-Lvjz`v9>eghUMzT0n3?A*JxOoi4 z*Wsa#-uaLTVIZ?tQhztGV(Dau!RnYXkQ6$Tx&53QJf|**W*NZyG1^sWM}YMS zW#M4LdbNy>wq_Elzh@Yq$lK%EObwWC#Fw9NlMosf@?=dk4ORLjbpSa&+obE!ggL(# zcy<=Dlz+(djHIx}KPZ5|Z=T<6@5nb4Cr6%^ zuZ0s;uhW+hc?+Y6+VJbIUl@(T_P>`uH&@18@U0li+da1*{*Wl!mgKDs_w}tK>K|eK z`v)|Alw>~h3NgxZ7dALLa0xOfNz#aNv9WbtyU1c(FX@V;2pb@~bKwPrcnP6r5R^D~ z?_@-#{SHmCxNk)GGS(=pGs8hbyB^%+G&?z21Lc{}q%?<~lwFQx64UJS(Ve{7-EavJ z5@uUVw!Xnr3IK{~3zck=Ql=9bF(-qPW|lXQXRprw2A2_mHyMM~4uM*L_I@&+LKRif zVa{9(hzF*rQTVtaXm!Vxp-%8ONjxf1g2Jd7@s-Q%tS^bT(1m73IkO{Xp|Sy_n6==9 zTlLg=&dg$Mg`#7cg$0GWvJ-4;oL7;O87(CG^d zfOD{+gIQB(^%rGFpXrjr*xIYeumKh4hNwzL$MLzUMfR6cx;4apGl`KVd{a2|rwkD6 z`el+)y+nM@jE~wB;!r?LY&F5W#COBs_ZkDdoSXyFTLLj*EUNQ&V}f}HPi%EkEP@cx zRevps)pC&9#;c(bFHCtH9=mlBuY>u|$ZT)85|;DM)m<-AB~EOK;UMd}AU)``vI*_m zi!whizgA?(xr~8MqBApbE=;p5a(CU&xB8OQ z9sUysj6@hyC>E|s(0w=iie%hM-m{zE_$hO+J~fP{hsFkTO?SgS6N(1MW~0WPE2jH= zZrp1wX4Pr;2=V^lhrki!f1zE-|G6^~@DAUvOA8)wB0rkSc=z_x?|q5&NdpcAYg8Gw z{lsKwHa*N-`Qc7Um8^_=REdD;I&Ss_Yy2Rpxay5WVbqLZt^pBQkcgOxd<36Sea#KCXsj5T1` zVd|-Ak?7YsC8*w^Om4P^41SREPxGgrkBv8Ioe~~gJEDzO+ne-WIbwLf@xcWX(oIJ8 zKNCX?mmPxQM)&tT0v8|U<6(>PPif-i0c^Sc?NSIGdvH?0%_NVmjgrZkP%@rEvJvUw z)wgMcVh74llInbq(X2n%PFCO0qXBcpAV(?Iw+YFb{y&n;ZNH(lp$fu%@k2QR;xI3? z(~?>iKkA(z9cWx5e{>c~fh|to*fp%@npW+!*xaD&*zVUyMUyI*QQ4!^;29#GitV4dIDk+-Vj1;1C_X`#djWQXJ&3rbLUzMcvbQ&~lxh0{ham0L^ zRkEN%kX%?X%Ws>K3yLP*1%>F!G?nQ2=?d~+as$v?#B?MQH&U`7%s)*Cz)7MY1lf;JQ<3os{BeGLsC1-a*id3TP1ytwPc^)Y}AUPrJh*(4Zy|DL^Y{Gc8%)I2NTlx`;eZb z)a*J0ZES#-dvet$2nv?)iD9A#<_j!h4JM~T$S{)g$!Ah*OArY3kDAh}5-=24T>e75 zSwdsc0^eemHz zeU==qD{_PzOtL@|XDTO~rFtb1CwJm<&vk*~Hzu)XBLe`S(kXeVHFKQEltq$e+Qyxv z!_$!HTtx~Azjb$TS)vc4K@2cX*+d$NSZezhL?M>bx0ZfHA9>?{kZ4btEP5mcUzhf| zdPR}(L!E-rBG*T^hI~$##-GBnqEqFQhH#Spb*rZ>k+0&w2k}&`|2iS3IKm26GX6`^ ztXCExb=S9ZA7BiJg_)R2k`yXMmVgGfZCqCjw^s?sRrkvk)`gzf4wC!qk=R|W=KI$p zUrls5SfaJ1fj~6rECGu7rk9*Et8fB^olZFMn0qdJ!nH5>ELZaM1dMBy>mD4~y^n<# z0noAaVndLdM@r6c?ST^NS`(G7#*waaFc{Y?lz0`Dcol-h80764^zv~v6CQGHYw0ma zYB8XSQhmCML>z>Hlmg{>MjeG{()f4^i!B5By9(#7;qw7_0tU8ZD{SFW`@qYQ5(OTd z302bLJ*xSw{J6Hvoig;t;RBKZ+AluHnQt#hzL~NhB@?{j8~0)b3{K>7<~nWsuNkmI*3 zdgYC%M0gLIZh!ZXwPkFfta4;Ejr5oy?$T^j-ae{qTqTMDpL4_sqhf5rAmNw;Bb4^|6T|c>1O-E3L>Q0F(aGC`yHm!_4F!9=U zA3RV|Y%#EjG~qQ;lOm!TxW{mthnV794?JrUxu#pGK2XUwh?1~MRD2b!qkCO}Xt|WH z^!K|ngglss<>AZ8UPD=;^0UOyp_DS8gQ~*!zt@b8l1>3Ng&sLDL^>23=%;_v?kVnE z9s1$r^Xqd0dvVU9gOlsIisU1)Y(`4yikD8E%vuR7p>`73lK8!Ao)1#^ekQMX^4lEJ zygmqmDV3qrs*?#CDLfUSmWPIgpBut6Eb(z*mdFlT!_>Nt)YlkQslGb))Ct5^i_Q({ zfqp5dKgVCDh>M(6bxU0gp5)ndCH#~tSqckIt0tt1NJPni?Fh{*uTUO^8NgHR5n&Qk zwneKsQ+$5p{*l1_k<#^R$-#cXn~nKhmU=2#mqDdBRSkE0;6AXY%wWGRaULFYeiWaG zZ)lSoxxe|Xkq^!57WGQnSXC5x-qyyCBgWT`Ql1MQyg<`?s2RFtvt}NJoc9Y=iGJDB z;==&8hHAL4v523`jYIo;ce?57J6MTS8`m&5EYL$dny6S>Zkz7<;mB%iF!>v{{cJ?=PUoXHSoCb|^23C#rPcw;_T3q3BFN?J36Fd$Tijrmxzl zRsNW?5u9nM6=61@mZCxo#;ozf0`Y9nmka?bnzg@l!CQ8ytcb?VjLD1Gg(W zqX1Pvs=qYhFevs7S~aFhQ4n$%C4%}|m~E7)yM!S9Tk{O+2*kcrU%a2Rx2rQC3#UB( z({|-|FwfKj_0ZL6fwI|eb&`L+d>}+{8Z*h=3fSOvWQr?j#3x2S5l2N zBo;#T^0f)Z5F*S$*ybo$&mS#+TqJ6R*-0?fB2%fzFm7;Vhet)P&59?7ABa4%y z>klc6n$T5v3e{m}84M5`%fW=gvU|T~K6HOiG-qD02}&CxWhR{=U96p9|F*z<2@iB+@xPTvBi~zst$w{^OfpNs%q(8=VeAqt&JKDRf17hkw=7MhZt6QKz#Sv>0pMz zKaCTa$iP6!fdJ}SQ&%hc3q~+0x`G|luYzr5%bH0P1JQ1H>`kq$&4b4KY6q{Qw)5@x zk7cDFKbmgB0u*BHLln1Crmm3;uW4}^ID5H3h*_04#_q-6&P9%=!v-rnIcZu!W5~vY zgt>ZSc3MX~{?2UY=*B-^>m6145TgEWg9ph;YJjV6TU+!*X2Nq(BAr_l$zGK02+LK| z*esjkn-BDV#h=25FRvS`EpXOT@*q1FyS;K2OP_V9s+Jq#(?f~RQYy`y1dio!$dWR& zyF%_&&Yrc&Q2z4NWGINM=SH{|d-B|P9%az*R{-S>b zJls+jPxdPj3xE>=R|%sNhtr%y{#SvIc4n5Vvx(Ukafmz;)Qc&SgSDaain3!3!x|7n z%??Sqb_;qD*o-0t3o~<-R#2B!-&RkjyERgy8|6A|%u0Z)-%7lgj4W>=oeX8C6SWvc z9`?C*tk;itbLM5_`oLf1d*cE=Qt&_j#Rj1M;9r`^hh*nP&=yMArG=W<+;3xP=Czl{ z(6_gI8;vpSKzh1l7RvmK`D@RUJa#O#oEHwDS*@OZbYAoG^T-OTb4_=s7DiY|wd*&5 zFeZ1H7Y(CyNAuTw#_IB%M7EMe%SEd&+DSDn2KB}15g6$aKBw&OYa+eW7#)+5AbP3* zoNp3Tx`X61;f7X2J)V9e(XN5=b%CB*yR$uXV+vsELtfv%&V_}0DT4U;l1jc@Q=3NN zFnB1N0*VYNQnmq#Tlh|CsrwdgR<+(v*Wu$xoY@KM)BxSLKlmw%SL9%RA#Sf4-MkdAWS2suN8RJ7{+~Gw~c4VBXHQ9MO7}R<2|1>I0a?Nn`(6rp-`+@pNGye3jfDK}3a-4;BoKmL^cWBT$XW6vg^E=L8Q8VE}-T`v# zotQ!K6nXJ=g{ETxoS9IY;a4GKzZ!z-~u`w5|{?|zW z^5s->ioOv+D}Y1sUzecLq#WL%5HJ5%-Uw^jV{9=#>o^*DwJwpFg<2;ASfe_X<+ zwdh*lP^r7No3!mQQA->kr%>B@WEV?I(}#K%t9?ZTLqtNopl*sMrE;2b!Qw_48M>j4 zhEr>1dL!_~w&N-Ryk3f@3Ta2sIXrj~=%xq~`!)e6Kbi)`WqpV?B2G%=i4cCH43>ut z@gd05=sz3Dk(~*Kx8?!{hLkVZ*qj1P>Ab$Tp^zT{CMk)9X>O*g=T!lF&w_%#{0j>> zn~J{kBJ#l;pK=+$C=HcPx7J}e`yqj^l#iHB)8u5TsJ5WS;%0!xW|7Xm`Fv*z%i+5D zpYCviQI~5yKi9`d`j`EmBs2SiuwYK`?oqr9%>u?1+$YboY{KA)kjDFYb+i(t_@!Ti zLq(au_`}^F^(dOd7~#5DbdqzQ6f%4X8S(9*&g=Ma%P!ywO=br0K6SnO^UUJiUCwzk z8|Dg)CpfEaQ0VvM?%Wd%eE;6?yQP#>M}-ld5S(N5z2naxHzKEG6i(o0zzW(7l{z$SCs+n?JDxFCMyI?VV>*!0U2VY)}!nmw}wM(Pv1Sw7J|3K|gp%k3>bb9gR z3W<}aFTq8(Qx9kaXv@hk7eCmpH+GWRg1b{=*y=FbQdKkbvHMH*GmUK%7BIjq?>_>v zCGIUU6gYC&t~V>U&|XW7Of8~qVd<;Udq%A`*2$IptdP{=yiUG8+Yp8##dM-Y@$JgM zHc?4PlR__!8LmidoY^x@-&gi<>z!lD=?H6OhTlZitNbPNNkGb{%k(DnH5-}yuKeDN zhJXl0xZvSVL_S5NIdU0!3zm6>OTk?^cA5--yt8$SPrzEZ~R?4tx?!9`5H2Ah7Ky1VCf2<#+>b07T|6`|AM;k#kTqvGE8vLVFlU5hJY(IoX=JKgu99 zcLd%<2Kgs^ih)Z=;Cs8H2I$hmdQqm^;yWWC4HFZVFnVh_zPC358M|`@pI`{MF9JH7 zD(wvk{RQrv3o8%=6GXr-@^P>f%m3MFMg4}y3U>y4E>cPMS}?QWTdfPBp%hN4?V$Be zPv6-zra^lK`UbAg2NM$>d=$DrN<RK%NRVvQf-EfXcN!HlxW3DKq7+1pI8bmF5H+xf*$M>0jUka#zROC5=A8W zb&y2<302k?w3=T3BdU<4@JMM-m!AcloQ#g~BO~;9@<2Th;g$)IbT{(j5Q|U`^-2cEbfzLE_^Z;i!_HB57e) zcZ@2m(VR*+G38qJ^wxVvv~INEQ;+VK=6fJykY?2kOQLfgj!IO<>FNH<%nm*e%q`b! zeARYu&W?c7`l@Hm?K4h!(Z^p;Mv>E4G0G@D7}vNvAJ)Ju4yA=kUcO{KKp9~MK6w;M zESh8#ApZ%w3oJ93x{OL-B1Jl?Y$T5^Mq|U8-k2p6u*bBDRi#*DR{}toXFJ${0I|l2 z(nyn?HZKwtm73YZrb zuTq_dTGT-MCIGwHG!G9SokE=>&Xyf;Sk!CANf$A&bxxsENz;3AoKw@l`5B7s5z``z z0f@tKt~#2m{U~x4fT~O~6)$|f$T8GG#Q+ro2f#Kqyz}fBa?A63W;X7S1{0KVB%Ck; zanH{gh1u>p_ENyAreL?EFD58iaYXI z@!g8K($)h^4_Vx?e4S?&t74VAYA4jYm%pH)3$m40bR?)O7ye-P^b(t~CIgu30D?g= zfNlzDWdTUS{W9yx8^r?Keog>nz(ABefEDT6p!DhRn!~Iy>JGhGSfDQ4*4uf3m0;h% zAcsoTfY=IVQ3x8BH?`Cj{!?56+SBo6{0I=zPAan6h ztrv*i3P#wuoFI~Os2QgWC{`T_W^AviELTPPhcGDrW9K11{rXq*D{}`ub1n8s3?^7d zqIZ>dOCp>{w7^K~MzmShUA|>Q+aE!|WuWHG=$(ww1T#R=8CGaANPdGt0|WaAYPeRw zXZUhnBqq=HPRcA_N)&%MTdIGRvm&g_0ur<}4g`kgAWOY~faTQ5gA~*bPVpvobyu-`cYsp(f?$bqj6kjS5weJY`|W z5*F+N9Ct@^egW6NZxD(E5M6{X1tyk^*ggIOi-jjI;rsq-nZs`BAYAIHpakEbT8-P> z9%h_Q!|LM*Vx-D^I>~~|rcbWu#Cb-UHuG6}b7?11AQUiWz+fl>TF8!n_SeW@mEJyi z8C*k3I=}y;XM?T(kX_DO<(;>;`MFj zfQ!N+H{FZ37p&Ck%h63iw-Cff@>ddJqFtKQ_BVjH68T;dMB^;1jdiyUXsoiM0}GvCt1=i($}ISuiwNwClM z8>-iU>ASbtnW~|3-5c!<3)Rs;cte&#<>1-E#b8VA6BO*n92jZxK^PV{T0JSh6Et~H zGZP-`8WEpSB&p2wr^jP_-pHRsk!BBkw)XHdV^o_6fr? zaygc`&BBqf!@(Rlt0qK8r<(wq#CtEt&@ZkH84AYs2_A3ehPvPZ0OiIsIdGGB*7!UF zxi>RKsK?-l!1Bk-43m`OVAAu4ub3JnNrdoi4I;iWJ-5=~-0sR-w>Oa96bjt$xdKCoT zpnmVr=-@**7QaEo&_Z|;a=r#9RAQlexWe9n@yTjI?cYRWZT)aQ@(1vhwXIsj>e#cf zyH5A<#7oLrsS|^yi!T85+R#`K{ZUAS2q&kl?O7{#PSYy3i%MzEw}W1xP5-1pjcnY0 zD956xVSiXA{2*Q?q5^Vhya+D{R39H(b_BsPx(So0TZzTyNB9#VnL*Prx|{Dqq0_+t zN}pSe#yazlrP}PmU=dVW79s?Nt9KTkq^pHGCdCCDf!`QM+HPy140dct_tu5z9Za$f zpm~dav4jn>C|+>t)J;qf??Ya2ZY+AX6Sb^A#c5dKbWNyrXDg<6OnZTY-xh163T2Q| zaOiSV`t1mgaBkE@(*VTa8zq~9PczcO#6WZtc@n6@?!BummkNWgFo@(x5Uj{h6%-B9 z0V1eZI?{>F^ZA5z?6p(Tt(K;sIb>r@?7zQoS`w>y*OJPu5THRp#l-^f-%!d3SMQrB z?ET_+V0wWZ67EGqyDU)z9*GuXc8@ZU(1=tYSugn+7JZ?gt?9&|!bJlg#G(Wi=;(IOD1B zRGw!3_5-X+?VXrK9v!Uqj&jlRIh$rBJzvO>m7x6%|KGk{)sTA1GM>UKovz{bcE<%< zk;Br&GLgeT&p+s+dBtrHQ5g=1+-E?;-~4BPe*_7W@tJ(Yp(4o^?je1>k!yxqB!6Ln zrVa6N8~l4Wfaad~gD;_oTYNXi`CF;3&7w?+C>}{L znFLl=d^mj^%ugip&dLyMysgbBvRIYo?)U9zv7z<=V7sWt>E32;xQ*Ey+fZOZI+SY- zhYLlT=IRJR{ZZ1X``r- zqH=#^NjW%$={-AN-Q*$SRFUk((PGU}7?G;h(kIZ!fTNi86;H}IfFgzlbA&)NaU)rR z+b}HATGi_Hj`W!Suqb_)3vdv2irl_ic9MGtq6*llG61b~Du@1HYAjL_=>>6hlghwE zjK!f^mkYaPidk?MSl(1y3E`g(vwvXZXLDjahIL(h$*>WU3URoJW?F5nN*u{hatl39 zSTqS|6(6c3VLTPxC&uzeDsDUa%e&(1e;3pB2vSVGrZ`HLQ^%>-pcyLrMLcwqRxQ;y z*Rui>gBr?~fiI|p1A-Yf;dAi+w5={oB)iRE1o3%H24-uLS9q@_?cFm&=Sh(HEHfqE zJqR|b`kzEmB?wBppYhqgciVaj=zJHG_q?~qRyBi1tqd;&89)M)`KfU7YU2=uiw*j% zIvZ37-;w#~tKyu>f1bJN5R%VwIO?@Cr zoum}R6s`eysvTec1)}*Bh9O!d&k{8M*mOXkGVkX3uoTY!01y`;NTHM4_3a@_aSXR;8zxC0aDshk254FzP!=XpR~s%`^>8K`2-nsnz%SLV>~! zrv%F8d@~ULyU1w4+T~b7OypaL4TrCIR#qQ$F-b5>>@SeX`XI|*@uSqz2xozg2@q@G z;*lZCCe{E0Iecz;E8xaSJh7UYMMs6RKsuj|2TF$&G_Qd8+&}oK4~^SvQv@V>>RbY0tRY8r zA|rtAOLjpyr{jn)BN`jOH2xaA64o(N#?`D7lN|6K(0cu(02}Tu#Si4IQl6f^vK7RN z%GyRB0F^>LKi>fA_+Yo7VT4Vm7*28aHIL{(mIY1JJt!t3%<{8qwce621Vl30ttn%w zfWF24^}EJ|K;N}Rd4mtcU^$3|3b~V{fKfpV`SOCc%%s!USH%qw{LD;0heN7@H*G~e zYWJ%gR(AnO^%fsC**_Jw?AfIq$>Z|;UW;U8cM4=Ms5VhA`V>vEAH0p*DgXVnFcl2k zz@9ZWd8vY&KQqCbdbNyUsU5A$hhu_v0fDmXIDK{?c|sIc)NUIV#@CYa;uKk6H3tBS zc|S^cj~fnz0K19#iZiC*$MZ5sb8vq2b72^auI`Bx&`$t|bP+0JxNH?*FjXihL9QFa zu1;qwAgo6Du{-3VK0kP%kQIc;`mfKwpsO9oSbMu#Xte z@mfemDb0FzTA00*)Cq{40|-sB40(fxF7M+dYsRE*DZ7iR2pxr#e==JT5JLklI@TDt z5~8su#G&%K2gq_s<)%qaC*q`O?FOADhNKRqw?947qTCP%QAfgp|6Px8cB(5BcmN0@ zHX&`j&@)NCh6q|dez4r+aI({>B6{}NV=$W*9pMpbFuVM~gyP@SwfcV32h0^D<61=O z&VvMQpEqU07Bs|VhD;|8CwgwFQqsaZt3=~@sSjTmuCnD2x(8$tYd6v<keka;3^7=gKACbFCx>-GM-Re7^2D`K zg{F02Y1 zx(~+kr$|rr0CL1zk)yVcN|3gac#hE&6473B?E`1E{FN-8Q)vnEfPzP-xTc7hO+T>b zuh*YH?RLRh-l}{Zq|L}EN1jQMlrWgu>(GzX2)cRgb~KN4T2H%_IA}Wg%b*2aKT}1y z6_ww;uL~mGK>A>rw|gO}ju7*QT3N)GlE) z2Y+}{FhkaSfeSh+v3wEg66Cl5C?#{}VoM0F#*fIuGQxb&a*)=};v~ph2sD!M{JUIG z-4LlzI)}u_#h_{k(%Jx$qEy_Q)DyAt2okhX0=!gN>+SM}Jho+x}hT z%SOy#mVqdCLo~Fvq)2p8@7^=*(%*STfippZ3qgx2opVJ-Ba942HQ?DPaN{!J(tc-^ zSGVvq$ru4}=DJu+R3e=)mq-Xms^JVgY}C--3#D?^tV*%05}^XMrY^t|xmf1Nl(eUT zf%5CFO~)8QZ>9v)mUD;)ID%`5&2aTZ!(1N#vt+=Lc<@8o*FuBpXGep;AutT326PxT zcZNCGfD_!>Otpgp{bpZU@0x)ozHYoPt5zwTpE9SGvYtqeoae~Ioz%g!caSa}Zy@br%R*4wQHu_~|HdxPQvPQespTe^~r=@0QjLiTt6iKn2j=(du z(`X9*ZDin(fZ4kGgJ^r#Chx*NC@^OSA_NHV%XSnTTe?xLg;T}fDX7Y|+as@VYL2#~ z*i*WYax@1)A?iiAx(Eepzao=W%jvdh8NK$$`1GvSF z$@2ugd$b-iBj}tBQsWtY1a$vHjqin?jF`TG%*QN-D3baIfF?@+HJ==nGJ%}1m_cs@ z0mBd^Qf*Hshj zf23e_&f9&C#g+_rp!O_;td)d3pd=~8o1iF_bHd|*c$`U}#Bc-0l7btOws%rJvuPXE zi0V{}H@*Vdx8Y>Nvzg&>D?u3{;xivkS`wp&40Ik4#0pP3&{bbJGTb@7myUc^T-KE z-dFcV8uO~yD!K=dK^U}P-Gw^k-IK$Zb;84J9VTb#G0QA)kw$sbQ~FHVCd}Ywckr#Bhf8lMW&EjgCy|*j zsKgEWPjIw%m zN5baxj}7e#8)nj4WRkC{8fN~K`Kzx}IPr_K_@F;a3OrnBhPY@E;+FWM1OyJkHuS%T zu0rHK*1Hu^0|Y3-*!ijlrR|ZKRe!aS{>*z6lpHNg!e_lJ86M~HSra9LKoM-50E650QxUW1r=jK z_Ng6NVv6`xI+dApowhow5$qI;1x*E?ynTV3fA!`+eMg&;oIhXSr!Es>#-k@p;` za!IQAGNZl>B-}E868l#n#Y$vXQ=W$T2C$75)xc2e*=R4IvWOqhD`0@+F_8PQ<$O>I zSLO)tQiaUHs;N}F%Fj=cu`3_S5vhJF|72XyQOIBQRW&Q}4_lJ)jJ3**`YcIQoE?t(an6&PBn$vnL#y2>P4rvPKukN~~P<@D)zgD-nxf^;xqHL{eGk zI4)(|E#T-V;Ei`p$CkH^`F%rD;mE41w_fd&nD_=fXBdFS@B!e#iva!kc87Wj3EEaI zP#wC4QBUQ4#74H(67)$py#7{8js8j_R5~dKWXMhNX9RTXloR2*n~OH1W7}EG%)N3Z zfy_O?AuD|LID_D6l#fnDUXJWCxC6C;A@2RG`|>lwT~q?S=#^V2|A>^b@H<`T{lS{9 zu2!gWVO%+hKk@qyRmQoZF!+oO6#|N^ZP6f(L_o8jE?n=D;w5s79?T1eLV~cxNG(jk^t_x+LO%_%%w;26pzeB-5x=|J8y`1?V)fMSWez=2;?8 z%$Y2$UZ;hHRpH5{*wB-3(C~(614jG6gQ;9Yf@T$8!jE;d??d*5;n<$oBAXhf z4*<(}Q>IcOUJFbJKpR#%V}FHFgInM3n2Gs;|M9uCiCH0hb#IrRRK|MIBu~9qGlc94 z-Srao3BhP+ni7-*or&{`lw=W9JXVbxw50|tOB<)j*x~Bf-oLtJs1E1hs>BobQz$cm z3P-HKybzO=86e10_^9y9SRxd?aUF|~MyHTFPk?-W^HIwTCY#6whDk++B2VI@2AT2; z8?y-qM}iYRh=55el2qH598)wm__r~Nh@FefOZ!_CtVw{b6jdl1j3_S(vZTWe4#RsF zSaY~gK!H7_h}V{52#J)=0KvWft_}IoeBf$09cP})KO|O3C(1a7@L{%a z1S&V*SpZ>~ggvu(%2xE$Dq(!d22e#IJl66P_UF{aa$Ht|GiS0w!NaKGM`~4wQW#2d z1sX!laG3x=V*So%FG$!q0E~05+XsjBv6{fL-TQ9u_VkVw{+Xht2~){@1vj+N(LWtJ z|DEmjcmX)?o$d5_d=rvi6BB=R850HAGWMJIY%ES{D+E97W%3mqbw^2n5z%Rz69E&x<;^nVV?oyAj}-~Lp$U4@ z$@b?!_Bv?Nvfs+JlfXOvb{t0&PV**YII{qk!-G(*5Wk#XlKFDXqDJ5hW7#lzd}!|x zVs)mQ?0(($xB6Ux7{JVJ=%a`W@npV($!kG)i>y$f5l_?kBuNq71K1?uS9D>~ zIm{Zc&{%I%PvPqlKLSKU#35EYY*X-1vb$?LpevswjdIQz{kU)p_yiXZphUsgY#t@l z$)U%177V!!_r8iG@WHGEi1>~JJG6l-*szKRu%aojCU-nczz(Vt;U4mV|`hAiWByfNT&(v`t$Ep;~4+Z-8_k>4gyTg|I8Fmm_V)M|+`A z@RVa8U;@7^upcM7Nyh1@5rf5S8wf*gBYuNB_`shcX%=8IBQ|FNoKy;t^|%(sTC-!= zs{s$!WXUk$Q-d(z)CitBivbSfSCGQbNXQWP9(-(P4xh1)K=@f2%aDWc??U=b*#(3E z{1LVu*5&`PK$$jz_LpS8sIzF52VhcubGa@$$U^OkTcdV^(69kNw@P{C z;zy=3q7D0}x!VQGfP}$-AxO1mN)hL_iE&8s$qT%fKr0qROorquUSL${{3Y?s3)j(U zsg+FVPE5l^wJ`!g%fL41{LAVTeaMZZ{{cvr9>Ssf2F0aJ(Xy`zb3}r!}VZuzV z6J6j3vm_qsYO|38Abh_EQ;%V>p@2Q)fFEEMvAlo+LeHpbk|CQz03HMmMl1mLXMCdy z8H|Es9n7Qby0Z-4Ve#U;7%4Hk+Y!CoQT`Gp=H8CE6nJp!#(BqTBjYFC^~f8@X47JB z5d1YpcEAO@TF?vc5&!~?3ITGe3RBx(ZDIx?zfx`@q zQ%K&;&QF05q?ADchTyL)0Uz^FS(rTIdwv%8M7Req52b}L!L=VnB$HQz#1zPUz9Sn) zJlZsNfPn4*ifd*15rYAnm^X;3W*`FsZBODG8q$*}AQ|m$N2>9j`TAh+Nw!8nk*J<4 zxX#NI5%vflPh-YOz+C^k%X9ifS$s_YJen^T&ZJr~;Q~QfAtp#b*A3T|z z6UEA`=CJTcl z+d;mY4VqMKh8N|0Nu1$^^d}TN)nwBicB4=UEYQUin@lzKe+i)@dQInq&sF!`kzj*$0Xyg;Z8pHW(tW07$F%PsSi^3w7U zY&GsFk`}nNpKC38CPl|QQrn#-9Te!mYam~| zW$abc>3{fa&do3Mgx#+`Qk5)R}%+NrV(o&K6Qi6PukKn<}zKa*^F} z$nH3tfbblgOe;cP9G(<@f_cM-pt|26$2qKBsqNoV>(XY$VrqF=Lrh?=K72dN$U?lDZ+r3k)>*vc0`lBy0WK zK(>@=3ki3ep5iN^UH-;i&M=3_@Z8Vch)Y*o9U^Cxkl0L8i&!I73Z#LeL^aX@Mx5B=-eHv3)lnA_3Syx{-($w~!P;sRzS53fb z5jSg41awm+ffxG6Nt4ueL6Z;hN_ zh>u8`-sl2b;G7OL2Og=fj#j9leDn{qMHtv3z-nA+U)jY9;?*HFRec@=AO}!Il3ElF z!AI<+#XC^ib-o5zbXNc@3Sd)iA?)dyZRrchXsKy7HASDUd zF$xaL66yTm+7fF3R!TSy!a9uME-Arjm!=DAqHjDfbk|@zV#q$)a=mqs1L?ceX}a+3 z;f$!ye~P6>}T>ct?} zF@a4H-0d6H1`L?d@>6>eh6rJ6_Oe5T{Vs(`0DHklHoS6haWLDF$aU9Zk6>_8mWiAI z=D2795Jf6L?IyyEGNR@kgArBNWgB*T^WJ0;bFUf;9~GF+>CUnH9}zthomcF6zu02bEb)yP6$>_izx{xup1{YQRoU}v_x!Ai{LOyk3DIhTl<|I-LrhwO9 zLrB`?XZ|>6vu)T&Je|f1K$yyspkaR4HT~M%Sb%YK0jK7_(s5;A>$**ALhTsDsA*NW zX}h02$d#$}Rx&UwOBM z!1)txFDe00I9MVFHc^H8VAV80DJy%;916n-vw5~l)=UEwtdg%FLt~Xr4h4dOD@%Zy z!JkCpKMt|Gx8=%6vJ2$FD{21oV(jCsjxE$LxW{h=VCdBufBzUn7#aNC>f3V~5a2+f zuuBM&<>h;BNfO{OH)z%!Q~PfSL_kzWPdX*6yj2IXK{>AO*L+a8m@|wj_bfp%0<`F4 zzQa;bWL@o3Tur;WO6y1in#}!BLr(tNPb4xdu_%I@D{vm*LVp>;lF-fyg1HbPogu2D ziN)BhArx&A`a5!O<-4IMY{f@&HWNM(fxl_}CugJzYaT$v%%4~-8nKBc7biBVn&0Bp~l2qNR{bjEf^^XvA`#1>q4UWOg`$d8igzh zHav1tQuu6zvdZA^BN;*0Wfu>BMA`jd`->y&m=?KcE4#DFvV*yj)XY__h}lcL8$f?! zHS<>1X$47E#u>)`(AA=lQKgzF6m)22DkgMMQM-guV{r(g#>`@hrBqX_bm`l5h7*Qe zJFYzxRZU`(YF>LFTuDiZ?qA246{lue5-rqm;q{^@z!wBd#J)Up*|}vcU;;+Qh0((= zcjpNlUjpN`6K|I2>y?AMx2UT5B~d1jpv!;*urFws zABlq$$I)e?qe^0;QdrF$D6=fP62{4k5kN$MLFw@X)#7hI1AyMbl;!QL1|t(GASzf8 zY0k)`65}>Fv2KUg=}S+@-d4C7E)?aPJDOjb5MR63S&>^#5zUXdK2$=2Z)~_zK`@N+9@-rk!7X=RE@_YiUjJ^Mo?6) zUwBePC=g0`*rPQ?IQj`PoWd=#1d0&ivLiIFnwiTX%M{9h$}C*eg=s%6?OI@*fM%!D zaE6xx)R|mUzzXBxDLg2IMq%_f(1v2HkC9Z!ICL3|o%|+aQ;XGZse#`DVFY=s3kIX~ zOZc7txrDL^YOLT~*3~DT292 z*svs+;XfgV2x$N$BH(Ei@?nKpWkr~UWrY@p4*nW-4=Gy;6+`-xWS%X3F@y1IaO|jP z-JB*e0+`7Lz7UEuP&aO-C;pF?aCA|*j-H0Vl=UHMR6 z-@BmKh?Ch$)d&D7VG*PE5=#C@5aJwzEQ2hUe+`?k6%^8K`%m=IL3$vs?@-oAM!XzW zMa_wAu}rJPZRo@!0O_@LO_?%s8C5?gyrYA5SLdANT@eI~)dL?GHBl9p#}hPWuTi=0 zH46Nqqp>qx3Pr}7aOKAUP}Xt@K&9wfr`GpT$td8>PXb~iuBlIk%$*wdU6fv)3R>$COo{xBe+U}{sxsL zmQ)w;-ALt|W|#mCQ972xHGr`Mp=H;vP}+LjBPt2mqo!-5#k9ySCAz%B#76JIq$XXr z6iKm_C|aO>*n!AIQr0GB&1z?fy3#^ard{gr&!A@tk+dYz{V)R(JW=0`@Z zdt1>rEiob4EIMdVcG3mVB}O>>z;q`>orAIytm3N4Ylt+7QBoPbXxxk{7IO)Vn)w6< zfk7p;3q&fk7oqwDF-boctP^iNvn40}s0f?y51hwBY)(Ql%Vd$FD0`4n-C))m#HO1j zGJ8SP?EH)Dbg9WPTy(JFpf^2c)VCP9S;z8Th@nNJc`AM;QXC)2ZxASiJ-mg8c}*ja zv3P2seyLW+D!#<3sPvd_$A^b{hz9(XhfujpD2R$j{v*>o01hYFZeig(?v1R(sZtF; zVt^)h^qn`-kpfPL#E^u*Q~}Vl5ay7^ELb8lM$Flt3$@Q;)xY9A_5eUj!bE;45`0@P zqI(}3TLXrRi5DH(mM;IG<9^Y`8FxiSKyV800&s~|=KuuoF969mtI>Z_4j3vUSYQA% z;>>0_7GB4T6L+A$uvV7gY`xPK?_$~S6apFhU8GwS2@;%ZLQuPEal)jzz8aUq96#l z4mVW9CP7?K?TIf5pWb8Q5Jx2Pf^DqSqhj zs3$I_mRQDr4PB6m@~M774#uX#U_gTW88^(+gv|satoUpX+BOFbTs=fH?_i)r z4^uufTOJBqjQ8G?-jea#E?K4d1ynYf7k~k!cok0z+XA$I-6rV5z!q7weJC}-J1vC5 zj3cqc9Y;E#E>DEm_XKtsdynt8pO3&GCeV*V^0t5MmF*{w>iqZOU=+_*Nt+L@1{RVS zm$&%DbyS@}LJcqv8krgdwy5gxB38f~@PP6|>DS%$9BqHmw*KDLdt;9453J05zz1MS z$yrSn7F$2n5k$TBcInlUNq_u{!Z~`Y0n}GPMcqFT zb9FKQ>xZ?Utug|KE>fu%2Pb9J(wr;wQW5`y-`zWx;51|9oR575(sn5d0wH?=+Em;$ zL!M;db-WhNi=1h{T>4D?X*@d+Z8oQgK9{K@@Y1$p9JYDDVyR}iWTl7;vc62Y>2n>apymhk#&#rH z*ONwo;QHmWDTofks0M)Qa?H7rSfy)AY87V~kbvwOx>1iHh=z)S;*M!R!H3EdC}q9a zf+Xi^muBU_L)r+FovRNxZL|>~n>K?=3kY+$MG}O-!$BB16|z9YuuHo{25>heWS@|C zflzgtWnR-1Z1`39>*Hf$P6>Au-a8x>P=y5DyJ704)d}&#Q5~QR&`}IRGV6#caR`z` z@CH5K2S}*seuAd@(O~E-61wvL217;7kYAR?0Hchk1Ysq5dO~Rqq37S0^asfl4%IO9 zq|i)uAo@?7tXXP4-?8vFv)%%50#G5TH`hP41{Df6q2UL(THg*@9@seB!iE)t)?Uwg zDp!y7)S~yrp*E(ahl>s)lW}99dE?cFHU@W$ZVbX1U>4o`#tbC%kyu4b3MLcL1TGcc zF1q+BcHuHh1kr7Rq=^h#fcJCO4}(9V^~2)NI+WwH0H0RAr=DeM+!bU1PC&80@QL#W zJ=}FWX-ZCvI7({;78K;;f=;B&OthZw?c3*J+)8Sm6@)4xIiXhs0BGgJwm8J?b&$Eo zQ(UA__>)@rZ3(bOqUIFWDH#F7aENfA;DZy~PDT!-aM^(YQPkGbT6PR6wuY9Qn{bY7 z8L)PQnLJ_20dp5@7FdsAKFhl!%y(ndkYeS>?0m4A!)Egtrosst44aQB0^!So)KdX;m_-yrP{+#c+z+D&VGKvFgXDbSgc!hGFBk)uY$S zLtA*NfdnB;Dq9nnkWyoI`!sf9Pm>&oF$@q9O$4xc?>3$5m;<6Oqwy0wZS)sBkTS=%sStf`pf-nT3#wFbtiKN3z&gQFv zwOvN`Eo!U`C3l?_XfIm(`#Gni4J#q%qT|0*iLmd1s6MF_zt_l&{2h?eXx>>9IEc9= z-jRjtz8f_D?2eS*aG~;|)!4CpS+5U8{In0cap%o%)7eo&*D$KIiN=e%R_nM{D5=yY9pPlh3<#6cL~n|8z?bGPCfZG`ZAj6E!4 zK`Ep*@(`wjkgpHrz>$5jn4ZdnKRi-68Da3*R&*i;JKM(Wf$eR8U#(3aA|aLlZuAlh zNr^PZTLIt8%p%yi_COQD8)t!;Jq>+B~gSwSZJFN(2P|n zrFlc4MHhusq1It|0|7{*d^Q>=UO+@%diqgm%&#mDDsH?#m*uJ@d*NEf6Uk~jTKlSm(#~DjhDPRmRq=eJV`6cbtuwX7{Wgi!jpt8D-dS9Kg;SMY>2bR zxj*A*L_XVkBW}}8M(J??cT|ea6K;;}H!!#xa$zCq641bjv}m8YzLU-F-S6t$BY7GT z55Pc(QjCa=yL-8IPj0r(FjQ`sTM-6Toe=MkNt)jV(9~RIL^%(9D>nz4Yj>3S(&|cT zp{knE7mU`NhfVRvA+K9KSQ%JGN zo&T!CxhFJe-npF7ChB!vE<6l`I@I}(KDrqK9HDS?wJD|^2Ga;$(vf_TgCnRQwOE>y zVEu%KaEchZFRQ5$DI5bi)}OHX^duuFvEO7?=w~Y&LNX%Z`B} zPLM_w1`dHB_9C3s3Z|^_brRsxN%<)y22vw&Vfh6G88Z%uP>YJn!9ojoJUzb#%0vOW zvg1Py&J>LdB9J0@KXY2K0ztlu$>?z2O^@xv5|O@QDH?{Um?{skfm;E7hsC@T4kx{5@jV&c9&j1m#P^im zC$5d$Pe41w^iR(dR`UKPpc98#giK&_l8KJid|@DOdv?Of_8FatWjnf3n#d)H+bO3}2oZ#=c~OieRALCc?CT*~s^IaCmAwffvcL?&V)W%Zx*a&;WXq?@8=ChJw4R@~S;KgQMaBGX5TQQ*QLa#S-Bt%ydYito2k#fl9Tkr~uMLgXOM-wHRv$<7q7i)F?CDH*{) zByeOc$%F(R;{U60z|#V&l`LdOgDwMLKg>D9chfh0Yfo7YJ(0_U>)p=sf| zvqr#rfFoQZvRYT`z}INd9ojeU?HXQBExR=#-I|Z9%;LKAc$U1*B&$jpt4a>5N)4+@ z2&YN}r%U-y`|D20;ZDihPRY|w$2$qP>3Ss6^GT)QlS%@UOZ_KV z@gB+N9?9Px$=4pq(;mss9?8WP$*UI0F&4=w4#^`9$sY#E6$Z%(2FULQ$myuSk{G@& z6X2ttj>%yh3Cqtw{I_#{lJ@Qfy*%mN9Xp|)Tpk{<9CCdg-5$@0dVh{9yA`4$2Nm7X zScwygr$TPWK+VW_D-`4UxoC!Wt%;D6QWEk5_*9Kh48@E_&5TWDhy+SHH;7>bto5#Y{ElM_jZ4;z_J@On=+dyt1mg19EW=0Zj$&!Coiy15D1Xpp)JG`Gr&Z0 zhscbKXtM(Zw5P~|KeUI>G2o74*WfY8Krz$r%;d!;tx8e&)1h{w6&NdWp?7~cS0PBG zTgbQ~)hlC)_T5NSc#2S&8b(PxA%TVXN)lJeaZbW&Qs$W?ji*{>rz&N&P+vtkY-36S z3{;!zP=(>8);PyB{)z+Y@DyrP80P|X=;hKh$LY{g+pK4<&9AeqoGvtn!ae@v@+zD6 zxG%UJwC5CyOzw_?1B>qgrHlp*M}_+RuuJ{j0qRC>YvO`FWY~10#e=6`@nIpuL`>ko zR7kUkP7$u8jo9+hOOr(E@Ee{XWH!BK(@ThF-Z%Knh&EDq{`kE!W=Ik)4+D~EoRh(+ z#yOV&01Ypto1_{{Nd@DDBRWcvP=Wx5;HB0P_Jn zHr6`nab-o-xG~-Q@cX481Tdyk#{IAV@BlUnAQiYB808FBn1UCWSS7HdVs`gp4V-jA ze}@PVKddBS%MkwsAc3`J0ruo&fX3gi%=sjx!1=^I5MkKFQNFPOsuawVQb+rhDrR2g zv^}l#xG8}uJk3~L6hp%%Aov(bBw0`ltXP8JHBz``!S*L`77PFS5Jtn2Fk_ov`0WHd z!V{`*=BO{qF~^;g0RUa2N|i>tB#YiLD3xLQ)$t|=$GL8px^ps<448tQb-c03iR7;@p~vU+A^r4c@T^`c`Byv~X@MM2Kp z!{COR#1<1e_3kGDhh7y94ih)uIa`qRtE-1e&c#kYhpj%m>`p*~MHp)laa-1rkswc$ z985$CK!a@HoI};+u=haC7n_tM$xEZWN??ETBjL5MJy=aeW4Ew6=aka6j-LhDI_E1= zw^0(*c}f$H1A`2fi6A-@fI>;J`N;sZBN7r0S=nNW22TnS!oif3F^&%TOz>D@j4}Rk z4yv2;D8fYBT(1Gex4Ht(X5Z?O=n*5T5TrlX!g0qMe+miRE;VfA6SzG^NCF$8y>@DH zwmSgXQ3-JZ$C!C^u|XFHYY=zVK!K86v({l;h=BFVl_biDK}_@qj^Yn0Lu>P&AvuBk zb|xA(r@RM`R4254H-gH<1VaGE3aXwQ{Hhes6-Om*a(H?ZT41y1HgIHcwnrvtnZga8 zND_ucN`5+ZVUinsp2?n$1O*C;5YKKTwS{{66$({2wv5ZZ)sc}ks5!(ID1=jsqk$bF$}3_88bqS&dz5$f0ZElj^T!u`JJT!prDmyQL5T37 z5H{R2uyg(k)V**qRaUIs{Ad0o5Tm`WN4>3g9 z3AdQFjt@}gwJA0(oI@y40*S!LWVc0*K!qI)o~-f^U{ z$WjrwX|2_3U&O?xHq>oS6Ta9oFL|{MRoJ&;7May8!sVzhY``q}X{)LcO7mc}yUj2A zLvj5ez|km8UiBnDa1(!-d#i@HDz@x$bBC{1ATo*+1T?w1*2}<&eA4MN5t)a#1E5$XZ6e7SW*=ZoEZB~^i^1cG*3^Le)k|5coH!CtNDCQQh zT<)JN@hnvF%qEelV#I}7?@U{P3T&4J#hlT1L}r|9qZngcY-lm@XuB}1f}rZC97-nM zd%}r%JV^eLepW}P)sO>G3IarfflHMG6^9itmqi7R0QhJ(4ILsM!)}0*O?1y~jqfCa z1~q1yZ5}F)4Gq->GRLN=@^xV?6#)Ih)Y&SXNO1rh0JVBwDWz?$^s&?}C?cVF1E+7a zNMWhNb8PUF-RCu;+~zY2AlWcwQF0(8MF(Q!l@US2{%(zw8}?qxiAdfEs;H&`z3Ai= zCRrFxi8UN_;Grp|Xw<9pmk6T%SWN52JmnrA%j&PM3fB`u(*T@qR;Q6jU5=8{WCvwVDmceaXcfOHz%UOz}a8~5(yA~30qz&T1W!d3;uH6sC~G{9>q z><}(z0n8$aiSUSP^A_0wf^f}iW0u%K1N+rF;y&kaO9DpTojaLqG?nmJD>e)(SZfOC zgSR<_fC{0Wn4=OP!5g;vh>{W$LQl*6egta4JM zcn0x$vl0@@#Ueyt8D9E4Tun>caK7?GWETS=bl{vzVo9{f{N@kj# zGU%DBQ^P29IAZ$Tzhn~eWVEpW%?1%-WMx4T&CWOzm{m*bvgv72Fht1wH++oI$rLqC zoB^V^;zYV!%ES|HTA;Nmv-1@?v{!dAfrpr-1{lE5)y4*mC_abc0$kCZoR%rs)HD&Z z!ovUx9LNW1HWe!htU`8f8Li9&9%mSZc{G9rOOS>#48{P2lZ1(wjcTAy0U9=s**4c$ zOh=<)Nn(~3?YFR0`k{=BNJuv&6A_(oTcE*BVD(_Z+4oWprsh)=CMFV@lL)viv=ZB zuu@4nV8kt7ICvZoW{%o!5JF5dCW0)AbfUKTEHMR*KrY4=>zpuLrH!CxUH6YPf0r&t z2-5rI&3NM<)?VvX-V>`lBN#Fs;vRg*O+>^EG(ZGIb_G#;7#apw`e>?HP>(;ENwUZ& z8~T#_UVvE{WIgJN>nv6Q8CO_UP@Wt+QqPmKh$AXxmD5f_!Y~LB_I{67%s61e^ni&C z{XTZHYh04?U@EamB~DBZZifR=XT|R-h*@>Rqm-sizI}fwqCzV!{*~L5R?dFeNsc0q zyVDlEW6|$>BKyKUN*2r}$ZxZ_QMzeo>}p~TehmW z24Drq=n#ihb-{hH6Uzp*WewQ^`U%1f3PMCN%$UN03ed2o7)^L_NuR(3rojj@=J8jH z4Fj%hL1N$0Qx(|)D0JcL05#spI*2t)M4i+CfegS?;yZcQb;8CR3*I0BvRZlcFvZZ2 zx9jB$CC5{vQ0jxz#;>PyVw}K%$D=GFK@%GC3I-s~5T--4BfWPFI^9!U2_Q%=q5~zp z56BpAtdY|wp-^b+LIxz4e6mVnUlc2}MV< zK~8I%8ZR_umjGIr3sq4zGI_vyQbcs#h8a{uJX$Ssfs8rM4+U1||DVbgCEajsM8Nha zAbtXRIUoW8F@yO`CLAKK9CLS5qfsQLYYd2wD%~JZ&+pKys9vvXH{QfBg8DN2R2NYx zXcvb>oM$lB9TIS-QK&1|yrd~xUHjv4hT6eFB9A`326F2WRAZsLg`l6TLE|>* zX2Y|lA%*hyBTKJ-x1e5&f;bYbcv!aV6^^3Tx-I-PvdXF%c}rs)equI*#lBh#6O+P0 zUTipxvE)!RMe`D+Az)v=!slEPZBkMyeUyn6u7+K@7%(P-0#eSjvrUJn9i9cMsj$g} zV96V8GIs^VN#*%Fd4#&}XJ|9R>rWOKRI82v@a*6#w8>?tSI;`4E()*Fqbzg!s0u&u zuqHaqzIr{DIC0zbGy1eZ&LJKeKzqXSB3BXz8fhF`ZRCiS+68ngdoqcU2F3{$(ja6M zB@!-PFu``)k zwBy>0K(83Y96w z6#mla@H~7z%__NWG=tLuGNqLbF13S(h;T3nqE6iD`V`V&jzB>$a$vh>Jvzv`bf}wi zO==#qzy{N=Uq(|s7O4>iNs*hL&6>SLWm|W4?q{sKK4Q4 z_&n1UNPMZnE|65qt+ouT36>-SQAHpD;?P}UUA$@vc=2&yL=nn(dAg11MH=%^i04Ji zO9ghajZ_@;p1o+Mu4pQdaJ%R>AaXFUAHmFv0`2M)MJV&K#Q0_E#L6D$zPYdND$APZmD8R1>M%GA{A9e$_-QLM8yOVL2TZn z>`h!Giu2(Q9aJ=EO|nOuT#6K&RLJj8k~c(pI7o{^$PtjCg;*>dxqqQJqOt+BWf|({ z3Mkb$R)I$MgV{jaE6%`u|`fK%XV2NsX6lK_~tTcaBv zfWl*ky}(Nw!+~rCAcPJ3&CAw#_3{5$Cn8gmIU}2APC5uzTkXBv4~A*=u)RGViCbbY zFsjg6$bkxxhD%hIj42ZQOHaW?Fw)sH&jMqQlBn2BGNG4JwaU7P541bMuKHH05^+N( zm!m{wMI4fY4V4Mlr8S_gF&;QBOO`+h)RvO^^g94AW?3wUYW%i-5pGrQ*!EwlmP+{IGkVgfs6uThUHJXhZE&rAYym3r1L7NF3bFPz!D1=}Nw~f{ z<%Wze1kW`tmE$Lc0N3S_05vR;!-02Zt80fR7As5A%CBaT3RPt(mTf0ZlwxgDBRVZY z_z5Lc+G9nrZXHWDS>L?;tFos~yjp`DMVjFA{~xQQn`r5!-E%~Rnvsf6N>cLo$jDVX zdq6r|4*;^74PimwgKPNpRNKSlWG~6&oHz|RGC2kKIC6eQR~*=Fu)=a4#wL5Udz^adPAje78)yx+Dv;XNc9oQS?fm)wJ3|eW1p5eS2K5D|oe(NKF8hOtQC78p8A`6TJ$+JOn4;)+x7+@wLP@u+wa z60jmgqr(H7nEYNSmQ5C8M&E^Uo1_vZLlgse>$$_#KB(>07nKxQ(q1|K#&4C-(IR{} zlESsp(+fa`2E>r;b1>FbFPP;b+?5dt6hk55vL@>QD^ZOAPH-ei(7@4(Lr4xNuxA0P zG+OBb^|1{S;# zo2V*LyFK|5P6_k; z8DZ?K}A^AC*8V6S3}=2i2m1?i;>6 z&p5kk770^f!aeL;I|(MdkAmU8&10egkiB~+e!(3_XG&r7wfXDr1^u$CgSYibqJg;@ zWX1icD$k)RW%Hx*?&mhL)Bjju!!o7k*WSwK8u`^2F-h}?rK0!K2~En5n|jXj8iS<- zs5abbqoVhV|C*&EaHYPF>3zf9fwe6J_Q zXeVGw0t&zdkGMl?yySSJf~T4B2mA;?F1Q1R#9cb+@I_eWR`7) zLCb$GczkJRpivC1L`5%{<4taB}GQ>KMs(pU;%0F}rbrw)lCk_-4TCJFQ0|m3QaWsZO`Dws0Oaswq0F;LW2N0gZn}hfC#Mym<629I95~Ts` z>pNH1G2A?T$&)BxoVW?SfUAqNPiTY#RmS+(8dqbWl8_hEx56K8nJ+;zV}2P07l=%f z3pw=PY?_6Dl^h|1rx#s7St;@l#{VKVTr>C6k;(ouDX z5}cPR>B(rn57Bul4r*Vpuqj$b2(Cj?hseuve!QgF=l>&(Zh0V?^4~$Wv+IP!MG;kO z)`VI`#M(Ri)Xm5t6&~0!Z;uHrFeZQoDAA8SVhgx}CI`Rcv+EK)gP)0Vyt*St_F!*= zjp2eu-^ctDLhG7?l%xTTiSXM?;BApmWQw=lvs-gXif{rfGBXFBd!MH<^A4`C3LVXy z19E^!d)Sc_xo~qL9R}#RC(G2)mc66ELhH-x(rCH?K}6`=q33hBaC3~ zB7Vwz!*1F^J0LtPicwdjluxkSLHI$|FIo!#+w`4cC|t}gPpG9iYxGiAHw@EC*wCth zn*!SSRu*c8;5PP8v1d+>@%7W<3y3NW==KL)(_rEyB36`fA^{+ zE1q{rkFXDH+5P)NbAn1EO7!BG@eDT)i7lc~ZXauQxDz-M?bfF|!+XotXJ`AfJt5*% z{Fo*Dd_lyu0RSJ2Huzj$m_bRiVb49BQXkF1Ox^x4B>_D3$VfP6SXoO3snstdyiF1O z6Atz8$(ELq$~has4(SSUq{jVSC-R*|w<+M`v43op7f(B`&~~r`F6Y9Ry*#$JAJcUN z^?wPZk0Y~PglF@#v@^2m)6fxDrMS&)j=R~4en&CY_3<-jy@z@Nxet*k|F&V))l=2u zLov4<+10BL5EgV`DrgI(GpVBuAu;w$+}QL!g{_u04ZPzMVPXX_ramo}((OEgTvg7Qia??m55`pOL7r93O)vNok*+<4f{a4uwyECWbkx@_ zk5Sxaj7>+e#Xaff?9N0=@G0&=0pvj&uO5b~_)KNlt;qp7@%sSGFm4JqO>a`L_D~rG zYT=6ZMm7ZI8Sz+zizS!`p_`i_W;2YOOvov-uDqjFYbdaB2dq@}*Xl0yL^zBo{^SW^ zGE>+J%Sx7r$ku~&(%}>U?DM4ckW!b>aFjyZnTA;PrD#7COyQDi28T<@VI~!l1!5xt z7i1#oDa))~$6q4aYtjrXCy5dW+-~68ccHd@65$d`Ugn-x*;B4QlZzqbSps67XMZ#W z{XST)BtBhGZI1@ZjHrg@+CV=464*>tOWP{kUS|r9)1%RPe{*F>r12#92ppl4LMht- zkgSwYcb38Jhi~IX)gJyqT3JFBNFnG1=+8o%=gnSAtc_!q;WN*~TzeT4?1NWfoX zrw^;PtlwY2GxC-m#6F#w@++$@KLjptild2r%=zA_n>l%foP-(bwx?$d+J*Sh}K6@p`3TUOaJ3;2vqbJE0af)d2 zqWcuE5Hq;=S{*X1UiZm?)i(NW5A?6q>!jjr<}=#S_j@C2ISRt_@TvSaOc?&VZi{&l z0XHdFvs-`PE>x^~w6~?{PWl%yWTkNW68SxkOwO$Rg20)RPa^U}?%mTc+~Vny zZta4Y=aof)1^F3sQOqgvUYM+5S2Lz19caW3^|9W(D=~6l~BlN zqH#uKM{Y#uHgg*%YckIxFXr=6FAL+PGUl@4EvnHTPLbUDw*9#ZgOB96ErUEXn?CSm z;c3yZ_p=v@t0AV8*rOVBU<10P0y`B*l*w^##IlC;DK(}vwA!>z4EQGZY9f$yl ze0)dq)WQ>UbKV4e2we&**ebMxWD2?82;rRP?|MCIKx_>F5Df9XDZCRTW;~XE34_Cz z`t7knsw*?2ru2A+ex&M*`zm1EHY5qy63`VJAYv&|v= z!I@bP%+Gdtocrads|fO^dvvS88M8%TxI>^rmz7n|F9;4Y*b+(kuFEm$*KCd|yeces)fL?<}K1bqEgvRYdM7;0;9!VHBYWJ_`sf)FBPjvl_JN_k#Lb8y8~__@I{yMn3zHB5!gH(P0SuV+ z;4QZ(aiM7|^_eiC@6K36iG9?KL^2B{4_vjNz2hKrq`3|1bi}kl^ct!&E?C~C4$&74 z1J9svh$YM*d1=c^g%n3W!FCyUw%1ZTE56@7X_xWOJU&9^AX9RPMwx)oZ8=YwDv5z{ zbl}m^yOKuo>x5$9ef?RLv+OUofI#}^_5K?MVeJy3iiK~nhZR;0tzVM$6=w|NxGLgI z2sgCgcbYnSP~JEMrqhCSm64tdowIigone#uS~QLU%XNY4F%DPuI>?}wunS`4UgZMx z4z_wy-;NR~I^+h>$}|Z^tqGK;!F#2?xpYZoxQy&sS~j>_-+_&paXo#?uJu4^ zks5gENJTu6H8vBONP^fDcxOzwOmsd=?DFjGWj?pW{!0bDlM=Lzbc~I;T^ThZS z>~v-HT%5(z2>vRXgm~BB(_8MLq@sg9m|Epa*}U3lAly)rLSvE*cbNlNxLh6fj!Exr zw`cY17~4LeqUxo=T`~&U6{Ro$<JAC>I(V$`ZX1Chm zC5Hw4=aWmt_qJly_^?$G`d+VH)Lk)e+uqc@FzydG+0{EyQYct=zfNwW#=U-M``rtM zHkRk$z8>Jzf1z|Nzm-jce7oI6z95~H{uYa?8ORh=D!iV7#SHk{ckU=ujT4{-4_7_F z3Sv$@wO0UL?s|l;Pb_(sVZF3GxCHsH zL_0K`H@76zCDNaP80x9n0m`!4z&Y(~Ur)tKTy$-C^y}j2$P7X?hIsut8wN6MT_!FT zh1<|SL(-)kI9&k#W5B|JYUpX@h6EiR%qO!nI%9!XDNU<7I+a_`V0hA$h(i-ZG$bMA zA^@taW)veCWAq)iSW}5W&p;`0s-be-N-p7LV)$ZSPW5Rm$OP~;AyEpauPI&!^~KXx z#&1|Sk(niU{WUFx#)h;lCq7I@nvCj1rBOw$6&gjghor3h36Bg?6=if+vsIwyHeA1- zu9yrYvC_qd113Nl*0S}l927OoK2DC=jOQ-aBKrHSUn_V1R%3K4>m2=(smT5>d(8Idkb@AGpL++P=e z4Q`lTKZ&JDnl1(hFZCkQav%2;niIfdV?m`6NjySRa znXRSF0Rq&;07MI6VPN3tVMLn`&dqENBlrg1CfZ%#`)3PL#^AeZ%5HQ^)E5xNpU{E{ z0r?dS7v z;I*m7OA7}_K=5j{HeV28#S+kc<$%+FT+rEFaw_CrqRASLGix#Q=@uu!Ih`%7IBEB2oM{ zWgD|v=>kzrm5Jiu!TIL!=FbLX}rZwN~sX- zqw8a7POaUaEQJDjDa7WU^M@$dF+ncK3jAR)>5K6PkwwDj(mTK8b{?^f9jqe`0(L(l)zv5B7LO1<%Zf$;-7$vwV?PceP0`j&bJR<38|V_+7hT^(U$sG{g-0^0nMUTqi7`>EGo=%EOlU z3naHoH1SJ2|L#nsR(hrR?{h%?S0V!|9tvf<&nq(xT}7`YtUpt<1KZK`YsX=+iZnV? zhrf|NKUVx(i?8cn-gE37L-8Z1J&w^Wxx5G_MYP7_PBWjP&zvWvNIgVWU|8Zoq4@Y= zP`H!@ZVqP@%f3;{OHK=gPicWV6xeuJhqH1mf+hqjJ; zT^1`s*$WvOLWy((qv@G6nY(3UmZvhHBAX?>QRXk%JZ8eLpMKM%d~HI2A8BAuB5e*c zy>%zaPJ!SrGKU#c+NcE~a|G(lu~w>8lEMg+uLI>vB=4uuFuTkTyB3r;ORzM8UZPYT z#4Fj_(;KXc=wgBjb7vgTlp*O9ib4uo%CEP^;0yDOvyYQmmHQOf=pQ9#9M5MEGr3en z%R{T#s=m+TcBt1c0d>ZIiIAB8%ihn%?|crSJilR+n$Sd><_JX}y=9F2u)LBJq-Ck| zRi+xuI+g1!54lSKF_^Mq4{u%EiDAjBacjZD0m%N=C|6}5g{Eisb0}ZjMq0MC0Sl!Y zd3;c7css?d5Fgk-Kq7n;7))x4y^AxOgh&7TH9%hQm!APZ@=r)L3Il3=fES7?xm%(3 z&e&DNj@hrucH)e3kK=N8nlq;fV}pqA!ytz?J$cSd>zM4|B?g{wTH%p%=_1 zZ7oA1d8I#NO<$`8o*(}1p+pMYYU{6I8fXr+RvkTJuBNYc>qfH0vhESF8Hw`-C$)0cfXkZ{W6XNlcd1%c~>jAHChr*N+A}PFoI5etqu7f6|zJ<>=?V+ z2*GH-&%YL43|{U#sqMv;)rCIH6A{AmT0W;Gr@l&s|4!b!$Yk%t068@V5ND#qn-7ks zZK_HWtN?~4*!K=DOp)yHAyjL&V zZp|SGht^QOiP_graXmaml@+tQ(O`{}-D||NL$7_v?D&hNI=#V5v#h|4rRv9i1y2JR zP9l1ej`5=JCMOW3ihtMr(J8%VR-L)~<1K<|dXI7IChU5De0;XEKPgj6KY!)4LQRhx zRUa~{$HPAshAhrcA+NsX?BO!5+}zS+#6H@&#AEv1HBh7bK>qY_Dc!#zV@xR$n9En4 z=wzwxq(vy~GQIlp+MrIcHmry1-*DURU(sEcNo{pSjdz|JL9zDE;-dbe)-IzC4LUxC zzu2t$!R=Sb}+1|vGK}vBpeqWUgT~-St7wa6y&{%qVgB1Q zQ{erZ`Z|fQH0Rc#UV;%(s7;H0LOLU45CwObZzsf(aZ{tZahb58_X*2P2$5s94WIL6 zOt9uCN`|}7@8XPMugr4i8|K&-*e|2(IixSyu&lZk6)&35*CVJ4cb#Sp=$+bla8%_> zG@L;LQ)Ej6&6P#-t~PmtwM|C>crPHC!u^m|;+GrOYpNQ068-vGFk=+C10M^b-w)^J z&xLFPN-0O9@?q|lSdvIgcqN_q7zFzI+V9MJNJLnc|CGYuap47mm3D~hA+)-9wr41N zDE{Fd$`4Q!P9*jQS294 zN&yk8*9Xbb9U$?{9F$c~J6Fz1(vTj0QU1xrO)iJS=p8<~EXjbEiYJtcxi2G5ZBIe< z^dhMA6_eXId%yS1Mfc+;1oYL0R~YClZ;22^c3GeL_sTNIr8uiw>|fG-9DOrNslXoh zsQ4&{xu4O5IoJISzHkN&TS@&8 zd~bANBULysn)g}O!B-t`L7Ev+vg^tHfwcD`n17Y$Q=&aW;FZ)1i&VmG0DeO?~D>Ykbq@Ab~ zYu52Yw81yAD0o8gEr{ zLyFk`NmX)2>`UM!^uu?RkV(-HSb@rn)=*?DG|48I=TAN!$bg~i94+iwE56KOcnYuD zi@zGy2Wd>VS>&4eKs=j|hX4pv)QeSnh7vs*V|vI1iEPp!JC$rIx(m;E?|^vce@@JI z(MO++@O%iUkqG^j2x^A%6&pBn|1Am~_Fy=3;G!5a#KE7*wd2NvQMK}jnVQ!WoU+FU zw%LfI-Zc+u7XvUxuH9eQyOItMLuVlVZ7ZgWfuz3%Wcd14$Pp}Ne+;xfENbG|X-&JZ zm}Q><7q>*+B@g~suvY}M8Dd5y*l6oB68?K<sam$GQ zeY{vrLcyr&5`_T&BtRkJOl!VZf!nyDe7W|d{ZZh))+h$-x=ysAhh`fZ2mn}1XOcl& z7luMt7fOriqa-oqz)9W0feNA|N9R_0F6!Dmo#m5eVI*Co{xrOeaObNPU>x6GS?ZpG zPIzaWAA>n;*(24T7!%O_TP&aJ`s@;n7On`UejoBqh2wm;w)m3k33n*6vX3^N-_Q_j z35DMc3kHGZCy(1u+w?b&olOq+4f~HOt1(p~=>omOn5ogZaOv%tWWh&);c127hBrRe zT-F>fY=8PsJLB;w&WT#}HGw5D}i`P3N{;tOSSZwpn zo~ls<#)t1-x)e&vGFHNUN_=NHyy6cw=*J4mogYR~YAQ$B;EjBxEY}8?IWcD z<0iUy|o z$BG{@uq`7FBK}E5;(>A>B%bUX9ikz3LG>`&0Orz^zk6y1o(A`Z+qolYgoqv8hq8Sq zTpr~P=px?&40wgS2k3|x@a8{8hb|mn56d+gF;ES8{@l7}NVd!bHMeWeHW8gX2_+f7 z2+};vcDC|t#zw9fx9j-OSKHlsjHfZT&D)*JV?-)nCOcpRLZK%izf1Hj{K2E>X=-8Q z752@ha3g=g(8GvtP{>nPao8`Vqg_|q=&j~ZoLqBpG_N}H(PE)}L1#Ld%-5PoLcmRt zx9E3RF29V5&-SEx;RW63=#>j#W37!*iVOKdT-$%q_1Z(^2P5JWzNB56(Unq8uz0Ip zY7=j7!4^Gve?ahfUEt(SBnt{Q!MhETT%i_{#^DvW^-WO@Q(|81ZQbc{iUFrv4uPN7 z@Z-U`&C+%u`Piva3xM%qnI|c&7GdTg&o<`Z(DBFQXgvompjrX7<$f}TKCjb0=~JnX zl%KWzC~a*==MfWrKJD^(<5>S_VV2(N-?J&2!^J)ZbSBEBApeIu)B!v5cBwcLdZFg3 z;+T7_xdMW33$?_awMa$NaptOyz z$ia_nNb&0*EN;J#V2-qhh3<2};~_>umeSiK2B-YsZ!wE&yGh4NFpB+GecLFFk`7Mq;P{Pn zTXLL}PNtwQZSPcXbMvbN-;;HAi98*h@2kf~i~Mv!N_B1LJ7|JE@IP0Om^l%~9D@qq zDs!L?#fc#+3Y+ag`&aOWrS>Rm{c;4$!)}fPzNPm^&}ZI- zA^3Qyvat}Xh8O$fdS97+hR-?iae=C?ZMW=`s0rn+FYe(gpv*2K!FrLb#&MNaVo;nc zHun8%P(UlrfMU6Z!vM;Dau{CZzoh-Z#*|0%iD}?oBcfCv`^p?*8g-0*iUfYJjBCJD;5cHRY{kF6mFAK!-k)~_Euizzx7_!U>DcdccP`A(4BL28c_S+bT6HkS-nIXo zg4kES?0_Xv{ZEtbz-PJPuGN{EL2kGvRh5@@1NF(Wua*ke4kSdTbSNVFr;lP|6>Tb1 zlo?BsCa>_Wh1>B0GC#0lCK#3kC}ePft^USmt`avkDJV#$q!L|U;QO*-#_6@PTPnCw zO2&>95TbdIdrOb@5#rW?CmU6}T5C^T3PzNRY%Cij*JvFt5md`X(eaDvJaY)>)z^iR zBKBY4<%;e;CWL@XdTU~C8b1QUmHPFJFY!|=XRb|fm1byYoLb3V6~lA^DKT@|rLh%{ z9zKt|cx9J%`Hx~0B3kLPF)~e-HBt%9w5c>C6tNV`>k)iRSI;bpZb6Gc72eq%W8R-% z=#KGdUHUNMV)!CIUt+z|0rK9?bmS&!euzC-*hN;V;U;pH+QUO#lIj{$wg7d~0F%|u zC_m>L5GeI4Nt;6%nsgcnhx&&O(>){V;__*Z^QS{$w5pxR+i@I3P|`wloHvs+23wOr ziH%UJTarWfgB9>!;4cJeUw#Hv&eKqt=>a{dAz;;Pn(Ysz8D;^Yv(srglqNs%e7u-X zhe`yUz?-O>ggw-T-`|5|&Z2rD&AaQjzJ3%#bgLa%$!80oGuhYy{XB+GD4eGN1f3~# zgu?y(h$oI@_fOxrY6s3X)QzPL$`U&~%5Ff_S18^Kp_YwAh|_A{@>tIhL3{%s9ZIvI|P&tA*Byvkv+;VL97Qm8-t_cPpd`6%h>~FK5B;w6L zmYp^5>V&+!cZ!w30Zu`SmU)(>&TuK=dyB^MZRqZ;1fBj2Eg&u|%rch0MgjJZ$;ioD zgP6Np9BYhjrdvir3WNgbMCx!|X_=>#IP=SY{_ z{2^?70%k=WH&>3GtQk!OhmM4%AHVj45&Gz28vW^ziI87DvvZPs7ZOeDyE_3v*xpMN`jU1rUA?Q zD?d?tk%!{n*y%6zV+o`-8OY|Q1gH7ZP}D)-8SKGaF6Z)}!wPH@Idtz_jJeS!Qant@ zk72c8JLOc#g>WeF# zl@(uvoZV`HIIl4mpO``%A3^mCi3mJJF8axQs0(kl89U|v7U)Eva)o^Azz#+k<+Ni| zfvhUW^KRu==SA;#k5xN>+LbZj_J&vW5qFZ4200YC_E+v_r({j#gy_8}j$gaW`+gQD z+db&qA(>8-E#fy%{s&_G3u?<}2$69ScgZw^0&@7Io>!R`!55BG^;NWCjEiG=oyga_ z`h7pWH%8;X5o9OM>6~Ev*eG)5D1sDRrJcuGi&#M9z0WsHlDC+U@|w@1+&6I{O7+gk zc^ySe#Bp`-xX&Pl`y==uf-iYiOPHDa!lx-}gyd2HJLdl`B(FjCA^P;CR)wr|GW2W} zjv{`nYJX0QoTg%6*c!5m>gNKk5?2I+shERMf00({NdjOzy2)!s>JA8=oIg9-@rBNx zg#cS57enmN9=X*E<-KDI$-BFbc6Mjk>1k&>6vI<`ET4%(nvAvOfGJW^l^9 zQrX@WEJ=)q{(LucI)_&my$WG1>h%S7pypNIL%v#C&j(>R#t?WG>nOw#)ec+#!zzkp@a=edoej)S) zL2!E)@OqOf%vA*rb8y|nuqve6<9+3Z8Nww-0UtzYJtrnlNv`F*tq!@3)x6n1Z=BNn zGJn{~-SQUj3(=L06915c{`IZ`g`c>h&kX+R799blTa|7C{ZIus#mL*^)b zDuK8#Lu+dQM+A@vM@L@~@10FM>}kWQC`WGgnLt6Nv^Im5h51-L1Qq9&BeuN0R8fYD z6g7`tSvYi>a%e*J;%mWcN#9X^%&pZoEX4_%&xCU8o@Y_TTSnNYshupryW09^N0kSK z2B>wk)|GvsBoHFult)ljpr_`LG|d(-eA>YtJ;c~M#^VNWA({FHHINS>*a>a&A{wNW z8-@WG&LMu#ZU~kB1-va$Mw7_I_cy}79fwkc5~f6~GcI97o?c zHD#+1h4W@(r%k!-?)DX!9--=KazL9gUNKDEjKCI-u-GQ1c?RTcp%( zLgY&b;moE`q(Ssy@W@OL>=G3#$_b~7tP))Io2#6++AiEEZCVZ$6XsNLDbq=ear}WU zFENb1OIC}MLU%4~d@CXY6D^dwMfU_mQ45B@q@Q6g-wz2TN4h+;RS1YaFI8g3R?CW4 zK6MHEB>&|oh9!bVnAu)oB-mg3jv8N@k&KT{SkQqVq4^0NU4X~3Y1>&$x{nZJ_o7(A zvJK@&$jBM}lDbHc6!3Q_U2Cj7)AK6I4xwSYzUotD3~)3i;KU6bzKvo#-Ry0Xq?u;P z0ykqZJ&?nd>wdx?rr!$xJ)VPlomdgyI6SdlWtK+!?3zpPKpXpO$yMW>h{sIMScuLa zD|atuTubZFK13-pI;R-Ak>b}FqFln85#B#+7(FE9V=(TxYDiKaPh=G?Z~Ub(^(?wK z8x{IqVrP^)(&b{3e=UG_knS&52$##fTIJW*BgR*0EKV>oedmYXdy)u(T$eNSOxYi1=JqM9?v62T;m0f(frOS#)vE%j87P(s8W5h4Q+67&;vcED&Y>nm%Q_`2%2` zTxBhCUfqe4ubDIWjE2kSGDWjovy`xc z$!cR>L5bwEL2@@@YFzf&W2S!%U!la~c7d;3JG|3O>14InXfIC+)7+S~Zh?HFa?rp7 zx7d9(92fESzm-zTIqN45U0O^*xLl-Y3olFwt*HQBKxrH`rqs{2j#F-5q#`P$kn1EF z;7`oJ!xf%mijsH8E&pJmHAP!~KJ}ivIY&G}dGu&`twKmG9-dIA4Am|OiHjX>7Jt_-BXh6*bX(E9h7Mq1#?Y8RhenG0 z-5*C*kyZ7Q2lQ`zaTyVwsAa{S7GEe{iKxDF)>j|1c9-7iV8IE-tM5(|4buXji~BJ` z2aUty&sa$qx)$g7@7S-t|LC7!p*<6c(TSKQA^Tgqd^{hE{1k} zYVrC4^`m;~K8p8L3P_|yC(boKyGY^7S;ZoXe9F|Xi-lnc1l{TEUrl&Ta$W;1;>)X} z_gbNMED^zp#YnONHApd(neu8ow{;eyv~E@#$CI`_M;sKhGw8yU+!~CQ5u!Z<#E-_3 zAAKcn9S_qju8Roi_vr0YD(}B48pNleoevhs-Ho+LmDxC99TOuo)tAcFfWaM}I)!r^CgRqGvh}QlLckWCl z21!lgVFVjWyG=TLD^BPz`aGIh94QPcoSSZ9CddEpn=j@#qFFA1!#hN?(~a--4kNfA zbAB{~A7X&RZUf6CQ;W5!20`iODh(asiCNx<3{FTmKoK~fgkKm&|KLV-o`Mx@jyrky z15-{$X_dmeByPg2IlJxmQ5U<)#=gRxBUi3M!@rwLF~7L@g-MpPDjFm1of2=e(CbP& ze?b$FZZpej+1s;OBUF)It*wzjE>|h4_;4c|Tgb@0)Z%Pp zv4bBoot`G5Gvt5oGq)|3{LI*nZz!X8R-ru8x6gFbb{8B?V^U2CvkX~uYO99Wa@#5s zLdAOH(_c_yV_BRZiqS|1@G3NUp$bNkw8

          Qf4(@}I?{Y+-4mf$y6m3T?xAh42CL zy!dDM%4H|R`&cUFo+q+6*i?Vhetmi6Lv9+z#==x@JC^FZWthYiK*r^c#Kh^OBzNY5?8vB1}r9 zQnu*+wjq0zG1@fzJ5OL%fp;Hx`? zjbF!}WDUL-P1O=dveW3{BXnA(p*>p>#RMs3rUT=b?3ePc6^mkA-X3==<@JXLlM`5( zmKOoPN{!{KsmNr5#xBKVJ)ItIlva-n78>km3R;yk%@6X+9?QSnWR&NJ51p*V zosV8EgHQeRg##v5QohTYZ;boOl#4`6K3^@7dl_Q?nK(ol9P}Ia+G^p3%PFdXwus`; zi>NWw@Wg+He{0KKe;lO;N?fr5)hRIs&v;5oc?n1Qpm26lAIE*7J~Er z#yzwUPOCeDxr`&2zl*ehb!bRBfsywu^}1OjfY9K&M~zjy+`#r;)gksKesZzw!PO2I>J>J; z%k0O8$5Uf6?nE4Kqk>XxShFd|(`v%#!nHzZW*(?tYOaj8k4lL{Uu~G0D$hhcSU}Jy zXZP`%c(S$Eq{bCJ)vv(>u?*=5Yj>e?xfVx-U|&iI7xFd7k;;Y^isVGlDqOIoq&>@$ zE~DhlZecA+;4HN<(e(D#Tpf#bv4bsG@H1(K$g_; z5nlJwt>>QMPZ+gssr7KT`B!f*IkI@7r3)l|R{BCUIr@l(TF$^kdZn)Ki=N6B*Y`GE zrkh?*GWOUnh`(O-9y&SR*>W55BWfSOe?gLggeR}DuxR=-qD0MLRgeF8wRJTLz#0oEz=*=7sLF~J+N;S^;46H=640HyuctUvt-iV2HluWfpUaS7!l zP}gi~2`(Mwywsou!H%^R1*#j2tx7UNdu8Hw&50cXzr!RxGLpMP&1%G?{yL*J@>nhO8gSgK@5ZBv{GdF4Ct8}NfD*4)JKcpG)bAk`@2;)*u zV@aiAs2eU2%U!`4CDxE^`^OCx$4_XREzuf@gL~ePPp*_QcpJi;@D4n&*=rPQNuHm( zOat?@?^v3OKh9z=mt~LcwkXj5)E7eFReQy+nGFxTD0HtT`rU}FicVbVy8|IFEhHk- z?Lj0FHXq24uM0U!sIl6j?`+0vaW2bE#vv$EI^+`9D1!wEH<*i=aqDS&p-}svCyXil z1lt(9;}%?;PJT)RbfNkjt!wS4Mq${O;f?^qFV6)+PLXl5qdrAi1gPy!@d#6VNK3v;#5Vw9_&N_*;A-rF`=ZR0)s z64Ctey755^F6_)V5q}lOo}1XflNu@DUQq3&#Jo}(=cMiM_{jbhSxe-?E2iKWC(ZY1 zG8$`#N^%cHv;3%0dzf27L0T=o){O^qtiVkLv%~TE7BBN|4n+#LY(kG3cPKrJ>=@No z4P9m!6Iyqj(7Bz4zC^dLc`BbYMN>|_dpjojyZx50zJOaH_?2R5Mt$*zIIiM@jeaD8 z8&Puz)LCJ$Bag-jyi@gSL^qU}2vobrIu>_L(noSl&ewWHmxeool1b zOs~;N7YJ9<`-YRu*P5~iUg?lC(RIm=NlAF+xus~W$ISPL$;M@HyH@W%>oBSQh#<{M zW4B=XK^8fgzYD^;j>Bk1RE^~0PNumZ9=@ABJ((MQFjOrnk-~~NjWnSeIqaH5yie=g zO_Lp1qSPd%0ddn#*=l~*3r)ZehyW=`isphpB3fjM$T{9%k?&*k;OUhs5*@qUM)g|q z^`3}@E*iSKvJ4^)mM?0D)s1q)&lHn9$_-Zh-Y@b+*K>KmpoF7uX3;=~=Z_(+HnL~F zAj|#}CamcmitP=ah;r<6$YqbBQ$o54)Pi(8__E>ykWp zIW+uBX0uAGGwyIRDAjL_2WRKpDl-e6IA_C)P_FUQw&b0X7h)6VbZPz*(z!Ja2I z=IK9LXVJTcxTPs0K7@rbXD#jHf-IUew|>I&6*zP}>j18d%k?5aR8 zu00K4nkmEH{4+Hzy>dAZ(v`8(_dsjd>>FuglIV0k{)zN%Np5xd9bR@9(_s*LC(SsF zSWcBsk)VM=(x2=*T&%jkCV(@NMeagnOC~x_8bU8gM0_X*?7-V`Jk>2?W_bQ^BKmBf zEz45d=^NwcSOZI3bL-J)lD+E9ngy$)9FYqo&g|bh0ZERVFAf}J*N+JP)Q@vyMB5=q z2m5J{yq7z~`oCkB9M^v?ET~7?${Z;>0NT~9m8i?r(S8;c?W z@LP*KKt%SaESMe>cg!|{iNeu9_LxtpRr;#Oh9XiZR1}&MRMBrqL>bkjw6^8L?z*+6 z8p({$J>pr>=@UDq=ip2c^s=`>CWh-zHE2oR)S|W4!IBKaBplD@G z0&jypr_~ZLAZ)kWoHiHnwfyyZm5iF=iU~AB-+UdqKJw^?Ds>;J@kE3?k$AxdKs}2*PPlZ4fE%$VsWww%Pr3MZ;IRous+Eh2IhOpx(q?T?kQqO6Gq5hysKhN!1X`~6KaaC zr$#BCx>UvI@;VD<fiy7lUH~@tPfM`6ScFqwE>3WFPw^?3}QO4 zEty)}X~V9Kns^CAjjg~m7HS&VV$?k6_2L;nANo=*d-Yw1WtS13Ma16;JT}Cu><$J- z*isSnj{eh|?+l|emA2^zttaFhEwTOHD2t0MKxeW;YW=%&{Oaa5BD{%AS}n`bFfRSR zId0pb`4Cx77=&C(tmx&)PPKt86&_vRbbll7AS^X^+9xQ%pv0j=LP`Fo;o-APpvMJM zKAHC8moHdN=(4(A98ztz3V#!r_-wxTyf>SZZ+ISr_ld!aZ*PSQXTSUge=YrT$$5Sh zg(wheWk+X^8EJ_*)75f9h}QINMHXBlwryk^{hKn?bt6uADR$y0b?m7FZl2QZr9mSU zJ@fMPswjyODekp#W`;s1uqC#(z1(zB-qJ~oG8*$l0e8tz{u61> zoNq@&rOydtV6%ncQ7po1sgUFF>O`td$3n7py={C`dTA!Bqmu~E-&wk(ZeI5^FUFWY zAO@YQMJQJQ%B!2DJoSVe%6O12=oB1_r2lw&5uvRKp26MbR)dg6Y0nmZbQ^zbVT< z3MB%zPtVCqD1B>^Aduc-EYh)rb^y)w4^Sp8+5mH|Z&M+qouy$c0qzv<={gDoQ zhM%wfDz0yD`7PZ;TkmGHLvEdiN>rWIU>JwRpv`aKKF!gVGEH+nDORG1*oDI%d*6D_ zVr0-Ni*g&J7aYVU*J(@^zOSx950k;)L_Ku=9{>nI_rGv#dXA&4W-!j)_PHt?X8;SM z7-5w#Fyn$@3ws$eO>`eIK3oH3|DE4Ou(D&I-R}@6+h`B4y=dl#1<-_G?SU8>a#L86 zE+cCb+?2{PK@A5S5^qDUlxM>n#OLY0bSC}lvCzU=+VIc%K^Gag$mN^h+0=TO4NNJ) zeLkph>MB(1a^i?`#6np_tAL$<8$uXj%8Z;2u&2-rrA3D`7Z3%rQvO!CSq9rY7-psG zEpnwB(MJrR(=AE3!o^Cq`3x;InE zs|GJo2qHH~r3Hlp?Cc53#sVWnK>)U%ghOs2Ov?m!4Xc@NKXGdz9iwF!ZGLLMaDxArj5Ks!*Ph%}FQ6tks)s3S=+ zW04Q68iLDpk(ZbAk@Z`P)*`K7beeBF8o@BZMg6UEAZLbMcUMe2ESO4O=1G|)<#a)u z2OE~GVJ&*%?aj$cVAkq~T~>E#w;-#gVjyJv#ELdo__C6xh9Y^wnA-8_r^rkUYd@@P zJ|>{8K}>B(?j|{ibqZf;gvIIbVJ`nJsWI6c7;1oo2A#Iorz>DU#1YaGayi8bAtycR zFlTC8Vfqi<6QjN)QM*M@{vg|sXk`Ndoj2sqLBHmPMM&SUNo`3W5Pqt(kbx-4`3Ka5 zmlYnlrTg)Ca6TgwLB7G5dy6NqAX5=_L#6~99xMIRtVOj+HG3yX8t>6fkkC*K?W9I* zj*x`0!iZ|>9Xy>*Uf(rB`58SNlOvOoWs2Hfi|eKWq-e+6#+)z^GK_U9H+4?vWvV~^ zqV@O!S89F|w7@m|Y%(gRW3446Fq6F}Cnq$HAYYh%L!?95=)OoFg$RZJEX>sv1PtAB zoc7W@myTTka^vL^GKl?s#$lp$9{}%Klu>TLQYgWY1QQZPGUw8yXOkB1Tb3Rk7Whq3 z8XM_m@2wbCQ7ljGdjdR5=Whhy8xYD`1fZil zM9BX%xqfAjgNi%u7G8{7G8N2Vf<{NM49NBV(fHs-{pMWfy7(ZsnLmPDP`ZSAbK0?I zw=D^EoLtwGt0%Iaj3h)m2c&Pj6MCKuG!c=i?dQ3WRQzKP>6}@s1yjm2vsMs>+o@z! zYE@~9%Jss~)vGv2QtX7Y#YT0`ja|98LogGhh0N|afE?B*(i@8BuV`5X`c(;`g4AD{ zK?`3E_*gAKv6`Xdk-p{GKctp@Kf^Cz3Fqq$L2BB++@^rPz#&nU60qT}!VOj+udZt_ z4+mc?!AprTh2)`7IKwaiHHTx>{M}hoo`(l~#0aUKyf0zp6hT5BMS$S6+Nh?!28xSU zv#DOoYAV)=OSejXbw89?!l7B22A1hKr&{nA<A7;-R8p z9B%P|?E1C)V6SCJDbX9BAsha4ZbMNz!_UaWU?3Qrs}+|*3j4>4uT*I7F(6kFxyK-z zq=Ru#d^eZ{f(_6f3QYu7KqwEndQSsTxQYr=3wzIit0EV|E-aIL4;REaj>bPz6G zH3^4TfFlfC2QqD6=Dw8y2opg!sz|^Xl>&SDgw1}sTPxC({i2K#el2DkeJf(h%^b{K zcn1Kv?+Czph5m?5j7xSj$Vm{d!{J z`T!!c%F3{o99rNTd?HPTsy)G%L*}^o@<1~Aa=bK1aWoC5d=U0)0)0yz!qFDmu>k2@VoB{Om8n zNoJd?LWki71El}~&Yr$##{|u+7W}g-%^i?}x^FaV<%lgr8BC<($b3MkfMGd5TWJOn zPnLvj=t3}b=r?De|Jq0$Y!`b86IYk+n4k`v2VG>2uosI+xCBEw26QotXmF3h&o&uM zI3jPh7NrwugoKZEvPtkT#fH5nh;b-{LMwE6841Gi@pt%7Tl%U|Kp-SJpmRDho77-Q zVrRO)!|q!&O`|CXuKpD2^!3AUNe~4pq`UZ}4X;u{zYG8x_J|kyN= zP>#M6P=F`||Jk{Rd!wPXT(+VN^j0$cv=+y2O)K?oTu90QN|JSLcYUA(_e$iL)f7(X zk!vQm&VB89uBM&SMKT5ikxQ6E1@ctFU`B`-`#Yv<4kpp;Wr>EiT8yScP%#{cZ+W&3 zvyX%FLO=t$K^DErBApHws^YW#diZl=`Nz^mI5;)`O;XM7!1awLPL|suhQ^}62nZ1o z?1Z0ivVbZY#u+wwAmc$#)OnH12mz=;kt`=?vs4B86(a7!-ausx_<}Es4I8TQjW=T} z@DA#;_!cn<`47t?G)Gs|E=0c)R;+^zD$kZev^>)Gd$L#f!oDvZ6lOXJ`Lj(=v#j5UqNVE9an5v?L4_ACbi zDWww$m9o+($cexfL_UVhsrUyN+R2&8Jtg>pHI$v=)M-_XfL+65Yx0T()Ulu~>M2oQ zfP4?gL)!3|{-~hn9Obj?GY-CG4wcs=0we2! zy@epQRacBc;F0+Z9NKG%Hy@)i4@qk3KmfJQ#s~`Un8t954<&M&gX?WXxEyEQqTVjD zlZa7#Bw=G*R9biaMyXB`-IR!#NIN%{0RSm8TKy(t&PbD2tk^5vdl^ZfSS&Pm2BDYC zc2V)cn{{j(Hz=|{2{WOM3R#$K8%Rm~aIM2OusJdr+92rJr&l0%8n_0kBFNfr;59be z#YGc2PZH||ji4pWTL+?z86c2GyApWAOl1pkvg$^BIJ@AW$}3w_FoWIHfE>*GF*Enk zAix*r>84iq4A5P7g7SNsfNUNC=!;Eeo;hr4<_c1IuB&gvy{&94s2ERxq`wh=es?w# z26$&b2@GH+@s(a?T}I5dXS&Do0GDHjPA?c);X<-wFXI4}u`8 zs=uHkts+rETx38kFngK%^5HvJ2!QVKZcaJT%!@-#`-D&~bTCdyIM`$|mOV1l#o>7K0C z9r5^%0X4d0^;9uY6<>HnF;kfKvQY|Pg@CR$z?R1*K(MQDkZ-45omCy6LiT5o{5Ph- za8B|kj~h0{IsiQ{YC>mzex0XW0jKn5&Vnq2xGZl`t=g)b4Ks&b84QIRRllqQ0-TBB zbtH`S$fXFnKT(DND`*ugoCNMYf)CoEGEV&)8*rqD)zjc>OQ;EXkNL5$yJq<883+mI zz6hviX?X@a;8c5|orEvsXA}-i?16O-nv~*W5PTJ1eF7@}-YBq*21oQ|NUSsk6}M%q ztbn?=)!RWt%V-y%x&y`ktnbMdmo)IaV~^i*ER|G1%e!uD)2D$j+FQ=k0SJaeTj}8E z-jJk`lm@ftmT5u2L;Ueoz+&&8Lr)}`h!J2&vOatM<;vOPvxs#S+>hRbb{tV~{p zY(zu_JrM!umjh>9^h5)p#6|UmeC~nYdzl+ipQ6t_W>>V0*mlbbB<_PFIhueA9YN#J zA>EFQBreqpblBsp^AA8zMxO^lf#F@f(RpK70B69Esw3h#t z4NWYzr9_uJrmhd>?iKK1`-(OBP3SY>!G&Fb%NP|99l&1MobV#e2E;`!;8uPp{1dbx z-(*Wykkp7(vlswQ88DZeb}lHF6WlQHg(V996SzAbd=a|;`GlvV{bCCEOi~!`a6@3Y zQ*4Y4pPBH25Ev-*9ub8U$zwtuLkUJwN(F-~+pKVe7ND)gXjD(AF96|8Y8ZR~?F*NV z2ha`x5jWc96kcw`JBL%60SujO0@HuF_%M{LVR#432NOq8^!dm>^PKEC;tl@WcvDzh zLy3PL2ErUs0F<~9pf%o_RSKH}8UukJXafMgt^BDC2lk^&6d4$5p3x$EHjG^wufZ9{ zC<`Xv9j>rdq$#@Lo#LcvLM4|F>P7-d8?{RUOmT_GBg+s2_#KGrUDm2t0oCx!MacsO zSPYR&gzPy3Gp9OxMYhdm9RfbovUNQ=P?=nG7&aVa9%@3DJZm1h5H}ww^+QyCg5*D*Ab5 zs5;jrdPJbZr_*|a`bw*3Bgm}u>CxrAGd7nz0V)qk0=!5NBCFW60>H9%_C^Rw zI(appf5$hLM|wCwvHB8c1fT>p4=$H^;^$?viNJuIL>A2w8m|DH!pxo<2SV@$nV=B! z$;7N_5i7&Li1vAXy?02vfM|vpxOSkzE~7;)*uj3}p1R#t(h(}roF*7o?RxFJsc#6- z09XLD%q5DKD3a*tNgX{3E%t5pP^eVd{~$Bd_ms!PiEiF6vx(U!pZ$yijvls%djR>I{}_1%9~wPgb`UYzWs~(ih%6kF zM<_#E6*VJ=K&KZtt+!*nwX6x}CL`_$uH0mU4KvB#3lDqcOoozUdExpUZHUhmeD9D$`SYCoIz?nP1nge@UA5bpPwNMQt<@DR69PX_z7o!)Pk z>OY|b)(F~#+$u=+J@86rAVeG0z1hsf85`zcZsL&^&Q$8XFu^1lWKoiu(zaqNVqjiFRV;&$fkAUxp7 zOwE@Kywk@~SMd~}ajzV_sV6jWzD@cbkd3`0&txx-{&H-56nsP_Xfu;08^)_|(g>0T zYazv0@f6+ggSjYDVA;uDOI0RC(hOi;1#3Q07UUPE>?3R%d-~ClcV3E@SaD_Jw~7b8?!s;uWW)r6r~v7LWikJ)X~+^kPL)S z0V$IU*kS3&^tnG(b%|BFlgLz`l~#vfhBRpWxQ4AXAc-=BJwFUy$76P5XiI|&=#6FXiA2?pArSbVyNNmBR030rt=nFuWn z_r$CH?-6T2?^IH+{wB<5c&L>fW$01%aB-Lk0M@CR@jtZ%?=1ycSaF76dmTqUNq2z? zZy#C#OGT;pA{uPWqBHIB4Sq$Vcra`EyU+@CAI3hnn-0|?D*=OBbfB_yVJ@QukA8SKa@Q??g;FVH2;et``Ra-F?1_QY zaYCHmpE9H`CBECWE!)8sMKB{a11e0MOxJ!wDG(wpILQq(EWnM6;E&*RQyAI1h6DL0 z#HRBULUNik^%xYOBKys(p)U3vM55Vk6H!})IS3o5d2?~j@Veu%5p(0Rf&d^xnWDO> zzlotV6VNB2>L8#$MP+G}>gfR+_CCOo?6o;qAa<7(j_F!3Zc2Fa=AnlTy+R@)qtrN6 z5>4qKCoiR8w*dh3eb+k?Wk6^d?|@t=kh@p`X0jRZ}eAJihFZq6>-l zU|%L91zGre!{(NQl^BCC_7W9f>vDSwY5`Vf+i`VQu+gd(n|@VQde8J%1SblDEb2Dl z3X6YX7D`(1+L(oRR{^Xi1shBaBMv4+g5p(k z)mdThe#5FrH!@ESZtn+L1o3|qG9Cu=4_nF|Nk`7#{lH@eQ~_|Vgpo4R0z9o~CSG(g z$3NAzg|pi&+eq#Ff}7ud19;3Wh{YQilJK)AYM}3cR?c6`rcz!BK{V!BlFAFWl*4;} zPPB$&#W!pIbK}S>trG1_fpmo1BxHD)f=A}3;kDRMwd9EfR2NeT1{i^#n${`~EtWzA zmksO`8v%Q&YQ$krF8~Qr@rmvSmw}>EgCHo2g^sRRYrM%6k|Kg|&xFz*y`w$oRSkKfw2l3E#kACNN`0buD-jhX{zznxuA1;>ai#erUaNA2Mv#t(R%Acc`KfN+AK z<}zQ=vF+r@22sw+09P95vYikg?8+DtpkY1gJzFx~V7wQ(kw^3U&3V|0YqG*!-&8ue zEQKRM3Bg``<0m@>eG&2U2p@)GkJv!X{2W=bS+R~oIzeA|1J-pCg?uPs2-0q?m@iEh zNA(#jcfhPDhu>pgTIG(ky=YKG^Jo#3@SlNF_ka-&&k^SF(#y0>@3##x`Y&5x($bG*@1p3RI%vD{k6n)=#n zQU7$`a%w6LXy36@)&gON&`tzsdhgyFFs8dWfAB|l~Gu0Hst{O45o2tZ%M)GuPaGEJn z!YVx>Rs@yAy(qg^&`1!#Obr z9c3kLr9lb9;!$434srmXSOOAzP=8%WluwLsu7u$TLbdpSdNb`z= zWSaC!E(>aOaEeS?`+TiqqJY%nc&EY*>E@gK0EvmyrWpi`uBq_k9bIoqO3*)I9&QkDF$Igpiqeg&9HZu@G`lY%;SEAoN{4}>lAj0o0{}qg{aJh;XoBu$f`l`&iII z{3uO%5Vd(Z6|K`G^1C%)v^WL_$Gyl@9rMvwtd`~W+>SEz-ZSGw5r;7waQclK0JzIO zQh1>(pr|VBD_}Py1fiB-btxd*M8aVQK^zb{8SqinQwwB8U69zQv)-LAp#V@x3fj&n z{9+9PBW!RYbI1#FA;ThB^uaMQ-k#$1SJ{8Ho!j({rq1g?Fxv(PF}%qX#UHLTXC}wq zTVWMnN4hk`>#Laz?i1wgVG-hJ07^i$zw8usAd4`tH}{W0hhV*N7!V7HV@XZ;nX>Ix zu(sITNCbC-HnGb>(V%azf3>!OV7%QkA;Iu7xVPHQTwO1-yCD4dvfSF*tHw|-p92MC z7QQHyqoj2p+p-|Q=$i#t6rQf3(r%YJfdKkvQ1JRK}1;NN% zS&4S+#~3agy`9|Ab=%qjFC8w+^&+_k`91uZWe5&#swQE){$$AJ2HM#-lj`)s&cM!nM(qveCc2xVjs^8?wL;-fB-T!ugVwY5Fk#OG# zU(Ox0O(!W43^voh^XN>-Xhy|T!_eFxRL`1fW)o3bX={lDo`Gf201rB~*M#thLM7}b z_-qPDfXK?{E!oadR>`;*rd8QQ4b1vX`VKs%Uw(GTBcoaxsfR=lFY5yyE9c4AWsOc1gX2j2t!Ujtz7S^ z&JskiZYrFKmFYMe@a8@S%rRSCZPhb%uc!r@0(@HtSHTpkk#fGZ4)u* zA1D7vr+&fk@f1w;zlV+C6Eget3Rl3_P~Ia0VD!u}XpZWoQ5jd3T+K@0gFo zVs6Z6`Gg9_wPZtPl(%u5mqxGHu-#mn?b`z9Z4TFX7g)QKs&}J`VedN&_&|$x;^4Rh z5NdFOd59%?IP0%e_UH0&hGp`oJz+sIrM|}1e>S&%&PT3@Pi7}91OTA({Dc}f|2%M!IQXlo<`%2Xi+?NeVkKynUZ0cbRE+b>;QMvj0| z{!act4>6j8<6pqkOn!V;-FC}7dR!0x;xB;{VrT}nmJD3805lvQ9p+6!QI(z5-)lE@bUHJKs=Vo(G_C1|Y6 zleiFyPsu?~VeR|n+>o(-1k-RJ%qjkS^|B&o6#%oV9kwE&R2Q;ToajAyl!&9D7gMB? zl>*Iu_;ssG!K44^$re442SP60{U(EVU>5 zMr1c1gR6it+Bgv@?(&q8vA9EFa4Q~-jV8Lj1Qya2ExB+>p6$a#2Q-@_$heX2p`G}A zD=qQWQ9;daw+BERtZ~XrDvw}4$4;epo9v^KvD}iX&{A6+dsH?UJN9K|a@Q~T_zM6< z0X1yMMFD#=FVpZ3&1{+rf#PINBv}`RM2l1*F%JqvjMQ*C$pk!$-eu`QcvG97B$@bd z_fvB4l9|L#%;ig5nZdC-anqD+tfJ%{`$0ehl*^DP0tX4Ze_v2?lK2w;HroG!33jm- z8_!Sf1QXc6O~Z>W1aq@tpnk*W^G^|caURS_8 z?siZt<8Q}=7~md4xFkM9jwjtW;-@}v&}L_vxf|vi5#%%kG971-+W7Pym(P9J$_P<2 zm=&Wu0XT@cz`7k!R`0W?r`<%+%z&}qFHB$(zA3>>1Mxy3@cgQ+wINT z*qeHoYJnB=y8A>jv9Lg%hcu@sNvbBZnATVU zeFFe}gRP}Wb3m{GX{}~y&?}|l2rd%LMk9fU>;Cj(%;%M1TmrH}r+-dzgU4ta4F<|I zx%zC+;GvVxLZSg#D;=sR@4z>o*Qa=Mdqr=q*g9)l%@vc<0d=!BV3lfM*ldQJS?FH> z#qu&)=5IC$Vvyi@Ux!tO*cEv@R2D%@G5XT`-UW3#)Kfh}YI2dYZ4_8B65Zt3T)!p& zfn`hz&4Co6@?|nDp}>at*$|Q%V4~UzDzfB@=Em8TqhNv5B+#D>;pWOnAv-)h$nG*9 zhTERi+qKkYfY8H3$GI1-%deSMQSvH3=di7kgV%_DIl%dG2pu?6&E?D=GXRNR%WcKpNq}usnf#oZJ??LB!QQ z`104SGS%IdJsulKFbnWYS_=xu#pYks?8(zQlERLuMI~3%f7##pe}n%Q_<^GPk3FC{ zO={4X^8*l^q9|wzL-C%wva*YT0d`Au1N6g-!Ci#NW39#Uyb+1kwH&SKLCydOEnA{W zLXaF8s7j!_)&;XiYb>nQMlAZ7wv2QQR#3yFl9~e`4 zGn{Nqah%PYQ`2#P7V_~ZK=RLUWiEecYGNxinUDMcCQf=+(KaE1bFpC~?SRMudT23@ z=`T%zLMsNmsxf3fwVvU&J|V+Qj0~)51~)3FgpoxYpPnWE^(&uYX)3XDW$s|&PBu^Z zR*YqxtF&H7cl&Py(0#k1PwYT%Ef5_!tl}y)x0rbj;f&v@3i*3?4*fKU+g$_!9jAy% zATSCILw!!aWY!a@?r9Sd%dsM?tVmR|{sFGi!zOI6g32FHeDXZKs76bbO9|#u+n9rP z{;GRLhLwk!xy0bpL6yqIl>{bQsb&%Rgc60)CBLvX8>yEckz#qo9c&vToquVn`QE7h z-E3xP=G9y*YAxdxCBmc5#PaDhH8ZIV0W;t_a4Ez_r{~(D@yfqF9!_T+RH75u;Xf)& zFM%;ExoEMWGzQIWUDm8u1B~sRQj3y;D5osX_abzbrYyo8GHJ7cwV$-Pmf29C4n?vq zD#RU2D}@V!v;e)+>S2~kXeqmT0)NZ~YF|%UUu>(05e|u8rZZZ6h>iRh#~y(jX=%^p zwXtbMg7Z;M0KrhWgk_s?slupPf+=@Xhk|Jm2&uMrte+W)ctn=aTSD;U8EWzy)O&-d z4lb9Nf6zI>gsHPS#>fXn0SkE84L9+7Ukm9wnDt=dXL0Guq%7Mj2|T-YCIhFptZsN} zXNW<0F2BV>TsTk67&QhUU3{QHd+Hc4CC-I#^azXYxF$~irv{+ak&Kkg7{ekd?xm*!rA!2B zrz5D#QP|%w#P~MQLqy5gL>ZdvCk#kWhQol*De*gm4G2aFFKkT))AhS5phaKjS>-S7K-#- zj^V|dwne%ft89*!^(Ix@3bwdx2{eS! z_yR}E4M4C3^=@*c3XW)t;AC3p9;z>%4t@gyAvlJvULq%bv^rB;gcfm{faTC8v`qDT zGz%ZLTu8|Dod~*s&F3Wsq_hC;XC((@xNUTW4Wv#)VBosAIY@buk19U6a+8%4xe{dk z(m{yGW&iOU6r`N@YkpC$gQ||=4FOZnFdzwgl;!^%aScAA{wuDLQfhGuwKDD6qwyGe zi!NTGgddbU`_4E5LFEt{(^?or1a}|HGm0H9iaf-dB@>~1dbIuPVw%)@IUbAL+;1Sc#?Ij`t=YXiJcpK07Kf^Ag<9Ny{F&?Il~nVLtUO&1V82T0d4 z^*}N3E?OV}`l+*JDU3;t=1t7Mf?;T~LU~B2@joZnUN9>n4cvDsn#QTjp3mvb>qU^j z{EdWUh7eZ)@GPts71#X#J#$e7=TJPJUR9Jpsi8ow5(PCg7oa$GXPCrm!Bbpx1qlVT zRdN0@U%$incbV*reTeP$<63xOUkH`*AiM)uEKxn%3+h0G2S>33J?aSV@As;$wjIi^ z#z#vew+GL2_jT9-QzP26cOm)ti+}8VwHVGn%ooam7B(e$THqia?DcvqS~PLA0r0Ec z6{{m6; z$rZ9aMtI=v`v|A!oXt>X#CT`VD4OOyJHlF#TA_Hxf63kyB)qAKZsZION=gPMF3v3M zpu6n?Sv?LuAb>*vF2E<_EC!|!MWlXi2(?%~*F+G`v_W;cD8;(yqZRg|i+;Ex+Ba?_ z0!GL>)Ju*5vkc*SgChlxp!9tt;4(SeX`+)vrcg#{{*2z5TI(wOgpa`mj$(6KV1z56 z{P%I<61)Cb*}h`iGKZJ1n{of|L};NSJzicg5MV7Rg5Q@?zbFwpO-ZnSiZHB0U*4v4 zp73I-4sEDEUfkUseC9LxCAJ4{i|!=dteDH2kke2XcLavRa9Fr)P0CxnZem?TxZ9o+ zfqB4-5msjG{1tTKbnwfKhs2{Vk75K+6&v7Q0&kp6v?!&FsqM<17jjGTzCCRfeUpVk z@9cA4R?LP{7Hx3!rI#@~#Ut>Y}N1;!qqD`HYNJ2okLc-KlldNl!V$ELKPP@>|uK3b3YT8fq zQywN`0QE7XD~)VT+7$Vt_GX$OH8srPSm zW>n+fh_A5Y4cHnOBW5GF8F(U8iNM6*(y(%586sr}hG1f#i^Cy!Z36}_owbbQSwje{ zGJS%iV7!$FnN*~(MIm}DE~Hq80&FZv7$Hg={1*5q;IK$WfjLF6gXrY^ZD4(L+7`$q z*tHiKM1f)?3lT!c9YH-yKd~la13)ITIU$aeM25%a2(v*ckYhc}j;YeIn6i=;fa-EG zkP3uZiG?^KF4vc7Az30ja8wv%0+AxE1b7meAg=&0xC1D`3W1}zr9oZ|8-&y-VMXh} zf)Tm1X{@}ZKwf3pb#Yu+*&B}Pmxg63i7-@#BQahzRRx2lYM5BODHMUn5RQ?~kxj+& zzYUDb!V8X79IQwYiIhUQ?86BlGs)a@U74KBR^DC@y>XqghXP0h>5N1*K(_+lrqTag zu!xFrmnTJwoD{tj2#iZ=%q(p9CTd#(NH8J-u>)_xqa~${h_R{$%DVLI-~YtumDeNph}7*Sy1&%=|1#|!otLo=8a#goQ>28=BoTss&* zSZ|93zpSmAkBRX zHvC2wbF3!gkW3S){M&e2B10n+?V(CVqXLUl68`;DeJn%55S+4XRw|VY;IXO$F*1Ts zhJ&H=6N8!al0|Ol@N*ckDmgDyr4AZ+l0A^41#=Z;uK{5hTB}V1yT)kaig0!4AXO7` z&Yy^2d9toq0dwjs3o{58k$TiT&?vUYgHbij_(%&@lK~J;CHNT{GO9sR>3v2&MB0fZ z4TV1+vIrue)MuPNGuv$R5OtEAH^R(ncBA=3g1 z8Ys@9k;(wnggJ?W5<5|_8qvBK!U&QuXV5b~r903CMnO;v#9(L%B6p_LP6C=>#A=`_ z3WITh){SP7y2Y=6z(g?jA!5B`Xmb#D1yR2*G0YU(^a20t29YA-2wC3xAF{(@O5|j6 z!zXgocfqd-^a?>sy@S@zEmv?F%eb5hM0)Zr5gsYC=}u#g@u*rc=uYgdy-w2|6I~=(@x>T$v#d zb7(^C5AG*MU!m)P^_qbFBoJo^JeF?aknLUd0bkiT!W;*MTnHKRF$h5YS<)04GweG- zNsVFYZ!|Fy(8!J8BMIwXV6ocxjFK4E=prGQ+gYt#$4Lw^xTyF8&KQmzxW?7-O#$Ez z47>OPu46p~?<(!)Y6tLxvJpmyj?_E}SpZH|4zKXdGj**k{+5xA2OK+pw7|+t)4-FO zdSj8E;XkS#PGlR>P>Tb7n5jx@J|G;K&VW1e05*Y#Vmz2W}g|LDxI0xf+m@i#>1KhO}~`dyCaBwVK~bMWW!i z+>%VLFfM8q`10c&9ALCd)~G1BTs4&06r$T{dB#brlt8!x=q2c}M8cB%KrP8; z+!JlH?Z`nSeOq5)C`Y0e0LZ#;Q7>y6m9H7BYd2Jm8RRq>kIfcVJKNkpVrC>@Ad3hwP+6+_e@bB6Bp@fE-FLwME^C{R^c!ZfTzNg{vo3KkkN5XNMEExUQI#faDdpF zA3ukjD*HClM$%p8*Oe=|fW2assGh$7=poZb{1NTP;*-dkV18hGk{3Q8KjwPygpYpe zfF-(DhvP^(FwOvxygsvH9^>vCR9j%+Be11mT0B5t6q#Doj2WKBbHXC#cnx{+B$zd5 z@)*Pkc3)9n2iQ|z-hfCvy89r=k--_EurvvJ|7dZI9yego+E^ld&{LMBGTP6NaTuEmPqF3LvCD#9e8LD`%F3lvcy4xQ+~HBZP`vV=>Y zh=xj2IuU;g@EmF{xGRwogrq6k^*@ve6C|Lpg&>D?0#N5^xMhH~(x6E>&1D5Dvf}Ls zBLAx(;Mq7`Xj0mdv%fHVVIx|TdO=;RH~Gczc>DqFUyssNhS zx5LJ0q1F$^hKe`D~*gA1Tl0rYtg)`eDm97EmM(!Ao&bUkxa zw1QhNsP#WB^57sh@2K$Tg^`c)-&BRw$Z@2PtOCwbT>UMU`k)dcFSxvs4r5CW1a+5I z!WRS5F0RrK$Imzhp>ZqohcW!IbDmO2MU-Bjb9LCfsQf#1d8*(b`l=d^g&nIMR4T|; z{0%Qw`L804wKeN8P*obwzMR{m4xBgvNrzJqOo2Qj6jEoT8T(Gq2NanhJ~~2EM>MA- zR+|oE>157L1Bnqa=`$Ds1ubG$RE@}hXz%8BY3z-a6=E$!c_`A%zR2!)Mh39^gUG4i zcZVVWQ_Z>Q*+%DOWH+MjMW}uwp`7g`bcYK$C%FxZnB+q!dbX3h5k5?3el&x=X$frQ zM3HGC+LkcAFV*uZ895syy*mp3anhPC9I|J4ZL-^OpwK%*r@029*zdq(137U@fBYnn zZ78IJX)2SFoWHNBTL-h)50C011xRbUsg_;VO456)j;q+(w--39W_EY9hNIXvc ztI`hzo|1TK@Fn3h!sq**vww}|m%(2&e--p+j~)bjnc_Fj-`wq6kFGHtlyQpaqq462 zyDH_L<52%vGl+L2t}k2{y9DS>OlAq(6g8J_5u=+~jpj4}Y#44+ap-&k`BaW8<)z{a zQ9%2i+=j(Baj-2{)+l5on^9XZ+!pSgFfxGYfEN~mnjm2Lm@Iy0lRa2Jk)~_)^E>1U zl?j8X897Nm4+1=J0~JtfthdQC%@m<)lEaCD9Y}&t2*hN`Ec+6v-tjn^I1R?_ zhC7aQ5U`q3a!bh4(hb)s0 zH9%}#AaV(#4Ex%B52Rp#DCU^TfD^sQis6#fp%xONGW zG(!F-a3BO?L_F!&OhCZ&K;iWacM zAOsXo0xGldW<(+PS2$paDWdp_K#B*{H5Q)e31PocCKMU~V2UV~2pYC4N_V{T1+c^W zEOpH^uVU2;yf}ki#b|qVo&^L@fN0^_5VLh7o49Bs60AJjFi{MjW#$R&NhGgd0g)2iW9gUX5}D-i<}gn{f5xkyMl(Enfd4|qr(m7umZ2tqNx1rJ9B zy00XHJ3bzduv8+z-AbpY5~Ji*f~aB$T+qhI=I-Os=7k6YC$HjT5M{%{2FRRKUav4w z_z*g(3N$%`#)^!65O@bWNJqt9flh~`#DPr1Ui}y~u$Khz)`2g=z`T-Ufynn`fynm- zfk}Pe- z#Gwc}T%)9Ze&l^F;Z?BET&7Hd;uViA76LL(9>$q+Yb=XmT2(WzaO=dMOWkEj9JCJ= zMu6a{&q$@OoeQ|aAOjpa6X|7J^vP|otmTiwL8H!=2R!;=DMIZP`2`=@vOb1^n`tq& zje-Ve+>hbXsYT$C$Vo<69->}7(NYvw<>a7R7o5`slPE+8L*u}7pwvWsTX_W*heHzq z-5Mh0Xb>KBsH%(S0pcU%G0=Y38bx(60xS9;$A3f_VwxX+(GL*HHXZd)m(?mrDu)di zUr1b11$Az9E`(RFQI5bxcz;zsIvG9Y>K*VkE~u8IkwI2e5iYDAt0N`4^HgoA$Rh^%}`9{A|MI7katC)-b7o70=o>5f)w4sO#}$( zFh^!YcOXI#A6ggj9)~9b_2wW9dFNaS<3@qE(Z9Rs(S0g;-!&HPnu@l}!cDYrk6g?d z*Gd-CN)Xdazn+(GJuaSlT&!^2&t$;QWVFjPH*iuP=YvkeYEufXoUBf9jA>%udsD$KNrXGI{{?*^6ig6j05y7Z!2^rA}i zq6)OK^=V~l>m+K~BWl?rD%m0`*&!*}9;n$KDA^ro*)pfuF=yE=X4x#J*(>JRD`wd$ z=GiF%y$L^RVF5BqAuNcnq(iN!Zi0h0s4r6jcUT*D0kj$mu0WeRDCFQ|Pt3X!TE=kK4VQnX2PB$D6(AF<|dlv0jNFojPqO6Ex1UEzgv&|xd zyED52yJdO-Q5PYZxY}aqK_K+HWIQ?VLuqO6OT25lE2i5opw#NERO9p64wR*?jZH8D zg)b!pL1i596N88olJ}NR+l@q^%3v5xD&wvKieC{*Qc*%kED8{*t((@IqbLyK7zu;9 z#z3RPG}E5W`1eZrhH`neVbIZ(jsX0U7INod_dia3s(O{@>0{0aKMg@#1JWwqQDKePe_hP!2WrZNcs< z^gJQmuiGJ&+g=TA0swdL7zxe*us3(gNWarvKYD}v@7rp2;4YBsmEhy%h+B_8Zm|Y} zEugXbX(FF?5f4!g{yTK`7B>ZD0jWHdyjlfM5n;*WTqE7-);*4Oj(aB9gMaept&-UH zf2hsE^4dkV)1umG2Ve*Q4B#ajnaw5?g7LzMnI$GvA)V1%Om0I&s#7?dkHuuQ>}o5l zN@5lyrLR6&TB4y$-iAUYvw6!V4;$Fipk~~rv{Qx^0;5U2BwIJkCC^MmH&eDjnkdL; zm07jLWKs-S$)*z8E2KDTiUZ}=(}N+|Dpg{>E!ed&&DG+&a!GIZ53%IDFqCt4>`h;{ zc8qf&SDd+<@h|`G<~7%$i(3Ax#$iFti2(D7OAG<7w@h~nHz?1X_2gOX2VWNJZm$qA zvD~Np8YRW+L4IKMwfJsggOnKJ6c7C+fOXgC3@;o&Z#1sO()sw+r^_jwv?=xG3>f1L z)V|xem=hpySDBx9ccE>hAQ1u>o2L(87!3Bmq#SCLkhsaBdVoSqD_ESWf1EtcY7j$N z;h2gFe|(r`jAm7ZAb9cU(y)y0(rO4Lq{-}%2J8}%Q?3n`A_e7@2~t-k9(wRT!y#3W zR4_YI1vL=SnZ_JiO%nrhKqdsqO1B%N=y=gSGsZf!%TXyH5J?FKwrk5Q|~0!(fG!qM5Kyh~k&AY()TUTVk7Q+tze- zVynwawk>UNw>s-%B$k#rdU{BHA6lId2V#AVp!4V$#q9Y?eHGA10L$21z#Qlj|1`rZ z5qH9Am8|UgGcGagfAiTlTuQbK?!s;t@^!}mHGdxkoQdG>zDJG(8_DIvfZOtzC~Ai4 z0W^Hk>oMD+%lA95|LJ0*|6R1yd#d*&)iufD9cy!o)@Y8q_xgZNi2N} z8NUsK+&~R@Yh4vskh3|9B2TKeP2DL;!uW1DOW|y{j%b0E(YFLQW{S`i=_&*>4va3c zTBOnA`@6xBekQSB&FB@u05Bm*Y_($GO~8A6A!7qta+p4BusZAhxMc;C0fz)7#4rYR zEsT9VFXHjI?vy}-yn-Guz$g6Ru_=T%ikX}O7E&wc=opA>?Fj#IxYg2Fu3DX+9c~?x zLFPqA>EB1YY%j)Y5rFFCO{u{}BF>GE@JrDftU%hel)EPt*TT#cFhCF%Q(o`FW}txRpD?FFEfWnKYu*t{;;chd z3kfpsF18IypfS{t;Pe8JIG}O>F3iAnL*DHFNSc;oQ6N#A#`}tpg+2y&yb!pkVkSg` zy$NoF@hXr-2QV3IHSd7@JTg{{;`moxS7ZhO(k@)&AUG>INUmGjl@pdz0~kiy0G%ov zU^F#Fj1bnmqKb>%XA=@0+X6$H$fozbJ&JaKk)(LJk)-#jR|@B)7FB2KnI(?K!y8W@ zO7RV!P@{*WhXzRcbtL+Q_O4;$^B3SBX{*4xk3>Uop<@mA@_hZwsA|Kam-Or||Vn%BU zrE3vEKZg+J0)m3^0n{t1R|PWDx97PeP7DG&tF~52!V*eW$tqh`fJx*SvWhpG)l4}_ zKaScmRt?iKEs>dojz|TM|x@a8V%}3~}$dqd!bkfB1z|381r~yo1!NNN5<@ubTTeKF6 z$wClC4fm=k!FJ^umM{BA`~;_@7#R3#W37ykn4Vka!iTBl-5IN+UXWH2bk zil)GaD}>9(IW49!WEQj>NUeC_e$+--5dCe&&@@6ODN9<*Y!eA$`zX@n=;VckJr>Y= zb&%ih%M=)ZN&&G+!RsKT=h%_!$e%}DnvgBZ{fj00h1g5!Af_-nAvS!89;QDy1_?3v zl>J?NWzgn~zv1S1ua4Er8bM`w#Qr*f0Iur!KaoL{8s^+au-t8r$3@YXaa>z7UVSVe zhzBD(gQgBQNMwlTD4~Jd^Fs8rJzAA#QUEEor8=WJa|O|AbKPZ)qzhAZHJYY5wU$H zXr82>Xk$#iC2)QP!4=B8=GZ7z3FZ|hVX!^33aGXsL9w*Q4yfMgR4PFm6KR%sBY54G zCk1du7zPea7%iFf+H6=W`6$AJGE;dLilzCi5eZGgDMApBOduWP&6oufW;^N^M>-Nb z73jq-ISxDmy5lF-K@V7d7PsGTE~*RPqj~axU2^?E5CdrH`U2NLMFfyKAIhDCQ72q^ zwhs?7IV8v%@;&*2{22obb&1Hl0K5-3;XfWQBlFSzqhS3~I;&^~d}W9`OKFV{hlGVE zz|@8~o^OgkSUr$dGa?HW@QU&$W*&IKECqlQkfEX(h+gSKTKJVBiz?bs_=7612y&zi zeUU=TQN#rlG>*%B%dYUr1ljR!fh5RPzzqV}4h9XY7TpniR``tv6wOa;CQF$!Fp;nV zuNX(Du_S@z!4|Ct?7MDIXAuKfkC}#iU!D?3`G#H6V2dNp!8kZ=+yqf!45UQ}GAR~b z?vIS9x-?cju04-I$oHbhRvif09MPL?sB2Xxf{WO(Gn?IHADde@QUhJAUdaO~vdLzL zIK}oZbrk0Q{Fp$3V(?@cW#@GX33eZc+KmG(wGudFD)zX0@T!O~qEL|}0L`2f&_wtH zT{lfPYfuZ0M1l@I1$g(zdy?|_2Kg1Dkt5`<0V6+yMap)@aR#(aRUtML_Mhyk4=|SC zS3iAJLn0jMBEW1BU^Yoib|@*fX4JGO?4ap|u*}kNbHr{q7B}f;i##>D6fgnWy97=i zU#qHGyq7^!?!d$^iChRcI#iXVi&~<7NQoVOcraWb*|=BOCnQVl#S|lluA~Rw6habK zmUYx5fLpU2dRP$qmXbOJ$N?&jTmhiTv)}R1)ixz5#-dh z^o1V@fICH{Nj=d}Sjw)Uit&@OXEky~bwKRzfGfcV79xuj!Dg|A$IK;0ELyZFvUcXP znI3h9EX^gP69RDLGuB9+!eXX?JLeJhFgd@bQJMHEY59`4q6$U360kYqy7k%_w#(ecDHMoGYQ5tibP7ZM=JqwkVP zyMaxRVpGLk0aEOGO|hFxmQrtHUe~C(#w#4S&&@H}-z-xX#Y9y>-w7g9mK9beXJ7@u7_iAV+J) zJ_a@S)Y+$uEYTu%6SmPjB`AoCwpwX{GLglAtmV+FSVmZttQnPSKGiF@{5s@MDR~je zWC8M}8l^UD%_t?KT*aAXT07C%xkE=M2&UYo{mL|j=oRLOGNW;FJod|dXWD+(?>che zjGx3S1OyH;#28sbFj6j((gfPpGb}iDn<1on-6z1i#%Xl0Wl@gXI&k472In3Tm9-8%)GDfx7nUc{9ZI@WiUsRcS57?blesQeqg)|#Rc(eBCtpgM;lnGZI(t9b<0M|INg-2VBwVDl{p~Q zwYW3E8c0%O;L-7q!2q|s)`e)at=ttg3Jd{Gsnf!{axsMS^6Nwt8tB15*9aq&oHBz9 z&vD2O%%1iD6o;P|fBYIUWTB?wd_ZD&EBT1doDFN;b}od0aI6`e(K*~6P>cp{+Utz> znhO!*8-i(~$&TnC5F#aqf%`POxYx0+V&r~Xr(cPy1CjS9a#c;EXs%Y_h0DzjqG-^B z2rn>#04O<=#W5wXNnz9@^mH#VKF z9V9H$yc7ec$JSZ78e#uz+{`6tzEm?(%Ui+AHnbxKp9QW_V&{?osXU|2 zX$w#-3UcFht>SF<2UA4JWW(Cd^!pCH5C;Z|3N8)j*IV$`oU7SSo*R#liwA+J?<&W% zTcY9xo_h+kdD!3^ImU>6Xa`x%3MGK^=^b#&Y-Q>q85uo7FEBd*v)3O zU@#CmMEZ6%8ox)MW* zQ#ELGl1T&x1R=Ucb3s@dO~{r^NSiqy-R|-b*LDJxGRVkcWB*C$RHX6J^pFf=7$2M- zy?P=c{_~bS!^G$+7Sfh0s5i;g@v>PPWdfJn{|eEc_fmobq|*V@-l8Y;06YKloBZi}Ze>27n^485R}jFd^&MNi@@vl|FPoCH!C%GICd01dHR(XK*F~7*J6& zg}wrW3b`>;l#DV^(IwfFJ{|mTJW+Qio8c)-)|{bz!7iKk0ILUAiI^uMhPuHx2obfJ zE@4Rgb2cQNfIRymhRDNtT=<&Zq4+fDX)dZ&H6}=CM;pAhV}@#zx4iJt1Ks3%*XdIBJ>yN?NU@WLuk z3xEI&^NQNO3c+!aX(BF*fWSkg;>@_VGtl7kTcCzB?>k@7PC{fu@zdop8Ysw7s>a4@ zwfwTtZ^#*@#QVq7M)X=w$OJHQ#>3gQuz=gn1VCH*>4FXvD7sx?lZCGKGalc z%4%%Epg@+=Xq_PN?Hw7)$u#=%Bm2B~7Ci2oi1c~SKshib4&@fL+Il&fbd}P}!y6}! zetGM=(TaBGJ0VL%HmrV;JLHYNR$GvFm9y{PtDAs4Vf~P1hXzQg_HT=sVQe@_ z>Nes3?|Fq}=qIqT0Tm;lf0gR$B$qnKrMy51Fc6-<>`=&QHDWSa>i7h0jRMXQJ_1S_ zV!#8io8T4ranTJ}j9xHE#uZK30y_c=fuRCL?q|q-c$$2-_A`Q181uCZG8XP3tZOxT-X*&*e9sIq4mQsI&ub%tD#^wLR~o3_F;3_<}5gEuF|=Adso zg2^EVCj#*)3d~SqnopctWMD`nI~PDq#cM&+-KneqY6CC|76Q?$B#E~-qShtzBvuJ2 zHgu*a8c!Vgo!fAUZ0H+laXAnXd>d}X<)d>g7;N>&C5w?zwA{%a!QciPP&0QRjej1n zF7}0&*&|7}V1Ts4s~i_zuA!+8d8*C=yNrxFE@| z7i+gEwQ(Xa7c1J(45;6Q{)*hBPf=w>5m-ZFnL}l(70Yr|hsY2~+T}%}O2AY&SaAY!cfe^+PdRX4(ui1~>f>(IBxX;o0)pNEU2X;G-bmj=OQJB*Z4r0!xf_&s(npT+F%5kZ=wFmL+g*0%CyHEZt@`7s(G270F)t4Qw-TLOvRWo2dYsS{cg ze=v!byRH~hTeg6PO&-~%B%Ug=7x>wHcjW3M1^DTn%h^o4m|D!%@MG%2E3HJXy<86X zP>13_elAwidj$7%=Btuw+TDa2{8q7p7sR?k9;{`6pJzYhm^i;Ks2EHse7z_d-s(dt z0bLwnmY0J;>NPs(XPA_1-iaKN$(g=GHlj#uc*eVuD4-01I*ze6s#Z2eUEZ<_vqMB2 z;}KG!6EsBG)K%gIrJ-x2&~(oaQc!C*2r(eNONyTY&k9bp^jH3LgPOA|lR?iiwjO<> z7Lc@#J$503CPH?z-Zu;r9$kKPF~S7u0uu;*oxCX;4xC7pRh2zvS(o81H{<+-YW${e z(r6j&j>yE*8Q$sa36asjX4p~vrMe}3&mrq@N@EgG1~LAL%d5c|5D9JI9X5Cx9@`}k zWvp9M|E3HOK4h{G1W?62+XKehXr~TDBjhc%T142d6V|>s^im9LmA;HNoQ{{55fO;W zTSXb94gg@C0rYp#&v&xzVJ+cnj)0T=MNz@Y0M(Wl8C4GglPm)zZ2!9@HN;jJT08X0#08mQ<1QY-W2nYb;CjU?X00000 z000000000f0001RaC9$bZ*FvRFGyuDfB%62JkA2X#Q+lD0FE}Gsnhr|lU3_+TMfUk&`26B zfl@Bsa$B--3L%LafTV-CMe2;rq`LGH#uHUe6)m$Ez4T5= zFRmYW`|5;pzqI;%F7Q!}W{3;3SLzcmKLu!w+Hx-AlpcLdYf+9?Wu>Ej5dFCv4sbnq zGF?ifa^{PFq1W@YPy=jluQbc^d2L9UwERTvN{H`kPiZNnos8f~Ks*+1$KFFHuNJ${$vfN~To|>6|Gi;%CL< zJjNE*>T`Ce?-bl<5mw^e-T_kG7M@NWgiDKg_Z!?cDy(YoGagK({B7u+s>hvWpK2?N z&$5w>$2PHoed@^+q<%ofYptU$|F{VO^Yry@2%x%UmFlFkO+c$VzS}$_p||xd5pJ zRSXZLBa*}eoTc3z`VuJmM&vJ1yua-a!usZ>7XM7D2-R}v)8zLbq}Day@Ki}=wNy(N zvr#QWhlL}+y2I_Dkq*56WkzJ@p_s~fis_hHbR?QcnBdr>nag2hpTf_hnry$v;7?<4 zMayub+J`Ej|Ax~nf(N8+=C+871e2ILnAt^}57#RNT#cQf;a+3L6qjt^75I=g$1E#5 zr7`naf6LESYhZu`B5?@1n~F76+5|=*05II^W?%N7$csa_pN0r6ZO+lRlA4(*&wWut z6jyh1UkaMXWz+-0uvjVq5U?lFC)}nahG#!NiQqpqhke|?CX}0vnCplRDG;&2ZJar> zLd7vWD%|*ZZCQ>3?+|#X*tHV|Oe%4*N|u%4E{cv<ZI26n!!}5`rQa}vFP?B}MEdiuVgv9p)G{%1L ze4mn-L)FTZfL^3XSIn@wjeRMs6@8F|Y2kzC+r?I^feOXGW4yE9pWScf$;a{5HeJC0 zIY7q0h9#kybWp;(A#HiYVrNW7HHPO12KnzC3z6bL+eSlR6v^rd){PcBy)p6#^;*NJ zr5VUd7C;U%2?a!~Kb5!lP-qqVr zTFCF0m4AI;?-VeVq(LY}oztTkKMZ}o(4emoL)4!iGKp{kzND`MpNH{qX>l6xeTBDu zFGx>QRbv=9CA-ttD_=qOoy!cr7(WK!k=vqezK!Wv>talB-y*Zeh+59sVNPA!+>`fj zs~cJ;_G?#4+3n(uULc8LzE;Y&Z6qb<$KHo}h6skeW1yH>d_iJ%#hjm(uveRV7ee!9al*cKfaPxMg%w_%Jy#4wcI!PCtDIb z|Do6c!sH(3z;MS>BY<`dzQB(Bype&acY-~?CBbd6ZZ)pB_vDg}urfeE*0Eo<;|%i( zjF#aOUiIL;<>=$LR^K}>m6=j`Gz03_2ft%8eTZZsH}OfEaYES!q$;G6)PZUUe$G)K z66|1VuT^!AQcs!ld*HFrdG^SXOCrkg@Wpvm!;=hgg?-u|+1|@`i@}4OZL`BGoUU~p7KRRp)DpPZ?miFov3hSLJl7VYm7Jc~;>vS!`q+~okhLH%V0cB%c_ zUZdbIdVPr;p=Vx_m>4x>pj&BZsj2#GH1>fYEDhRW>zspJP-T3ZdyZ7igwi;Dg&$G9 zow};wRN|#7y<3tY+nuUkDouVAINBP$RcV>(Yo5~j+Om-p!`g-BtSwC)jpbfpTyet} zX)d~DSSsyIpLkJe2DOrCugHBwT};z@J7f~wR~{=G3wej{>EU#6!poLOxF3zNg zggAJgXEAOLAv7tJk_==Yey;NQx!GLFYP5xQwu|>I;k1Gcbz$sS4bz+;L#h?_64qhI z*wv&+Q#RqTtf$TJ@ApS}eRU_`HfS#zK=l$1lm3l35T<`F<{B|x2HG4J5jLun&#qs` z$dMBkFReWng3Jh87OLBFKt+v)eGeA*^Bo5V7|DwYL07|EL~bG#Eh$Ihvo3uw2JuS2 z5l;qDOsjNj7_R!ZxP!+x8@*Bu5a_;8d}uQ_u0!;BJ3Z}cmqdnN2PZHyX_o0Wa3An{ zxHvdkT>F0oq!NX?87JS$XX2xLsb<-Z570-^H zT7&ekpeBVlyjVw=FZ=ib^1p1=fCfkwk?~<62 zxdF_`!o_5t5P{fEQtm7i)roWoq6wlB!3o96B^WOIFp-jI)TZb|Vc3#Nm>95+&{(jH z@^umY&rp}O&?x{4(FZ7G(MXQuQH*VJKKyH*9s5*XZXdEejPMh`hnKm72{df9gJks* z!IbYDE)+@>O@88|akBOPm+`?aCpV|iH>adGr|^5!RH%_``e3e`w5$NyouuI)EF!x- zZ{n!lZ~5PdviEKGJCl(^aLNAhbjd-1X-EI?JZB7bQf7Y)5JDTuxRMOGU?4V=_7_ED zf}SDMAjDh51?u}2nF4Sbj`$p2N|3F z9$I3AAjSk4h?L~;O*?0{q(m+SVpW6>l~HgjX09JF%vad&6lQ%}My9+TtZ#f_awgY` zh(pE!`)B;S{JMGk*#42C7!XaGY&Y}c3(DoGr#j>By^JI6%<eJqA$DgH5_}8KR7tWCp>aXy(KRA8?L4aexXAi=UmatcqUy-M4kh)oBnEh=1|JOcLQo9Gk_o!){i<@*iW@2#f|qL5Q9 zGy7o$HsBu4$7V*Aq7w=Pulv{TUN${aj&2TTOcw@(Qian(jYeievX%!Fi20bjSY|85 zCPnZnv>51t7LFDhsZX(Z52}YA8l6RYA5jyJXJ8078yXu2GeIH{@6|#yW*gqPuXYhs!90-bC6;+!X&rAK>8w9JeO)+mpJd7S{ls<^bthP%Va zm6`{0G7}qC+2z=!-A81a4DU5YB^_nWA~nN>G@4v7%AcY!IDx>fh7!z@baU(#s6kB9 z1M;51?COz%jc`vPZ&1gh4KVJI!1VN|DHg@oK4pgNNnIbQc03ITRTdJvcQ&S)j1(in zhQ?VjWxuTyj)Q`NhRb4)qgiqIgdj>`e}^mlNMo5QkM)f{8zrel{j&wDTg94&^UK%^ zpEhXbapI%kcdA%%f6BExsYK2z@st$KaAt&uT_$Qk9_cuj<=UYO|00{sJkDF#WHl1j zJOh?$j7`5T41+5VRwJ;V1z(b(9O{<`kL7`8fKIr?0=erRX9B#&K|oLIF|Hx@oc>?s z@gQyCX`fjXAekE1d|H)bj#3^E*&BX3Z&u`w0e}{bIumf1t92`r(VTHkGLAui%$jee zS3!>x>rW&tE74k5u!EVknzk@8<{93pRxM-<+BcbN;z!NKI!gW+no+0@8kaaSAHN5g z^pgI}PhxrH#c1#fbQz2>KaUm(>`j;~$HQ2A{yGjcam^<(b_bF(fi-N2{vXWfOey+< z@FS*nZ~ELKm`OMWz_(fS@I$RK81yP@Vs+QaQ4d|p<$m&E4Cr#Wkw^MhN`6w^h$ZNI zq7o`LD5fMSgdXxT2P8KU%IGyaM0p9eCYy1g=eNf8q0j)wZm0lt_^aZCKAKa$Ka#-o zjhUNs*(nsgqY$*V2M0e8^l8K@Ro|j#>`+z$lO8@HYAbN`$a?G zNaMo*+A=8-dHm5B5kJG3qhz!qI?E!-OregtbY;l|Q^kRJd6MN!68qD?vSlmE%ox#3 z?z0peV*een%F_!TMubMWC=^MZQFlm^^pFRcS1LI98E3b1r|U^}s-%|HK@8keTf6R8 zeq%2mUza~uW$+cjVbyQ`Q4{en6)<_BE1*peh(0(X1WEgkH!H|amTKi~j5o{4u2lx7 zwAMXQGMfFRWbDC3gaq+CqUX~C)ufTk!s6BSi{d4TN>X2A4e>?L2W%Imqt5YJ)HG)y z378>J|4PX(&T6hCxHA|<5qR*zV5U*rLyjU@+J4$@n@^qK-8o^C*qv1627HRjeEBem zrhOfN-MZ*=({my!qE3Os1!Oxr* z!lGL%%&dkz2HIt_8td(})jcuQsM?d7t7B2Y_-vnLlk#DsrQ~d!zn=ArU#=$nX(_f4 zH>odTkEb7jQHDygaAckZeBO;a#gy;)ut8~xe|Ei`uLIV8?JVB~kNwg-_!C6vyqI0J zcuD^-1A0B-D!TC$x@Z?VhF*B_`r8De9VLg%s9&ni(NP)Ouww9TnQp8QMs^$B-JsUQ7Q$ibaY#Hdda0*Uk_a%K;zD20So%8665sNw%a! zfVhc$uBC&91N$sWv_1c3aG?D+$Fa-Uu-jxbVvrABr_^wD-w7p)Y!+;4#WBi)t-uApV9PPjROBWn?lRw119`^m-wFk;*cy4_Tjz%lf>+} ze)MEvP8iGgow87t@BO9UX(vWk9+1%fmAJDmqB-9q8od|4R8?zlcB+quu^GN(W*Ysn z=cv?HvbFR-2z$fQPbUFh9J|P5Fa~k%@z3+GWvD2a@@=6pMJ?oF1*Bfzpac(7M7K)yj#4Hoc1VC2$_SFGs3>>w9u8wJ zY>hbaJ6R?e5p10ZqD_FXEd8L}D5_LpM8UdX>YY18p0HgMpZwhC2rRjfu+k2AgB<&F zQ^I0%GZZ9A40Y3WYF+z6KonMuuT3^Wh0D;vDZYRK#Vo3TIYl)*L!|%C|Vo4P)XkAIJ`ppW9gPO^~*&5?XGRBkTX@eSAwjLu(ILwGMhnW z1cQaXayA$#;VE+XP~oSUdw!r zPR;ByH=GbPr(a`bM##zxX1;5AJeDUlL%yIvYscdGcR@@e2z^Y1iyC{NwKz**xK}9g z!D4U-EM6wae_%1$P~-Ub2R7!1hx+2tO>O_T%Haoguhz-SlW0 z<)4@Koa_lJqKur4e4--_A5WTiA`2e0%{v?_JP|rxm9hEaQiKpHwhNaUP~-<$J`?Un zpsxdEGo<4Do|Q2M;fD}QydgC4wY3j5p_+SoO|@ zd<8e$_ScS=a};L;3;15>2frn?6Zm;-GM!b0!gf#Z)*47<7pn%J>}Q58d050SgZIca zPj3^`^6q9ZxfJe{maW;Gw$TaHJ+Ugsh4N6mJhmv987o@SXa9h@xtcPS_(}f2x{Z$a z2l^l0YsA4w&9-hpLFyt}DL;hbOnG$^+Ipe9>Y zc^e>7JlijL8CJ29XB#aB>1i}&}VdA)|il)aVk1{jKb30 z(nk1d6&M;&fWA2mBP13`0w_EBQX+#*#96j0QpKmW5T#8L7?dT@xLuXu%q%cfvVm1r z?WET`wc{B6tQ`L3z`2O6AH5@8Cd!_uTtjS3C+xgbnziv;E3Iy_fP6CelPNtaL&!Dn zWf1elLU>9BnZL~6yH(~Q`Iw&awjb?tXESN5Eao?Q*!QCC!oy{Mq1)rCCDPnXrH3w- z5Mz1~f|R<|FJ8PGGuF@MW?3fl9s<;kGHctYqcq$G|4~mIH>^?5 ze+*erlvJGI)^z<+HlT-{`{qcrlC^VGqp&6oGoW`ib6Z|<*dx1Trs(>-C%$Qo#lnI{ z#qDKEuFz9}*#g<__^&gcGN*D0);>%7*UYj~ysF_zMl1v6obTJ0AwVVm^Pu7t`K^v? zn(hY)MAJAH&ZfJZA!;{Ze=UR-<)%oAkWfpvMYq1$)dV%ROM#rdIozd=3&Mw$W#DPO z%M|{zWJ|0fWiZCTPI7=T{D8KLgf(exEP#cD>RMkfZ9uxe5FTausGFTg96rtKoi-a) z%#pnZ$0Lt62{PG9jR6%v!s6wl?$>2+Q$sD3xv$#p+}2xc9DmC@`c(!Ee^P}`-&cza zfd!1x%fV$Y)W4BbZ1c}O^TA6AYUwUw{|#Kx2XtYUoqg6Tg7zbRTArPyogEsWHt=xj zqF|HcCFB{4kAIEe8{O$wus*AwH$5=Ge2~FKD}B!9X=~(+`m|~>MF2QDBuvL(RfAIBPl%(K$j+Je=Jb7e3T}gZ}oPD~4B2#Z&W)``OERvL3mlf*W&+L=~YXURg4i zZ15wOEryC4Ez9>c?A)9yG(67ZTt;3ftR*=_-HkJf(ZRDAFW~V4pUr(%O~nkqo($pq z=SXNi>BMl8yD(@f&}M57O1NY3!^@>ReE&?MExj3PUMm(@9k0BnjHjkR-f_c;szy@e zFofg%HDrf|c1@KcS`iIlax=t~_l^vGC-};IA8mvdP*Z%1X8QZS_g*8M4Vn|2O*IufC{bnoH zpdlw;#b-*;xIqk_L9=qcGu13~Sq$c(e$XmppQYek5Y?hjuISWkFeE|+zT)E+Q>iv< zsV$8nkCef-B)n{iS$ZoBpSH9ub{2vzOI@mf+|%L)`U?JArZ(lJxfIrG=Pko;Nles0 z#aD#T!%kQ$Acq@>bbHpiCa&>=Kz8i9D>&@vK4zM(X0o~lA?N@x@qz|>G4_B_9BGam zZ3rLH1)rfqqFB2K7ADOEo8bD3>eKI^^^Mx9?mogAPG=jE=ncfqT?OM_heJ>;VXIIy zQ%?PDk76g?3qaT0jIJ*wG>~_Q&4!`pOtGakIon<-mJ_cuv>D5aQx59}7CeDx2;?FL zCj&bC>rcjxcGB?Li_=sMHNWyNy-O2k(Amu{=9<}4Na9<0T@2Xu-At(aTNNVQ1xQ~Q z4Q%8T4^e%t@$jb)Y+G77uQoakomS$IcI$)oXwvNvc~uINl_>4#>cZ-v3hg`Z0IP}< zS%fUC0 zviL=l2xjjtbc#;6X?$RlljmQ^R3orGM{x|uAIQiwxcZ|cv=;*XXi6Xo^e0o&Um&fMzB%@AyI@j!$%1dzFPUb z##|Ml+;Xw%WhIK$D`7`*>1YviK8C1lVo4z^ZW%I**N-H!A%8LX?&DWiXbAD#6%5d zc2Eqr4g2eX^VG+2P{RHmZ@Ou%OJy%^s@@%XdU0ucK7c84yIa$Hc`m`K>T0|wi;iPf z(8}RnPqg@1vlMT7x`Mr<{WFwkItL|@N<;sFFciM2Yf_A~)2>*Q&Ge+O`Ey}`bLEV* zuh!bdK`SCC&bjW}r$OTEcEWYCQ;Z8I9?|LzE4!+MC5_Ot*tPNoz@JCsX<{Ft~jH6ZwFmsubg8r;nBJD za;rIz@d(bMzKRyw+b=j!B~!UmKqCFcWz3vXopGL5FLYF`P|$RmT2*&5UFV| zv2^vUGq;(a#ZQGun|0Uhyn?Jg@c|B}3Naq2c7Drc6w6?gjl5ThiD*%do3MUbdV7&v z<`br0Y?dja)BG)0zwTIdhat0$7XMP}=Er82_CVn`a92<17&kT6+~G%-iEk!9^zI6kc?`WNT1qH0L7``%f9ddqQHJ!ga9EiM9;Sc#@W>|TL~JXWg`_^rNx@K^nr4VOhHiwiC8kQ zcO-KdsKt@X^JVW{uWwsD!4GJZC4+k$)Vyrp?5$_U- z1u3krg!=xZqvRcY^rLJ!QLBU@dh3dr#>_6*US}s>N*5LotPrYdy zW@|@6-OssgW4Yw|XNA7n)1>e70Akv1O4FP>-20K>{2121!cog=sC(GScKuO$te4xz zbKyLt`+1|0;~Ph1mZ|;JNNc~!HB-o*(xz_;<$Jq@jo12tt^W))kK6(IVPENlVASHv9UpG6beh(sB@ck&&9zM&bA&*GnH)_7qB z$@;o}dx1}x13*6AqhvMBBh`OQVwHUv);|CqdY_F8CWsxjDKT>4MSzQBCRACBTvBOD zY)6wLnnPEKCL0+4I!MJBZ5?k$)+G=K1keWMu!_VC4glUTkeQb2*aSaCiBo<-f!P!H zll2-k*d%m<>Wq8~tn$3l9emn~XE9|lspl$8FfI{QEw1*xqheSvhC6D3*4S;W037U8 zEClRiGatu0hHqX9Naq+-=9;L&YfvSvLqb&F*Ds?kO3fE=l~+QgZD4so;fIk$%y3vZ zr9$6QkTIvf{H8^UhUwbN+~h>OP!6R4$z^NSj_~S7BdM53oMIAnz}}1XMj{qf6Bjqd z*ab!bVO?|~4U-xiY~cLZ2MO(BEXZ5>@chH>WrznL3zDx&|4{o29P$lC@(a#Knwim; zkQ6mC`y5+>%g@%RukG&P*(T7pcKu<+0L+#9YT;06ppUW$Z-58Je!9JwizIby>QtiP z+4ez*Fh4cEm**Y;MpAr~eL3dM}SN-0k+kNHUM~~1I2du*n|u)|KG47G!1ngnJvPY2{nBr2vQ3M<1>R~CvdqQ5 z^!L>TVZ{~OB%ILG5i*y;4c2CqpFtA7rx=6=hpXdR-g?Z0;iOY1sR`4HC-tA}GW5#r zNT$X&e$~4$wsXo<4eh!J&?%ucsLWBBpl?ABVPEnpWhTpvP+`T(-BVghOyw95uly=q zfCi<@^CyMUK3=>wbc+` zGH>S)v<16i8s}ZK;E%0)u}IR>yCc?(yCaOlQcwhpKJlwJt>7dO1S8%rv^KzkaRu?< z8|@%YP|FJ5_ZVt6Q9?sy_&5E1$Uj?LrT|*HGMa3U!aN$;qdA*&SF=%Q01uUT0R|)F zyO5vuM*+pm!l_fKZW9;9C${dlB}fu=@)3sEZ?ucp`#8`}cS|^CbWi+Z#%5VgmXj3g zy=pV&Y;Fz2Xt*qF(CsHFE6NTfti6HbgGyUs;M zB-AzwL_SOPG9R)lZPMKsf(d!$SnY5UxxI6 zjSk04B@Qmt#4~ZgKqphwEyJM#8Z$u#EzJbq$Q#M$Of=pfOK0 z%$sCHvK#-1AD;>@9{F>;9OTRD$JvdRM_5~2F~A*oF21Vkehi@6^o11YLA)bRT7TPn0qXqviu z-`u$B^H{u2&-E~rovr)%fh3T@REaYL>H1`2NRTnwRcS|n^$BI+V8VK}jE=Tu5~{yv z7@x@7H@>(PWczZZCR7P6Fo$n=b)u*N?q zfWL2^-)-e4dpNs;h|-mYhc=dgjc_br523a^70`AjI{Z3*lf^t906)yDsy!QbLC1ju z`c*-ceaM+V|8I%@EgSO8>4RA>IlnOCyE$*mCzJ2UHxwsFo|dnL6IHL%mk@aiqlnt@ z>#tuJjl%Z7mp?aG#$E8O7|Po{w;%qHDBG6gtqu3}tt09mVg36DG<}q0KJyAO%5oPr zI680%GAK#Xh;p&9bzZy3Vq7ojilhh|AiHzn1%-GCp=S`3ICt-4M5g@?O|rOeMEEk+ zD6BKXK|;G8+~hPnIavecnb4#(hn|#Oj%59<;DlTC)OpU#Vr_+@W158p zg}Sm6Y-^lXk&+oLB{OVcCv>)!8NpqHNdr3tE1VLzODMrmphyR+`NAw$feYQ){0Gz6 zVkd@?RsxnoIzzPrJ|=mwy*x=eZj8lcnI>V(+AMD;hpk666heS=u%UxlQ)u-UWk;Xs zlEc{AtH`hc73YSiN=C==xvE9>mr}Yl#C|i0ktTdoIP|9s5bXM8l2N@xe9nxI+7#kY zKul~k!MwzG!{GNC1H7D^1JYXpF<~sK^LJx{c?VByby6&X5YbhCEs51~klMzpp%E`k zc^n?QbrG+F`OnB~Z@3bc^Ul>>FH$8=Y>D9@>$)I4=(Ms4?c0koKQF&lWXQRUfli_` zGjT3Vvn+D3ToJM>DyyLWvBlhg-^&+fN5fWCy$A^sp|A`dZv)?pqR#r#JojVoNX@Oh zHMKxVelF(k-CY;=#va|#nS?7{Ib%l%B&OzHxwDBLQW{|Eu)7nsVqhOp3LPlJv^^S1 zPebLJAJ%|wi8;h<{G_X3=}_n)4_S|I{>2iF!j9|E6vnsulGGjk69*vx9&jQ*n#y?h_S5fuiS7^jEYG4X@p`2%21N(e2>wr zKiE!I-_WB0bHyM>Db}|M$(sH@lFV(tp|zn3!hP{WIRfG^FSOH=S{6U*ogf`(TqA#U z7D|CFPT$xytmm3m?X=k3pzGM~*GEN@Dwk2&qtxITBG*kPpOok^J?8fB=)6lEz@l^> zh`MZwK~!L!-`SDQVKK0r9u#EI($?XVM;ZdHZ^U-|Ipc>F=pN7@)LD!aqI35P77mRv z8IR3;Hk4nLr1W$eG;Fygp{nKZaC@^(@WRKcI@G{exL7nB$$^e4ACWphJ*cSTW0Qo01EP zCf)^w=*l#e==tdi@?UZT&|AcGBogFFZB8S~>co^Mv(j|ehZk5E&Q@e2HLetoEPq+-}mVmwXR1P|ojPjC*KNXlgu!x6g%Y{O0O z$;s4+D#iE*bRZa$ItoxllKyOUk!vGBPZg_G<~X21BuIKaf~CN5zM|Vh$3Nw`T>`Ll ztQKP%s+#IaZ=h_{ilU{SSo;mY#m+=EsX}&*>dgle()jz3o}|?5Is|QOfR}r6)h7rF zmhg#Tq6g*+EMg5Nr$WdulJm)DQfx~Q2=tGd(yJ0M6j)sTLcCc*W6=WNVwU9dL6*Zv zL^zLgyno$Z&LIG6tOKqJygqgwwS>l-qvF*~K&2gA6(Q`24i0^m9Ih*Jgd0q ziutCOoHMI%0)?GUIPsW!E_=eYFZe81^7I6ZYnAIB9N4{&g%<(PvGrm@kef$J&T#F4 z66#tLm9EB-u5vIK*DREH6_t1ug2fo*?HTm)aWxYja&2qrF-U4Lpo&s`x{E{{gn^U- z<#|ROg=o_FcnXUx1NplO=dR)N0eAuiwqz@8;Zgg*%aIZV9-IkP(&Ig<`K|o8w#=O} z^vB@?k^$N;KFFDGFG#+bvLGcByy6@8Vg(FNTCUl#%Wm1Fd zL5&Zk{3(3Zmldh?0gn~CarY1{FkQBcfe53#mI#@rN;Z(=w=8<)ji^L;51Vd(_mH(^ zY@w`jWHpWSm?7@cY;a9tW$;3j7O7~~paO$#{y=or$f+q}s!XVu;UsZ_eO98~CO1Ka zCGv+$9B`I4X6|JLtU4dx1l&|#nyEBA5n1|)z%gJ6t zS)%f@#L%IXGM|I0!uP+|jE<5{0X2mlIWR;z6dUNLf79+M?pz)E;pOw|a{_yD&Z2{p z>$!^LBe85oO6iK1PMyqJ2`r&@64;XXy=$HiQuux*uXpm>9MZf#2!bh7*(mhI`-5F#8!*W4e5b?DX2fkU#5tQoKolRlq*>Z3r?#hq>4yH$$;$$%`C4_9)%geQ|%F95>&QDt2tAAe&qgFPUJiBuccFgGmFLp++OSXyqI?)u@#YHTq18@A)jMUN!R%0rsm35VP-*;gLO zbYkx>pvPxVk8pMN@xub~Y|xhs0V|rdzjVP{cBrk=feu+2y>xI~GB|ah+~`i%Hhr}G2ykZ~ z&nujaSj7`NtXTD9Wp5-)yOsG*1UgGGM!Go=2h{DJ;7S9xD>|bz;V>xn4O%s(N>LDU z7$t)GT9|E=sk?+A{af=4>IlTXRA0QGw707>APc8F{?m5lcQDV?1NG3=X@RoYZ*|Y6 zJq$NGy>o>O+4pF19_^%PCQF8(TQO1BO!3k?gkyH~xF;@Q4H-st;bI-tNrIN1b%T_; z6h*hw8$$%H&oo1Md|u3z5Y^}P==Blnj_b}c1biSxOO^Z|_-@(`hYFfL6a^*_MLy{WRswGvz|R;1RA$U4Y}DkF=Nr0WkUjGE9@cnZ~FXBi9-8_U6j z!m@k6WAziGUVgI(kd5RljyayKYJ{A_>;Z}@1Jn%_~ zeTB@)GlXLathlP!)H2{4JE|Fy124%q(a2{nogo<1%Yym(3dd~u{gZ5HhAdrsxx-z@ z)s;aaHLG+IIqrAW6cGzJ*d|*TVo&s@@i3(j`?OMkP>u?ajNGJOJh8=-ORbF>4ON0sSCL1AV}}@4c|d&k+38@0!9R@?naIFE$$LaLAG}w7Wv?RnDHZ$x#0C)MO}#tLH|z z7JKsCcpkD8@fR(q9GDLCu*;R&-u|a)6tl7WH&ZoEh2<=aB#!^8L_1TurQ5O#Y6?wX z_0wM|HTHN{@`Dl$cJR-MbH*X*rkP<*xYYpY38+;$I!R8d>f51>_B?DWERT&i}`EM zlRS1TwVW3YpjoY+eRN*)^Yh3Gt8-0vs1`<8NVV%XfiNa_m=_JBbVu{oe8%eXoJ6*g zMaxC2Fxp8qEe7?)=@A&|5k9Bv?`tBx)EFI;ksx}i0Gw|URJw!YGvS6-Lp`2;Bhjvb z@^yipTD!A7bYlu&>O)@Nzs`k)dntnW_>xM#TvMAy;V^h8n*xdqDN?opid*(l_DPG@`6p#vzNB37-%ADQ&r$a~BtCP9?q zDK3TQhfy$YZ85IbmhIWzGb7+~W%zCKQ7KJ5c8$Y0}CzheXmXr|cAQeD4R>hJI%nCm-t#-oUr{sRLf!##?46iF@f3OSbZ-yFh}Yj5C_{mL zRtY>)HeA0giTDe{q|@AeC96n2JO21{1_FQrHZasS%$@C;&V{C90i2moo8ex^9m_Bz zYgOahQJ{=kcl>bP_g9_MwdsN;0BT zp%5?sSKbI~+GA`nKI9s#>~OE9+F|3r?u!>;83Z%wwtu=F;Pn#Ag55< zd1Mz$OVfvX7OQ3$885z2vj)qffW_ly=#kS)r0lZ#{rwVCD z&^bJK5$L7}68km*C_kD8#btenHX=?+D=iJwMmSNcxxkpCmK; zgRo#u@a|E(49x<@72GG!vuwiPiIB$od3CfBrTC>^gF{7`!1%-6AN44j!x-VZSagze zpA<5D2^sP2q0a00aLX>>3QcAP@IH0D`}54=-CfRkGaKd#jVCy(Zcym=79!E*-4qdOI@+o%^g*BkkK(9t$PI!W%N z&5D)VIWtm`7e5QL4bs`pF7L9-SuNTkp^agqI0f|^bM3k5*N=cZsr3l=T~ohnEmZi} z1oKm%;~7(55duPD@fV2@y_Th1Z7p8X_3E$ts;m1-oD| zeCy~-s0UwOj>5RCfVE4b=>#cF?f*dSQK1x^_;h;l}z6 z?|9g3<%EV~dgac~H5@p5E>EsnyqP$01FCp`jE`s_mflPEX(2G^Rm&2KolB&j%9|9()wK zKT1Rwbb*~NH`>6As&O3mZNo-FBzm?Y#nJjV(WXZmN_NIi;3Mk=5b9bi`c*8K&VI>z z9)JfqEL72B&%8l_a|xQFa$56?`i;=3y^DB0l4dX;&2A>DN*_|GC|_(@zEW+5jc60k z#gu5pNsS9gpmtkIlGI5Fi~_Vm_!N3?FV z;8Tz8m*#sQWRPan3`?SO9*#;>#_8$)%ghcw56mssY<$&rZ_bW@)B37s&FwQzdC|vT zPezf`STV{dJ{Z@yJ0I4-EDoiGN?yKXJwO>@20nQdN-UaW6d?Zzy9+Ecn7WKgU?N32 zs%#{WE=FU+n%<^iPA`uo#d4pzbvkqk}xG{wPt5O z!`N>&#*ECbDSn@JVAo<3kzH|cSSViVu|Qw{yb72X7OzsBhg#G?`z8Rp*)$IiADu#- zBhHo`a9Gr9#z_}3uyszMQ%Tc%ahy}r!1)=9?Ge)=ivftkajrU=tonFAvCI`;xZZeKKc(1?3+p z4jT?5z`B!2!ncB7Ip6$AZH4)h6gxlZ2IC$kLlG^K6Fg!xIu_4I)bjW9?RCBqG1i$8+|SNEU#WHu)__ub_aCnAgT(70viw=(r-01r(Uoo0gz4 z>bb`Uqesc#M(VMq4unAE^`VqaV<8eoC@dX9CyG1rS@GS9xzg4HOb=Pyv3#9p7OP^F zyJ{!YyO+P9p$oE=S9Bz(EEoP@_w*8*u_gnU>i~j5F@SCgX=MRO!u>Mq$s5H2+kQ>} zWWYd_J%API+o1I6@S4M{G3pMzSy-Sh+}7Pt(AfixcQsf6&$))BOyPF{AP(vf2XYB6 zkmDbnlI(+!6yJhC(*Q?6xW77Mjjr(9#*!X0>Rb)Z+j;(BgDmwY`aE)`^!7 zOfDgJpFQ1AZk6EqRIx4s-|fIe7dapY%7^Yht1Sjh2{fF)3$${@ldT7h~5fD*twh_l5?mT zrwk}o9SLS^uc<6oMf!&@DF0*UAwT{4SM)1$2R(Bw_DKvTSVy9Fm3K=boJX|4Nb5$l zS=L>?WkcH^LBM68=FRAxjL`%$K++jjXfjBCgF*uX`v_{dR={WYa$Y1R&-PBrEMH0# ze>hvJf0eT$tjq!uv^5R{hUOqk$}BhWalW5oR+KAN9;XojfL}8g^nO^3`jKTMG)jUw zoQ_apW%41SayHV>;oX^~*Eo&0CEP@OkZX*$61-w^6TalO%VsL{732?F7CI)z$^ZR(8*RggSoVaE~{>;fEjM{|Aw*S~KN ziUbf{gf9gqmWZzav-=JEJ+uR;zoKC~);|OA;%6vM> zg3G2)uIR*hMwvGAS$cD6CsH63FlNADC<0o@j(_&o$Y7P;K6x2jLrOZo|D$Jvt^bkG zc|QpRxR>UwETSbJ<{Aj%*|z>P)om;&#|FcyukKA>{Y&EYZRUWB!Xh`_i?9l!JHR4v zFLO-aO=>?Li%2XsS;d)Q1l!&?Cu%d_(#7ZEBxE@a@vKR(&-NRt*MRA}x7nGhp>o|D z?F|dn(Li`ZmO|y=*}}zOOYIXB?8h7!Y4SlB7B^ZwDZdjmc~CPG9_tzrpHU>K%=D+n zV|?DopG1*n4}7;J0~jq{!!vR@mblHrk+H+U95|~c zL`SEa0Gq^nFUQa?t_>Lq#`XywZ{~)&-~j;T#xyx_lX%wnJOjBmGexM!;E3b$J|ivk zYjAujpv{tLNPP$%XlN?LgrR&(8cfSC(IyHF4gh1NL(PkatOgg3p*)zUmj>|Fu<7tZQG!5VbS0{D6r1m2*2@6hPrLpT<{LB-HQ zcoK5H1}9Wvp?bK&-huJSYC-MaL}P9Ja6a+}@RhZ#TEyzuv$4BQ_wmF_%37%tgQklw z0QB0>SP=bDNQ4L{r>*T-D|b%QDz=MCY0kHUUZG9@q(O~r+qNrhiSSI`+UM8Xf za%sE>F9=j0A6s?=!7{oDlc-yX#pXx&6Cs&F(=oc6??a)}!2n91TaCs#^N*$4?80CX zR9Y4y1cj@27N4Z6g*qn11ss9j7)RP}YoQEwY)JRkh3FkjvJIeli+-_$4YDX+aO%`e zOc3uwUT|(KdbSg_tUtwRSmAU{sC8#6rguzxfrH-`YorQgkWz5ya#Q;42##=W)I`$& z#NZnxn}Sa>(!#_*bQ5_JsKf5Pt1XubgRd}%4DiOuu*gmvt- zQ_-!Krl2`wV@&M7zi?U-t9jRw%B>KfK|#gE0`T8Z$_Q8Qn<(u4;&@e{hRS2)7M=4o}i}D48ZZ!!8SRYC*3-+B>mb@@FQSX2o3iV=@YZ4isK&5B~ z&(W`TcCc;+r-Q6wMt_?ICa~@YAo9>*f3}m%=KEzeCxJNQsqa*tX8!gAtV->jm_;5P ztoDv_(egQ)W+gpe$dHww{SE)$zFpOjddf1M!YiGw;r4dN1zVBB(!?^6!$8kJ=%abX zZ4Xfy4v5@mK*QhsXMcYL36t@ee8iz5$rkP*eZ7%uhFc_mVS%O%@o^jcdpCgQp7?_= zp@>_2H^=!~sjkhUOo=ESNigI^PXq!4C(!mcX}@uj4w8U$s8yAR&+X(2&pby)^BHq! zMVDT*7*lT|XpUBoO28NF#^oKl2wBRJ$pw!biF1mFDjE?P#%~_5fhJsK@EvW^TBR*&N$YU_m;RYYm4BMVjX72toZ((y9BG zKTtrD|69XPI{5HcK4OI=Db<5N5pW;hxsy^-h_@G&IH;;=1#lq{3qT$ve#?8VVy%~2SUs@Bpc(8z$JnDrG;$~b@`h6ZzlKs0e9S%TXzEYVuk>h+HFnE$XS zeV7Yy5O#{(zFT&ZdkCTm*r_rAt#m4f{$Oe>QW5C|adnf*z(kD2p<0&GehS|kohb#CEh&?zx=3DwHFJG>UG2}OK2ttaF#|!ar9V1+p%Lrm(oprAWEI26vPy+0eGq%U;YK6 z`4omBS|!gCH2>IiK%g@3=J~J`&i?=ra%z2E%RYe|@sR&ey7KU^)xYnfFVxP|NTpys zMn1I_wCd>YdXTY=!Ccp22B-U!JG(o~#nu{*nJSSL#u)WMho;d37&&rr4sV#Zu_dqt zUNa5{y)iC9b|OFua&0ul`cbf9C(zS4w;X$Uu@x9A_jbS0v)U`}si4 z0^CCa^w_W$_a+01|<+U5$Dh+|gfaNy3_8!S&NUet5X&3e^_mhm+%gc>J=><@xVf2&0i9{sS%> zf(e9CqWy%T56!0@#Dc@x(37FikrOMGfz~lVQxyhQOV@%F!u=Cov9P`|YCYoY0Kujg zP99dq#5q7ZpN$7fhZHogfcV@$_^A(#+iOz z8^1LE8oUzLF;d3WtP_(Q@E*{5{iFaJ?k>d-=WH6{UQ7`%wO|c)mjoT^z{j@L@4BWt;H8y#vf}B4y!JB%u zjA5xAt;>gFf_4Fcvg0E&4(N_dYO4uk-^iTR2% zrr^i(GDvf9e)Mx;7>us&i4@RJ0EcuDDr2~86<{z`C@4X$8^f+nXDcAAM)|Qje46g_M6XTM!UK11>t&7`PImu_wf#^127ea!KW; zNlqu?q-pI2ohF8)4yCt0J<_7w5C>65!h-)@k8pOXD-?JD2qHEiZN1PlNxp^%T0VZT z+~jbw)2SkQ_Sj=En-(465o$2I{J@0b-_*7Ge$xlc6(r+YMC#6i1a6-H5NhsGz|1-5i}H}0%TNq4Sd>1QavCRxbwNiD!b9@JwNZtv;2@4!ssRf%87e}K zKTDk8;p1T&;D*_^aYNhSS;X)}UH?#?c3!0$ByBQGM%yl|3LLr*#`32~PxSzD#9NW0 zwvS4Xwvu>`(G(KVUUTgOXSMv5ES^(o3G#q~N2j=^h?q@3u;;JWpFiz(!CKy`d>y3C z$S6miNs*K=nA+>mkJJdddF^&Ik91m3yOcO+I{C|>1zkT=MY$D~-@UI3BHcjxV41gj zA*qfK^M~Ze{)4b_XN(Pq>xVKoPTcv3J!)_j2rIx4Yz z5$Y1;xBw_6bLV192(HGD$ip(ie9&@`*3RN2$Xf_BlJWezTu|K*sZctH#K^^#j}57(#ER1k{#u zhzB@=Yl+Qp^+dy59{{stz>#?HL)zCugX(8TgTNs$45bEi7&Ui>IoN;`+}cdFg9H6$ zUs~^)fhN9gyf3R(DV(1&r~JmZg6t3j$68u+Wek^21_Q6B49!?p2`=-RIyH|veCzoFyO=AL!HOsoaEW%iVVDFXqx9}pb4ImDU0|2%Wci$e4r7U)(MK^f9ZjwZI7baD9Hy; zD3(kOKV{ zvky(5MwJDWl|IEIP3n=1Tw}Ub-cE7fW{Hj+(BMc?dOHKS#f{1H1igE-9y258oDEXr z8GQtF|3i)Mg`SLEA+vGLbc6YPJaV0F&heU8PJ40oXR zEQG9;ggl@mDa4zgD3x=<rO000t!WraTEXnP=26xdh2rXArfA;)|&H$SvI^ow)m)& z*c94fY?I=^2Nuftt%J>fw|)-5fFBFV0i-*(2L2b26u3;110?$3uTxRU<$%mSE><-u_3z=JbyZ?Ft)a(pqGa zuc{hm{*?KvuTwbji?jHkKT8TcTxf>4Xc6L;_@e{_4#GC{zlW|u2aoZ6QKG=r>u3IgUp^=gI9IA3js`xUaz6>PXGJg{L zS0TkpWLHz3hWQ4tjTY6wQ0v)fFQBrBAJ8jcfaEcd`?2MGPzqP(2=G#c%)zRuRJ+R0 zPm!@JAIcG_ek=cET+mU-U-eZrEAkIplJShS%8lc$4Yw&xGEs5YfZdMD|I9f0fS9eA zVqMNf!N0R79GVFFo9nVh7syJiT=no3PSqdI|~JRxMB+x`t6t<$c6Pw$>8# zNjSXzR!fckN+VP{DF@v6mwSgh-{jB@)Gs0a|0=?*!TPXjCl(O(UUFiM6ny#)^sB&RkIf*~<`wvyd zxuP)mj13h6imYwXAdW;pv!5$X=NxTw;$$*++gfwfkErnQdO)=+npXff!2z;v z-mBOqr%iz0*@-IGQqG=Z-Q-gBc3O)qDcPH)y)5yGfOtt$kTwFP)* zA(xH24Wqgw;(z!xO3wy%_Ov9^s8s*ef=vbJG_gf}UB~8GB2dhkEUjLrg@sk&$)(uP zlW@@RhG+vu`@n;#Ttk9p6<@-Sb=H4eHWlwf_J!ftp4cLr8m12b%Xm|!QXyUoOb9?5 zRyt#Ug;9fB-|m=+`GNoOxwVN|A$)ajm!4F{deJ0Ly;w7Z>e$}Dx@4#h=i#cv6ZcanGl2?6tiZewlav`C$W!>J z@XJ^t6uogBi;qU9kULL+e17v$%M2!)$OVQ;MTR0z;-dzc@(UZY2?s}l6F!K5Nh^|6 z+m{?uG&lISF^Y(ti_A;=TNJEGfUXo(C>o3?FAB1x!wn9@dly)9xKKcWJ*9}(mSYHs zl+FObz5cEZ`O$pfYB(Kdp3CIX&eGMUAMV4wC4`^+%%9O9PawKQYLQ(K9u18q^Q>9`~!D&kXC%;1=*YbsNqLyRftj;N^%7nLd|fQ06=2>&Sx)3 z*f{`0uzs9%Orn_vFL;r}1SBO0 zxf1oDMAtyzI>%k>Z# zL_@?ORy%A{@KCb5YdfGTpCpZP&Kdo5Jt33TLz(8W;t(wbRX%35b}kvE3KC!ZN^7?p-}LYV;^7wzbvpHC%Q?- z>8KHd#cUf0LvAB}gFE=ZpCV}%U@{{%X91j43X%1=7RFk$W7w+!57%VLFyT{!FyPb( zo;r&G4&zsl!p}&^5ceK@Y-bLiv5!FbSsKfbgYfS{`c2sdgaG^zwjS2y|FJ-sKftF@ z(GjsJrM&i+WWT7hXq5+GQhsx}E;`6U?TTBYc7xEc0YA4&dFA3qrZS=p`=`0v1L7DP;jirJ>-BNU>32wfC56#sA`fSn?nE|1P(?l0QYBnqY4>}f@2-bqwKn~4Blb! z;=C9sF}vFlz1&g$5+>%}j=2>!~ zuPp%|^H5osJmY(Q7WYKB2QUw%g)qUjA4MdSSA)b9$b7yd8%I3aG;l#!0|j z|GUd``b1fLO#eKZFGH~~z^zG$;f`&dBg0esi!ZGOY11D(nVl2G%B<$F@JQqNrW0&b zk{ou{#P(DH0}XtW%b2erH{dfLWXT3-fo7adH3|5N$w9oz$v`@bfnNHkRE7;^$8EsL zJHf-tcc`J9@J4xJnH3qe_r*TmGKUj`Ob3#S`Bi{;fIMMyDTdoYzMBo2RBeVA<$Ot; z;fC}l6g<^r(;jxCPzfy1#T1)NHTHiD#?UBlU<6cc{2>q+rh$OZ*f_UrAs1(Appg$o z5Gp7SP7A|F%w*r0g4%D_H=pz1tLYIgeA(H?^KASQ^VDH1jL#(5rw;*CPc$gv5J3n` ztDuW)kCHjkQG+n~r5KKo_g}m~s12V{T9#vxaQ@3J^OW+^@(^q_?kSQMxV4{aEqW$J z$2?NoohBU==)h|rU%cebr)S2CjdNX)U@kceyV)pZ8m%HZC~{@&RnzHz_-y1t-@Ax~ zHb!{H9XgLl_Y2~#5HH@Styduajp5JlhhiFH}jDv6XX$jX?E?_b_(6N2Z1iwYrj zyKpm^X~*H&iDLVI50#6#rogJGhvPN=|-Eqk7IGlj+9GpxmLSG!7 z6n=tv!-t@{-yp|1tX-+?-%{(+X2oJ^d09hDV6Wuyq%SwqWA%tpmxl{v*rZy!?-D5P_cV|u6Ptgl>=OCSom~+?O>p=yjB`c& zz8yS83;weE^2tFYl|mA<9e)#5H{<3#hmu38#JP>k{~H2Jclo9%+-v1I@;C?beUS6m$;XQE&vLtBzA^8;={&6Bb}J?4xR3WAb8umI1Y zF<@^t-jMVO-@mM#Gtm+O>U8JOofZ`ySVKS&4uv9ldxSTi-KeJB0!JZW51!)ltIsG= z!lvA-<_9OwkgI+pWScgdi+adUnmzLeP>XVJ+Jn*O>V6yRe#=NZDcl#-tlzTC=N{BI z#5?Kq2{6vZm3&?1xX*M-Qv!7}XZ$iWbW(s0}*9p)XeiJbY4$Bhh{NdUXYXDYC zI1a)(jNvXR!D*MK3u~fpJTP?EU^`;SKH74*)Qj6#>G0p0D#Xq>zb6vWeHQi1OSjMk~8GFMX~(OM!Kx z3OLE=z#qDhF}4O5RsEc_L=U;xyI4X#bz&(XF$(4+QVphn*Iz?O+T~~dIA^nM*hoB` z#tT50%95aAe%Lkr+TB=yadZKv=D*T$Wnk;NO>08!7{sV)Rk&%px06fFAb1$cWHy+ji6@V%DIowu_;SyK8P7(^Hu{N3u?a~crfK%%fq2$SXIdu~Y*;4wF7 z)*Vy(ZwN#{R7X!bC9J$v2eLsquI|@-P`H>gj4JmmK`{cf=w!abQcz@F?NeM$yShs2 zNCTS8{ZK|Fq7v)Y4UdPO5cC+2ey7i4syEV~2Ev=K$&k1qOm@8W~7s zg$R`?(outEm84vmA=D3z4=_HZr16WHpLTPvme#``hy^5*oC6DCxZJ>ThqX+GPIn&M zEGmsbLHc4wgNdQW#9T;~>18b#DG0H^Cui$IqWMfd>aiMyEDAO}a#B+GY=yGQ;O`?D zLDyv$4}V12{b2ixBkY(Kxo9i9v&pi9xsue(Rjr8GOS~IEe`7WCR@P|+Nmj-g#{SUN zqL5LgnkW=>Xl5!VbWu^egi&L02%^T!Vv40yQ>=99+jWK$hFv?ZJrq?@{&e4^UtYJ;OreOE1mlcwV=H8jfUrWh<{ zkYm~}0nySbF0Nk!~gA~WnWul`>Vxm%5 z%^fJSEV~lM$%_#{M1Vo*@dVZ4Z$ATo-oljS?W_hP6Dc4nSP*H>$fOeEHaM|vhu7&# zPsrX@xEU@K<(oU2Uz!kKyVqHfTTT(pkGMWnFQU1u_Y_xBFD+-A$bL2#(gNn&tQVL= zXsmEfC*br=SyZj3iV@XY$sP;R=U~K(`)z?IMub#LkWet*GE`jDWN%f`K6@2}Ddj#w zh+o)w>S70opCG=E^joBGlD$9jFQP2XYx#(#t)rR*%J~+g;2$Y4T1bEvC|z>BiinNP z3sDz=1O|Nt%8MY6i;M8ZI%pIO?*t)*(+h17ml3^-EgG0TJ@_5-6%+m{*RVu=j8$l` zMdj$ZI*N$Kv@`Cx6ibm}WfR&dGpLberUF!r$0Lda>eNP1RIXomQbZ^aN_g0#HAOi3 z2{N3*EwTiP5aO~UG_RVO%OT4Y%7Dr&T-1eWKP~NAV4Q$vr_*qTmjcw8TvNab@@{e6r%7H0@xkuQrB$(kpA%_TQ03;&d zX%+Heg;`}qn1yA97KaZ08g&mTTM89J`jTXxEqyVA@oRAGsA%1sCNct;$p*dNR9R`2Yabf|}~pY7(}4 zgG5%#2#(CNx#xWy$|NWQxvcC1Hzja%QMrzuhQO5dGz97VqkG@Gpx20#*-F(204ZS+ zqxTX@{znkv9D^)_ESG-`o69nmQUiKl2*_+N-y{@m-ID^*k`&Mhz@C%!!^7(et*RNQ zccJF#S-`Vg3@e-RA(>?pC3u&}_55NAZ&2lZ_PN$(YL(WHi3kM1ggcj;2x-w^LV9Ma zIf;f$hO4M;a(9zl3eh*Bt$s7I;f%zNJUox?Q(C~|l-h`l(BwtUiEXh=tHf>S#3KOd zwRKIIGIAMJKPSASgLYTvoaJ2+1dP=K9~d=J6_>{oG-j_+x$iX!{Gy|=GhGTr#+!1q zqf>{ZAEgxOO!p*JA+BULwRno@2H|9l%dKMrsY_LCAAYE&;;=_F;;UXu1B&ogovML~ zzP((S;D_o+)b@(>*kqB27hZaD4e8UeZuKTSyv8HAN`n3dl_i!`7x3Lk<(p=h01Z(( zmcuoGu>_%I*RN38dfX!_3E88jYox`r$Sx(iyu-vs@4=)dUAGiTv6U!VpnceZ$V5`s zCT7iQXNkJfLR6+*>hRB?X9|(DB+~sb0}?z@--b{((kNX5!vK3*(KjtIA=xZCXi#?2 z1<)l%IQ+nLCq$itvJy4JG>K7C8NF!Sj4Bp$35=Tg1O|aYCAAAgDzq1&`UEja zKNqYMZ#}amC;g}ho9_>t$3kpQLNd!_k)kMjkW$@X)*Hm8nvy7#Hy(D zm~O|1hkA$x{FR4LxlAaCibwt<(>wqUC)sXc;XLk*ti-8O4L@RlCU^9mH`0*;PKd;i zguqk*(6bQckj5-nA~QzJ*`Euw&tlcT;ym^MKuf|zekl@sTQ8z}9~)Z(hKq?89om*I z|DfZ3(Z(5fMMgky3hx4NiB{(T1n@5a$v3Ofe^L$@DkE5605jsuW;qsK$BPwO0!%LY zP3W-5lEOr6jUOlr;SMU*E)qyR=g6xu0jHAiBX(jxqTq<+_yRVufo}|P!&Fc{ zi}FZ8Kqr(KudFu@LQG(x1iGezi{&0#RCOSN)5QW@y!mU8FmSRO4?K*}_|XMV|9#%v z#9cjjyzrRyr?4~c0+k(IXcb7)>E;TO&n<)tJ4*kq^qbg1F?NmoXQR6l=rl-j? zLmPwjMg&0%S6f$F#q5UUP}5gA)`V@X-XARG;5xtvDuALO2)Pb7RKzAhTu|+aFA1OC zW8x4;B=Ul7tkk4>1~2*6f?@?KG&Uz)pb5|DEOO9)IaR4|Apj^K#hnQ|+q2#WbyObW z;8pw#7kiw%fL|;FP;(3Ex4|Vv&QQn=4{28{5+YWkeBPbtnhdBXE~b`P#(xcDac`Iy z=|J+SenAe#ro&)Bg8Ugb%+rL;1S72YY!BKt2Mt_3L^JPTphOQ-J~LY$3R{f#-jm*v z@!Kw0rTGO^HklWI0i}2qPYc@uw13?u=)%AjS+sp9HNiVAgu;v?vBVumI-o93gxL24 zb{Tt*@3)_iz#t~jk3;gdf9#d*Cy(m<_u^m_&sIsB53U9lk{Fk__{4Qook2nkFb^7; z8U(he>hK~~z#H&@@5k-T6IEgZ_(ldSuH~3pMNtPt zG7!vm+5?Ber9&LHdB9?+X1Qdghzqj5Ou6ZE9jTz^2lU2vBwE*#Mu6b@<+CY>4#TJh zfa-G0xsh0>9dJk06MKih|;fX+Xh;$`dGMz1V^z=W3T`<-kMQ2$P+w z4>)bK5h0s4gGvhsbGbzlguuf=7&#TPK*X?1yF>f(-ds@Rru@U zV`5GTcNE?`92HQ71l_w~>Za8R@xxIapbXGa3_>#Nh$?Xil0@(ZJ>LgNsOWxzruxxf z=qwVt^8W@yMa__3mc;<0jHm=*C3<>7X%3<1-?Vcbe;o)v^DA~~T~1ORB|#I`uZ?RAj3$5ULSQ23Kt_-zTWMxy2v*C`nR z!*Gajpx}cO+)hRgq;T1R0a4V}(pq*5DYk}|o11WsY#Fe2gqb{H$^mm1Y!+CLVLr>d zBg}VW)R1E3$LxHtn!}gV7{6fUg3}>vEZDneb$~+*%;2$Rz*vY(r27-hPnh{&=E%!6 z%m9{Iue`bv+^bT6AOU~x2g0)>n_AP3x3?+A+6=*M7`};Yk zqzx+}=c41kREe%JQ_{_KvF-*BPwqSe^3 zeOa#$Mf|i6x^d^tcUW~FedwK@nqVUKlqU}9!K^bQSY+gbSGPPMoqkZGn#bOudFQ-i z@tI73p6GO4cu$5Qr^G=R;G1^DAal3k8*PN}e2hITV?imTHu4argOINe<-n1BvY4LA zgg-n|I2mE^*;aHS20Pov?1AlVfM2amA0i=^0B-aW3rUGI##;g9^lD)yOb3qHkTwTa zLg^1!I~z*$;fu_4O(?}i3gEpmASF?RKUip+5YUWODW!Qsp+y&kRH4>kcmn}QqkJ|R zC|*EBUV8dbY0R%I4=QfFKbPgIC41po#uLeEJ6isR=|B`P4Cr49u}*N8aF^4^FO8SH zIhI?veLP7k%5^BxTNuJW62g;&Eh`Xayg$q8AZ&=U$GJb_Xhc5SdLwSrO-AW)0C!Z1 z%@b~p?Kd#E8**VG=@QVuiL_{+y1tXm?%nU|+#`7!5f8vXh*FG*jk|lfc291$&M;JN zm|GDBRhabfud1sO9Ah)|1)%E3YlcsxD72FgSMy0YU#4bBve3?h&sct3Mmu>wK9 zi^=G4-c66~!xE9b1Yh$#qh|W{G=x8y3nOz-ra`cK^*$AL@H&_psK{{yA%>cM;Ua^o zbSWB!shBDcv4L9weuu@p6AmZ6XYoB5-5zim-Ng5l-Y2e&+)qF|#PmW_19_ia>AQ!=3O>3oHp(Eg-!0V`G*sczbrj z$@UqYiDf&wQkuvmh}$WrQ3w%)ta(w4CRAbw-;L~C91J4Z!;j*AV|t^Y|3X~2d>cD5L)((L79r+@5QvEDFVlcH%)pXpWu%*; zXC~`aoL1b}JU_kmic#RgAaYbQ5v_<(WdVnKbBeGgo>%iA&&>h-0 z?(G_0Pc6GOA>EpftIXoM^mvxM&Lpc!8LLVTt4a;4N(iS)1gA^+Py6dm$>C1P+fK>T zPRY+s$;(d2&ZlH#(Xue8*%Wkah`KgUryzo(WI$1}8t2&ze$Bums~|^Jn0~5&{ZYaC zfM?5pXXgI%bAKs%&gpc$Qt5gm((_5B;gd=NlS}<4S@9mp=N`%59?91p$*lgE)(FRpN`3490|+MK>W9Jev(WrzeyIyae2MqM3R#c~p(+Qw}9#I%uB`trv`Wq;G? z3+w)I9<24Qd;E@L3yo7`tPNj+uMHov`A zG@CM;D620ymmG(B;ck-nL?PY?)@-=QtU5;MR=a)-!_jA*k11hl8ffrjQ^rPesdH2#VM>hKh5 zR2b(1bm-;MG{@=CQroO&uFbErt(-12hr&Jn(9kk~Zi%jm0f&+{10i}!v z4M&Cg{jf{@-2v)GZfoL#K4jQ*qs4=#U-4lf!$eHrz*I=Hh)xl%q>b3}(Myv=>hK$$ zA!IhaX46ZEXWlpX%!oEpc>egkGiFE2YO6)wnU;{P6pwAOtX` zQ^x(T|L_1d3Lq7@9T?>dR+xepm{=vSqhfaVVhx;hL4St`5I?LWVapKz1t5X7W&!r( zWq`)tugv)*rNH^bJrH5o#8JMn0jd3klqyp;I#=X*4*SA?XG~!mEyxvq8}u zViEV_-tG6Ify#*sDQ<|3!#Nhqpe=L zB*{yoyh>ny@+0B3usv8!MPs+HI_H$qwvL|#**fPdQnyhO)Oku1j{}1YmWd!b6o5iW zvH8gWv?CG{4q4e^iUv;#62ifhlrfGD`AqOwVvI5VaSp1R^C-eZ+gz^!#kaZw&Su~0 zk?0X4st}|<*TQkf8h;82-7Yn3_wnbg@Af2Wt>_ z)bP@k5nhLem8>3!~{bC z#tN#Q9Q>*j&lN`{ZgO~f6Ix)i=QeO;aJEM#Xqmzdo=6ggMM{1;bzzble4fdkjsyh? ziV)9kB(;Tl`V|UQIJS(-zSWVDHRT2X5Lu(vwjxuou6ZT`v<@6$6{tDH7AS;Mi=%-Z zBFZab1sX)6>wA=U_5n$iP4mYWeLK@D_@!p4WI>4Vp%6CQG_Z623)H=EF;!Nq@Vta8 zT=|eU#jzzVLl9j-ib?&5$x~sd^B{_~q+A8D1P?Jp*$KCpwT=$e9mX5%v6m-*t*BzT zQceArz4D@D!yhD~f$_$f@>ylvE_Opyg`#^fu-$ zGB0_x4OQ5;ViuXzEyCrfFKoap`Dv@F5lZu5w7bnO`$KX4Ai&WmOx)1LVA65*ai=Pw-{YK zJ`o%9ai`h=SINB2#c7Il@@Pdx45BiGs|pH;t(k3j;Gb^@aei~e7mEG}b`L_2z_f=8 zVdap!(M3y+CF6`(A~(}=6#@9@(atdjwBeQXbslkPJ2Nij&Nt}@4ggM@c$+vvKr6*H zrxYT)3n1Ar zWl?e&86gB=;7Xv9Es<3(P|6fKiM9eHCQ%gPNp9inS#(!e+ANga7@QeRER) z-Q`nZ_XMP#d*fy>)2r=Iv^o~ zRUbr@4HdAs@*`nK4nPE8!pgeu`otGbnkJj?Im6JT5IvBfZ~_F;g~19U1>kFY3&8r)t#NIo0)?&Kmcsg1xnN-@G#1;jNY0i`s+YbopyE@uJEB8iFch->o} z*#UxZ&1++p*g*sP)jHxn=Wt5`M&6w}nQSzb@K`H03@cb`3h0BkIfZ}j;Rl6v|8DLG)bpr%1@`B&d5p#sPi>0`Z$ zWT=m#j&6xFB&8ON@u60kb%Dp@ZGbe;d6dhpj2m5n^OzK@!c*I1`vvOY5@fX;Cml$ow~ajM2#yHBOuXqPXHjx?IY{6K-0d zwJNjo6*{z6cQJv7n5707z|hsk28}2_hvEWU(Vd)@DcRIC5wpU>016z)2WmDID+;Va zc5WH1%mf~17=?K>f(1*EhB6Gs0ECl-iI|OQpiTiAHjmjh*H}zPqhd*7mKN=|uvGe? zjEqP~HzgAhop4*A!A)TGUd@vmMUxv%1S)F5Ul>Scn!JeZ)?+L;lBN+XLaeIe%W)AY zO0hE3d)ka4q=Pz=kHiOZb7F6HnxCMh80F@%Kt}OSzB-EqB~`FeNjhM}Enhfz91v!X z+HMd+Ofx2eEQ)lZw)reE1&u&1#ue+FFkGdLpl4n8k2HUmE=LH``{m7e;~&;u>s8(p zt2`qZG9Ka{e8){h#0@k+1VnZPQF|B~23PuMs#s8uKbc9g$S527lKWnOSs7$K>Wb?u zRsk7TSXEG-96M6ale35;DrJ?^PC~*k2oUyuk5|k%V8Zl(i4FZecC%|-lJQ_Fu}LLP zOb%{`15szi?<$B{b;F~SrcJ(me<`9uD=z+(+mu$$e%VQmB9FV%7QJK9?|UNq!aYhB z%qGZhv$#>ZX=m&R4+552ck?!eL=ITCj}1hdhIfmu)tOtis<;MV1<2?ShgEgKeX$eE z2DN1k*#i0r!VL;SL@~^m!h#CWu%;MIcyURezyzkj2r}mJSBnh;u53YK-_cVQ*#anZ z;p+f3-pM+MHB3aE)Bu4Dz*FKodDnHq#vBXYAOW&kdGs*F(2%$5t4WAW+Zm(5t9kuWC2m#4v*TGW=8*Q7LE_heVubFxKS~iXI&j zaHmnIE7!cFDO+9p<8g-C!9gOAKD`EV>k(9Ap}U2kpR7USHtA-=v!)@1^7kW4uYR|n zUW1C!a-haIE}I7P&7sJ5~U$v zU%tZUToP?kQYw9vi50GfUAh=BCW8V}&a|^lho~K%1*)mA$%J6Z8*MUo1;t6_`8#=p zy6HWE3S5E?zLfcH5EB5R(`y zl#-2Dkb1=6^#H-8#t~Je1FV@e(pM1(K_I-WP6~M#R8jq+{a3W(+Kl9{p*VHzv11IO zeX=9G5DkN{r3(xJ6V?jrfP@HgXS#{*N4Q^BbnW8CYw`+}DaI83(&+F!d_K)8xo$Lr z(*iQ3l?^VngNBH3FbJYf-0Auh(qN82K`?S)yJtN*$hvf>n{!QS9<#s()30AfQ&72L zG;Kg2W~<7|oAUr%I$x+rJjjW}V9`Uu9r!|sCNB>_7tcQSLE`v4(-la3slqOhRLiZl z46F&3Bm+@JAOYggU1D9lY6^JqabQFd%6NIYjp;=i^H7NAMaxSCcCw9B9Q2;OXr``c zDv)ry=r$m7Ft8uN%!>lB&mfNVvT6s8(Ztp)2}MyKtHCk=lZFx^@+k)nxJ6>%)B*zK z>p>GN5XoylD0oN^&zo+kVoL?x)~g~FRYuAUQ|Uy-1Q9`O-lXhJTqKI~;SU{DG-yq- zN1I%V6r5DZ?@*FAM0z+#i$cf|kfDWGEFHOjp*W(l0kmZq>gNh5)i_pxM)!l+K>(gF z7;y2udpU#((MK2<5N+Rh`%_*Gauk7wI@|Gwjy*3UnZVKv({60WV_wU}&ZVKUhGmsB zxVWHbnDnr3>_SAhzu^PxrK7iBlfVYLK#0Ln3#Y4TW~5skFS9Y%%Gjy#Io7Qjqnyal z)DJ1cw7F0L6HjiV7STCaVRsTWV2>Rt{_iwOQJWxjak_Tq$#%`Lg+DY21P#Zued`uj ziP=huGx(NAQ$&iu!zJ?EtFbRVd)Q6Ai4Dxh(8ZH z68uX~!9_6A*)-1rV~>)k*i15^mr}LLx`+?7JHf8{R;m(lLnoJ`L}f)Bl7kJE3D~7I zpsq0&YP8-$q6x|&3`cKT$M zJj(dD9G9rWUR`7F1|AoFF(4RQAoM5Iu82ouuG6u9a0g>P_we4sK6sQ75Q4#+4UTJr zd%{TK0=2VDlJ|3oC6H>r`+UxxLstYj?;QO>24pK*!;4mNTiX_Aux=Hkx69-|2oPe{ z%$xL@xCnzzZ$b?M?d)@Tz#YQD__lw?djaoC2?oLFdMj@ z!=ylwRO%*x$=K~(Zv6yy_?g8y|@bSLSs#apUBv zFIX_-CWLCLl3&)d4uoT+T4`}TG@Hm=WK~~bN39NV%s6pe_yGwf=l~5S3fjI3!Eupk zA}))7z(b|t%(%8Q(BS;Ad!U3f?>k%3PC{ry?NjA48YZ|ws;9 zds~W$h!(AJa-?hnGPVGe<~eKuD)Iy38)FKw_BFv`Dg{ZnzB}cHj4uSwH7=FoCxrml z<&gk2ERn;3cV??=hbI;*OVY}(W|0b2Whs_zCry-MZBrvUEkgJSB~;pDMX_!jOEy{G zy!@-Or%t?DgB?Yh;Pd|80IsM1`7>icd;X^7qKdRXTe>I$RF`vYHKHLEnRG z`1MrV!{uZz$>f|k4LLG71^765Ehaf37+DLQ}EoRL96koco7n?B1NOa1Du%rUMQAL z7Gp-=g>svu5+*|w19SW<pzn>i{cJjQ~z?BuUV~(TYPz4k)l^0je}w=>qk!4HDwD8d3pP za0R~@GdvNd#_!59%lM0(*@}vIAmN>h6eeo@Hb4gL{8Odr8S9&{~kt zCcKY=;l9mdq5+V-dnbOu9Y<$MVe_^5>+S{pva5r)^+}?Exf*1}{irI>p(^Ri=_L_UE&gk-;sb6RaS_tdL@EB^fr0Bm)-THj5C&p+eU`hfCzy*)ELu=a~{ZH7V1e`g_tN+L(WyQd-D z&cu(@?t`()A_o!px9qHQEp1b#g<;ZI3Sj`1$Q-8+5W+nBA+C&R`kS71YP*F^p*!Z= zdHVae?qvX}>-XduinKfbsB@ZzKZ053raVVZY0%Bj5m)ZseS#9c-UJe*0qpBLSJyGzJblTNC}5np z3B7=;i?mN@gacK^_}Cg(W1y0d7t^=GA8wg1K{I2183h-JOp*&Z^xtfG=^;nEbPYLL zIi(_uc?rVK_ZapDs{4q_-?GL3e4panBE(@b@UZF33)|9Bb%zq1mn!MWXul89c`6QS zU$C$#T1E)2LsEyx%W{6aq}k{HBaLo(Aer*tLAJB&gv3P=RczLTT1CX#JNwkl$RQOT z*fMXA2`w-tfCebhk3C`wxPm4JzvHv(5

          TiE_NUBS-dNZ-b5Df=1uR{1Za!nuC<2 z0gZ|9+e_eWkx*ocx81W_b4iMD0xU8!2cCPMr!n&muCNLn&71>rfJl4ToUkAYBB^;M zk9wkz#IVH!6Nw1Bm#PymtRdeSB1SitoLXYg*bxBei(x;L=O;KTLTyFhW|DenBH|0m zzDg=olZZ%Uf(GzCra&c!PwrEq&kZtxhH}Ldw3Lmz%~&;L`gwgXDODdeGKPq%Mgdrt zq5Mv9P@dvUpU&aXGT~tC|LiW0-?gc2iLq)!rXbCBYLFYhT4EgP`@BBfKW>Z z2nBpkRz8*h06~@j08mQ<1QY-W2nYb;CjU?X00000000000000h0001RaC9$bZ*FvR zFGyu?(1X=5&CZggZ}bYX0FRZtrYux@aNKyeN3?i2|g z+}+*XDNcZ(#l5(@J1y?e;93e4hvJmd0zLn|bDz%K-PzswX6@zMhm9-(0Qx@x0Equ7 zGT^_6g^G-bh=_rRjDV1e005u^R5hCs0VO6NfF|JoNihHb`aki9r87C;|0~!4C4eu$ z3g8NG0@(bANdfu*JAgO9=RfQLaR0BJ7r+JJ`A-1=HUQ+moGZZpKkxg0>#hFFtN(}m z{xi`3ngjsI=qPIb?|n!Jh#Uau7$C(FGd`Nn(X6BS;Rk$pDJB;D$*U?}Z!a$GlRw3u(RpV9=?Mztm zSoN@YuOO=uyWhS{yp>ia$y~R3r&M$%@ZF)ksAR#FO-4X|18UJv@EDh|x5Oo9uPWrK zxv?49fz&b88LTzhxy5vp_2N0@V1H2&TKO%r%BviF!dDadj;}HLJ8$6Hsi{WlBnG_} zuBfJz+gbXDZw)_K5%zF8^ls4AqL!d+Y3y=U7=#o01`@n4J3m3cG5o%if>fhwyLN{L zxVjB`ga6D7%1RecnT&oUJ+F=QC+f@|{#&+#5nyoL-VS_?S%~ zMwi3uSWROIEJDBLINhXULR{I2#w&5%uoxx3+q%->UF)(vU9$B*b zaBB((XM=H4qdB+i`9s;M^6)^q92+%51AoNp4nM3YpFU#~vd$vapaKeLA&O|lz*lNubYNHYfD^YneV>JK7^$1IUrqsMft%;72 zlPf+khpQ&q-LrN)S6^mTa+=!3itvu7sSnpsy?0L3SBMfw>q$OZ;f}aMk+=1=CiG~o zM+xD;Z1m70oH{UVYswk$hTcd9*$!b1l@|>$5_AqcBaF25r>BH%wP;D0;3uDYwJ+wu<}7NrmJZKeW;ZC2dEu zXd8;P>eSuL`aq5+a}z)aOjaW0ozhCwD1l7i;3ZL* zf=w9O%uPHV%L4RjmR{CxzWv`1DL^bv9O=z%V3olb0*H|T;2nzUFmc<)nIfzCok#RT z+Dgy5Wzd@74 z$YV~DHYGCcZvu^qioo|sClCSn_{{$on0 zRWTRRwBNV0w!WWr3LfP{1z}?IKL+rdibcp2Tw=IVLt>VmCoz}Z1y^<(6SChY4Gf9! zBp8T@LB zCt>gILvuU6xc?_hVnS(j59^l(NEYz{yJ%&Zc$M=^LCPg?&O?dgGR68wsgWKB;k0G6 zJTO7Mj7o%M4kx1HMilVz2kVcWdm)zo1Z720I#^(Z2~YaFd{!Aqb!|dnYUvMn4N^jJ>#y=E?AaZWOUXLZC-4ztcdzXDQ2pH&Izl3uR4a4f=z| zJ4{3~qC?6O^pyy=DDuSB3qBJ*j;7>A%Q z%U_$`)LD|qWAkvE>LxgWnVP{IonI|c6T094Yt-{WCWQkz5v~GhE5a{=kU#nxQ8jIc zdcc1yr?1#Us7GVG!`hpwcgS5?wO|S>YxaJrBdvOoy9~L!cl@7%wewi3OpB~~4=3=8 zd?&MUN;Ax_(Gup*H%$WFC4FseM|Tnx0W;SFkaPaI$d&g*gWJbMd#sFHqh7(6{O}rU zk=e%_EEl%A-~Wub-}N4LV};PZd|0hR{^;sQh42MSg931;PF&A2%e8{N%r;m`81AlQ(#eG-i1y^~Kh8#On#L;k-F{ z+_HYLw-2|?vj3&J`tuHX*fmDLdOSZ8$1&zE={x!bYye$u76U7O9mta%X;Tsz*)Q4m z8|n!&2QEp7gb?{g$N}&Hi~vH<{hPbiI4iRlfru{^2m98G*4vh1^Av>6WDkn)RZUoW zx-Uw);Grv|19HM7msX*ip+o?FMq^M*ZBr8!or-l8oJtN&@`M3a!jieJ5@w-M%JxsOd^y z+i9BStr8={$~kHs^@=2mW{FuK)=D_bKTVO|=uD&vqz-n?Ca@f@oNZ|oYXYir9w)DY zWy>&VqMdy%hfjBC)Omza1PEbb#)V(RDz3iHOX|=u`Q$1PFvj z?&@F;aQ7&r$JW#2kaqejt)!RT$a`ow^Dcp?ly^X)Ik1tN;6$qu9a$7{(}Vih5WW2q9HIt&o~3bzgf(_oA}^D@hRl989KV$kN(3t! zl-#CK^@7Z3H59A_5&^DdMKwmOnLzkp1rFB3QiG1BZ#Z4!orea!nT;Oa7CUn8j%jP+ zkbBJESYI4q?adcptxTP)71AGELz4!ep-T)F(mQjL zBC8KA2y5fL;7l3Qc7nem{?-}B06+yx&Y~IEN@gm$i|zVW`oW0ksiZMU_^InLBI3lB zb{c*c>a|Q(YL(q%8Oml=5M+MM`P9DsW=r3wcI+l>FV^A&yKgZHRF|YRypEW(-=i5( zMbg3#^7KGc(aZphEFA!LJr9yC1C-zym920DaZOSaN(QbpteYlQZZ*oXN6V$LoJD)v zW!@g=(;uQM-Bf=d#O9$uf$x^VWs<$kJ}|Q+>1HAJ1%R9K^kedw=q`=Ae+EM$FUiFM zz2r&0{wO;3X(b-lBgww?F|CSWAY`hrpN}HUyy8n3e~c1R+nm|tv0i4`U~*95A^l>D z#7c(vK%z&T{%hRgKriO6h)%kQ&r(n54YkWK5@te57v*g>f&(&o2FperJN}0`=46Kh zNbSUvQOiMLk_~rjQ-rcL7p4b9=Z1Z(!!z-Z-qn|cBV> z#s)$672PpU{f}X%5b9IqFW`@`h!HFGt&;PJ5iU_}@=4`cs>VS%*&fQkfS9k;pQb18 z<39?6WtC+SXbVZWXrER60secM!=KPk=e8tcoMYo(WN z$3oGOBji$Jb-7U@LbpOL>GZ1wC45vRmdzOtZ;k3 z-eWxJpuntW-yfc_Wgv`aX1F_4igEw7lR!(GagkvWnc>Lx+iwCpdT(M}glte-#AG|R z9!HtwRLfgfBda6stzORNKTL?D`xMf~60#gU8M(rWc$Fv^geG|dLHV?wUFEGM34_r$ z5R^x;gvR^TC3rG?Q{AdU_r-W;;$50f{(PYtH`KTY%y>(MbaS zN`2@irKTi+7Ay>>_G_)P>!1erM)SSs7SI|); zk2A?}RF_FhJAdNEoG4n6b3pIU3}i+}W0K>?fCddk4p2cL1Yw?g$-s!efIKVo1D~J! zAS=%v7QU!I-8>Rl-TsDGIQjd;-y_G? z#&0~PwqGo#Gd^P_T_$dEYYS-eh6Z)(k59fc_=}aY`?ucrtUlsS;wzsw1af492H*>*FsYLgAZ}oPg&8m!ssBh!Txg7ys42PeEd8cb+CLkX?2Patn9$JdEJji=2?@2;-^KfA@1C4&!}yC3;{{D6;z zOPHEPeX7Ia*VKwREk1S-O!w4&_>tILQ*wL;Evu;9x=V_#{i3BYBD`$}uYgD5&gfKh zl=#D;ngo-!h63_QySL94f0p+>G?pGph4Ph8Sg%OLt#@(XpBKx zbS2RQ8a9UTx8>C!Oum)jk-d6aO7y1D)gZxngG`I5CI-Er6C#6RTOfONsb-g$K6e7>d# zHT>L?MZQvhrslPje+JmHY9A?(W@#+dY%EP}ECD5#^{7=Md;ahhtnEUUHD0F(e6K2$ zxY~LtBbeMbpRjM}CrCCujREakT2K1E(T#>HYnPQAZf^TQ1&z2v@SQ==JF}2?rq(KP zb72)C!Jp8vfbkdflD22T`fwTBTa^H`>3LnW={b9}=@&eNyKwf1Bke=VI-78|_{qCk zua`Qn@YMeppnzi{!$c|^O!>t7FeMi<&bfZ~)8JY9{#nS8ENssQt@q(|*x%r>$9Y3^ z6cYDX7yAzQwg|?Om?4*}#Dx;DRp?ke6HaRiH_#lo8oh5s)kWQRmYhE8)kkE`c#P?f0jTpxG1f&&mU39t-s>cI&}J!BmJ?;7dxP>;+}G6 zu^G=%U8wpyjs5B={AvG_{7psT8fGou3cdq*qD4)h{Qc<|EIAfEL&uJ-TdRTG5q zip%2R}<;&imH@w(5k&92l*U&4O#kT z&!3%nGDXlZ5m~G}qiY=Ukq+p_(6K(yTSXLRB%PNYS~jzbSKvM3>=TLf=@-EQdN;Fv zNwJK#1V*d_5%N!M<9<2-rJc;OCn5zLk}lGxD1=6VChMGoZuCH)^)&rUF!kZybbdm3 zQYUdD>_?j24xc8MqO%Sw>$zB$q{WGHMr1_9%t0DsQv~;LpH-@ zGTi0o;sOyjvGHkJWE}^&tAra19p)GA^pQ{=4DijS=03IrOAq5pU(*3P%HhehNTQhF z(FmB!lVJ}a*B61)!HOp*HP`;w_?Szd^|#El&fOOBAJt zhPeHk17u4&ve?D5!tdg8j3+G9-Z8i06zIh_`H_|MUrVj%3{~Mv>Ei^O1Az8VkLo$u zXte8#iz8^lSHc8eEDMv$Rj zuVl5J8R%%COo;n)r#I>YRzL5EhKtHWZk`RVj{}6!N}?}@Fm`@Pyd*xlZH#u#Js3V; z)EXGJ++X~2-n}E3<;RZTq~ytZeR2=((iDA;!{Rx=uetGM{V`q57T|fobM+N>MiR5(Dc#pZ_%v92Q?v8{)c9(r{aFN<`rGfi_>+4*Bbquo{l z+Vu(+U8Te_SJXLas*VpAxP!8^6&K>8aY*>?zx(V*aVi*HPnFIuuQ;SkoT*mqakOTb zwdN)q;bt^lpl=F6A8?X0*IO}@8wh1qLU_{X+ljN)dCSU_j1R@x0qaZ|AYT|sGSNSdv(J2!*YVTHA<86SZ&n%jQQ5<{~p z$@n|XkGDfau~q7ku8{Yh)xc_S>`uY0 zI&Rp9TqCvO0|FPGQVXKp2LdDGrW3gb(nGX%wavc-+aB5s^V%^FYMS<}G;-tz$QY?? z{f7Zp=S*nE402_g)GnN-IBfGE92pqCttLz9)*yna+oC1+2md)%(3h-&o=2l@HcD63Dpnz!< z3}s^qlh6`9+>;TiP4dyNfbG!E1Rp1yeWF>4aJ#MbJnqa~mt6xX8-bvsshz?V!RLiL`>|X=J6Bl!419V&)hPi!a1w3crik zRqhA+ioOxnR@t&@F+d4x{h3y3{SdTkptP1 z^yWqv*%Fh3SKiU5G6QLEB}z~c^)Fgu9zfw?%69^cjUaQ zdgXuHJ(S@eviXjdK2l`8Fv%D?p~ftigL2FIllrg7`bAgC`~#&t9Uh_%E*sAvrV$fv z9?!^zsNTQo<>+PH&qJe!nWHl9t_Nbo9`=;ZTHk_bC0lfCb&YTzX7!`OFq2yy8~9*m zon@Mzv9fsbt-N6l#mw1ir4N}ONy&{lQraw^F2RlhVwwaXf>l|1f)Ks-iGHzEf~ii1 z{JNh%nK}6D2Kqw<&@Y?wp#WQsogudaZC@3g@et+Udt5V=7A4rTrj&hvK@xallot1l_$S7~``h{zGH=c(pTo_;+GKl1$k>A{s4dqnzC8xi zoKr{Y_|ih8!4z0WkFdeA7Yv2HkW)dG%+OUcfW0SVj<~xlRdbYEqk4=ie=>=Wff_qk zb<9p;$rz8C(;F_1&s3_HwHD`^nZsIgVnJ4!j{QVv_wC4jjDg;DLwVIp)fu=$-XvNA zSoMS+`>pM$#kIb1A7#o|f3n;C?X4|vZ9z@c<_KohQ~sz{L2ZdqJXtlb7SX4a)K7Z*7p&XaA5nwVsH1d(6%8l&d9G^~1yN>QW!h%4fjL4s;{aAe zC&#E*^+_4|^z$`!Tj6e+`IW)(reGy&>3GaBVy&l=mmEjZ>6xP?WI7QUc3X6t?qU`u zoJr0D>w~YZ`KvG)2sBG6XI0Ul5xqdr?wfDLRv)Y!jTmc|CUGK#dH7%_!tw$`J5!Z3 z#>qOHaJ?lS(XkhkSvr7|T~jw5EmKANi(v`DBo6@oK;EwlNA8d}237D;#+rsqAVjSV zA)8-D?F(i6{%2_BxbiDpF9Z$)$Ru=)=4cG30$B^gtu~LqQcP3v%Zy`d z(t2)mjDKv$hzorX(tYN)7nFuAZJf z$AOuZ12mEoJC6Ih(Ya3tYrXzi|2|*X)znN$=dRE-gmbGc?4odc#f>x0`8JpIU}1Wz zitSKRF$9X3-AR@ry&r;KH3sKK1#9oYaRQsBg7?1`+UGo1%JgP?)KOa3o!Sewq91jU z-ga)Dp{=Mo%gOW@QL?2~5xnu01kk@XnT`KUrM|h3D2b4;s_+QdZz45(!^K} zu9fQX=RjMOI>7E`4Yq1vwec!{uIbft#<2V zv7~1blA~C2WUuzpf<%YqKSswdcKX{Ibs<)>y7y53 z()jSXA8)GArODoK;5oExO;?{ZiL~bv?x})ErSb$u?+^SPmL@l!q7W1X2#zN%$<5pLHM&}ZZtJmv%}W(j z{{I8BIzMt=)f3DRT#WRpgqP+0Cc;eI>ulD#HYkP$1!Z!?;KZr;_IQhZwUtM}+lD930<|<56WPF%A|QZS zfghLP*GRa#aAwe&EcACvg;2LT#V!&eEx?h4cU+K|Y3R9NLRj zv8X+LuiZ+(@_CJJWl~#h1Qec&#T8C&cZ0=2$$HpS7_gGyn(*$&$fHV1gdWA-JyZ<) z#;)0E#Ebtk`9QIosml{ipR->C0`m0F`K@zr*me@?F8gj<0_dSA>oO4wfWPdDc?~jS z9brnRh}*MYV1J+Y9yVC3vnAA$#$w27GR8w|DoSjVWyq&p7V8%fdwE{c- zLkf1eADGb5KD)&oj)Rd7B4N7~(dSI<=gkv!M;2%4Ak;%TDiv4OcWu~T7H`=EoP+I8 zezKXSgjW)_UJfvRJI)2aNsQLk`(ftbVDQpt_W)^(>@WbbAR@##cE7C7s+91oTp0qojSkU;dLFM7GAC(jEl|fljRxn(j9et zE+WxxKz^=3qfUSsp+`*&tukwdr(HhT)oE29Rn!@gbk=gcz8F1IV{8;1M`oKXTNGt+e$c z9GjGWg!9Eg<>>W~z`ybJshP5?CqGBJsIOT0Zv%7=Yz2T2@t z@4S^{0LX>qrzAb@j`oGH!aq4$lwBS3|JqBcVR#79u!{uH9&i2vADpy`)0^;`whRZ| zD}$X*<-^4-Vzt4lSh*SB<0vH-EiKYQ9;Xw_Ee|qDgJ5ZgKO0NtfObJlXM&7QN8?AE zMLW9CW+Wo)U0oh9YQl)(81zBHjkU%Pz)fY*0 zwKA-K*mR#|XGYw66|}YiC7uJUaw4)#OF`O(M#@-@B#Z=(;-^T9vm9l5;v0Voph35fBh7`H^S<6|Ml)6O!}M zZP(a0BEeZTGPvo~XZ0F|rIQ0sD{~S7sb%Q^8xbz787f~m#k@fxjr@p+$Zf#}I-zQ2 z>4%8J5zKK-S)g3D#wzkTM1iFs`d1-<-}k;4;4*k)&LqU<^Bmz0Kw+(nWYIl|Pi;R2 zfB3S2C4oieU@vP7XC<1xe}$xXMyr*Nv4qRqvG73rO~8$KnyH;Hf0c;UU! zA#Ko_j9)It&V!zKAF5>yL`1H47RBRIn}q+eO_G2v;o&z3(jZMR&WzLu3{ZloY27ko z`qs604C$;TxW=E6Ovk;DLRVNwR+Ok48Y713&fm2NG=X~1Wiezp$|ILe(T^0*fOP=# zSUziLr=X!2;Jk5|#y*g`zegIdK9;Swr27e44CCTtLd~MLt43zWO^B$(TLX+e-78 z*r6XeK-`Mi-r{_@mrlA`|4W>4cy^C+KaYU3k1!M(Oa0@_y6Jr>7Af$E{p*q6KNZ^5^J3U-+A0<^s`)V{Ne=saT5hKYK;XqEZ<`L}~~t9X0(MW)k)dNua)7 zN>DNe!EM@mNlV`x!a`>@H6wNTl}AII!w-?vRcDHEU3C~erd6gqSjE*2_ zbv0!V$=h?+nKT%@i|PDtVaDJt^+zUt3SBDbuW-~pAjB>fW!&n=plpT0lS`d1llt3QWF9euy?2AOi&*ExhI0^fNQW**wk;4DpghDmdt8 zV}B#8awETJTS7zdfM{YFQEc7bP3tLAB@e&mGe&MA;rL^hrICF0EG;+uQZ==0lxpgfwfz6zpG}kOiQG18MVFPq4F|Y=H+~{%> z*lQxtO<{T^QQj6-XP5A^H`~g4YXO0!;>-7=A7>HZ^7mfja+_Kt(K2a&KVq@ubgTJn zBp>;KlA%Oi4M9XF71AYMtd2_Lg52DxjuJ|#?$MqymWWeaeC9IR92!#H6iJzN>Etuv zaYhbbLigGjH?@I4_Ld0zD=d{AtRdmlep*(Ko|t|8j@h6-EH@Q-OcLz;bnH zSvUg$4^}8o%Znlv>F`VhrNnNz^p!F zSn)WlXP}SN&BF6%&}1zlq7{&+dou$G*?mk3`7x+@3-3-5AhR(%TPM-OPNS-}#S}Ez z=GthR47Ex_?``ukh1%q@ldcCZKEOvGHt zl?7+vtXT0zrKg$4tsGGH8Us852S+f{r(|a;6ivBZ)H2x`y&gdFE@~M+aYt%$qmQJ) zxlxGha6Vi^#VCSzjTh-hn5?a^94!X~wNitvYJw=0@7?IXpqo^PQvnvy zh8E=u#*=GOucZiouH_yD*H$i&dEacDLyxWHL5GcSXiPMwiHP(-dDs?7Q}Z+f^=}8g z^c|?&w8M;+-5Tbn{5w^FVCr42m_zLl@u5*Z6N2x`)HjrOW)Df!>xiS8F-%!5p5QT0 zrrQ1XG$p&W_axD_gr1Zgl6g3oN5UT(iJp0tF2 z3SQ((q2sQ#6tyBm$=E$Kjy)4|Jhtm5*;WZ5aMeySZ@*1#K-{#R3mH|K`O|F}=lH)= zUcU#1W5w3*o0Uuq=8!e5x?q_~3QsaAq^5~Xi>oBHNv$L_p(Y zAV)mfMQC74qRg4YSOcl-#4C|O;&nWLG6yv!YHz_)6w7WM=@(Zf{D3pCW0$Q;t1|68 zR_fu7rDOvq)uMX4CICl3xWDO*Xk2Ig5sr9+sD*KKEt??6N9}7E>_zTNGAmsg9XiGw z7RgeE*-GakO(o2o^{P6aX1#H3y$Z0Vh3Ni9aaz3T$p%%_3Z_r4DiQK4imG>LM{P%v zD?}ZA(U$?%EvRaTGQ$t!4EL~XZF|Q#i>d^;ibrtdJmyY-t<)O_JD%Hec3XE{@0Ep! zxf<3B^XO2i;H=l|39f2s)jlHuBQj5fXm#_OtPtaQN+1L^Fpd~}vo-NW0Y!sz#Kc~k zlbaPAfR)fzUT6q;o{*qRW|*)+PW#FUJJhwcyAE;Q>C2vh{sT*^$7am6Ph*Xi)}G zpPHuRU4j*jt<-ckdtV?hZ)1V+5CmSz1ZVZVJg0zja=JZ)Rq`#vJQINDJJ8N~Dv3R@ zXYXb2e$zcU6v}JHvv1FB^70Gh#*0MNCOal=X(X}xnt2E;+sr~Jb$4^9ls`U*YLjfB zO`{kyaL}5~{kBY(V-hBk7glmmoi#wNV@DMHdBjgKE-hYq$%9Z|8LYlhQRcu%(o)SC z3`*BsK-92*FtP2((o$2+g`^w55&WF|S;X@8%VaxCf*TsQ?nP$=FS4!v!}bp}N@Dtn ziJ=O%$d0D6^q6nRDlU<_aw4jQ@n-Cck6<;$t1|CUceE0{k8XV(?Fh`vtLs+2kKkr< zNrfkdbCs>=da73;vB*5U%hAK-;CzDL*Zw8em?JCu!16bHCNXI|e6cr8_;Sypc4VDC z4_zhy8B3j^jn3oIaLyjH1f{rR9EIHjuiZQ{-s+*Efk~68_zWH8>dvSyD3iDLi*|L) z_3U?+`o};RzU_v2A8!J6t4iW zJr?Q1Mo;n)57A94c1aRaq&)*>(o9seZH=`%PP~D0*ZrWnH8b}Dxr%lKGX?$_m`e%J zK4~$WXYF^#efWXERa-j>+1fpNY7ViB_r!@oNhW!iC!MC&hswsyEgMRa?}nBN2lain z+Awds!D;(&7S)bE%3r6d)EUcnoUYLZr&2wbxLqKeuC{mS!HA0d+sYx3wV)3aPKj;v zi{0V-rxj^tL-*Jw75UU2J-Ymz?l+>DgUpCq`4^@8)yo#&y`y*D(WVus2$AD=TUE2# z*XSc`OzLUO#A)kddoNJEN|F4eZFYDgv$shn^2ZU*0QTxAmyt?*HAQ^;4=H$_^2uPC z_;Yz>6f6V?q=8Hn1h;?wcY4o?L#8M7mr4KO zOHGSa3?0#`Lmk_4IX$`V`~kKBmgONdLJ zr6<>4=*805>d=;?_IOdbcI*dR!OPx#S?h*5GqbX1{uhK(KMee7ymCF#Hc;{xw5*E0 zS>M2F(j?epq3GlUEHedkFfh~Xohq&jcsDJ{%@98nNFW7Nu}-Zr6sM}ihOnt+tgYLI zO3yQVnAKyvFu^o=GK(%MOwIoG%lCr zl`pkbg3h0!zgE}75PiRNb&aUA1P>xq$jBSoRz#Cx+?8!^=yBSQgN)+SB8q#)9ofbN zLb}k`At?4X&dwL&V5V|BS5on6&@a!LE7p}Y2gkhL=_liuHGUV<8ape!II#b5yhQP+ z$d)pShU!s&mLP#kolt-%^OEp4)C8vf!wqVu9%>6{;T!~cPF|1SOUzf5ifrLOX3HW4 z0zsR%^{oZYAskj|9X<*2hI#24CXVO#K#WLU1WEL$kqh{uH<|;j8ny&y$Cx=8koN0w zr}{z{e;uV%?&5MK8@_Rk%k}gOtAH%2j%MdEj5P1Dn)SW*M-7D&7~JOzdV3KSWqO=t z;73DYYCHCIC#;(>5$K;Vyb%k35}3YCg?G?Nc9P8bcNNGsC$!eJ43FtglB3Od#Z&|* zANo=t=YXWLbrv*B}dsGsw(vqXNcL|gBkH@ci)h|*RHtDxvCmuV2pV5OUL_K2uRnO0yp znTj8nQ83yofNA?qv zZeqc0gGC@W{QTBDqbysC9Lj*&MGDCifLRrAeZfzz?Z}gBM3@VEN4GU%E6ce4W|xL; z$B1e^ONZkZ$prAqOgGFY^7OeG#b)?8wU-bGjNsXfzWPxk(czD6?!)+Vxr8qvQ!b74 z;}+xq=a?Ywk9Z)Ha_{ZC7`PLlH;96QaY|^2HYKRv_JVQg_)e z6-09>o=a#IF{M1vLKLmeE4~A@;^OPq*g&R3{+{twR$=t6D)uz5r&vF~6kfPAM-~H( ztggsro^;D)@FL0wnM;8@Cm#0>I)6R34AInuD=sPHQL0ijS+J& z4gAYBQCeBcQV5XVZg{tVF2MLH0r=1OFAI#LgEHA`;!9*q{qg!`T|t-$VtO-PBNpT< zhV#KtP~AjtXrk6|?n+wF7&L^`vv)wi0;!L?P#-`-$O;qKlBP2=fAvhk-Cq2wh5y_{ z{~To#DG;}Gs)&E)EpMybaZWkMBWRyKG-M1*V@c!4$WG;}c*U*EQ4LZe^;q@y*@OG4m?G|;_M3tA)*}}GRYSG?V{C2tqXn;z zInciuo?S)ku?-jFX`GNRFPH>NlU9_c z$l5J!extH6;$P%7S6)9bpbA@+JaJG_?Glqd^7vaoyybUMV)i6B7*^@9TZLnZ?cp$I zkk8By?%|7!9_yZBMv9)JA-vm^qPR=PZP1PHsw&okGZXn!-kMOzVdyh-{BhPkoWxZ4UtGHU{k@5J?@HNf7QR4&*L)8(U z1=O0uwds|rrP;nyqAzb|xKKn^I5*QfV>yxfe&w-oIJWS1Z#35e&Rt1tRm?T|DMv$e zV^CAeaqwkK#FmTrZiTWG3a)fI>YK1Y>0gBkqYf*eBo8sXMGQ<(=eur7I&&qAzz>tW zfycCOo6P2ez4^8{UK1tt2WU&tPRR6ki zGfF?c!>q9k+YIL5Xp*)>mFei;?n`C3{P+nw*-Mh=jpX+k?md;2>xp!EtUtTXr&?*_ z%lR>f;bId0IqGLb*bw%-l$ms|%OmaHTh?4-B?8KY9F@aoS0*w<&+DpLsJKt@6W8@G zzvdl6QOYL*LlOg(@KjlHCk&ChUdGClu8*Ev^O!922lEVRSxa3Q@XGfAILBG>@O;}m ztX68u9_jFHn*~1Fov@U{Ioym(*FG{9Z|6u62{InSHucRh+9Ke_80}FUVY(iZaeaF7 z3Cf8oQPoc-GX|ZlCxy6{UqoR7>ZsJoLJS>rS~wM{XIeaJgvVG81hc}H%rb>*6KvtH zqSP6#OGn*LWyv|j3}$+{R3;B@`%!O0nlYe)sylhcY1i7*)Ws@E2&onRte#uiY)NeP` zeb2u=tZlO!zCaxn!ArGN;H?wwv`fyuVR{>qR-ZCk!HyW6<<2FS(VV`!lZi6*Q4nM(-#GD{Fv@PH0(1o!zZM7Y&f(r52p}jNpxJa{U4_NA`|cc@SDm zO9T~2%ekI{=`q?Vcf9GNwq_3TgH~X$WZkL&3*V|fF?!VTQ8n~Oc>8;#%ALVx2cdaV zhFz&K-rZI7#%&bSeRa0$@@VqHs16G0H?e>=@_PzTM}8E-&C}TaiRyZLkCuIm%*9S@ z*p^{Y+oi{!>q=F--|BQU>`0M~Uv{ru#)C*dhcfte4NKg7-l@0_ApZHV=Jm_%*%kYY zMOWJ&^`H#bPIU)|N2~@ixNO`VkmLA`Gg2qHR3i4%T8<-Jse`wA>t4I=;8UdZF+6KC zij2jp>FaEO>(u+B`vL3#W|wD6&oLVbTe$PkL+8Jgwl{Vh_tSej)%jgn0!D@?>u%zml>=vB1@8 z%^ilV3EkrIRG5KZH_LWHCuP)+!~ySokzT(SlD%%!C%uRs+Rf45m39{#Z-SXPtX@=& zWS==U;gryPdGqf#TJ?4Xeq)ri{<-nD*fKebOn5z<_pknW6&!y~@iA;K)bryi^qa_F zg@E2KBhh4xG`RUpE^qtvLzuRmjOZx%A7q10$uN#4JrS)orW)VN<&!TX=NXhE{SE!Fr3>j4PrNo{<`>D&w@zxs>##Gy3brte9C9%! zo=jQwX?t!rkedtoIrwGYs?g*Q-L4+{V}&ihPVs7ruO#_6o)AHu-cabb5a+iWeW9#e zk0eE7tgyVVylqz7St5o)+5TyC3XEunOfdt{v1^32w^f5|?-3Y!-Ahvq$czx;`R4?R zQhZFJsK3#^yUQl{KL8Ox?!Rli%BYo+eNei(R=-3x!R{D;PiC`->CbMv5i30c?N${% zP^s=r0v`WYeD72D%}fHThphOq^itMHkxVzPiuSNJ$3@I{8dL!{=3pyUrDbL1bfYBLt?&`{#pPdG)N zs&2&S73CLG)$Q1>pG{N)0MI^s1AN_)A449cwptE9NHa17< zj@u1^u13R3l<*hz)LVz~F(V2JO=!xM^OCfY=K`^w zo^JdZs9`?2#r?j`>_PELof<5hU=9yE*8uyu?ySk&c92lB)A#%2I+}<;`rI)z|1Wb1 zolt(FbO4y`41_{)N1V*FE*c!l`;lDSFk5|tPiVqF^~zolF4Ap$&+swkrq%=Yl; zH0bfDddJ2rb=r^$yEBUyATAwzVunt`K)R(V;n_oW%FE*!1CH2Tnf39DOY*ahKRt4; z(;s|h5aZf1vo3j-*u>$%G*;@agf4L}U*rUn+ z2&o-`hNq#bnDvlo6oPy4!EZjx&iVy&Xb#^%`+h^upcp9W(cMxQr>AM8?x@d`u3*wi z$O_&iLS7KIwFi}pXKhGQ7TS*1lKM*=Hp?0Qu&6MHe+d@+Vlfs8WZoYcw& z8AQ)DRHi|NivoV-qXm_qfJMs&&QtaXZHN^_0dVny@c>bg72`qRRmkEMZnsi6WFo5G zq%iZx7-2KP5Rjmhf?ce;E(I6^rkvYUNgElhMW5>@4QYvR;R(sug~BBKG0_+!0Z*%e znF7U#{<@}*IC&2H3D0x_v9CM;={geEdUtx*OtB&;6qfcjrnp9WF0d#|calyZ;bC}!?dtXGeRP-7nB4|sG=XW~t$M0ri)IE;`? z1NoZphOGOksD*QJy{e?}T?!zL8q)R3p9MSveG)_^#gHF*25^}OWq$z6{1bpyMTPL| z+7RL@%(IL&DG?g;s0@pl5IZ(Tsr%GvR*)#1VQ8AmLs?5e@ha4pOyQOcnOx#K1E9Wt z!b3k>i-K7PKsw5l;4r2I*}O+e`7N75m@)BEAtAS&W~A}sKlKB2gMx3Xq>RN3KBg{W z`PvBOBQ%Wko4SzJ5&lksj+%{KT8!<~=2{*3uuVktyDnOp~kmWj&v)lkd2{{J?_TQ_I>)C^vFb`P< zyLgmH?M@OJE1AxrU~zDv!-!AaJ=0uSISoqp+cxm0xSn)gXB!qi=vyoZdD46bfg01kgc`w^u(@3##_vvs zmZxi)nBBN1LSVtT-;AOBC5@UI4%BosH7bIaU5s?^PS0a=Wq6^s6Qjxgn=PfW295$n zvNsC@PpmbI)ESLt)pIm*)J2;|wl6Vcr{{h_K{X}>*$Ll|LB=7v)~vC4!?Q;=*NQw5 z-D0}(%g7Z-1VzFJ(g4EZ93k-~I6jA{Xh9wXVZ+v+ZP1hs{Ln=oJvUBd9u1a?BRfEB zS8pDR!eR|(p)@>EG-xRNG8i<#(pbXMoFR@h6jIv`VI(f1MNexC^fpbXxXBasmYG1g zic-fruBUfu3mqVk(C8r|koO!MT9W@-Yl$v0seVb?O$fZwhX(_gYK3jgsp9hGIjn(_ zkfNiP9taXlL@AREJvClWx!dXMsxalCoEx^>lEzBir#`&Eg3j!7Cl*y5Nf8sQ-h*)K z)79Zh5cVrL@rxe$_!bbG;!w87Xt z8|`D^CDLXOr((oLRtL;ue%TO6sQxN60$9UTj(jBHJtbqdu$HfBq6L#7iFYoEAt^oI zV8;_F^M-+3%#e`eu&Hm7xlz(BFpIMQSK@_Iyo9&rUdX=@f3}cD<1sk{=dGfX za=kd8S(649-O%s=Zn&Mro(aqJXL|rpL-trTM@bk(N6Oc5pgdFf$`5tnF`=sx zM&wNUD}>~)2KEntMu-9(lu~#n(LUn2^@Ja8QX1v^whO(Jia^;$TCNLLxu^veDeHl2 zQ|yB|*&9`u5(hUB)kBf&FejsW?fbRR&q`+7yG;`k*GfWqX9R;RSG-Vt3=1Q)7$IIJmq_&P>jLiN6{grhKq?dld9fMPU<)Eg-b9R3f-my} zA!LMqJM9CINXH~*Jhr1)k<1Zl=Ln~y?rm7AG$lQ-m@*XqJw*Q`Q1R3T(+29~E{fnM z!i!)o&xwJdIrilYe*s5N{YVI88K(u1MHSk_o;;gUY++X`JSJv?&rcsF0+a35NJp#pNVs=~yOV;MwWFp{(`EHfxPg+deA#wc-^th+`duz<+I6 z0ioFDA8%S-3<}hrNlD~_uU^q`9&!d%P6bM88kuYrG2$Kyg5D+%P^mO3*{e>tM$7UQ z&gfBWL8l2r(+E~omX*7C5CfyudMg9k0jWrR;pgR>*0s2h5TcHC=n`y%31D{_dYJQq z=XgEOPE6?5oj^_Eb66FiJAL`(mG0uH$N|unzc=z{rvo};NSuo!Kx42+0k79 zJTWul^QbN4+yR(O&%+Qq6pCZb_%O04Vff@#3A=1Hk)$2}NEH3me>6VU_MO7yczuY` zfG(CZxs@Kx>qLf*6h`OK3rueqYiBdO8Z$pS&up1@6O6QTfWRhtnRllWJT@&1gh1XG zj4MSaAaC)T#2hgYSjL>3gOI@zI41U)p(e_!gGDKv6slP-!WbX#v9CLk(px)&Wtvm< zhoq0DsL{ROyqzmJS1U$#?_@Q!zG@5b78(qYK$WUNllUO%guIX{9(qTJaQYY+OP7O! z{w{Fg$!tT3GzQa!!4Sf!=juP0M%Hn}Z{T#JMxoCy>~J-l?gkADvKT+8|{HnYV zX^(-YOx~9?Q6zwry{9NdoE-V45x_Yd9kV4wx|1{MskmAGs$l|^iQEWy_oJIk-_loN zC068AcU=o8xf8-@U>IK-cJCPpaX})hr^N@)*wu|h>8}%`9=~V^@bLVnDoex(|1@ZF z28Z*^{jTQ;2P3IK@1531aKr)HjZuqMtLeJ!L9T?7W5U8jUdSfNK@rLpf(ogsHYYz%2ipld4ANuF)wd;HMI@vdtM)f}iS%?_Ew1X``p zDsCP)nd)*>(V8!{;^olinfS^akaa4HFhoub2?@B+w2s5`@E8M1+G#Q77BasEy4nYv zMNnY#)5R+T2N=W`j*!*T?VkBOS7b)JpIga+i=OuDFTcxiWo^yVuhUT!HJb)@RY(Xr zIRyl8mTIGk6{S`Ry9C}`a8sC{Up!XUgT zF2|O|#Uc?wzUaQN;U~pCA2+*zj~v-DhJv8a_&hejbB$>D;B#7eh##C~M}0edVVh`F zNFX(d3@+N)ZD~##T`e`m5t`(8o8Lm*rn*|YAsO-LkGlwGO31Am=R>2Bs47R}wQy%B zMcaa}T6Obt8S}Y?v?S#Cc3%bVHR70ZFXc8VCqz*3pddzMD)S)zk~>n=7tB8u{{9IQ zco&dY9N)=Mq&r8jkI%j!L>H#(=ZR`noFx-p>o9He#6wh!-ImlrEa?)a1^v@iEMKmq zl%I9y3TzfG!kf{YEBqvH9Ag>BwuWEfwjbU)fgf4ru!SlnZymMg-$VyQ@N#)nBxxyf z86VHQ901Fyz|zMNArWz8%LgMPAYll@l^F_p9L5S3z-O6(SKbdlN1&S zNG+deR@QJ^>zbBjs`KmKh@{aX&fNDY(Tlp+UeGqD7NO__vjCO^c6rVZDM!k8C43xN@0On_O@)`O$ssE6Mn-^yT_PO|=Eh1b_sVgp!fByt zIPA1S!<>#H;0?y9+?e2te+T+ zDWh4cThMKiGkA?_P%~nf%0j+n4Ssvdm1fY!5zQ*%FM6eH$w9e&I^=aPL@|JFRD_Z1 z007<612(UJC=A3*o>AJ^1ONj^$V6hJgOH#L&uWt0G2Hc3kPUV?OyWAuKZkw75l5#f z5#+w zKB@~4@;tM9a1oA}BR7I2lU$LP5>Ic0WCMpnxehVFOwts`!;RAWA=H>qV#6?eK;$$L z??@4mWCN{;d_f6KacAVxsAU1M(aaMH=-6Y`+PqSK#VO($5&`J!m-fWxLJyOdnna*^ z8R4xa&k~Y%OP*fZW~q<+d7+Z-hW}1`{oF9XCRBL*QN%hqxa$1w8EJ(@6pHErix86q zkhrI*vO{0bmVPXnz}8n!V#0Q@@$JqF^ErTb%UW@s!eVm$7PytSA0^T%nfEJc;((h9M?GEU;jqmF@w>q? zvOf~aBI#e6qlhy6T&By{h%N=w_J3?#OeawW(K51GkyzjWqj&QMkHgT9=#jZPwAdud zo~1WRZb45W+F%MK00Cp8q0P5^npy}{7!^Z>*!z;wl><@Lse;0{p=}^c8bJ2mU7n_e zxkP6x1os2BzXW=9cU??Wb^$OICge&rSXx)r6uw02Cis&BWEvlO1EQ0(!og?`sr9|# z77q27k2>j5ui)_?8C!q=NR9Xwy!N5*A~2uVG;9)50=9-AO)OH;PJ-3$+@AT%)03d^ zx|tuD&C^b*IoZ*B>?vIdR&1lK!-=e2LW_lGx7G7g6IwKS2T9O`KJgTud-b*qhebe% zf}}=7b((L;*PwNj%VA^}34$ar6|vMuX_O>9fZU~foo)po000^0p(aal(a&x!f;PZ4 z*aB~Rmn^dhah&)q1Vkg)2N6~XH1M(v)-#FaQ|%2* ze?0X(ZE3jq*qjF)BC1jF(D1nI8;}9H^bdf?^ybaUvc*d{31&THjE1--;;&8AhpJet zEI7OrQuwAZ)A2sJv=H(gJHtm(vee)(@HpScf`Esa{Tf*{`bbA+GY;swX(Q=Vd@0 z1fLKR2E(6bZ3yD*A7rcK8N_oZdL@k+_-7d}EQ~BH5ai6MYa}cSW}Gc@Uv%&pJM38$ z)2+jA1No6Iu5nA0qZ5ili6>q3?j0nu9)aNJIAjCQ6nf_kG64q-e=XjyP+m+H!QAQL3tUp#~m!uW}6oOv&1 zGV0qY4dp~nQnzdRWdV5yBbaLmAs-tL+b%k$04iz?KiUVLKqZzI{3IF7`kFyxd$U*_RJ9uu3(4@X&Y6AB8@y+Spi>y;#Z z!AOa8BC-L<4pSOYoiAI`7-`vRbOtd#(YZYK0oG0k^Z+LW+>j?a6@YYE{qrZ8mO0Bz zuCP2{NI$)j;lx?tWw8LcP?zoSD^xM9NP^KD1rQ>CG*_3l4AaUy5778DSn&l~l-0!< zyqlDIPLz$|q1z2zqukzS3c-PpJVC=iS_VA)=PIvjeCHD@j~YhwMwTg7WQkkZdYgxw4yg*1cZP62mrH8d=>v6k5jTEC`so5~j~0eO@MFYKTK+)1t?|+J+8m13KZ1Q&n6_5a>ikzPm?do43(=Eqm$lDm^9j%c`;GfPbOU& zY7S|635^~m7NKpVGrPy#h`}0s*ZlsM4hb(%we5g>oFo@pavmV)#)7B;)gHGTNAc@X`GGne8rQk6`&=HJeENl7Nt*i6AtLClbthcJ3IA zO)(l-cEBOxxGgmF7YcJ!A4ge$Bt@OgDL08DwftlvNjqE*Em*iv9E7*yu)ryMC2|&B zZ=_cNA@5bl_q-L*ulB&E61J_S?90f9ewap@#sB>zbmhg%UVh!aILrp346H{4w&$Wu zm~4Xo23XvS$RXl%mv@>AC^0Ai)`J5e5XfGFpd3ZdlaYA<6`lUVUSZJlow5psk-6!xR&#i%>51u@ z2mXOo9^#_>LOB_n`e;rmFwR+9#Q!K6bY~XIols=rbiZYjd?~{og-K-g2T~RaMJQZ~ z1BH1E$rtq_!ASqC${l7wj3`j@+N}qL&Q!8a#g@_R__f2ibX{8@*5C)6ZLS9x?>0yi z^914)#B#w`SL1krgWZEZmw7($|Lv%DFF^%0gpkz|GrO^C&#*q|%5on{ZpQ7$=u-## zA+kxTWe9BlGD=s&dmuQOby1ae?FJoL<=%Ef2*J8SBPC+cf$P91%5dN>`@k6TH$^`z zc!dKV9Pg1+O?LHMF_brc42niN0uot(we}G^Ig-NOcX>XTQv_syPg+?2B%u!_vhWr% zGkieYk0~ZlUJ!-CK|U(9DydIsRW>)#(QrDJ0Mr3*B+OC2m>Z8BAc7eM!e-k88XYq} zdeZ#B)|=1G1KkKCO*a^GD-~w`M(B6YaCv0|W`n>i1Ok>yJ<%`E6Y(Vce>Vn31a3ji zuX)(Q2YI1%7@*EZ3MDw(^x%U&WG;fI!<{)e<2oE~IPecg;Mn=b@T!F17%(65JKdrM z@oOrx0jS?}`xoR!)7W_LsS(aOmOyX|_O6xm^LYQCU~*{RS`ca2eqqCqd53{9N;s}G zNgLg(3BNWXNDEM_DVPRKCigsk?$q;V2}-~@NodaRb&sy=6bz58V6bk=7IMkj$E1eQ z2t_-ROb|#_lTcH5m7otY0xV7%EGF?MavDo^@UXdzL`?M);s8D|+5K(>c0|yFZ)-`I zz{=GGWY{l(m3B-}e@hLwyYv-iXnABRI85s&cfhcXmsx+Hn789oe$$v(E)TK47sk`b zWWR9T0Q#nFi2B1C9~3nGb^1dhPM9G45KxP?>>HPf^en@V7 z=(#Y{Nr$^WD9&+tkkOE^a$Or^Vn!R)z&MvI$LwaDK<`edk3QiVCF5#>-wkBJ8YXo| zbkc^o0}39JUQ>kaosN7J#J_?I-Cy!C_|{Pox%N;e!77fO0P8Y$+=Jc*8~2A_2m>%p z4zP;!)0lmUM- zyp`ONMc`2$#YhjB(K3mj-SvW=&{D4iJ{!>Ghj?ggbbLH$(@d<8yi4h0-3yf)OoU>Z zWurf~4#=GmrCh8`Zi(tpmLn)-D7DbQ0L1vo-WFBc-FIJ6zORPnvl`(Gd_Ea;A3r2D zp_kOcWKd8XSy2N(xZ2>@( zngrP7(4cS_6CDhek&g$3MC}T#VFd@kElS-uSIWL6^0GI0C+wdWU?kA4m~Pc^ivW6gZK#YJZL2MCHRV90o~|V0~xGaf+e-R zJM5+*Z_%cgWe|0o#zD^+>FNzh zKp0cqG;kih8-w1(Lt9&O=)Z&T&Z+)VwIZac?)&M2hGSoxH_u2)c*8Iec;^g(*nT^g z2MQx6vbrnSHr%GH0Hl|sJs?*IhDhoLLq?xbOoZ_m5HVGV4%HEY0W_9D;Hc&SN|J0$ zEuD@DaHb#~M4?1#go)7s;e};RC}TiU7Ohuud(+(@kyv%5y*wQI#EEQ<Hc>aP{TB4jOL}`pIT}NK1m$9N?4i3(&!> z4Qa2}D^T!~j@W8G2m=u9lSw^qh=z1xDR>nz%0@(pmf4A>(3-Sx zo5%x}h-QeA)(%Q)LxtHEbopg^3~}J};;ZTW^Gw*to*5s{kmZC0vKV=6lvuO2;kg+i zr`6GR{3PL(N7?6RXmUYJ>6^@_D$1z7E+@$W0l1mAaMDKtS^?gtQB0(?cuVoPoNZuJoC%4m~A{S z3?~aj{$rv8MP&*r5yCj!IyqjM%T-53$LqEK!z8A{qs}T&QC}hon{mE#4!?%Cp~be2 zq6{ z<2LFs(0mEhT$CS?`|uZbtH1Gj#)u*wa7Xr@q8Fr88rOS8TWn%Sn0Jl17l$FUh zX)VR2>$L%P<^FE6;m<6)^%{=%~I32Hze986xWJ zKt~21o8Wj^Y}_kD)M2(LZ;#e)T~I|G%6U9V5h~E*z&&bfy>Smj2#qDB!*!&eXrJUN zmFyXeX(vw?k%NPV$;eh@P&a=@?HAUv&>)M)1PHN_-F^28h(Q?`{}&*duiw=^8-n-m zo+rWMhtN0?mrAB73eogzp_*+6F;dYxvxlL^iJ~)SHw1RB;_R^bwNv)YYyQ$WMi@V> zJSz{!@_6FMs0mM0?)^*XK#b-S0EX;}s-0QF?t4#kP$#65qCk9zmh>NRCY%wX4$6xW z>1?VX_P?{#kx`hj1+R+{$j6Xb(DO#;CZ4_)knmX0V+WE>s;D+OjBDi>+=l$6E5`1H z#E`H6t#1u=xr8As!|}jHmzx8gaaS`&ko?V28O9G65S}CuH4w_>|D-chamDQOYSBT{ zPem1QNlOrVV$AkBh^N`0AcTg%UCkyt`7A(!DF8+^J`#wQCyCm)rNX`{`3fdeFP@bG z?{iX2_##8ppGnaip)`RonW*%~!nwI}6v28ep=#gnA_#_t4T21^1hj|t`~oEPl*lm& zQiSh>$(>;dsbiOzcc*6%2X!#_fWq><^6o;>v84Ob=5D#R`NjdGlh!oJqBd`;)jc}v zP8$i_y-}9}&0aoP9M{Gf^@BMwsPqx+Gpo?>0Uqnh-UObViFgf=WI-e`G9>*&-1DKb zcpJat4nH3dQ9P0p(DYX3wxNYP_Qcl{A+UGoi2yVPBTiPi-#qB;D2gGmvAnBxk8&$1rzU}HdcIwQK%X1)5a zE+6>0WK0W=`1S>@)wl7x=xTo2A%3^m6&Ls71bA+hr$f<`mn(EbJnKBVT@RQsZ zIxey>;497^M43?GU{mVAmhne1o9ICs6m*%yHDgUOjMY^!Ygs=wnNBps{x}nbGTJ2B z9V5aD=-2|UOWyf6%kh197P$T@ImX~b(O!7$*r+dFc&NJNvEYAZxyaAgvR zSWZVD0s{E5;LWFl`Tz_6lFVC58Q!HwUqwpd7m6H=3A|P4! z`^zk-!5F3wWaP|Sf*{@lyGE-Lb6OQT<`>f%WN<=rR{T9>4^YOJ)y$IKG%pH%x|JBE z-}{n^+er~4uJW6tw+Rdi#O6PeOxBb6Rpf(nN!FZo-bSAc_^Dpj)7;`(H}ur%l(HTQMi z2tA=$;!1D8Xk)-sM~3?tDyy{cAzgZEdcC9rZNz*-VGEH?=CUGCZ? z1L&fcV#&%?0T^WvDz&8RiKe)pKsLJMC_(OMgfm1#Q^ z!uC6@MMK~aAQY4LK|O8&h*Bg+NGbvz2jUr}U?W@34Uusx_yK$n*1=%DZj2wF1z_f~ z8v=M+j^h%Mt+4{NhX2NrgIXGUoh7gv6z2=AQ2msF)SPE*!wNiglD|P?#C+Lry0lIt z9`!!vg$$g$i2It3if5+5$Q)8uS0mzaI`A#)7V$D4mc!A0YzN?!AY_4 zd0@Ts!VRX;G!%Q^?H~4N0K>1_c!)}=prirrl6LLmOr( zJ5<(50f3co$S1bYA+kZp_J~Z1RA73I*zkj*F>Yc6Yp`OBy3Q+3PAbeM561g2Bj%SBqWy!X^er=Bi-XJ_S-ZF3=$E zq;cr2+P@dcg1Sb~7up`)k4p7oMjK6uU*Iao`{DZZmll;HoWa}v6k#hhqTOK^YA!4(z68yAaA7Ga zPsO?oEPe@@*Sr%Rs1ayO(J2S0%!)p`RMOZ2?pMJjz#<7^kxiCR(g0CFuD{kvzjiS8 zA}B}tC(Wi>NVH)|CA2k<&EO&N0D3%+W`>xw#B3YDO-teEFLjXW@>~O5N<0h^NPuh7 zj{{LMBEY|Z8=FJHB&MX7gf-Dw05_YA6BZgBEmn)#X{SNB4=hrsr=&a;Y_c^4T%i(Z zR$?X{5!XX1*Y?%xhU0Td(t~1{T~%qs3T^wjK2TM67%fDi8o}gOHQHPpZk%`u`vzXg zNhRVi525!QWaCK}3L6EFffK$fkRAl(1$-I^Q4Lo7!pWfpj@TT1$i?mJ*JSdL;8zHM zF+sp5Wzae*5|ryCvj90s8JmSrBR>YwpedqTmDgpeU>Wc<_KG z#es1!ux1?+H2;&xzorB|(+G$@5atqv9#I({pYz;~8bP^0R24dK`Nx0{J!3V`kt@c@ z_e7xTFuoISHu|xJJo`J7}izG-OwkAGZd6#6hBO1hyq;L|ba@6GyGX7Vi4#KZwB6QmBkL zKD9~ifm@m>H3$vi*+}Gi2?!RbK7N25KY@Q)ZUy&82q*^HhdYcR;s8U9dix0&I z24W$25s}hFauN_)MfPM@*22Y*H^?}IO707wdSuLi7|VUd9WuX6PbIYYNbtZ=8f^ zY3!%VSzj+`AJn^14yDI;kPy2?QQb)+DOXHJao@OBjxSLwdZW5bf6TBg#@Bs(pqw2j zJBdkY7P?+r+r#k;71U#hUH?rpPm!jzs#8>tQhPK@Yf$MM%52em*08EkBf05=Dp-s! z11J({9eY|Xj4CFNmC&}R-)W($HBI1(~`}saHQD(KF&Kh%oSR?A`M)5@z z*y3@Qi65Z`Sw$7yj7X21OU42!y4@0MCS+7bSJ~2IzUN$%EGHLzWu0mtx>Ce8lowD_ z?HXmX&&Lf>)k2NWeZXwOx7?IOqu4zNzB|c@T0@B{2d#7^i?YX(_B?J(%4a|>^P~cf zygqCU^QMn=6e9N!TZ>s$9-7)@_-cm-&b}-bZh4W)pxaxOz*CaLrB;!Mwo@C$u%@w9 zgeiuK9KnuwW?>hHfUAPd5mK)}pud1_+m0-anAZC_o5aZ9$Lz%p#NFjjejJ#`sXO5B z7(RccG2Mq8%;~K+;D7DI&Ag`G&4a**=MQ>F-lpD6!-%G29TRGW`^_@%7`qcI7s+{; zL%Cc_U3`9BuBu%EDRuUfM;7$Obz+4t55NwgiaMqw6){|5qBbPgNn36pBC9L0m&zz0 zVO&W`RDB3{A6kSC|B^qolq`*=zb|dLy-i(i|@Bjw7 z5b@w>70yRS4S)Z>fUZGU3fDRLR6B&remJyL# zX0!+c@R3>nCIt?nA#G;(uZ}rKTfnMITL=CmPg9REMAdB zC#e9vZ6pRLUkh3NE7bwq1Y0Xz+LZt>$<{+yFT;R>2blaM6Rn3nzu8ogYIRU14mVL} z{0eskMc4wAOjF`(v#_u?hbu~O_i=>`yaj{*f>+z#ZwB`jXtTjsgJ{gtHxo$V3$AVN ztdd$IMcxCOhQ%((FW+DvV*u)+yt7RdmG-c~n;dPCK08neq(CaW@$p*Ms2}iSfk%06 zA17zIvw`XmI}syb*sWk{vHQX2!5MF;dsB#r1$o4!*Sh|@Ldt`AbEN}JAdOSzJ07=A5;!rLHu zfmA+P!_w1XPHA?9NG0M%710Kg=P?m6nd232h9;Q9w~V^UN?r^NH~7W%3ps6|fF)sJ z!b2O2!~rU-0H`&gUqKcNOF@*4)HwR30T_sjFuE(hS-)NOv;d5aa@G>K)#9xnnivDO z2VeExS~p;(6#k!;$n^Hv^^jjIy#CA2)m3K9=FW^oKE)BMkw3E=*EzdUU>;4W#1mx% z+7E*XRW_@cjQqeMEK7}v7`Qr!djr_8hT8(nJl&YBsM+hkS#cLt-7bUHCA7OL8Pe}A zJC-CEYz2h6udhj1#g^=!Y>7LNhdF75?MkTwzA2H#spfYO145ShvgE-9@yM+gBKK3z=+7Cl_^wQ5EL~A zl@SjLTWVPV?56nXCJ>a1u3%SaHpqe_LYHDZql8SbS zqo0+UhCN^yt2i@AEXdHxCh%CF6G$3g3V(`UXtoVZnMlYg38}bH5O5-a;U%#w*CmaZ zQsjf6VepWGIR41B;v^$CIJv?cY3+xrjS~bG>tGHwCWl0Gd0xPsX8}}*G_{yrsnJr>i3L4x%0P(i$)Fu=})Eg(VT{Wz}n7oL!ynL8jP-qM6F zPA4`1%pXR(7D)^UoAyHxeLzCm=qyNCg=xaSHi)Ae_vZ++yI%cxUMDwDpHNf_u?!-d;d_oO#4nQ6;Ft*)TYmjsYkN3RPz zD#5fu4iRUPXdihY7C|HjnM2pbn=3)8WUGv+kQ-qLv`4AIwjmY-cX40I3xq?2hTFue ze9rV5TyUt8FRB1@@QY%Q!29p={367TBsGFmNk$HHah=CH8`%r{L z1VcKsKjdgr{YgbW6{+D70S?~Xlp-Psg6X?Z(awJD$LAX8Im8R3g&)JNsimgII~MO} z8#xKpASOc2IT0PXcM}BYL}&p4TP1YX6XC+_m~AsG^yOBQG)o8TF8N=bfZ2)R(-M%a z!HqLqmGUmh{zV+pk3Alw4?7crsjN;HmBBYGhOQP#!V*;?2}El}rcZ7KNP?ge5D1@N zMF9REp@gAvG$e`cmd5rl$iKFdWYs>J?&+08|t%LFBi9tdX8E`JRMbVQA}XfL8rjPN!} zfP!~o0s-@KC3yCtFOPZY|J=9=s80{yNVo}nO9uzGEfr*LaUX`4jfG-WcuZEYAx8&$ zZY3ze;z5rFC0OYqwTHF^=sDq=2<#`|!Vnb|JdF!%8jvEx1r96>j$h}AiPZQz2Q6A=h5Ce zzK#0|e1k5$U~<8V>GIlOezIj<(sI&r(&(Mp7jPY&bVTfmr+^F?4vl!?;XPIuV22?W zKDgz;XG=O3YR~`*k#xJLTpo3EfX?Z5lb@VrOSA3TebAw>l4NWkdl&$IFBtA2nc4!8;y564T$CssHWl|y&r) zP~2C)jNhu6yWOfK#_F@UhDuSp*%lWN&Z4-6U!K1H-F@%00J_v}EWGEuym~qd7FDtY zZ`u{D<=Xbs1hhZ`w5s(e{A-?I1XS>`9qt$Fk$zHT_ z(yPN94O2Rg6rUZZRptF^bk<*nPVu%+^Jngy%-S25Z_Bd7rlx0>)pMC0ET-~EC zsrMhB@f1i6>fS% zKjaKEe;{IUzCh%MzCgn!z7oPnE_9fQ2Xg@-u0lj6Y8I=XBY$96;312ir*9C@>>Qc^ zguGbHbt6AlUFh@MXU7@=mvIUep5U3d!SS)M&;AC3? z#<@}GVCnaILwD>>?>b@ys9wlc-PKS=$N7QjwMwo*TlEfxgY32h+kX_j#w~GE|Iuti zqC^OYm!cKkhyNz8h!G!A22bW7%W!~_-zxblE8e{oRP za$b{b0TH=6BXTr88vy1?E-MY}}VT+?N#Gmjv9Fyxf+g+?Ibd^hm0v z(B^HpO%>dpKr)rzfnEETr*-t?-##;P?p{1O-`VpPnFC0J<=|51Crj*|F1vDa{8Ti& z2{CijGHw(TC?OdOM~-Zx=pH(3ZqUgdAAyytJN}S)N(FZ0^S9u)N#KWbg{rc?r>|Ur zTR57ct-;bt!61EER;ZIqwA)YLdAFLi*_f=W@*aX}73eR3O~_BnuLuwE0kIe{761T* z0cYq@RgtOC2DEW5jTVTElr(Dh6b}%W5Q`G7>BRAP;M9AcI+;TAG64dysRuD3IWPkg zuog$KBbgY91{fp1Lh|nl?BNBzLJMMoD?>sm&cT75{{-05Z53T+kU%^d2YhJL=vp;U z&~r5N{RyTm5r)2ATQs&gI=GPHE26=^2Ijai(JoGO)2nPU}IDxHIQp(Wq62DXY66mx_zjzBmUfkl?gM+fGFy z8cvKxmK{aS4nY>t?P`K2o=$9ELGA+-FY-;2sli*KsFga79k?k2areC@pMNsP}o5lN@7e&OJ01@ zVv2<`dITa{HXO2G`B{xSCTdOVPF$slQqlcYg0!67KOww}dD@~OvIJy{Vks!`sko?@ zaW=Xy<1(fzR6%Ll6t$Js7D3yU*IFUHt#%^)*^3$&nnkKA$II1^l3gCV31d7P&(wg| zo$H5rb~(oGyG0a3+wg>|g0I|&KQE{81ndMnZNM&EZRVkCMnT5MLh1Zq&>YL-?RF!0 zeJC9@MZ5#Ni-Gnrw>**eG(C3Le4!Sbh`p@P8p1J(IK6XfV{Y{4$1??cfsghjCrA!w za09~zOf~`(2pBT%0`zPOgTx3&cF5qL0)RooVE|%8BC?8{&O`cuhb zWa+6(<*G}HI7++>h|>eY%&mVANFx;JqIL|mi8(qqRiVaanU|Yl zdyA`xhpb_T-uGVZawV#M!TXMrJkLj z>@1Y|uu8p= zmM%Gh80$h01-SVjX~8oto%kB?{X>#+$w?Y&AS&_4@fM1Eex#3!2!s=P(EqR#>YtMf za>E-xU~nml1T#CVWtp|?fQXW}WEUuUiRL@g5=f4#$nR}OMPjYZ#~qMClzt-zTPBSR zLf+s`xpP=xX|px5%r_W#=xZl!93m#6({aX#L>&(hKjCt|k@U2gew6{?#NJ67M9hPM zzT9(Uu6M-POjqflPU7v1cY&%45}5JeB^mI)Ec6=6_C^5;qB5tO;b}S!1uA;rkeF_> z8y`qlz((nCd0>cSJr>XHdSh>Qhm#cJIn)>+fN%>58FyAtB4=kWf&VmK9w8e~6olIx zI7GOAE!3i=OCgr|CKwD+s0@l@By5e?cZqx3AjHsIgHh@GCn6`=SR9m+5{~z2gRZy}7JDER zSQ)n&BOFbrL4%h91wFPwk4GzKol@Oy$9yy5Pt2$>S8;U?^E{sn@)i`FCLv_DZA0;; zO=yWh86Lt4TADOPz>xbFTknoaug!~sWIi~;n3xZYy&Cc~#EkV+UJ0%YYTO7WlhC$q zQ`{74)gxVR_uPdzd;SDmt`^P;+ny3t#CT1%nQ$S`Vs(rm?Uyz=cY0g!1TFSJhPp%{ z+rqpV;nemel|2lzE8w@aKLg-%Ky3=bt049j3&55qAR!Ywc5))8P4+?sFt9V2hSs5r z4yJ2DIpMcvwcE#GOs_9|)Fi1JQ!*4lFtqBmtl`Eq^X!%jVBN=RJ`J|Ee*Y4$T!S2_HnM;x|bbk6HySs+AW^1)G2` zj`TD^jmY6v-JkJn)VR_pfUXP)2E%TCIwY~BP|fErkRJy2Ifxf%B|taNN7Yzi52uU~ zL|D2Uf|AW{2#}%@$}Ui}v)|#jK9~>HCSCj*2gUtUU!YT>V27VZGYAx$Q_v4Vk=0uC zH)M5)lGj718F4;5JYUC;@Y;=J1Rf)S04eEac2Eof9gEYx;gz-rjdKo02nfc_D8PJ3 zL3txdABZPkGq$vl1v0-)89FASZfRC*g14lg< zRJhg<7zX2|RY)Y0js?Uw+8cT%=WEA_*h6vW1}3R&83fKdCg1mI8F-fe0lVQ!+Pr z;z|@{l!vz~J}17Y4st_mmj+-}E~kL2;9W01Uzzx&hVba9IaxG8+=lIELJ>BV`iH; z=?H|?!DlU3;ZfH+W*EC(2HYfa9RS1XqJG-lvJVujX_!i}gjyz$RAOj4{G>rHcM$0R z3cbqU=JD8mCC#kIhoQmr3JJ&`RG-s>%xU%@7PC~Wz%Be@`;orwr#;Ywa651lV9-oC z1W_~z3O>A@m?oaXXuw~}aEE!Y1_-2#x!@4p`oz7u!gH0e!OuX408fDdwD1UIfmjg1 z@CJFUJI`<80wRZEr&v?b@{GX5#3Ju z9zDbAXmLLT!|WR6|Ep!1{FB#N$~U0$h)%{kk8F1utJ4&aJi~ej17*}1&>{!CI~XI} z*uj0vokUb|r?~j3Qo43;&~qYm*ib;6{E<KX%LJq$g4Q;e}b#Uew;AwN%lgKKuOnF6Joq1 zHHd;K<2PMOz`^2!1+np_WYO>%YeQ7E5Dd1>EB~G*+I;JMin_?5pi`^AZ81x_$09eFdb|&a|X#-9)nZ9_p8ZebETRKqF>@T6rs;zv*;jG0_v|M5im%= zmX`)Y8eg#qG|=>0pdd1iba2}$_Rx4PL{xE5aDai?gkZL)TLdc1sVST}0t58{P0{w; ztwnZ1X)|R(?Ze!1XI^zyMKMQIwMC>!T3TvAu6eVPlk_++&m6H zd8J1Wk9P3A^`0$`op2q3|%(+_f(8E)={d{Zc30dLh5VZd;>J$Z^sh3Mu|W37ATX zAQ(}A#5gewBw0QH7+jPo@y2BP+h#$wA~bEugANby0@THCmu1uSAD&Rc8mGYK+~N7^ z8jItW2*I&`4akPV2!qV59a@i$xsLl1y{$+tltKW>ZwoZA)9!?YXxsm-Ih#t}^yYfV zXW;;iZt*6T&~$5y(jbJGe38J(`chJHcZslH0lf+VoKZDk8P7dM9MH#1N`gi(yo$9> zY+O%VK|OLZTQ{7Ydk8@rCL`r;gk4*e_gUQHL~$FQdL*mfK^Ga+e@Y|~m;CYbvS2Ji z+H4Rr;!iW`Ya&H_rRsgA3J5XGZ2$(;9Ef3bSe2hmAj3`hw%ZCA?MT6AJ9v%O%u5+1 z#o>enW|3&Yg{dYA@_&#@q`b37L~|@ZE5U!_+>yYX4*_NaYk6FiH{pN>&2#5vP;HY# zfC|axA0BpST3rQUD2-OJG)kC7 zQsx*k23~BHs5^&kve6Q#8kBF;P#L3|3Ha;GdIG!@9U3b?Zq@bInVQWs_)A#M4@GTs zLCTsi=9rxO7rW*tNe760r5p(11Srh%QQ}wYIRFal@6}y9*`DX*+v}cRBI#_bItf&J za(17eg~(3%sgaMNk@sJYoRh_aF(-gbs2=31v+VB|Q_9+zEK>F7`ZTQ8QvQ(-D@ z9?l?^N~<|%F1L~)!Y@3#d&$-8DCh>OG>ZmpFE+`Mn$Y$z_AY~3O)9AW#VSo}MMpnC zGS(G=%_vHb(Rxkh+?mt51_bijqI8%|1OVkh3| zp++X2k)C{Z;bB9Ts~x8Eb%CF(g;tmhF++S=;jI=`?)QiqYcCQ1i2)ou6J?d$_%%9! zdxYz_9yQ5Enz-z7LZC?qtdun*ad>gBtF!@OqINlWz}>noEoFFr$&PKbPwcx0G6&~h9GV6J0`|@F2hjpCi6ksEP!2JE8{pl*$3`>?=Pwze zj^HH`gWHH-AzSKpbdZ_yRw(zKndA})1JFUf@}&=tu=!5rKoJe*fuS2!!XqQxY;|hx za7j4CYU2qTFeRk#{^MtL*(i?1>C%sN8|GoHk=qIOA*+}-N1jlEd+-nmL~>)zQev|t zqt6PhPzKM?@Pb4viXIRaf?aIY$=#ii`*RT-D)CB1f}L0>ygrP0)CNY<{N6 zRVFyq@Z#z9NVQJ z<(4}Ph=wo|qouUOAS1z%Awzgijyui2 z{hF2Ll^spqn2$`X}3lqWs^1@)Obr%`3Prk)DPfJ zLo$$grYSB591eU1JUQs@o<<-w;c?2R?Ab|DiK!+OB$))@|EYEko+#-`35h&yMCY=- zwvPa)T{|pjziKSEP@aP^J^(%1wQmxCr!WNkW`b%>!1ZcS-u7sbd;(I!-x_}4FhZd4R(s! zz6!x{k!d0>i-5pGrQ*!EwlmP+^IM>XGw(ZJ(N02SL-51mt0ALI#Tu+^WUF7*V`|Zb zxh5V%T#_m#Ax_2)9d1tOgdy(dJMqrO6{46HYxn>B8qgQH(qsN)uvt8N|*?(r;gD*u7|O>^a<(mJsPWj#$`4f3tmM=*Qy$yCf3#Pkrv z`L22d1RG=aUnq_=)5MC;2AQV}M$R@v7e4|n5G%9I-L03qUdAxS@6LU&O_ywMOv4}A zWP9MY7dry{3d=d{qma;#Ps22j5bE|f8;VGQ5oUNj^>C4-q8>|wnHARj+Vh(qCaSl;Cl2Pi3?K>SC>1qrkj zeJ@Oc%KQsuddc2lX_605AM|)Me$F5;@(SEr2<$VejMK;=Awq7maRZowj>8(G^U)i4 z99~x&?I`QxNg$h(t2+v7l;)Uu2ZXV?d^&dERGnGfVQ^3)1R-eW{WC7QWD+L4f_nBUEEABBE zTO14`>0?+$4T25~OD!5{=M*JirbFsyFCi_ldQYeWr1#$^eOND&-x;dUjt69DAZU{a zh!$DDzu)E8l?bTrqW`=@#n^kodth&T_FMRQr7J(D`rap~tsNazYpsnbgorz9-YZk} zPN6X|s)j#PJg2Vp$iGlG=wstx8n>2Envx+J7gF)8k-Z-R(azf8PpzY+5`A&um}eZT z!^!}w>(YEvtN1gA-iCb6bX-`i?5%9Ai0fH6auZQ+HKruw@6!P~8W-1DaP%>Cg=&c466pTRsoD&FyQb}yP;Q?rkLbE#p#~Uu_KzdG(AWwaiF_$DEFb*W$$06e} z;OtUk#=qc@xeQh%G6+#Y)DSkHaDzlWh9|d+8bOf_=;_8EJ^IpMC|B%W%32s|;0jKoL6X!h*JUTwwP$|Gj+=Fb3x%8j~s z+1l2TcAp^%589!k&A}c2VE^285!S$k2qutEtpd*t*gULdP9RgB4rQ8K!@6MI!qf|- zWpdOg2h=)<@Tt4rdJ@i9+ed0>1?Yo52KvB5!~~bCzaOJpxI>+})1go;1Yn=Vx?_Ze zx~PH%!ED1sZI81lg3@DY{@MGLlq;IWd)lnCi-P}OK|cw2$4JO zLCNdr!xB{{Qw_2ABpQcx%&Pl;=1Sg@@{>cNLv4u{Nb_-UGM=7E#1f5r9X~RsCh<)- z+PmlD@;hUVYH3BUZ{@V!C#dX}D0RCTRJ5aSh2~it zXaS(07$)0QntiX|<$;F#&s(9IgOZtKS>%(w2+_%c+7NM-h@~h2;RiIiMSP3z5(l8- zJY617i)P69h3!O8cn=zhs}O>Pw!8w^LkDZq1cSPQ^ZEShQbX8dS`p+XM_ z<{(f@2ME8w{VjT)004oW002-+0|XQR2nYxO;wJx4000000000000000CjbBdaBy@l zW^ZnEb1z6`Y<6XIX=7ndWp!mOa%E?AY+-UPW^Qz3VRT_^cV$o;Y!hurf;$Aa;O_2$ z;8xsAad$7JCAhn5ai?f;DDF@kiWMtfiaYInznS~%&OJNlIkUU-?Ec%?{hk2;5dBBs ze+CBprU;Qs(d0wf^;0B8U;&3Yi9&;$z51pHro1^}S{)6Jt4TEPDoH~?*c4ZsfI z18@dd06hPTkOK7oi+BRO01f~*fa`w+cmZ4hp8pI0padxWmvaXA{`Y$Sm$&-w4-}yB zpXc)*{|8weMa}=W2MF-w4G261B<29z#30khapUHzb`uUd0ihugOnPF)0>Y(sWQ%hC zk=lT$P2X|Gg1M~H_-fi?94`$G;}GrmRjnJuemGy3&Ks6KblaYBj;x7cfgU=D&9m}1 zy`I3|wAi-I5!vN}YstG(CDljd$_2+Q+g|DPns!2DwA3_MKCE3Jy~xz#~IthfhO zrSL0mfLq*2RvD$Z%WmV+JNd4Nv6*KjeN8FZJcYo6{I5FKpXAIx{Gd_UtS{c@O+s6Q zXXxUt2UzDk2R}@V-QcW=tLDEt%!aZhv&(%^hK%ULX&sRK#IYifHf8LTD52HA+e+Cn z>z2;VY?&KI1F-%!bbqjT=@57X_#h|I|&!o|)j z>JRH@D8?dYR&WK(hf~OzrG2GaYjLs%Jv@u4W5aj82#gkMrsZecXGVOTRJs?cRSovS z;E-{I_Dc!=w4?x>5^OS3{J@*`HTyKxvfZ~m#W==ef!~OnwBhi_x|qYs`tNRoi>KQ7 z(gf?Ogu6|pPd;W0t@3Y6$ARDI#V|^S!Bhfl_IdW}_~rNSfhmWB-KC83Fs^WAGsDif z?vU#JTa8o!G7=-Pl6_v)s4^^F5M8yVywam^B}4$SIfI(--yC_llAy-9Ylg#HmGHdN zD}vSMO2M-ps*aPUTT<~6R*m{*0$3alv3GgL`qrvSne%{w3pRn`ZkTWZ4aO8CNB=BQ zEn9?$N40=++Ud~qTg*1hMH|)5h&77#2qsP zo)W>SJ<3Ja#`8V`4kne#HXEj+XCmSG^|s+H+rPY;5sEU^j!r$hXj*|CC9)>m%IPb( zP>~D0WEvLCd?=g{7GB_G-LsSZ5@#P7qWn_xFA1eG8;KHzb}!Z!4iIk)KI;4}mN4o% z_IqjZ!{Bxc+#B%$#Es`nC5BfG)+W-i**hKBNwdiF{ovpHRa z$s}J{p^>MBE#9)6T)h^D^HS3VEpS2t`__}bfs>N2G@xjAZp4_U&HRnsNp0!KX_2w8 zt+RMLxBiGI;ya!g5Kuk6Ww{_%>0hEco#@L}j)pYa6MFY^vyyH_zjg3f=$azI>J4Yy z`{CN?Mt>o$_l+7+p%wkW;)_Nbtl=4ysYLbF710qraU*lu*L`#7r_SkK?Q{w0^q{Dd zyyY*PPBB(GsMv|+toO!mRHjS)8Xhg0_eZn-_5qtoI}@ui+UA>MKfhPC0>DAP-^_nQ z9KNM1P(|=o5>5%5caP<-GUkOmX#f5GDPtQ9fDiyZmjz#TBb+p+or(Xnw+b3Fmlu!o zUko5nA+OaEZag)a%@{*`7_~K7$?X{}s(MiM<_9QYSb53a@*7xyZp8jN{PTy+a|h1- zC!YC=Eu90z?m7K{l*nar-%7H0lJ3nnrQU&Cg+ z@{8gvK3%gEd1jrCHX8&HRmy^80p?J+BaEV?u!(^f*9})buuf!Z!1XYf+IO|y?RFyE zb%K*M*iNGwdHuGS<;l@RETa=F8s($-AyROUm{6tbN=+Z(S9(CJR*O2uMB44YU*pX? zKT_3q@>NuD4UytT15k5#7UzzAEQA->eHuOZVyKCo6H4go1=M`1l9FS(Kxh89TWR+< zS{*?%K)N`z4(5;T?tt6e(%{mg9Z>vyYT|N%v=Tys5DhfM>wb|q?97;mE%j!s-o>wx z=od4f*bAG0TLc5cNZ=#MsHA4}vKbcxbW;nxz?za3rOnLPFGThu{V=WYgns&RXdgF9 zf?l21yGLm}ykEbKQWoLN;db@5@>QBl40Hg4F(L#6som4z0s(zeddl29|z_fkS z6VAcK+G&Y^C!zIONeZYXeaki(19qZT4U@MGjWelJ3XL z;_2I?1Pg6vD-$Bn=J8MHSQMuD_?X=E$5+Va`K#Xfu9VocKSlGnrf1jUZG__~-)LTv zIllBhUAv{1t2&yP)Qu5F)a9P(SJ6{y7?SKXH641I)Z8rIU>TXZm<58C2o~qwKeC_8`uJ5*?vz|T1>s!ja z>*W+=b;R1w1~3I935hbX<5~^>c!vqq%e>2njad|XrTg;N#2K5g-Qrx;n8$_-#aMn3 z?=kw#vGz%oi$d#%84C?U1a?SdU_GblqrzgxTbNtBD4;V+B*Av&imwym=t?|qy88R> zU_=636-Qkhk2cD05LzE@o$zlM9vXx(OUk3Z#S>|{d+P5ArCJqhA`j`np@@Yu?e!j7tjvqg)}~Ghq7@kBIh30znRcV zzsfFy_9{z9UP(ZkXIm7NDu8vyt2XRBbQy;pbsXIhJv8z@pCA94>>6!EsDz@N1j{zG z9V3I6&E{fyibqgBJ60n9p;Y6oa3T{qoPzlqh2IaIAM;g5Chh^WokI}H>#bB19Z~Pq+=5 zg&;(OuRtFizQa(VU?qw^(st}jvPdEGOssHIb|P++_v#a$s2{Z^3YacL`fu$EY^$Tj!5NzRK1{_)XR^U+J_=Q>vs8W$Lpi&mzT!`hkY@%@zINH~HTYuUJEyb@HVRht{!4^cz4@hr;%V5fTVs)~qiM z#fda4OItvouq$K$Xw5NZV4g%A(n*u_g6PONOSOxh|4gQpIz?vP#Z4+;bIP6NIC;Us zP_T}ZpYo$&vc}!%)g-Lp@-2?4EntXa7(r5$BSbdUo-L_g^EB|Tj*;x*gw)`{haOi}#MG~eo1 z<5=Zy6d4057H=?4Phkb4j(k>P$L3R%fFQYek%Iq)2jz0$h`1UXHOL1Orv6cDxfv%W zuEh=HTb9YrXvmoA;yc?;W>5sjsfZ5g;x|wk^ws*e#Nf~&Lyo>s>`pL|gx7MDH}qgJ zlDz!4p|b#9>{4*iBxa8)i%Ua)pq5e}RYP`eLU(aH<8D77)&Ta>uq#dqAU4CFgGQu% zil|C@-)J#G0&wLvU5&nc6DX{maU2=lC`B%QU(qkP;XaRuAh{uhH6U3TY6OuL9Dhsi z9zTm15cX7t1k^(XGQf>c2a50Tv0v^V$9_7QaQq7XE_1p(d(K;XT+sN&*bZFRnNlA@ zM;gWnHulluF>a^bbKOe9$?*#Lvbw_!28GSRJ}mqpRynuw`S@*2)euElmJwHRG}blt z36CQ^M`Fm7phcY{{42T63Qqqr(}~5&0+|xNGgl-QWgBS>#otPo3C@NzVV73EbH-4H z>$-SO%aziO#11P$Ei3YkP=OZtGy8+)mPjsra!Z;oeUSn%iPbF+HlbD7_-vP`+Ney~ z6w%PxFVS-35oSr0(0f|5vQEqchiD829B6BPo;8wEe}P16U+M5~UO_r}aqsg^SE!>T zt7aN?ux98*Na^M2%{F&*WU=NhC#8c_P|P65v|T10Q%$9YTgp!%i?bI?>smsTbUQ^Uz{T4%g86;e#NNau6J6dhg$6v#IM-{Yooy}k1 z4Lm|3yIWYeD1>~s_4A|7>6jgqIU=ZV%gSpgzuIqh>cR5bzcVqysdM$qHvS+fc*sub zlA<^B3RbjRixdC)4eB#!(k~&{{WC1OTL-Dz4FAJui11LJKSkg>H;~(a%XZwJI4;V+Nxiu2+F=%5#&IhbmCl@5-+9jUub8Lw>f&r?>%>-=r#+Kx4Afn`zvBhf& zrSh}?lB-y9i;}AxqV4+@OS^toE1cK*lc%I^&k*XC_&H$;#QvG2PTr z`eHIF)TgC@sZTfYw1ME(qMF@$!!x~8T7OZAV{Tc!$YHbAHoePo0KPr8D|u=iJU;`itHe?;5U7JB>HNwX%w|RHKkchNbb(`FhNwI=PE-pf)dSZ|H zGv+d7M;v9xLq^BWC&`lidg6|ZW>!wy*km)hhO$DevLaqg6{L&|Kpnq~q0&;F7N26; z5&?t%L+X*%Zkf|()^Rb29b_)MS2^Qsu{l?w?I(?LKaJv)=tNzR z?rsR9Gkiu`JEgmf3MG^RcepO8t34t#J<1V!CB#P!`-)3wM>tc0o0ojK z9*G;f;<0vM*3odLocU#X)>-d9e(7xgz~7VFgQWn6)$d!3*1v7jhri$tDD(&;8BPT@ z%!=a+gcbD)*$CGHb~%p|K}RZXlsTg})7jNo7EwC!hBvg#!B%lah=UWk-gJhKRQstp z`vH(~SrFC5JaetdRf&6GsBV10Wy{B=nRXML626^XHIjwqT&Js2&}t@3P1E-V)DWoo zmf`PX#P1-9e!7*V^n-Z*yQAQaK>nd~LW3FItgPhP*dImVu`~(4!%f`v?y~QXl)P_l z68K^TF`K^>k*+;!Et1-&TTJC6ND3wB!u%D4BYCK)2}f{_gc^?Hj5aX0>iMD zHzZKWWu^$rtTJiyW$-*}N1xPAj(1O3SQG45QP{!+tScs~%LM|f>oc1O&v@018O_y+ zNvRW^)ilCq5_V;g@_$*L<3HGP{&-h9gN21E){xH#=!)fI#hFhm_Zfb?_GC(M@BHET zyMBOAo^_o|Q-^Uald4`-@J@7G#!B8{fp$UVtFKn^rGTDUdXE2v*zx{xh=4-ne%ZT^ zr!oh!2kKsB<=z2h4%R&RWgGS0X-?O}WkB0;VgOV=w=t!v$ z%=|}5bs^Ud*O&2qHbPm3PUawN3vJilzYz2wrIp+eBskW@%XdUo|93=zf&xzzTHy|d zn#kgv2s?$0KPq2qS3*9Q*(B!2n1QPOAAh~QzUF4km2P;W$rsh)M9KPo3|_|}{ebg# zS;!jys|i$sUjPHn@8g|ZF*c0JbI=MWA%{blQgaUJbG1a^x+r-r73FV3WPieyLN#hp z*+{O~%dS2MA8sm{cmLAd*b?pMa!`-fD{U(@j@nAFU%~b`6S71R*V8_zJ>MkRYhmp&@cN^_o86+@#+Nf2 zlW1QjTw4nk;t)tQ@(a8cOy?yO#=kl$gy?4k9}kfH7k_Xs2e(rMSfIwcJZA)^h_sxi zC#N}A<=*15W@Yy|T77fHS^Yamte}j@8!7w~5nU4)L6{@=#+#C&j5&;|(p&jT?l*sY z;sAVIVr0){yX)~w;~A_##{q~Za*=oF}(ou%3!UC_CV#Pr#fA8A|g$w=r&e6rZo)4Z;>Q90{K1|sEXrz zACbv48aC+e)v(LD#^h?k=at5~vBkRnN7I>gL+#S==S`4*zt#lf`%%3S^q-#1Brj(I zbrNXP3PT{RhK4w2R*tL~mp=3j-2}lK@NYC`Z(j`hTDpWBR^#?5jEmIWy9RdLwDich zJna_kq9~=1+1Ve{Cbuxe=vq15nIz^uFeZSwwljbhpV%5d>2}(mIT$?8r_4)xgkRSZ1Jo@8;4nPvQXZH!w5+e+(-X=_|v zLV;R@KGG#BMS2NGc~x`{!4bWTW90Zyc^!KZeznRh+3nYmbVkLn>l_rh2V1GRCn~nQ ziKx-}l%7Zsg|3tzwt%iNTJ@`mud{C3flE<*a{|stNy9nwjR8a4m?l9gcBWtg)6u|{(?tTEc`oud7_b5GGMqCvNV3mP5_1uiBt$w!C|| z-g(1u?wqa%CSs%xb0fweQnu@i^z0`USLll1O_EK!7s^;JT>oK`%?e@(^teNPqU;??ACuYzs;s9iJ2Ubmds;;4Rp|WK zxa4d^r_O#85VeB(wugjw?pCsEi1-s7o1pX2BJL1u2)A@~2usikppGsTFSNA4v0dX4 zrNCF&vFFQ+R7JyQ@m)s5&}KW!NZIRxh0czoGTyQWzg8pUgUX5C92dsDy@?8~u@_#g z;|+e@G?k`G(!cE9pj^X%<-ERQWj44)&8Vu2^@J`)|Mli~GyLc0U)%L~H zTYd%2(n)$Izb);m^ox(;4$U51C?;Nkgk<5p(fhPQ6=wJm+$W-k)gVFwz9o` z3g5~GOvt8FLC%MLJjdNs7^N&|Mnj{YB*R;Sl}w?2NM3ndXx2c`j6c0(fq-4e(-bPI z`JJAc-iB>SwQ@&LRtQTfAg>=%F#t?r7qvlQwKv4z}N~QWm*jD|tNs&ga+V%8g z`~t^3*@DNv{cQiO7m>rn7|iM|KdO0oeah#JhPJ~AL)-NDOTD*WF_iyxmFaEbgBu&FQ7Mns;?opl0_JhpwEGMiAaWdzaNMu#tNrP~DGMgY#lyIptA%jw z+Igm8UWe5VR(7Lk`TcxJN=tA(m>iKQi1^mqD`lx$hG(p0?42z zv8+(Ae3fmw3#-KPOV4<)EJ<6Y3ihIEf=)G1RpRRq7e)}3shjsYy*CeE2~T7tOz);# zF8U%Cf3Y%4j#){v-4xOHE)H2X6KTD6Si`49{_AL19&EVuU}k4Gw0Zf@TfNs|r&_CAtX5!E5P8FG9;`GJq7Ma;zhWd-w9E>2fT#OLb>aO%Cq z?8@4mc^w*Zj6>t7-rsz&`$U#q3;^iM=yG!56?c;{Mib5HuSiE)CVC@>LR=oe@`G=a zl+s4n=#}i?#X40;3bVmaO95 zwU*Qd`nn*=3}zZ;`1=J$G_F+t40pXuAgsFoSCZf7oI>OOBu46&cj#LjWWDZ6E%lEc zf3S9_+T=5~tUIQHzY-r7zDF`e=!Vm|vqcx9j-#72?<|ujmSd$+m9AossUpt$&o7wv zB0HI>q^%X1OOb8WVt=TG@Mc&CRk5d>QpWhj1kx-SL~axq|C~vlKU3_<{PgPYH9wo6TSyk3?mizEbQyq&Y#gU`>wJF-J=H zB%y>;n*2tnuKSCPjtT8K=_u{5l4_@7wWe|zBF@0wUJ*?Sh4V}cH;w*M^N{6cmN4jD z)wW_Z7kNT3Xrhtn>?06J(8y?rgcaCnT5ZYXU)q)Q}|a%h?^I zcnK`l(U25QfmT6J-xei5)Y)B3wS9Z{<3g4P-BN7K1}amJn$yW*hDzM< zObZnw=P<7c{2M?CJAl10Ru6VUJ@bM(i>;z{OZX*7%nzlQeAv&Tc&5)eBHffN??Q+& zBvmaa#&SL()gqOCRl#N~m!in|C}r!xiyx^#>$*hv0V*EV$2P)KTHr4TljXfVTF2Rn zI@+OCAvSZEE*(R@+r$K!Zz?Je$A-U`A>Z=;>7^&`P?5b0epBY($n)z7{2p19j$egA zBK%LBQcD3>g*Cnp?EshC5>`y6c@Cfoj*a^9b&;8xo?@IyvzQGDHq1)(k1P?`H@e^f^dmZB}?|r3yjumb(I3Re&07Vh60W>@> z^V+?n zHAwYa<(Fj9F}GXp@VQ?}mn#?^6Jn*#T6(&JVOA$}W{GllGWCK?vX@%OQOT`UZo3rQ zDWYE73jKlXW`@TPcrYoez}y6Eo6&o^)D9B;MnmOsyey%-ycI6mcUx$2`z`cjp!PEw zJKgKhAvpqn+K?jEKYLexZ!fN)(-ROR7g-y9LD=_il`oDxS@;9NgL*r0#W{;UsC4wf zoiVn zixue9S~YCHM5^L{Eb3TCM!bZ^yF zpeKRr0~lWTY&i??2jZpKc(44x3#`X2B;fOF62QA~fgry2ZE@8U&l;GZ0~cJpcOmyH z?uHYE7beOSo7SNXj|!I(!qI#&{gPOW+wxZLTj&F|eNi+b!7dCM;lqQkX}j!(BOmS37k^qmSjywGP`mL=ZNh6!IRc@LGzCn$l*R$`F=J!rk;w2}Sqwx}A{ zWO(^NE7N3ce=A`@G~2I0)tarr+6Vc58}ysg0$)Tf%MUVc`ue=I@L&fqeq2}61bydP zDoAraf;`bNj>)!k+_(dnlPp`j224y{qN3!;`X}Nfv4p!#OBvx#HCki-qNIL*Lw+#VhK80up zdiE3|c9r<`$J@@QHyU}3{n8rkyYWmCi##WB#Ho%sW1Xdi0)?kw74B3HFGU-@ zn0HGc_H0D9I5g20faDZ5P~XBqpCH>z@(Quyjx&V@44_v&B| zwJ10G_iT8ntJr$UY8__@9@2~H!#{&qxk`w+>kLnF!t>3a|4Gb{Dq!Q%KVzu}4}*zW z9!M&~Kb-hmnsjp{w8wm7&i6hcKqY;;)_s3IlHJ3AOc9oc=id52=BCl7!7F@EKu0e1 zyv_L8WD}`~@Iy!NWv8(!y*;hcTgrJE`1FlE0krZgLUY)YiAS9{wnc)ekHu2pk4mIA zc;@>CIDq%T+XIQVNWAw=$G;}=lA9-rpJrp)WF76N?NmjrRZ}+jWOn}Q*Yb8$u=|LA z1+7uUQI%L`sN_CGhPI8xk;4`GF;5fTy)5*Fl=v9>r^8U1k5bIQR?v zJ%amg_=izn9;na(zL*n%7VqnGSN%I{tL#8ZZD!DWMBYhfT?-JfSFh@XCFFfqgAb`;9cwze}cmY-~F6CS<)mVL=95#d#GT6l>( zZLV|bgs}_Y!YjLyAUA+3zFMqrBU>`!X;+?YQ{^-d_UY(yt}sJb$uLIu@u>CDCiL)i zG80sWoBXi1N+6v_j*fXoH~4!xf7iX@lO-U0W1Ha>NIPKeH4^#jh=A4nxEU#U#FevY zd6%cqb)gCiJe>dfrQXG^L~)r{K!RZ?{+K=vddb`RIl!t1L;aw77vgY@pe$Nb*OBV@u%+vgLo#L4%ozM< z3VtaTsZA(@BCM}U0A}rOfVgwz4=}d+Oz^Mye?Euu9jFB^urb|9ufA5>n~W1KHlp;P z5-D+?=$3!LkP*0CCuTr!6e{yo!p`rfJ(=`PTVO-F_AZ6aGi2LiGSl_lGiP2L)aO{KBPZm0e#NL%KL_p^>~;HcnsMkMgec2hT5(JPX& z7#fer7C6`ouTQ$u>NR}?Haa<>#CzqZHTG7WN5*2rwN6Ts6q#aD>yb`a&UQ>3k(>zqEs(LjRJ;U!T@;5N`* zh`SP#(!_8ouEAaA--PKzpuoPCQDo6|$WApXhpO|@!vxeP;uh%LlwxGWrOQrTcTRLI zEQwm~)g$UHVB?OjTNx(Xi|QL-XI)h+jSvg6Y8j)p-AS_K-{7M>>BP7R>PbNckjYIn zX%z_9h6bjwM@WT7^%G^m>-0Gia=eQ!6qCaAE3uoLMS)xJAZzR{^+;>~&y_53(&dz^ z=kIkp9=j(v=+__b$=X}bmv;w@eAK711N{30!BWgEmxNiE7Y#ep*RW^ z`62!jTkfCp2n=D~g`S$}d2h07B#N7XKD3keQpCRD&7@4@MqWf-ckGTw3RP?~@y+Sm$`y&-{< zY*LB0Kmq<`JU86u>vm63>2d+BQq>8hiiOO0g@`~P35~c4L>fe29AJN225Oh-QnMb- z#nzFIL-4P^8oU2gDq@GpMKKbY!;G^i`pUHi^Z3!&pT3bZ)Dq`hFl!~NEwO02Bj6Y) zaU`)%CaE}OY^2W+lu3D7D@Sb}Pb(;4%hgzMx8%Fp;X|4t5fKm4Ks#Q)LS;tOWV*h3 z_NjCfduq*go0{#cNA$=h4OqGgfq~;O<##JZ4c@<0fnL#i#}84KlwWmA?bpaKMVa)> zDBo4kh}H;`luTP-#gAFM&czs1wYQn&Ay+n5L;-?hG?JU0P3h5q@#h{#?u9 z=rR{lSlJjAQZzCI(&{_T%siA!`}mC+d5(u)|7l|0is2;^3SE;9`kzxEsXE!yCmyWK0{Rj-Y z^auS3RmxS^C$crCj!}P~sJ%Ov z`IQ5v2J+WJpHSG32B1m)MGE7$jx@Eo{&gMWXFSG0bT7HBpy94YHVii(JAcBHsE1Zp z$#K+5iK4YxIP!nl*zNO7S_+nk<_Yuv%R_~i`CLlnPHtKjdI2Qvq=iyBfhZ|N+>G=#OplJmG)ylClBy$i zo&qi8!)^{OEPPxeCs-fAoT2+0i}OR9uE?yy_9)%Jcz+gyxRZy->RsRh zZT~R8TRk$vTYvr=$dE_`Ku=DNS0R#*Qjv5jlHuTD%N-KMta9buJI2BC4L>V+w|%_( zqZ{0$AT;q^Y%GkdnLiQ+)^Ic=iFm~6@BNYP(Oy?HLGhY@yd_%HfMNM5sU72F@3bM( zE{hI+8I25;;UDv^U#;QVF!qNw2?VCp)Wq5I) zFe`?58Nf@Eq-Ke8Z&$pnf!`{GSS>3n0WTbHF+704h8Od5(roJ5>(6$6t>4aC#XUP> z&u-f^bdQe*Ov>{%JO*WYSb+&~w>(aCAw%RLHZ@U7^ZrK4KXC*tgOpY#Ewd`M)DH2O zT+A!Ag?(Uwg?OLEBiD(_-+Uj?TK%Yj*IqAya0m+(tQ0(@+fA0=(4G;VVB zZ=#U#k(4)977;p?0qW^XwOu?&`H6ZWHQn^~fS?%@g4tsCD^k(p<72`tQK~mN>*T3; zHR?-iXc5%-2hpBpg0Mm6{e-a+aFLgQOzwdYjPw&<7@P9Bt~o&i9IWzhzS=7M(^|0q zE$1I7u758>b8x~3`TbjDu44)QFLT4F?4#&vzP6>^oTSAJ;W}Y`3zxF}ES~&Q3FZAP zOGWjO>Lx*)0U^P#t{Hr27y(I$2`R0WhrlHGS)h!WQY12gR6CQj&GKWhfShPFw?V@L zw_HvKA-2$$ZQF*Wd&=V;wu}lr@}=TG7@-R~YnKT${a==M2chj)xO;@&4g}u&f*PoA z92;Fk4QwHXX=A5ZnT6eblhO{QvCJH`JRlxQQ2VwWI$y;w*}`$f2fZK8k74HK%7C6GN=BOW4g&5thN6!d*gZ0dqZPgFp&Xi| zQ<>~^Z`eMzVgJ-_r)xl*p^_@Q9r2{EW2gAJCPZP$;+ukgEUgx^CcYnw(2O>I|HOu+ zD(i(0pUuML2lIdY1HQl7&i|=0?!wt<6;nndn48X-XBYFc^eLJco|Hh8xutJq)@|_5 zM#=<^vrIimCuDT6LQKBEtA*q=tXY*vO6y1>hoTMx_J zmqm1;;zBq4UhRKbX;V`J6+65~o|}?qY2DcB)9DO)Io`%5$%ZX(Tq^PsHd*1J24)}V zq9@>|J&EOrMi9K6f72Qixr3zkLWSJW>uIY--yXVz)NsSc@+ zTbvt)ucg76flNPsbTwIah`jXsAy z%z8w%l){72a3~p$V~CppDJs1BAkG5nv$3<+QHQ~&vh~sLlcd3)542||X73K86TB~T z)IB701Bh30!Asx6^(y#hxpktc00RRX5t0;1O)OTB$`cXN`U@wpcyuV0Hg;6el;pLr z?6^?ChS==5)goShns)gXOWtZ07)iBIQS#>f2uCKb@#yo4Z0q4C48}D>9gCXo^c3<< zG6i3utVaA)m}L3OtGco>Lor&2P{4y8$w7NLR0ocVlt;80KUR7959SUL@}nkJWv%*A zni%vg#PS1f6^&fnYVu7Wp_(u`?Ya%%0Jy^II}B-wQ@Bs$RKieo+`kl2TE5!Nc##j* z{ZMg+_hq#8Wpf}dnz5>cT1Duj5?O`dver~Yd#pw((c8nkGMM=lH6Y==RUZ0R?ka_& zo9)M;z5!ctGg-7atW14m{s27SGrwG=z4{{8_pbcFZ4rm<_g<%C`^5* zG>K{-CWKlm>naD$QDct8s)c1C+q>{wagsoauClsR?WqqG?J7l(AJGG_OH55R^@D&Yw!DjT&W zi<;`0rSclcGkW6MZYN)La;9utvHFdnP8EQOUK+-LL|9*ZkNZo`gt&Fl+a=!LZTX+o z1sXRFE_O*L-xaxI@f&nRsbm>fI9(*eExemSq|Ctx=dSE0kEcwR%9{=Znj=H>V>Vz( zXgkojB^aZ0dI z#|a;u>l?EG0bjzB5R4gDF)h5TSO83sdHesaEPUlEY(dLGAJH$;Qwo0uKy1|E{L-rvntXS|(UD*rPhcaCOZWkifuBfCtqidhl z-r(j~&g#6grGU6~&4l0l6G5P}y+$H<+nd2f;>RuE!``2DEi2F#dH^X;U~^cYQbI~bIdVrQXY>LG%_Y21DIXUqnbC(A`qpi=UtoeP@`}$ijXbaagc5KhO>*hV?N>(aG!>eA~ z?YJ~Kbl3DS-mG?HFIka;uq@;l!#D~4HvW9fqv4jpSQh%otEF2+I?5Imm9b4wnVWhk z#OJjp?NWBBUTcg$p3>;=R@{I{&UTxgXO<7z-}f5IDSmL2TQ?HuWkG?$qQ?f8031B( zl%16Y)fbgoRB@-P9Y@{XUWl`hT=NiuWUQcrBE>GlqT|*!s_CW(B`qsrt58vU-muyr z0!voM7J46ghU^vUFc%SRkF($+G*3p)cU~xCX)=st$wnRAD;WZ&W9TW**zP#cm7iRa zEYEGpF`*w_SZ=^4gTv1wa2PblYVT<a?@X|>1H58Nv+x2+n2^q$nZcaT=P|Us+CrYH=T)@%6_>x7GDC- z?YgFra6n3C2V~-GTodRV5@^#{NRlL>cW|oW#7)ZDQ!Tw9?ARf@I_i^Y_ovOzCIEHp zz5#M(0mO`mSvDhHEIA=o3D&6WF3y*4fX6eCF^WaQ35A7kN>BL4-NncUY^b9C@mT~B zsVDO?4`VA0aTkU`R4ihD7Q~fXEs7BB_y3~PWPB=%9{$u)Qn&go&z@|Qj~2nC+(mcN zHkCN-E?IL}mC|2_^gEdD!&fhZN*zPX92U6j^HAL1A3+&q+75Ys3(woe;|kgq=Le*e zTOU3Vm^SqYK{)4UuBMY+k%^I93+;Pki84(sJ0>o!TzIc-&CfJbWEFDY!cRCN>ers9 zg(Qp>`6{_-NY1>F%RlA&WX|a7th#?ej42YW22%V=aEbV=;$U#GzHw|@9KE17Mxklm(9Uj z-pPG&7|3$z!rapDu=p}4(*qp8aj^7A0>kKr3t*0z{!de-68`7TIJm)IrjY~T1QB|HjF+UIP>bq1@u6T-U;a-raMzjhp z^1M>Hs;Ugt$Bbl>{T9e~X+rPc{{#&|!0^l8&{-!ADog}o!va2Lqd1W%_nxex53WIP zrL~1bM%zGdd+ub#`6~8sP*yVgVUtrP8t~Ii%`>M3p}8|?U5)Hd^hZSB7dZs!NZa$2 z)%Fwj?orH|?dRyvI>;I#Y=H%zga?Gb8&}Uw@VFblQ_3Z$8)G48Cy4|VEZ=Z7{wh!W z3;PQykPmjX6a6GhB+YjTUkZ#wgrf&~d!t8^v`>qF$7DFH#VXpazJA0BWqR6FYP8iG zQG;IIU(WWEs?_I};1hIfhbRksd61`Z``y_pUd!~@XgTU-v9P^On+hW?8+3*9&E0 z6F&28f4?xClWtMWgf5FyhMv9y)C-6J`dywp)+#A7xGL!Jc& zj}rb68W(KH%%5CjQ|#`pikvJFQr)!0g?f6VqYpB=3I6hb_Op6?C$#ZIcppiaZCFxSvQF5|%mG4A=4Gr)X&_ua;=J(HLc~@N@%3gs%5m$=SH~04! zbe9JwMl<(aB(6>wt*cEpBAVnZ#Bny5>I7YV^~w5f$x-syq_z$qF=FczGjsWZrLqxH zAvAPqJD!a1p{7cVYYU=TI~6D`e$;TsF-OQPgM1U~F)N`1V`a;rSfx!B#O&^w&@$)V z4j>lfKH0oPa?3wA;ZD+rD%xXYul@s=80^}XtC?=n+RXL)i;(Z=CKB;3rL$pcaaVKz)k`#1nM_YRD33s zYYrN+@uHQez)MK9eeC2P=gdcc|7J@bXRHv?DFt#>cJESR^0L{Q3#XBr8SP zwR;m)uf-&$ug^CAD~t++rc&MLNjtO6H_cr3XQ%23n_O|#kqZ7ro+j>o-lfa3zMs30 zJGF|7;G?!zZY<^HJIvS-6VtF5D+NVBZ!-<=gKk-iGCXL&WRJD_7gi4}D9^@jaPE9O z!&*AolQNfn9)~HJwI>Lmt!vI_zI#`=7J$1GS2R{ud_F+kZA|5kr-xz?XdahOW@&JO zU5dh#!yp=Gj{x^GpGE_wdFilPyN{jqow$kXHGUF8oh%zW{F|1GO->4|_SUb<62cT| zNOoaV_(D?r5v_OFCyTKX0Y0p=GnaYlOclFBOdbHs97;Wlvq}l1YDk0h(=7Fa6w2bZ zt>yG4-)8YA!=0&p%y$Qo1Jm+$=rGV8cl79R{JE`wz&)nGFD?d8$85})Bb((5QikIw zXO{)x84|l69(9`$^ES5F6+e8jPMwgA@73&VIsgZg41imS7C?P*S^4 z8gw>`?D#%L)KO~{yB(!(Zxrb;CLyT{e8KA2vD0ILwy-!*{(J%l2{*iTXC1aWpe za6)MIud^tWK!icg%G8Nj77EN(as=%5sT(Xz#blk9%o~L$Apm2f%;~IWK#w3Ah z)D!k)vw;18T3Xi_W?>@6MhKtH!5%m?;L1C%K`8Gk0;L4~9rPGx&+LM-3o)5(xo`v3*3YUw_9-~A0KntwZ zexd$#hn(RDES3<10gpWgf1;-fmgIcfmmNmow}NM|*#rd6MSdafDc3H_cxnLT^|MJDPO zd(6u{!Tdxd<_X^X51i`FPJiZiAmkTOQg+VFY`!b1HH1}Og_brZQ&tBco&FEuGW6sJ zbFQ=OvWVGu%33;8iwK@=^Z+7h$ov%~`N%oBdR&0b^l%$?4tsfiENloArB!Ceow5i# zUt-FaXD*)(^m7J{rmaRvy3a5A0_hpnTv;ATzu>(gTC2hcJszx)5}fN444O7TPTP{W z7i0qyDTM$9{YQJBo6`Cd9Xmk^efc;YHE4XiJp#}X%TFSUx}p}r|8czUxEn2r_9p~! z$P6z`Bip(|V^Rq%!h+}(+A8u3%8^rhDpzkMBjc}hVSW}iY(JqCmb?92UXPSa&%d)u=ymGj~_Vyf?|d%m8^vbnCQyc0{SJ4iMwmQh<#_CuMJNvXrPw1*&_pN z4Ac@#5zdEz3qu1+nQ1}^5NR2U*+z_L*Af*Q(Nz!f&+F&ZafDN010pU~7Y!GkAH`W4 zh)~5XRB^w!d@{Hm7i4JSxH2?F4>TG-L{>mU)WIra0Nlo2Qb{Tla+PFjq+495ct!(9 zr_M6oi#!Tc%IyRN0T4UXO?AB*pjVv$y+Ie+;v*+8{?^R>5z5r*o66V%K_t6?T*jj^ z3BiLjk@s$*@vh*~bkki&$0VYgKfH&op))FoJ>d^t8TDJBNp{-B_^P_DddWdk> zW5(J;P|}zUE@!HoZgdqOG7E~A{5deWPH%&+5IcNBY12F*21{9QlH(5bw}`jz|CdIc zrc>k9QE+VmVLcwKHRYP(a)C%bD6xRDK7;oi8@kUVs~FuGswP zs~%#lppxkmtyFYuWe(#2;7Q$_fnk{TL1H>y2I!i9@!gv616&e;LFW$IR4{H8xxYY5 z8wQa=Z+R#Z26fDQq0h>NAuJv3v#`SSgCTle1wuZcJ*i~6u7_e+e{v=Vd8vwkXUZVJ zB7-UfUZc!xiMxc1yGKE@3v5x4lC1h<91W55;6T`1uQvERn^J6IMi^wCYMz7nN?mY% z_7X%~PAFa}rf^zWo)x_g83glAH`wT^18dx%JKPIv?CA26v)3v!+Tf8s_+cx7vx%Z! z1B!-uy7X5yN!SxxJ!EOv@|Ht)a?dbK_5nh+3#rSyM671E4u=l*rULP(JSrr5W)LBA0^`A7e7h3cCsP{lWPDQy|T_QYxIGlZlnX;1xe!99O!w5?O)(@E~QihB17tV65BiO z0`!DMEG2CP>$xLXK+T(lq4fEkMHa4KdPw$Q0!tj&%;Bf6IQUb(k~%PPd2%VMmmTZY z8LPw|_&NGP_5|%Fh9sojY@pf67Ew$o=~vVQq!axFx@WkIEv!2sq@9Rwvo@)*n1wj$ zn+NrAT?rnaHiSuPWnMZ~WsJIp=8|+TcpAqBhwd42#b^GSd@d7BSWodvPAI8_gAB(z ze5<<-(gUvPh1O-l=-kn`J_jcYTso05&&i)1w69YJJ=LY?r6xyTcYM`#wHrRXK2G5Q z-%=>{=DY-UQoERu_@DYs#&+rxrXX_IPH;K+%GbW6c3?=uK@LAvqC}nBhME>NwRs}J zo&hH~B}zq1-a3k^jyI%702!|DU%N}JIQ&c%TVE#}XT}B7=!v`D#zaU*VFXXD5S0dZ zu_Yd}g!7t2=tP6TajE<#iGT#gP)`V_MYo&Z%ZB#`GSM;pW%>%=t& z2G#BVB+1n=)3$a>pG_l&BMaArNby;`AtPq)xV3^*@mlY0Xsny~Js@sZ7>mIebia#i zpKpH{l9FV#$K6W(C2e4>lD%f5ozlwqkrF(*NZGL(B!n%_=RSyd)#($IMBc$RJLSW})mXE4}hZ~ANuMIaDXd)wAusg2-;Oc?= zR)}2O?I_c>j%f#J>9ZB8R(86;DK+J215Hggw$ViF(Lo+J!Iq6U^nV6(l%JzW)Pk7x z+F>68(C}>_L#8M&y195_RnbDDR8-VgTo&j;Dzf89W2a>JE{s}I673rB{W`RIZeFMu zAZqVHoQMOlgta0u1H_*G_^vz9PCy^v58yHBW3k@N;2f;BM*kD5FONOpnY08^6>#>d zp@H5w^^PraS{=zBWdPW_&v#LmXNGZxkdnEjdyu_Rl85r}8RlK+8C?V=6{FD9bTCw| zC!CRW_aKn$RfmRcOvuMSBtK8`6L#cYe^Y|07V>LsNkE=Th=H7d;v-)OWQr<&Qf6Ln zjmwrDpuMT$TROiY%%vs9MN?u`*rVboLz4*^Z&P1CJW@U?-Rf>5t8Nr9Gue@-qpm45 zvUZG9A0=EjhCdGuQg0$!ZcBfm!PWoZfzA1!Z^HTUfqmbjB_PVRx5XJrN*vRe5b&8G zk+<^FJbu%A10gg3vz@x}a~nKRYH485L?xM_WJ^=nEDVqy#{D|)P-SXbXCQ9R=I12V zBH((0^mGT@5dCCO9Rw;2==`&40d6yw>tK)t811}cOd_xa7E=?a?RhGt*US;-mQ_EC z6I2cP3EF>ba2Vx|k0g=DkWteg7hIl#rsn-GLhq-^zO9|aT`Yz*khp1KVnHMq>gsr{ z{30N+kVe6AIRVg-wM*5mkCk5{6GL_XGh>8zSG(d;@>tLgahcb0X1lp}?KDKt5T#R&qg5ceA z=;jE!MmEl0(O6HIU}0*BAdvP@S=;5)BM3fF{mCnXftpZykF| zj3`y3d5?r-ni|FoT3pa`DJr_K9niKF*2oLOyeQc~fw55LB8* z9XpeZMUw_{e5NLZM<7N0t|}5LS;ikv!UV$(OVyMFlJvd@RJfDQ0KRa zps0s*Fy}JW3$Mr$$#1l7>pu)lgAaY(N5kWX; zdxKm}7n~AD6(XZn#3$zdyu1px*@OO9)h633G{}3U8~*+<1m;vJDEC(L%ToDGuX7^O!9r)oZV9$n!f({HEChp0ArlQyy;%#Dj&h#b* zQZh(SqoOES04T0KHv^=j^haDluvW80L7C?e(#fSde4vuY-^DeA$76*G{!y&W3)JCj z39ABlQIv#P90#zE!;?*RCJ)gQudSaT6^vkDNmOj2t%h?^e~4X6I|f!K(&T;eYHZ}DCH$(o+x4Fk)aD< zTFf97&2vC^o-JkdDP2^cJJQBX`p_~LhNrIVBkDNH1gV6oG={1|a{0hT{?wnCE+2J! z{s2fox4-CKu1Cg8rlk=yl~LuG-_gZ+M1im;D7!6(WW}AB4XRAx)L_S25P4#oa`niC zz_$w(@Rs}cHd$oEFEnMTL7BGCq;aGJ;p zlEWSmQr|835T|3@|9w-i#Q8AnoWOn?F75U2z^J?4;L}gSBc4$fN->P6gxbTQ{G6_8 z7YWTY`T+2d*1&VvGmh$(3VXK{SW>jK&Z4DSIKU81i!D97t|7;bOq%B0>M13IcM7N$hJPDg z8F+G(ody0v+hNN?-ers=QmfnrLD>9YmTQhsiou0;RiXz){vPB$_V)}-d1B}Kpb?IL z?3iuB10XKoBp^v>4^t+-FmlU+LOApS)P&Ghm4nwD%&+4=*D>U) zpBS9O%&@gqyfhs9^JYQ?+cC_VN`f6=F!jQ*pF4`-ArC#)X%hgXru~2n*HjS(3=C0W z@(5@kL4$g74mL9|haWM@bm~kq)HE8t|IRGv0rrAFG41fm=si7jxLojomrYx?IoN;| z1N5_WBg9Vv&svrOjF=fZ5D*Y>Xh6iEVgTd@c%B@XuB>^I#MWamQ{r{-ZZ%oyYK&G5 z8A*59pi-?MC$WRhhzdby>z#IK_%QPy2723Iol#{o|HI~awgJ{aW^rY})Vc6AMX_GL z%ux%tA{sN>>%{>$gLj=a!<<2(aG`C;V2FWe$Rr>WIw2?~8U2C4Xpo@T2rqL(Bw9*E z+@{pDd3SD^(}S%f*{EV@;6olbH7GU;7(?O~+r{UM?Tz(f5k+-HGp>iZ#!NJPWZy(G zfA&=oW{wC=5|Yeb&LMRvmriu_q+q}>IUNTGw(;8aA#TsW4pVI@Qd_la*|1cYY69EW z!#Sx%riwA_v#2ub9fqpng$(s5pn}|D~^?7wY+s| zcJ+0UyihunSw5@DYoAVNkNwvMx1POr1}NSVIH~flhg@coD`c{Uq+m@L(<2+TdSxhz z0B4{u2T+d)lTB%G0H3v1q-Ymau%}W*jY^B~cLG~5t0d^;En z3vI+IGo)KvdswE{R+0A6bQjgo`?=Kxp=&cQ|uAiAbI_p+H(w zB@VsrPKF5%G}{nQH0{Iyd4-u$VE5_QQCR3y7?pu0vb|%5p3F712Bh(>*aAyJaYz=p zInfQcoTIMPOH|ci#G+d(ZF7*z?of6#wK>lsOjPy3;j&DXN1`!n`iE^n{|08${>wZ) z&l2o!S;CDnqWbohNcpDH50)o`FVrIp!zCD{Kn~SV1D@Q{+c}OEYlUAD{UU*~J|;LF zNLgkDNi{ttI7_U^Q#?6kdV0#fPoo0?VE?D4$hWoa$e!wBt>9$E!yGb;>DyU@=6KPD z=sBfzh#Za_=Fbfiuo=uaGtUz6d5$J$T(&c>^ZhtXb*m|d0~?>k|BuQaBSK`MhN?nc zM^bz()y!r{7UTQFQ0m@UnMdfSLbH=migFsJ!$c#JjYc47Y717biJQmA6s=PY*l`Mf zR>{;wg#6MA{kI`biI7cNIyz!Auw>{0eVffe+KVc=0$kPPWl-Bj#eI3kMVQ+}hsfFT zh9olz5X>l8j?B#L!UiKPpbX#vEp^LK8gDIxCeRS+B}{&&Y5@cYbem7F!}qHK0F!si z)!A)n2ZhPH@O4?)u2o&VnJ4G&Iw%_-$H1tWe?;BQXK6{WDgt%&a54fhh!Kuexn<&e zl}66IM5E5ej+% z>Q^)TGIDHRoVoD)OO1dcK;*H5DKk&Ks&P4T?~)wAjs4_bJn0!5;~*w|1q$?^KXjhj z#}UT1vNw8_G~TEakfL%Lj8U1*fq(|wQ3@%qOk~OCM!TBT|ODTyV@adT>t|@PY^dBsNLiu&@>m5`Z1t?THF}Ce?=F}$T zl!ly91f1Ffp(s}w&&qD50Am1BXg|P5@p-D+1)bKZBNL47iq9Vkbt;A^e8P1)YDze+ z$_+$i&0T{lqYrx@{S}KBCl}xn)b}!g17aiu?cy5P3EJB$+I%!#m5Q`Dgfv@In#Ar! za~jREwv2&zh#pADD?n- zL*j)6T<8WsVKNY{h@W#EwhB@cBUe2asiB7zk)nW-mhgam;KN9pSRVFJ>zveDL@S8%klrYw`3jo?P8W>C5?L1eyY_`! zSm1f|M+3cw&RjPL}~%Nv@sHVY7>=eo>7Yv+@=-XYdjEM zxg>*zszH$S@^cKjR+3@S=l_sU2h_a;Dq$jQKZA(EZG4CPCGrcsB$GA~(2PlP-Hv7HDz-;N8&JV7OPCoU)>gzQ-L9Y*oS^N1O7XypO4w znszkv-d+cm;|=26jK`oPE^tm9DXOI`buSL@XlmyMwuU5+=rBVUmR!(E?$E_i zro^T_9ML93JB(6NRjJJ(x?ZlLq426nn8s0-NDO-3;8GUP;BKCxYp4$=_*i<3>!%XY z0eC>$209+JCZGm&vOAVEKyx4@v^KC>Bvp08N2dTn%(mO_3*P}?{NZKZ%qI5|)Oq1m zA<$W&=XixNU?@fJbEx|{C@zKtSD(L@#^^(qp&U;_9q8F}4d4!Nc?{}6H((I&W@s_S z0-XdPLDl9jyn%)BeelA5;W|R>KW*|#V;&L95S*+^5adRQQ*0~QQu_> z{4*RjHe~D?MwjFeWihB%VNP3B5k|T4(hd#lFrmW=ydG&d%($o_N(ZLmsvCUG>~681 z^gI;2Q>*G2Jz&r40#LlwN=wP+Nf0>&%`l;=pJoZ>eSSg!DhywrpcbV;cKmqhs9kdS z1wItq>7D5UrgbnbK(--fo!|&1&XlV*etaXmigoN!!mcFWEJy$khI@82F~0rWtg0i? zI`$#UBqCd`9XZ4P;87Auu$@;db9FlxsnpgWysaW3*cx)9eI+3ZsA@HiZ-C3&WIjzQ z!ictkmx^UONruj536`rQ3(+a${31SCdP0l2lruLu>k%0wM5VICOs}u)5oEHK4a20z;AD0L1mW16%HdEX9SMIbr zBj_U+WD~1BBn4LX!$M-fB%~uPO_X^GfkIq=Hi-a`h_tZK&4D8k5@2e%=u;O$VnN-) z%|S@=Yv=&iVg5t)Dkj_eB?a~m{&rhNv4qlT4T|CpEt0nG53H%-@|&52aDT{1`wz~rN}X}de6p0bqCihDcHYt+A}nxn zAW&fUOc`LIyXBFjlRzO=xqM-nNF{8qj=mZkuqNiAkL6<-9r;Hfgyh@9syQvkfm~UQ zg#NS&Bdt4i`h^?4d|Bzlsp_C&hfO6YqyV zaj!}tP=&{MOTbinTT=eqS)3vp z5*OS*2|0{YHeI+4Cm|3s!||C0G&e^G;Y$>~?MDo-OUU(YC!0TvZ%e2j@%d`rJCq5a zbDuiACo`mi%(aRoXXq#_!JQBRVE(|dNETuXE{jV8oYxu0%_WgC?h0B>uir9?R0aML zK`>_)WK9(F;f4EOAmAeMs`69DlY^tim3lugk$He6gTj8l^{Nv~dE~M931=mUJl!`R zrP%J+CBFWJ`=#?3B>ni}QORi<9y}Lc?o{|gBVG3>Ofc+iy<_>Xd&(@l; z(;>MWU4aq;F9Pa{fJyqSc)8=G*oP29rwB|TLNq{O;gMn-fG-0ZGF#5K=HJ=BI0K2s z@yj|acshSY3?f9}KhO{aOA#`XZzs@YqGT0_ynYwGF8fpk!Kx>r03VN@>+@uBZI zny-jqJTt>xt8m_F8+9nO42W;5c6I;`xrLn=51GU2o8#L&OFQmxO&CK&ED$Ot#nrjW9L z9wYnyM{&WK47 z)qj@+2+J{`zs4(!Jad(ov+-kDv8tTerqZsSIfOTHxC&pu^##NWxhssc>*+89aekWs zVZpk*^f%cC794hh9p^`&qKRDqw8&_CgY(9Db}wK9(MHPpLbMuYjnByUiL^E3-As$2Zug}Wj!kO)Lz6rF-znNpH01$Tr~ z)IPf5NUxJqbL?^;n(0-|? z$b#S3kN*711{RvU*lT&MZmoIALGmsOI19=)B!3_{_Wr(D?>s&w@WcIUT0Fb5_85LQ zh=T1YQvNt4g7Y}D>X0{*JriEU{czlkHU;@JMN&)fDl-uf%hxHpRT)zs_^=S)27jb< z;h7_{Ehmwa5X3e*7L%dKI$MV>PaEmB%bx^DD!*|w2|7}PfL^eInMo|=^Wl^M6Aa~% z7*$P{ER}{P5Ehl*}vuUx=&3WhtE6 z@YQKZZ@^ArY8e(V1S%h<9HC(WV0zbu&7gfDrx=g0e%z>yn%(|~{-NC=Ns`vBO_1`p z&L?A^ROeKOF+SwmjJzhDg?2kI(IC& zc!T^pwXSsG0W@&YkYAS;=tRk5fVl~>9M>T@1|U)(?k3z0MoPAh zJA)ABV`Aa?dq!yjp6>7VBz%iXRA-;~g|CLuFB~kAUC_BX0FB`e7x63EHw&Du~-}mxIAy}yF+fXKXh^0cd0ca2;R>1EIz6C(r%nli1RXv`(1N{Nb|E_t0 zmpFxWYS)k=O(cG%9F4EDzXrQeicAIba0`Hfc5Rr~78p*$FcAUVFf0vm;hAUx=EJ8n z!g?p+iGWSPsk!uTIKgbtV&~}^h;vKpQ~vgGU`WLA{|S6%5NSt21Pg~JqTt0ARD-Gy z8X52`ZU$QUg8GE008b4aFePD#86(Kj(E(>O zz{Q7AE}j?%%*c=dJ1nqSkJf&IHQHDzacJVz@eoE{eHK(bf@0)R8d_A*mEGV1k;CcI z=%Ok#^0Ndk7uv}`8jf=AA`guJO&+ueiJ~W|#dG$*iETd(%W_LkO~aD{WNS{Bga5TD zR!;fEMm>CuhMU45rl{>y)6F{_rci{5 zA_~VBfXT&y;INU8>a=15r55eXpA`fXK8(50F*c+Gu!~s1KLeT}SpS`-DL=X2!yv&M zAi(gSswQt4_zZLc&KUakHdw;2!fzs=i*&+tTv93vKq2eGR8U2nr+9$+MaA}{nDoAj zKBqZY{rp2@&iyrHv&xZ(q({ji2>&Z&B_~->SuwD3^4N$qfebH-AE|`IBcLdO_=&;s z9Wa@VYok@=UKGWLVxIX;Ff?>`HWHxEG@0)$J2DdqiHAK*1}Py1o_xjhy89IBfwL5Xm&!O(Yes$VExjFLoOOaq?3pi6d*`WjJPQI!du?DEnhl#X>(7jfP@Z! z{>8H)`OCQh5p7pl8x45D*CV>)LzxX~=tG2v(SJ~2Di(`RCRjZW+V3vQu?)atC)MN0ELtzP&=Rh(aZto3oeF>8z^_9GDb5GYKfq0ZF$_ z2#gf^50gkMdYFvH(01&~VaRWwa=7VJqZFOCWvxr%a`wA)Zx8mBZf+=?P>m&+KS*Q1 zh8$3t9!?PEC`y??N~KAhpQr9S(JI-QRTyMMhhio<=b8$1C5Z!c+e!RSP9q%dc=#*n zIz+2I?RLsx$aAqjr;?FQgtL&JLJi#{aEbyW!=XLS(4TkxbK;xp0{%y-uGKOuk{`rH z(vS{82t3~{2fX)`GCLBUmDfBC`KgJS7@-HMwXfO@oint>QL|scXRQ><0h%0Yj?C99 zv4RF>3Hi!FTH#F;AmX8umgAGJ9A%dzq&iWCqz=MSr!bUMg+a#ZfEB13Z;$s`G>eA} z*G88YGRRF;Y))_oEp!*xlM(KN*dmHuHzr(kF2b&)0_U!NKrP7PjVUxC3RH#2w5n`D zlOW9k%H>ZktyR=+=?4H<|@B9SmG zp=!m$iL+V2G3-KoQsS)c>~JekO7_h~FJeOSu8m=NbL0U+*qIoHVyv8}jo$**R+E*j zk=)0azi*)?bV>?oH?Gz6DWqzYCnHoR`HU(99F|7A%#51SlVCxWDy|B7STKa)blb}L z2#BIWr3;2v2;#s!v_ z0}IO)aU35{MmP^z2KH7$PBAn&tT|vtfykS_z3m&zm!>PzS$K`-n#t#n!w*`OY-};| zF%f{Fy(kdAf!sjedu@v{7<2I%_8noFV%`L3#8+b62NvOr04)We4x>!^j|hU8Siq;G z^JLnnqr!zKD4@Z%JBt7v1Pa{_CC1VGwMnllFCDashhzjo#UN?26j~L5174HeovzCF zfme>Ul&Xo_#>{nTW-L&UC;*~br2OCxE4cJrwpQFv5h0` z4(&8WkO@FkX(c7bi!KB4Mgj*p2z2Qtk#!OP0pOnjg#!VgpHK`R2?Qo0t-=Q3Ri&c^ z(6WIe=b05~wP>)W;cHbOK~9!_1n3H8yBLS#Lnn5Pt$uMhsni?-y6g?E%~4OZ)ut$6 z%R?GM{wLZ4Abf=&Bn3krX9P$_OvVpn4tz?^aG8KjI!i$Spgx|~%oA}-j4PITY5J=) zafoke`3@Nz5-XlSw$4zN$wv2r7L;poRywRFTWBzF1&D^wt3=3xtlF)k33)_O$Pp(b zX9!H_6n+T#?vRfDH^A=7C0nM}r$=0JDilmt9e_^muhe<0@b%Bv>#d zaw+Z-F>~}-B&ms8S4Bi&13~CKaxI1QZ9}~;hZ0U6jZzCSI&KD?WnK-nCIjYdh}#1k zKm{WS3Sls$8Xo@a$OzJjuEGdJl@#6fI*bUj=_%!ik`mH|9@<{!HijyqIM}}4j@W2X z1JOT9nQS){*y&`OUP&V$l?eODlRFdY;Tn0Z(A=DvtEr!5k*-dqwTWKdsU_`=)P!&> zduDO}+JM18$P-Y44`u2wJ1YvuK*MMl&FW+`m_MAFy{8*k^bgiGXN5!cVJnI>*%!lmxSHn)_m4 z2Xr(Mn=C;o6(CKTaCz}-(V?iI!yp9vW>UV@0p6okj`NI#OXeVO6J)538bd}Cl#-+) z|9@cjBJB`_|EmCZUfDZUHbN`u0WmmG!uhGU&_>Y?uh2FOy0BoWVKIGq3gbHFJfDWq zRAYl!N<|?w=Q>$pG&BNbOOVkoKH`kKz5@!BT0lPdu4tv)l3k7CvTN!_yIv zv+xR3oBbFFtHUZNOpH^~d5H2uAkV9EOOqwj|MDA19t;9|0GW-~0!Q6YLxwax$l(B> z(H=kho(=IR*~^#!kDgS0JOAIoG0usPgtU3Pf9Ux={WuaYa9**+UulqR2vJ*Xe0&i$ z9;4dNhl7isAMr@BfQ#TlHGp)XnIsX@;=0bTwjDU;L865V#Jmbg5+N{;4({VIH2BQt z2PP+>;%v6am9d@&>={aEg7?Y*2v~wRzT_6{Lre)kIzrCPC0Pi96gt#`C~1$;1QPNg z9$zqi{SLN<@n>rhGuHAW9n{>32y~hc>IcUfv6*r*FO~6F7e+2nXtE|U0hCBVifPmY zoozHrM4Epdh>T}8gkh+FZUwtl+ybNV>~_8sU=yrI4Lj>upM)9j_Co(40aRKS!~ru8 zdkpusT{3--%>5RODEcM1-;mMXFr0MSlcNrdEkw$g48-)cl9e!aYmpbRX{9Nw?s%|<&|0_4j>0CuDm-G#uAgiN$#ozm3$KGG zS{Llv>7xAg0#xujh>&}>6kUMZj12g#70mAqn#BXhp-a%^_)0FaXr5Drptmga#Q_+| z25k`tFxHvW0Wu2`2pQ?@T&v<5-4O@R0?%7Rzs zm=^txIb<_$rL!N8S*z@144n`T2U`Y~-4$o^UdsteZ32)J>{%96K&9&Aw*YV}Kh&^v zK*jhL^2f$cYCIMAB5A+fy0Yxk7#ZN02^H#Pz)A3Nz-htvy0j2^Tsm7~b12M*hSSIj z|vq2GRofI#BOM)?}3^Qc%W8-F>gqa(pmydK0pn@77g>l692Ck`zKkeE@* z*(j`H{K$W<{1*_cL*6MNV{YIqJZ0hpD2ID9UC%?knzK!F&~L6d=vN{pY)TOtu|34` zrx9H6JK#vST=q}tka$C*8dv%Sy%YW2mL68oQy(Ph#{3mu^S zo+Dsc&gbCRM2B`5SUwu@rC)H~LE9c`Gzpp;j$iQ|N^EB5f{Ina(*~p(YaK2=x*588 zV6gbMam~L}4zv|L&Sye%b(PM1WTK9^!zdCC1$Y!N72fARpoKpxor6H|Bo{0w9RE6L z%C{k(W4IDGS8u?i0@H;M?WZ-m^k<${mn`ZFp&$YmJFUr4oP#ifc&D^=>+Nu`0Sr5R z)pPLHfxw(N6&7d^BamZ2g}Df!0t%(thZto>=D52+`e@FapHi1sotD zK#c%?14so^Q&bc`we=`emr|iZKA3#F@G-k|@wTNC!aoQTN01BsfC$0H@3kt|E8T!B z{EZAM`g~t|N>8=^wrU-^YIq^F#N&#Uw{xNy?|(*plLozdky0R zY+h>#+Q622X3J(uvYBjE8Lr4<3QR4ssKWo31BAj*ovZDWWDJ+`^g2H98 z7uQ_jdZnn=q*@VVRil>TS%@Mkl{KQ4g)9J3h^%s$^-q;BR0UgQ8`K|GilfTyR7qTF z$|#7ba+2kB)fraBQv@Y2033i*2$xHRFaRtcrG!ejm>OF&fT=K3fK62>qId+h60QjB zz|}O^m7-pzsyXYR-6*MGPBK7gYr%{V$-y^?vq^a)1oNdBodoNoD4qn=CUrT<4WaVi zjgArw_c=LnH=2;pdOhV||J}+<1Pd`9unmk-u;zgiB6DPuLx-?P_~PcCIQBqcrIwpI zZb)%NXtNk$s|@BIJHk~2R4H96ItGBcN(ke6HjM#Rh5*E20%6D@{fL1ot0+>aCVGhi z?sv!HPT(c|RTk^mOM4*%Ua02hYlAG&hJ-KKZ2z?e67l31X zI}#Syh=KSa^ZANzo{L?|7i@h|bj9GAVDfig!djEWC{#btlmm*e|5JP= zu-ZbXs>v~k>Y;?_sSxD1`M&CH>>8%{Z>W@^ft>zBHc9}U7gTe6fChlk38r}|w~S&f zjI52Hv_j9~Y!wd|C%OSY0}jek-QGZ)nm`T?N50BUw{yr@JOU9hf)PXVGZw9TQ?V!N z7@$loID@Kd7mej@1&eY9oF4wI)GnoWbL$zg)7gAp&y@yCk)0B5aLE5qIBU$am;CN<} zrspfSP(fI?XB6LXoKEPbQ(oL}{K~CU?goMjQ&NLvnCc3&g{R1xBDoE?aBNskVM9z*1{Mm$Uw7e!badvx#gKwCl4CXz-K zqc_(L{H}qp2TVJxJAn$5CB2hp5W>t}+&K#ff4Ei|_)QSapv)P0Yy!W^1{BJA4hMRA z4hMU200jSG0Dtew1`WOtKmTxn&ChUwgDlAHpH$r;zS=r@@ z$$Fj`sVrcWJo`{Yl0wmQykz?8DVm_U4Df^U@mLK06Ee+6xaFnH#RdZOI{3oW=7O3Y zqkdF51LBW_8;(z>Uktr2g1#Jxw4RptrsCKy@x)v~n*m z=gYOtr$EQB%Eb-ra`0;0E&$_FVaFzN8%dVU(|lHM(JK+-Htd+1LGYgxOc-q2d%{)> zgcVxsG|xs0?ezD_Ur2A0?Vcx%i%@}Q))34(!W*7SUw>4OAjPe$(^zv&BUxp@3nL9! zTR`;FX2V7BZ!)1WcLw*mhGa7+s+!_%yTJ|Yr<h zi;3A1(n)aIkOGlNLKtC5xd)7`{6XK_8@Wfff?Ii8vxB<(zc*^a+NttM4F>^(Sf&qR zm?0iwpW$!j$u`1-#T)KH5Ci(R(>-T=yozy%+k(hxACTdq4aK-+(3yi}se5I46J$mk zhBk?bSZ{}r>XH-SURgyh*)ghhIB0<(({NGG5G)D@1YYe=Xx|C>AV~p1S-k-{S-1d( zwa02XXsjkb>jJAb1mta$(SQR}Jc?02u%OIP zqDcUciOax5y3_u3w-?|V%!I6xc1E6JgW{ zaR*Ot6^6H#1_Q}Q-M?3*nUi}Ww8rJiJ2uPR3V+wTTE_4M5tR$qj|xylX}W7u8I0tUNhyx(9+2|rJ$-1EhVye zRE$LgnAg|L#cmMqjx%Cwttdirt+M*)Su|qK+bSb5fbWI%=B<-TYIBsPePPD!)V12q zV!1t~bz6+Sr@wghD)uc&_x!x4MPEp1!9mX-w)*{_Hg$(PWPxXwpTizknR$y&!iE?^ z9~cLAnCR=Mn<2msyY_B}SiT2$&c%28(|BHXt{1-N2{3w^Nn$*_P@dR|7gLuRyWfX_ zKEhE4mSbZ(e-uy65gZf7&n^*z1Snk)fPYFtIIvU~Hfd(YR4}G3A~6v_ zCw0v`EOZRaH!N&BnBhSI@TDFOAoUnCIF~6HmjN@On`$S7O-PFv#E5*x6IrM#z)|vD zJuze(7(`tnERdDH&shGqfQdw;=^8?Sw-}yUF%2EgjRX~66{B6+JrtnDF)HZO30#0P zY_#>p2xm|CFMnqE3sK)omE!-^rrV~Yy9d7IFa~2?Vo!ln+ zYOR6fuzWL_12P+hekK89snJ|WZr)%G*fMAmRMFJl7-1sN*{}$@hMTMi{}CXns1QEi zQr~b#YD>J|5*Ja);Z!5$phh=(4(XiL2}MpIvFd-j0N3X8AOXP9T&k+k%*dmp6|dz3 zJyZNoTf2tz=bGQ!pr|%N0u+T$2+<&jF`&vCt>(uQ<&#Cy>p=p(QkY(*w*hGumg&Vd zTqX&YZ!r!Ej74(87hwQUpnePq>FzF>2qRCxhPoCbVy|SwDFXdm#02Z2x*Ep;*Ug%? z7>9-pR5=}n$p=m8w1(W8SOL_1ENWI_(SU~Xj)}q+VKewtkxY#f;8bVT4C<(pqm~+& zf;D~|fdR(}8wQgAM^Mtc6hI*YT?Rkm8RqDvsVC*-0)6_bz@eE_u6VE-z#4U;h=vI= zg3f}m8rF~v8b~cfeQ`oY5Th0@Og2-BL^NO3Qc-8A?okm)C?lP_2q&toQ(HDZ01O0 zH<|TzqXeOaxS3;Xz;D|w(CW6J@V@1AVBn-;S{usO^lPioFhGUStm*;P2yC#( ztOXQ+%nCGf)2p3R?rDkIbm9Y=YE(SqQ_7Acar6b%w;mcQwlLvUMHsvd_>x=8V>(2A z|d+~ADbWC%fJJX*rNA_1YJKEDx1L1RJj5m0W2k^$$BA3dI|#>i%_ z_JZMPUgf={7oIOV0wH>fFd{3bw@4l}#Zd=5*~G!nGA3gMl7R~(GsA>r65O$8C+6`2 z5Aj9XNKzl$)g&35iqx{Mh$#0t_v#+>v7eA=k&%$rdxZ~R03ez+`ibp7wsPn<)>td7 ziiE>SB5p;_%N`;qFMmbVH>s^FKPeV4_s2KGSCo-9Y+<6qCxF(^8pL~RZ@@1V$3Cok zsI(PA<pz0?5N&hzZ9pJ=2#75zN~m-n{rg))#h0w6-EDP+7ob9UNr9_on8GZWExb1kzum&4hz!*<6c;a7_unL5uP1aG5@8HIgLt(V0YPIony zS7+}?GKZ9!Ap&ETJe=xGl1GC1Xf_JCpy~-VYL-zNl~a;=@kgF6yFBC2CEf^LlNow* zb+hy4`CiWW-p=i48!U-RBFA;Rwvn;#qJZ;y!04?osJ zWNvtA`vTFJGRJ8#bV1u|6#7uE|P-nCr|knX*bAvP>7(csewSCFZic_&~Z7#N+e zlDa=Gp@)`jiqf*nSri61kWFOQ0JcypJzR4v5l-{i6T1Kjglb~)0V>~~Y*IteG(W5` zW<+J!c^~kSTSF=bMdKs_=oLg?UdvuEH3JJvEc{VOaUcz*c7TCQ3OF!Ho?Gt(5rh$w z?4_Z+5m%@DN(?xL-3r*hQVkOLNl}Iw#vCZ5-)Ds{mtmO#5#G#0vcjHtk~4g>>LY+f zM;glxU;BWju#3E#rH>vV7%0#eBlgFwA4&luWaLMmY61=&LXxJyn`BnI>q8BrdlS8Q1QK}qf-n@T~7)d`4yeMidQ(gmJ&73EByQMOw5J_AH?aWTziQrx^ zHB&(*?ht!s;1^worT6G@$yDot#3vy@AdB2t(OZNON3a66$=3q^Ev03Q@{(v$;6W8^ z^Fp#Lg-5Ug-oI9D!G8hdJ7}Nz8C8yM>B`g~R z{zIm!8l^DUV3J7Us0jdaClsQtGlJIof!lvk*vuM{R@oPWjvZ@HJ}{VpaRC9achKci zrlgD^gugpxz+xx}%fzY;wN@bv7xYw-qotUJT8KCQYD3YdWZ}%Vh!nTCm6BAzOSRk+ z{}y3;hE68qUj1V4U4B?R%c4a{&f}9SbR}6xbY+>4ZWX8`2;4D;aIutAM_m{gF)R7l z5io*6XlU73bCQD~a0`!vCJAljNwfIv37@`iIZo&xLj1s=Rt{8*9vcd0v-BkS;K{_< z%Z?J7iVaW~K)n>3ihuTt-6G_bxaN)E4Q=1UmBuQ zjubd*%z;z^scIO93n|xvK1vNS2kT-^lo-k=>~wQrQuZgGIAp!=rN+$F6g|L_6!y)T z*>LDq-Q)(7y#{9rGi&x}1pP;EP9WN~Qk}03g(G7cp^(7ACMVYvv8{K5fI-(8K{nKUAk}6hH;qeL7J^^yOy| zrl$MfnoqU{s||T=>41lXvSeWqL#blQhJ2G`F(uA^6<`1c;kWoS%i<{5f_!q2oEvzZ zWt3g9q;7 z3;U^5YcY|+i&9n;peKxn<3_CiM9(!{((;2AJqbX#U%3+3*TaphjBjPU0Nm^9qu z9wJq+|55a#GEtqLK}k{(;VgOFdc5g|3*+tZyngJ%faj7Cv_Wp!v*%0ja6wk({Nsag zfP*pTV+9#Dn&Ct+A(6k=gW#d42eO6sHg0)4g94>czxu782MEWIRc=zs45y~K36m`} zM;8*?L*)`oOD*$0h+?}_9Ti$Stm?3CPO8lyhCSREU-8XCU)?bmT9q#13SkDq3StN; z%A7VJ70IIM+JX**VR%gtIyzLt$}GsvGFt_Mj)j!Jef{sDstPT;4@RELma3+(KxN52 zkE3FJ(nF>kdZ45^8H}`hmb8K$;{-flf&Fj4`qS}N&wrsCRkmdy@#17 zWE}m)W1>UPPd>nurOTZ62=J$FZHJDFke09U4JX4x8mIJlw9oh!>Nc`%gY(0L1w%l@S)?pI zt$_By@)a+DngcaE0kdm?)huPd@Gej70YfGcXAOi2y>-|J$gk*!pCm)b@bQxzK!|l< zkfg|z=FgZD7prP>F#O!9r;jToKN4#OPD|oKLd``eK0Sl_8bo(!L?Z_fXnYzDT~#ML z6cXfXSTg~`sOGDPb=>R~L+rwZ9(d?34chyVQ>aDcG@;JrQ2srXX`yKv+9z%SfP%45 zkqufJ0J7JPdGCXuQWP3l@UXG?mj%RI)mpim zfOCP$$tw!5YTw1G#HQP0B6vLB01cuF+P(_Gagk{vE{lM`L#5)(xVAIU;PYFchBNOw zU(rrNWJB=7;;SK}OvM_kY&R{I(VF(zgAhl)2&kBZqS`nb)#~Vk%9ONRXS96#APMbH z<`nU>4EG>50VM;#d}~LcOBlXvW`FV;>D5ql1<(LVK(@cd9R?~!T6)QGiXt}StUpbS z7<)d6U{j%6s~JSj2;aL6_AfxZh-l;xzQXYBX^Fr@iaykU+7Y)xAhU@cgzwMBO*0ef zWmTx<04Hm`Blij7;xf~0Qes;4qu}<6Hbpbi-jhZ701k7UP^*F!KL|DoFh`a^jflP5 zPr6(UnU?p!ASUxR3@QQzdeP-<>a9)f>^^1y7y(er!6ZR<=s7_DoQEC3 z2#MfMgB-RVUDSocibsDCC5ix>yuo98htcMHt@oc24bD?)0QY*PI3R#S?r94GY!fwf z>{${dss2%#m?AL-q9d3@CqU>l)B*~LDzJHfuXH}WD-=hnKtz=tDw#UMRQSoH2PK2@ zTcOqvVbu-?U{k&NeGFP42T5RUo-F7(3mO?rR782WoD4dfVoc%i;sC>L{8|Z&g|=_O zFHS%`M;oL&k}pJ#6gJSy(jg5rD7}rRlS*L%K)Q2)@ZDt)4JQo>R`%l*=_8sR7E8V( zkeXgowzJi!5n^6MK(8*ahZIpW9fz@z1c;MY<`NrI-a9ljmSF|*sdNNA+2jHm9F7u7 z@yOxH1i%Djm-;k)6KE_?q+A4mDUugMTM^o>L3;^pnU@DN$QI;fN{~?Qtfi=D4+iwq zUYVxs5mHexeFcp1sCt>GPbQDR^op4vEMB@alj_0;1l6m-vO*6+O$|r@(ad>)Xw%7V zS*qHF@Y*hMSV3*&_7br`qwx?IC&Y4XcT~3_V0x)(bQZfHHiGAw!I2cQTX39c^NiRZ zsc;rGPy7TFZTp7iFmS@qd`BU0L8-xv@lMvd$Wap8{D4Te=RgWacr-u5^ec|HC8fmJ z1~0Ax3j7jO0pG?7X$b(gqAs@vAgi3^NoWvGo2gmxcx=oul^!3X9rRmH$2t^;Mray} ztOY60*Is{LpgB@L9*7ntnXj3ztAw7i>g^Ke-K?XESH?d)*OwJ&2&y18O*n|M;rZcr zXk=ej4AfT9XRsiX-Is5bI{FMRIj&9Fh7#LW0yig+Y^l4pUZrUa4tzu&tV6A1Co{wH z+use6iKAD-Z?hQ?wsr7RU9Q<|6*Ehl$4*z+OIWC_i=i&-sYgVzOo4pVvOt0pgAsAZ z@q;oVt|q#DbgG>Nu(?a8xYiKO>y{eBKwo4|RcdUZ*#V{zrLD;S$m=O=3i@#qmxqsjH0l^wuEFVCoJm)B$@Wm`=*L+JOHYqRc}}YZfk%n8^)QK&m&F%{icS zXwz0Edk{*=uffVa1HjWqR*XjqlU}B-r*9?QukrHO4a#09VCuW61e+e|GIakDC)DYC z)kBjr)4JjKa%zZw4L8<04sT1Vh_GBpbv~gSDAGZi*Qq>NMV#*uV3=SaMgwI+FK6v3 zFhzJs1I^IrSt6FNxtLo$AM!mT`K|*eN^!9y7V$)9=4+Z6e)+BwFk;Ar-DD}qzWm1; z@8p6}X5RFhf6H)T5-1sh1@$sZ!J=pyJRJ?!KVbm9Agj0}LlFU>20Dpv@d12eUkqes z*JJ%GOUA5J)X4kuK3xZNKu{$KU)WJ|aNz57gvF0S0G2X-zaDuF*rL^I`}FbR0@3l4>46 z2Gj(&qnEwu+@6aWYa2ms?K}6`=q33h zBaC3~B7Vwz!*1F^J0LtPicwdjluxkSLHI$|FIo!#+w`4cC|t}gPpG9iYxGiAHw@EC z*wCthn*!SSRu*c8;5PP8v1d+>@%7W<3y3NW==KL)(_rEyB36` zfA^{+E1q{rkFXDH+5P)NbAn1EO7!BG@eDT)i7lc~ZXauQxDz-M?bfF|!+XotXJ`Af zJt5*%{Fo*Dd_lyu0RSJ2Huzj$m_bRiVb49BQXkF1Ox^x4B>_D3$VfP6SXoO3snstd zyiF1O6Atz8$(ELq$~has4(SSUq{jVSC-R*|w<+M`v43op7f(B`&~~r`F6Y9Ry*#$J zAJcUN^?wPZk0Y~PglF@#v@^2m)6fxDrMS&)j=R~4en&CY_3<-jy@z@Nxet*k|F&V) z)l=2uLov4<+10BL5EgV`DrgI(GpVBuAu;w$+}QL!g{_u04ZPzMVPXX_ramo}((OEgTvg7Qia??m55`pOL7r93O)vNok*+<4f{a4uwyECW zbkx@_k5Sxaj7>+e#Xaff?9N0=@G0&=0pvj&uO5b~_)KNlt;qp7@%sSGFm4JqO>a`L z_D~rGYT=6ZMm7ZI8Sz+zizS!`p_`i_W;2YOOvov-uDqjFYbdaB2dq@}*Xl0yL^zBo z{^SW^GE>+J%Sx7r$ku~&(%}>U?DM4ckW!b>aFjyZnTA;PrD#7COyQDi28T<@VI~!l z1!5xt7i1#oDa))~$6q4aYtjrXCy5dW+-~68ccHd@65$d`Ugn-x*;B4QlZzqbSps67 zXMZ#W{XST)BtBhGZI1@ZjHrg@+CV=464*>tOWP{kUS|r9)1%RPe{*F>r12#92ppl4 zLMht-kgSwYcb38Jhi~IX)gJyqT3JFBNFnG1=+8o%=gnSAtc_!q;WN*~TzeT4?1 zNWfoXrw^;PtlwY2GxC-m#6F#w@++$@KLjptild2r%=zA_n>l%foP-(bwx?$d+J*Sh}K6@p`3TUOaJ3;2vqbJE0 zaf)d2qWcuE5Hq;=S{*X1UiZm?)i(NW5A?6q>!jjr<}=#S_j@C2ISRt_@TvSaOc?&V zZi{&l0XHdFvs-`PE>x^~w6~?{PWl%yWTkNW68SxkOwO$Rg20)RPa^U}?%mTc z+~VnyZta4Y=aof)1^F3sQOqgvUYM+5S2Lz19caW3^|9W(D=~6 zl~BlNqH#uKM{Y#uHgg*%YckIxFXr=6FAL+PGUl@4EvnHTPLbUDw*9#ZgOB96ErUEX zn?CSm;c3yZ_p=v@t0AV8*rOVBU<10P0y`B*l*w^##IlC;DK(}vwA!>z4EQGZY z9f$yle0)dq)WQ>UbKV4e2we&**ebMxWD2?82;rRP?|MCIKx_>F5Df9XDZCRTW;~XE z34_Cz`t7knsw*?2ru2A+ex&M*`zm1EHY5qy63`VJAYv&|v=!I@bP%+Gdtocrads|fO^dvvS88M8%TxI>^rmz7n|F9;4Y*b+(kuFEm$*KCd|yeces)fL?<}K1bqEgvRYdM7;0;9!VHBYWJ_`sf)FBPjvl_JN_k#Lb8y8~__@I{yMn3zHB5!gH(P z0SuV+;4QZ(aiM7|^_eiC@6K36iG9?KL^2B{4_vjNz2hKrq`3|1bi}kl^ct!&E?C~C z4$&741J9svh$YM*d1=c^g%n3W!FCyUw%1ZTE56@7X_xWOJU&9^AX9RPMwx)oZ8=Yw zDv5z{bl}m^yOKuo>x5$9ef?RLv+OUofI#}^_5K?MVeJy3iiK~nhZR;0tzVM$6=w|N zxGLgI2sgCgcbYnSP~JEMrqhCSm64tdowIigone#uS~QLU%XNY4F%DPuI>?}wunS`4 zUgZMx4z_wy-;NR~I^+h>$}|Z^tqGK;!F#2?xpYZoxQy&sS~j>_-+_&paXo#? zuJu4^ks5gENJTu6H8vBONP^fDcxOzwOmsd=?DFjGWj?pW{!0bDlM=Lzbc~I;T z^ThZS>~v-HT%5(z2>vRXgm~BB(_8MLq@sg9m|Epa*}U3lAly)rLSvE*cbNlNxLh6f zj!Exrw`cY17~4LeqUxo=T`~&U6{Ro$<JAC>I(V$`Z zX1ChmC5Hw4=aWmt_qJly_^?$G`d+VH)Lk)e+uqc@FzydG+0{EyQYct=zfNwW#=U-M z``rtMHkRk$z8>Jzf1z|Nzm-jce7oI6z95~H{uYa?8ORh=D!iV7#SHk{ckU=ujT4{- z4_7_F3Sv$@wO0UL?s|l;Pb_(sVZF3G zxCHsHL_0K`H@76zCDNaP80x9n0m`!4z&Y(~Ur)tKTy$-C^y}j2$P7X?hIsut8wN6M zT_!FTh1<|SL(-)kI9&k#W5B|JYUpX@h6EiR%qO!nI%9!XDNU<7I+a_`V0hA$h(i-Z zG$bMAA^@taW)veCWAq)iSW}5W&p;`0s-be-N-p7LV)$ZSPW5Rm$OP~;AyEpauPI&! z^~KXx#&1|Sk(niU{WUFx#)h;lCq7I@nvCj1rBOw$6&gjghor3h36Bg?6=if+vsIwy zHeA1-u9yrYvC_qd113Nl*0S}l927OoK2DC=jOQ-aBKrHSUn_V1R%3K4>m2=(smT5>d(8Idkb@AGpL z++P=e4Q`lTKZ&JDnl1(hFZCkQav%2;niIfdV?m`6N zjySRanXRSF0Rq&;07MI6VPN3tVMLn`&dqENBlrg1CfZ%#`)3PL#^AeZ%5HQ^)E5xN zpU{E{0r?dS7v;I*m7OA7}_K=5j{HeV28#S+kc<$%+FT+rEFaw_CrqRASLGix#Q=@uu!Ih`%7IBE zB2oM{WgD|v=>kzrm5Jiu!TIL!=FbLX}rZw zN~sX-qw8a7POaUaEQJDjDa7WU^M@$dF+ncK3jAR)>5K6PkwwDj(mTK8b{?^f9jqe`0(L(l)zv5B7LO1<%Zf$;-7$vwV?PceP0`j&bJR<38|V_+7hT^(U$sG{g-0^0nMUTqi7`>EGo= z%EOlU3naHoH1SJ2|L#nsR(hrR?{h%?S0V!|9tvf<&nq(xT}7`YtUpt<1KZK`YsX=+ ziZnV?hrf|NKUVx(i?8cn-gE37L-8Z1J&w^Wxx5G_MYP7_PBWjP&zvWvNIgVWU|8Zo zq4@Y=P`H!@ZVqP@%f3;{OHK=gPicWV6xeuJhqH1mf+ zhqjJ;T^1`s*$WvOLWy((qv@G6nY(3UmZvhHBAX?>QRXk%JZ8eLpMKM%d~HI2A8BAu zB5e*cy>%zaPJ!SrGKU#c+NcE~a|G(lu~w>8lEMg+uLI>vB=4uuFuTkTyB3r;ORzM8 zUZPYT#4Fj_(;KXc=wgBjb7vgTlp*O9ib4uo%CEP^;0yDOvyYQmmHQOf=pQ9#9M5ME zGr3en%R{T#s=m+TcBt1c0d>ZIiIAB8%ihn%?|crSJilR+n$Sd><_JX}y=9F2u)LBJ zq-Ck|Ri+xuI+g1!54lSKF_^Mq4{u%EiDAjBacjZD0m%N=C|6}5g{Eisb0}ZjMq0MC z0Sl!Yd3;c7css?d5Fgk-Kq7n;7))x4y^AxOgh&7TH9%hQm!APZ@=r)L3Il3=fES7? zxm%(3&e&DNj@hrucH)e3kK=N8nlq;fV}pqA!ytz?J$cSd>zM4|B?g{wTH% zp%=_1Z7oA1d8I#NO<$`8o*(}1p+pMYYU{6I8fXr+RvkTJuBNYc>qfH0vhESF8Hw`-C$)0cfXkZ{W6XNlcd1%c~>jAHChr*N+A}PFoI5etqu7f6|zJ< z>=?V+2*GH-&%YL43|{U#sqMv;)rCIH6A{AmT0W;Gr@l&s|4!b!$Yk%t068@V5ND#q zn-7ksZK_HWtN?~4*!K=DOp)yHA zyjL&VZp|SGht^QOiP_graXmaml@+tQ(O`{}-D||NL$7_v?D&hNI=#V5v#h|4rRv9i z1y2JRP9l1ej`5=JCMOW3ihtMr(J8%VR-L)~<1K<|dXI7IChU5De0;XEKPgj6KY!)4 zLQRhxRUa~{$HPAshAhrcA+NsX?BO!5+}zS+#6H@&#AEv1HBh7bK>qY_Dc!#zV@xR$ zn9En4=wzwxq(vy~GQIlp+MrIcHmry1-*DURU(sEcNo{pSjdz|JL9zDE;-dbe)-IzC z4LUxCzu2t$!R=Sb}+1|vGK}vBpeqWUgT~-St7wa6y&{%q zVgB1QQ{erZ`Z|fQH0Rc#UV;%(s7;H0LOLU45CwObZzsf(aZ{tZahb58_X*2P2$5s9 z4WIL6Ot9uCN`|}7@8XPMugr4i8|K&-*e|2(IixSyu&lZk6)&35*CVJ4cb#Sp=$+bl za8%_>G@L;LQ)Ej6&6P#-t~PmtwM|C>crPHC!u^m|;+GrOYpNQ068-vGFk=+C10M^b z-w)^J&xLFPN-0O9@?q|lSdvIgcqN_q7zFzI+V9MJNJLnc|CGYuap47mm3D~hA+)-9 zwr41NDE{Fd$`4Q!P9*j zQS294N&yk8*9Xbb9U$?{9F$c~J6Fz1(vTj0QU1xrO)iJS=p8<~EXjbEiYJtcxi2G5 zZBIe<^dhMA6_eXId%yS1Mfc+;1oYL0R~YClZ;22^c3GeL_sTNIr8uiw>|fG-9DOrN zslXohsQ4&{xu4O5IoJISzHkN& zTS@&8d~bANBULysn)g}O!B-t`L7Ev+vg^tHfwcD`n17Y$Q=&aW;FZ)1i&VmG0DeO?~D>Ykb zq@Ab~Yu52Yw81yAD0o z8gEr{LyFk`NmX)2>`UM!^uu?RkV(-HSb@rn)=*?DG|48I=TAN!$bg~i94+iwE56KO zcnYuDi@zGy2Wd>VS>&4eKs=j|hX4pv)QeSnh7vs*V|vI1iEPp!JC$rIx(m;E?|^vc ze@@JI(MO++@O%iUkqG^j2x^A%6&pBn|1Am~_Fy=3;G!5a#KE7*wd2NvQMK}jnVQ!W zoU+FUw%LfI-Zc+u7XvUxuH9eQyOItMLuVlVZ7ZgWfuz3%Wcd14$Pp}Ne+;xfENbG| zX-&JZm}Q><7q>*+B@g~suvY}M8Dd5y*l6oB68?K<s zam$GQeY{vrLcyr&5`_T&BtRkJOl!VZf!nyDe7W|d{ZZh))+h$-x=ysAhh`fZ2mn}1 zXOcl&7luMt7fOriqa-oqz)9W0feNA|N9R_0F6!Dmo#m5eVI*Co{xrOeaObNPU>x6G zS?ZpGPIzaWAA>n;*(24T7!%O_TP&aJ`s@;n7On`UejoBqh2wm;w)m3k33n*6vX3^N z-_Q_j35DMc3kHGZCy(1u+w?b&olOq+4f~HOt1(p~=>omOn5ogZaOv%tWWh&);c127 zhBrReT-F>fY=8PsJLB;w&WT#}HGw5D}i`P3N{;tOS zSZwpno~ls<#)t1-x)e&vGFHNUN_=NHyy6cw=*J4mogYR~YAQ$B;EjBxEY}8?IWcD<0 ziUy|o$BG{@uq`7FBK}E5;(>A>B%bUX9ikz3LG>`&0Orz^zk6y1o(A`Z+qolYgoqv8 zhq8SqTpr~P=px?&40wgS2k3|x@a8{8hb|mn56d+gF;ES8{@l7}NVd!bHMeWeHW8gX z2_+f72+};vcDC|t#zw9fx9j-OSKHlsjHfZT&D)*JV?-)nCOcpRLZK%izf1Hj{K2E> zX=-8Q752@ha3g=g(8GvtP{>nPao8`Vqg_|q=&j~ZoLqBpG_N}H(PE)}L1#Ld%-5Po zLcmRtx9E3RF29V5&-SEx;RW63=#>j#W37!*iVOKdT-$%q_1Z(^2P5JWzNB56(Unq8 zuz0IpY7=j7!4^Gve?ahfUEt(SBnt{Q!MhETT%i_{#^DvW^-WO@Q(|81ZQbc{iUFrv z4uPN7@Z-U`&C+%u`Piva3xM%qnI|c&7GdTg&o<`Z(DBFQXgvompjrX7<$f}TKCjb0 z=~JnXl%KWzC~a*==MfWrKJD^(<5>S_VV2(N-?J&2!^J)ZbSBEBApeIu)B!v5cBwcL zdZFg3;+T7_xdMW33$?_awMa$Naptgj_>FX1 za-5S+rl2ov?^JJd^Q#2klXZ59JRP0ytH(x*{B%J|b#3Q6Xo5ZPKUa{LIT6Mjg9_g& zbD$2zi6JWro9#jSSMY|V_9$!pasOEI#;ZjJ-KrT0hBXWoS& z_;{(Zu@J0=7yIOTUzvS|&pGjNfvT=;x9pRs3FWRY?%^t+%q}CrdXcQgag|nLP@F6_ z_Wf&6Kr7CGV!4LH0Lp%H7+&PRr2W9glt=T4Y2aQXqEsLI${b=Eb)>Xmtu}6en|w!{ zqNTzj%`b0VLy@!Zl8Wv##WobHZnR6P4oMv^tgIHcrL3%`why>~fG5da>`|l{{lUWN zVu|vA(gO-63Cgv0-UXqLtsw><--i9xLYPLft7YtYyqk%4&;`e~s4MysvM{m9?J@T+ zJd$^|G(WFBqmv@0z>#UJm!vA5)v8%i-m1YZk@PUKD#h($d7ye(Q%F$5<;=0J{oN*$ zJ+e7QHd1?e{sCAWu<4Y-dKRWFp!3kT-1m{`*za?9F3ivj+jvrWBP$46buh->wf~)h z*jK*nfF)7=Pm}JzXSw07)tQ<>Zn!2@m6vq`^~tiYmI~JnBt)ikC?fi&k78pLZ7Nih z8B3BTukfyg+wlT2Kd@pZ7?uPmWN?A4{>Epn5;ry}C`hKH5?x>5`?6xj>9w+3D!5Tf z#*P#aqIr;eOON&u;?{vD8&$hnYfoJYMwE+eEE^=(XdN#RRLe!t@r&s^a|r0w*M*WI z_Fv%Titaupgn&zWYhrF1KLWy)`t^)2@lz{ju1#>2W@u=fTFG7&!*l^DF>~3au@#RV zK99S2WtVpOk75)eTIsSeGEJ5>QVGqpsWc=Mu@uYe5qwNn&n${=L5n~Y-q{{w-k)FS zj`3(+`Y_^R_#!`FV!hG<^4`vLKaqF0Cmy;lhw{B zKj#||DD^8zn?o6zbQ%bU`iBnFJtOMk@@bCqr$b@1s-4K&aU4WY(n55cHe|0X?Z9VAX7z?GL3HW&xqI(`h-BCO`3fyqHgi zN(7z2o2Z+FJ=BKZ--Be%qIx0CyX&{UeiTD=s~uU%XA7V++1LU7JcdswoTmT;ohfvL z!u|b-Cyr$IPv5v|2hKLsjin9B5<5J~Za~#nDBcU9mW@P+(#ta6<8K{|31D@fDwoU} zHP1UT(2)ScMl6n+FzBX-XPY7y^=*iP9E0PLu0(y~KB7a@Y5}^uChx^FZdSaPeek21 z|#o zoi*_4guJ|Wij}|tPC<*7d6uNka4F$?i^lS8=A!l`t5ciNb2HSR=&9e+-&x)=}>W3>rCC~6FTge1%*yNcT^cKS>9ZUY$lvy zol5H??J>f_U8RG;tfn!U&z=bW^Dje))|P5(VVM@P`?V2C9hzXd;HS6g1R}HNNSECF zA#8jCWKec(<7gPH4ndvY)4MW_-W0DC(%|q;VcTY2cRFiKD9&=D!U}i=nfN*)wob z%o_v=klM!3TzWObnje@xzQ$4JWR)r zVYOj9s$-D-h2uQ3;&m_i&MLG=rX2s}kD`pJB#3vad=JLUcs=tQ7$g?#G34n`T}v}06( ztSZOzZsk|!MelcyRXc#%l`-Jd4SD6;U7mic)RkUG@i(`46$k)62 zeLuZ7M&rK`WGBw)oM8OeD01c~f)re(oyS^>SU}{x&o@kxx0sOfn$M%$H*q0K_0Gw8 z9YsvUadq&x&me~TBlsYKFL_o=n3?;+rzvWLySt8dc4yh?X=ghW!&7=JpNT`7jJ4%}FGZuLprv|daLT+= z+1?c_NsNd7f$m@hQ-6|-20z6^x(|uaE0f5h?(e?bg>!V%?QsQS{ms#z_$=@zx+q%j z>up4iRk)N^&ZtT*fke;GdPi3}7H~ypMB!A@l`7 zaC;ZCw!FSnQHF~Y zHIH6dICPqFXhQYkYr$(t-%)wqEP2bHl#>^x>hPF5p2(NMA0{QDu!t8ISbAqUsyZoorErsgqS83z!EAPN8dO# zWvdW{^JZhGO}Xvv_7#{Oq3UUJK$}oD_gNvN5ANvf+4H~chbty!RK+#qPS-}S#rQ*@ zVayCUJg#&6VGXo{Ma(jayi2|oCplLH2d;KOI`Ay&=gA@{`s%1UpzKIc^CGlcq||Og zS`HNx=2USh(@BhR{DCen zF^s-TR*REDcP?yvDy3x>a>pJ6ZG4+$klx;(U12#7u}Rbs|g%ZgV% zbqV_<|K%u#C4xqn*?p<%ne>QiM5a5N_1#0?$3jbc09>}`{znP$lX zH)An9ki(Vhe!?H7-wOXdo`ZUwSP|biJh5J7mPY&RnoICN8~ba?RpXtA$4t&xh|V7? zcQ0mKOY6`+L@6>lrx?1C;@23WT*8|X-al*@JtX8~Fz&c&NKzk9WEC!N{G~DVEV?%v z75ZLcXOuhAuK?9I|w%m zhj__A;f;WEQS@FTqZKds$*j)e{1N27XoP-;n=l63qn9FfT1-K>T%>3VFH8xosQ_L;X&g1C)X%n#Q*K|RA}XYi>m(WA zPt3r>6`o^?l%DhAZHwc=G5_6?E@|xB0t$lwiAjp3SWR&bi;TD;z>~TlgSdfKzPacZ zzIB&1)HxK zaOSYkP6#?{dK8fBe5L^&o=~R@)h-B$iydwjf7dS~bFcn%ThY9R4q#!%(3n7nMvDC1 zA4gV^RrQhw^lyA|84;eSWyPHqUnpLQsJ?U7S0A%>m)_}M!3oBz?@kj9(*mE1`!PZX zjl<*5SVA^XPDnUF5jdZOUl>OJ;6`q}c zwyU1U2l>6EXjJqoQ%iWtm;N43g&iU_r7OmCSag!fJGpf1BFESGa3dRA$jH6a;%sEG zgC8@Uo+hF*pT(nWVQHg*@0%kEZNqtm@B#9? z_-FXaWhcY?SSsb7C$c!$RDaWceR<_WZW_kM!c=cNmg>7@n8Xz0qciUF+vLoK3oB-_ zXUwIPoTO;O#+iGQzh)I_HHl=yBoWQ2z5@B}OOHSZ`XpoY8-4QglAiHu0NwB+OiHCv zw(3RIRiJ=n2FqH_UI$pKAV4Tcv%G5GDWYu(S#Wmi=|I7u(X!g{8r>U9cydzUt2=~^ zU&o$g4Zasm)e=Xt)9B$NbXumNJzEjQ1Sw^v1LK$Mm-4O^i(*{f9(OC{^@j(O6Ihv+ z7XiOYjpeJU$Yg`YF2!U$ogQwKR*wu88tiBaT9q`-5Aw?%%fH-Yl=w8#0B^KF*n9sV zx&~>dcevk(ev~%Lh$x;m@g^)38V=MvzRTOfb#Pn2Y%R@BMsL)UNXWk$@7CN8ovg*3 zk6tZ*=#D9i=Ys+1K9Hj?JT(L$-xaQyD4er4b5S0b0bk`1?{&MnC$gYu+oD!}Ug7f{x zJ+u%`t2=_Zj3b!9eHD?G!pJj*1)-4*Xit#_knhi+%c-*Dw`OClGw&~8hV~De44YjNkyV@Mp9Q*UTN_*vO*q{}+y_aMJ+ND(BQg9ja9td!1i9%A@(MIazCUf>Lc*vnj{ZYQpHkwL)lS9;jbxu8g;jN{K^XZJ3%W&qO^~K+q^> z_wkx|vbEQw#uYu)ufYVd4Cx4KccF5*7Dt6(UrGoU@-@ei%7zw-s7=B38cF7qX z9F!e+=5+lDvJf2)CSDLgQh_gBc~{ENGLe2e)UMW3`5nRUiy_9_BhaS$i9tn4Bm9gJ zUiZ?i=bqtD7`1Mx^>DZOS8p#lvUsAU3nYD3`a(51`iO>F&cH-^rLOObp2`;2_cmRo zn_f>c_Si3ozh3noIyv6iavSm^Y9GLVL6V?%3gJD3JIWp^h6_2~(S$ZaSd=vz3&Sj{ z&Bix*Hh@ehBh1tD`3#|39R?>t&44A%7Jku7iNwaCTOM624^zDZdhD*Ss zn91Fb6N55!^v&{jVmMj^)VUzEYfgC}1G9Yc3uUsEGJ#cjeYVsXVz=c(W#&z7mcdd$ z_Ofml^Lrs7d5b}0LGTn>!d&H&)+w-TB&ydq1`ol z#eVXVe#}tR7c$^+B=5v0QN@rhhII?M>0DQnPx$88&cUOZZSaP7`U&)&C4FH|sEDc& z4}_xlZxwh{ye0VbcqdaAgFGbW>n*=7r?|0>1hGX;rjP!{g0ol-f zO<7?9n(oNzZ3W>0)+zJZW(&zN!5g#T6lDGrQj}i+rTy2eKm7=b35#W~ZF+`r3FRbE z*KBGDE*<5()Sw2zjl|`Pykeq#5yZf)DZt<5Ev! zNu^?_8!iybUBMY8){t!b#|;(7PiUJh(He<^d)|;wu9Px(8^WCM4m`2hYZPlqo}ar+ z1M{@+SelAI&SEc@WsmN*DA51Z7ee4wd&RGr4G+91bgw4*-H5GuGzTQ2U@Kj4AvC z+Zem!7F?W8eo6#%q52%HYwf2-V#RzgU4(_C{w}r8cKyIJVm%9ZjW8|+2U?|b!E*5H zyOCsqVi{&&v*CC`kxyoON(SHLKa_D$0wi0+KvTO5bG6W7l&hagd+J%<+csHk<30Qm z(fskc@j(kN?94Y2e-+1`o7ljU8Y$pjQ0=9}yiywHr0wwd$o>^sOXR{Urr;PS&G%_C z8f%A2at}qb{HRfTm|H?YS}nfTjR$kAz)c0S!}0kRFY|5=MGCiULXR4EC_Rhp7}Zw| zU1k^)T6dk$xt)f-M7OVbDxWk(Q%=2mJ0|+O{g$u3fLkH>m11c|ees7luHu7@ek6h$ zQF92?Sz)jvkH!hSQ}t^^H9)03MC9&*t#mLllE^$jThlKdf;0xD@T zN~}24Y8D(r{IPsC&(|jRYomS|7(}k6mTLV&&@wBOt5M@Ntt!r*K-C)*R{(bq3?p_y zm_S!w`_Sqj#TEm_O&_b4(;jL^&J-D~4GakqKZJWQq_I$6Ga^v^W+iDX@|nbsO>o&w zDvrd4?ruzDF_@U%g7!RexXeIyjSDskw+|O_k$TEotBjFUGvc%j8#^R4Gjt(*95q)= zuhB^t2v^ekhLg5wzgb;*uNNqFVCrD(0k%=d`N#$|B3R_{OSFsc5CAk9i+ zw_y4~7CD)}3&OgN!)QiSjpXA_rnw&;zMDNgnHzmDR4poz!iqSJG@%+f?3zTpPwU)G zlO0&1)Fh<=05L$$zj4z}*=l~*3r)ZehyW=`isphpB3fjM$T{9%k?&*k;OUhs5*@qU zM)g|q^`3}@E*iSKvJ4^)mM?0D)s1q)&lHn9$_-Zh-Y@b+*K>KmpoF7uX3;=~=Z_(+ zHnL~FAj|#}CamcmitP=ah;r<6$YqbBQ$o54)Pi(8__E z>ykWpIW+uBX0uAGGwyIRDAjL_2WRKpDl-e6IA_C)P_FUQw&b0X7h)6VbZPz*(z z!Ja2I=IK9LXVJTcxTPs0K7@rbXD#jHf-IUew|>I&6*zP}>j18d%k z?5aR8u00K4nkmEH{4+Hzy>dAZ(v`8(_dsjd>>FuglIV0k{)zN%Np5xd9bR@9(_s*L zC(SsFSWcBsk)VM=(x2=*T&%jkCV(@NMeagnOC~x_8bU8gM0_X*?7-V`Jk>2?W_bQ^ zBKmBfEz45d=^NwcSOZI3bL-J)lD+E9ngy$)9FYqo&g|bh0ZERVFAf}J*N+JP)Q@vy zMB5=q2m5J{yq7z~`oCkB9M^v?ET~7?${Z;>0NT~9m8i?r(S z8;c?W@LP*KKt%SaESMe>cg!|{iNeu9_LxtpRr;#Oh9XiZR1}&MRMBrqL>bkjw6^8L z?z*+68p({$J>pr>=@UDq=ip2c^s=`>CWh-zHE2oR)S|W4!IBKaB zplD@G0&jypr_~ZLAZ)kWoHiHnwfyyZm5iF=iU~AB-+UdqKJw^?Ds>;J@kE3?k$AxdKs}2*PPlZ4fE%$VsWww%Pr3MZ;IRous+Eh2IhOpx(q?T?kQqO6Gq5hysKhN!1X`~ z6KaaCr$#BCx>UvI@;VD<fiy7lUH~@tPfM`6ScFqwE>3WFPw^? z3}QO4Ety)}X~V9Kns^CAjjg~m7HS&VV$?k6_2L;nANo=*d-Yw1WtS13Ma16;JT}Cu z><$J-*isSnj{eh|?+l|emA2^zttaFhEwTOHD2t0MKxeW;YW=%&{Oaa5BD{%AS}n`b zFfRSRId0pb`4Cx77=&C(tmx&)PPKt86&_vRbbll7AS^X^+9xQ%pv0j=LP`Fo;o-AP zpvMJMKAHC8moHdN=(4(A98ztz3V#!r_-wxTyf>SZZ+ISr_ld!aZ*PSQXTSUge=YrT z$$5Shg(wheWk+X^8EJ_*)75f9h}QINMHXBlwryk^{hKn?bt6uADR$y0b?m7FZl2QZ zr9mSUJ@fMPswjyODekp#W`;s1uqC#(z1(zB-qJ~oG8*$l0e8tz z{u61>oNq@&rOydtV6%ncQ7po1sgUFF>O`td$3n7py={C`dTA!Bqmu~E-&wk(ZeI5^ zFUFWYAO@YQMJQJQ%B!2DJoSVe%6O12=oB1_r2lw&5uvRKp26MbR)dg6Y0nmZbQ^ zzbVT<3MB%zPtVCqD1B>^Aduc-EYh)rb^y)w4^Sp8+5mH|Z&M+qouy$c0qzv<= z{gDoQhM%wfDz0yD`7PZ;TkmGHLvEdiN>rWIU>JwRpv`aKKF!gVGEH+nDORG1*oDI% zd*6D_Vr0-Ni*g&J7aYVU*J(@^zOSx950k;)L_Ku=9{>nI_rGv#dXA&4W-!j)_PHt? zX8;SM7-5w#Fyn$@3ws$eO>`eIK3oH3|DE4Ou(D&I-R}@6+h`B4y=dl#1<-_G?SU8> za#L86E+cCb+?2{PK@A5S5^qDUlxM>n#OLY0bSC}lvCzU=+VIc%K^Gag$mN^h+0=TO z4NNJ)eLkph>MB(1a^i?`#6np_tAL$<8$uXj%8Z;2u&2-rrA3D`7Z3%rQvO!CSq9rY z7-psGEpnwB(MJrR(=AE3!o^Cq`3x;InEs|GJo2qHH~r3Hlp?Cc53#sVWnK>)U%ghOs2Ov?m!4Xc@NKXGdz9iwF!ZGLLMaDxArj5Ks!*Ph%}FQ6tks) zs3S=+W04Q68iLDpk(ZbAk@Z`P)*`K7beeBF8o@BZMg6UEAZLbMcUMe2ESO4O=1G|) z<#a)u2OE~GVJ&*%?aj$cVAkq~T~>E#w;-#gVjyJv#ELdo__C6xh9Y^wnA-8_r^rkU zYd@@PJ|>{8K}>B(?j|{ibqZf;gvIIbVJ`nJsWI6c7;1oo2A#Iorz>DU#1YaGayi8b zAtycRFlTC8Vfqi<6QjN)QM*M@{vg|sXk`Ndoj2sqLBHmPMM&SUNo`3W5Pqt(kbx-4 z`3Ka5mlYnlrTg)Ca6TgwLB7G5dy6NqAX5=_L#6~99xMIRtVOj+HG3yX8t>6fkkC*K z?W9I*j*x`0!iZ|>9Xy>*Uf(rB`58SNlOvOoWs2Hfi|eKWq-e+6#+)z^GK_U9H+4?v zWvV~^qV@O!S89F|w7@m|Y%(gRW3446Fq6F}Cnq$HAYYh%L!?95=)OoFg$RZJEX>sv z1PtABoc7W@myTTka^vL^GKl?s#$lp$9{}%Klu>TLQYgWY1QQZPGUw8yXOkB1Tb3Rk z7Whq38XM_m@2wbCQ7ljGdjdR5=Whhy8xYD` z1fZilM9BX%xqfAjgNi%u7G8{7G8N2Vf<{NM49NBV(fHs-{pMWfy7(ZsnLmPDP`ZSA zbK0?Iw=D^EoLtwGt0%Iaj3h)m2c&Pj6MCKuG!c=i?dQ3WRQzKP>6}@s1yjm2vsMs> z+o@z!YE@~9%Jss~)vGv2QtX7Y#YT0`ja|98LogGhh0N|afE?B*(i@8BuV`5X`c(;` zg4AD{K?`3E_*gAKv6`Xdk-p{GKctp@Kf^Cz3Fqq$L2BB++@^rPz#&nU60qT}!VOj+ zudZt_4+mc?!AprTh2)`7IKwaiHHTx>{M}hoo`(l~#0aUKyf0zp6hT5BMS$S6+Nh?! z28xSUv#DOoYAV)=OSejXbw89?!l7B22A1hKr&{nA<A7 z;-R8p9B%P|?E1C)V6SCJDbX9BAsha4ZbMNz!_UaWU?3Qrs}+|*3j4>4uT*I7F(6kF zxyK-zq=Ru#d^eZ{f(_6f3QYu7KqwEndQSsTxQYr=3wzIit0EV|E-aIL4;REaj> zbPz6GH3^4TfFlfC2QqD6=Dw8y2opg!sz|^Xl>&SDgw1}sTPxC({i2K#el2DkeJf(h z%^b{Kcn1Kv?+Czph5m?5j7xSj$Vm z{d!{J`T!!c%F3{o99rNTd?HPTsy)G%L*}^o@<1~Aa=bK1aWoC5d=U0)0)0yz!qFDmu>k2@VoB z{Om8nNoJd?LWki71El}~&Yr$##{|u+7W}g-%^i?}x^FaV<%lgr8BC<($b3MkfMGd5 zTWJOnPnLvj=t3}b=r?De|Jq0$Y!`b86IYk+n4k`v2VG>2uosI+xCBEw26QotXmF3h z&o&uMI3jPh7NrwugoKZEvPtkT#fH5nh;b-{LMwE6841Gi@pt%7Tl%U|Kp-SJpmRDh zo77-QVrRO)!|q!&O`|CXuKpD2^!3AUNe~4pq`UZ}4X;u{zYG8x_J|kyN=P>#M6P=F`||Jk{Rd!wPXT(+VN^j0$cv=+y2O)K?oTu90QN|JSLcYUA(_e$iL z)f7(Xk!vQm&VB89uBM&SMKT5ikxQ6E1@ctFU`B`-`#Yv<4kpp;Wr>EiT8yScP%#{c zZ+W&3vyX%FLO=t$K^DErBApHws^YW#diZl=`Nz^mI5;)`O;XM7!1awLPL|suhQ^}6 z2nZ1o?1Z0ivVbZY#u+wwAmc$#)OnH12mz=;kt`=?vs4B86(a7!-ausx_<}Es4I8TQ zjW=T}@DA#;_!cn<`47t?G)Gs|E=0c)R;+^zD$kZev^>)Gd$L#f!oDvZ6lOXJ`Lj(=v#j5UqNVE9an5v?L4 z_ACbiDWww$m9o+($cexfL_UVhsrUyN+R2&8Jtg>pHI$v=)M-_XfL+65Yx0T()Ulu~ z>M2oQfP4?gL)!3|{-~hn9Obj?GY-CG4wcs= z0we2!y@epQRacBc;F0+Z9NKG%Hy@)i4@qk3KmfJQ#s~`Un8t954<&M&gX?WXxEyEQ zqTVjDlZa7#Bw=G*R9biaMyXB`-IR!#NIN%{0RSm8TKy(t&PbD2tk^5vdl^ZfSS&Pm z2BDYCc2V)cn{{j(Hz=|{2{WOM3R#$K8%Rm~aIM2OusJdr+92rJr&l0%8n_0kBFNfr z;59be#YGc2PZH||ji4pWTL+?z86c2GyApWAOl1pkvg$^BIJ@AW$}3w_FoWIHfE>*G zF*EnkAix*r>84iq4A5P7g7SNsfNUNC=!;Eeo;hr4<_c1IuB&gvy{&94s2ERxq`wh= zes?w#26$&b2@GH+@s(a?T}I5dXS&Do0GDHjPA?c);X<-wFXI z4}u`8s=uHkts+rETx38kFngK%^5HvJ2!QVKZcaJT%!@-#`-D&~bTCdyIM`$|mOV1l#o z>7K0C9r5^%0X4d0^;9uY6<>HnF;kfKvQY|Pg@CR$z?R1*K(MQDkZ-45omCy6LiT5o z{5Ph-a8B|kj~h0{IsiQ{YC>mzex0XW0jKn5&Vnq2xGZl`t=g)b4Ks&b84QIRRllqQ z0-TBBbtH`S$fXFnKT(DND`*ugoCNMYf)CoEGEV&)8*rqD)zjc>OQ;EXkNL5$yJq<8 z83+mIz6hviX?X@a;8c5|orEvsXA}-i?16O-nv~*W5PTJ1eF7@}-YBq*21oQ|NUSsk z6}M%qtbn?=)!RWt%V-y%x&y`ktnbMdmo)IaV~^i*ER|G1%e!uD)2D$j+FQ=k0SJae zTj}8E-jJk`lm@ftmT5u2L;Ueoz+&&8Lr)}`h!J2&vOatM<;vOPvxs#S+>hRbb{ ztV~{pY(zu_JrM!umjh>9^h5)p#6|UmeC~nYdzl+ipQ6t_W>>V0*mlbbB<_PFIhueA z9YN#JA>EFQBreqpblBsp^AA8zMxO^lf#F@f(RpK70B69Es zw3h#t4NWYzr9_uJrmhd>?iKK1`-(OBP3SY>!G&Fb%NP|99l&1MobV#e2E;`!;8uPp z{1dbx-(*Wykkp7(vlswQ88DZeb}lHF6WlQHg(V996SzAbd=a|;`GlvV{bCCEOi~!` za6@3YQ*4Y4pPBH25Ev-*9ub8U$zwtuLkUJwN(F-~+pKVe7ND)gXjD(AF96|8Y8ZR~ z?F*NV2ha`x5jWc96kcw`JBL%60SujO0@HuF_%M{LVR#432NOq8^!dm>^PKEC;tl@W zcvDzhLy3PL2ErUs0F<~9pf%o_RSKH}8UukJXafMgt^BDC2lk^&6d4$5p3x$EHjG^w zufZ9{C<`Xv9j>rdq$#@Lo#LcvLM4|F>P7-d8?{RUOmT_GBg+s2_#KGrUDm2t0oCx! zMacsOSPYR&gzPy3Gp9OxMYhdm9RfbovUNQ=P?=nG7&aVa9%@3DJZm1h5H}ww^+QyCg5* zD*Ab5s5;jrdPJbZr_*|a`bw*3Bgm}u>CxrAGd7nz0V)qk0=!5NBCFW60>H9% z_C^RwI(appf5$hLM|wCwvHB8c1fT>p4=$H^;^$?viNJuIL>A2w8m|DH!pxo<2SV@$ znV=B!$;7N_5i7&Li1vAXy?02vfM|vpxOSkzE~7;)*uj3}p1R#t(h(}roF*7o?RxFJ zsc#6-09XLD%q5DKD3a*tNgX{3E%t5pP^eVd{~$Bd_ms!PiEiF6vx(U!pZ$yijvls%djR>I{}_1%9~wPgb`UYzWs~(i zh%6kFM<_#E6*VJ=K&KZtt+!*nwX6x}CL`_$uH0mU4KvB#3lDqcOoozUdExpUZHUhmeD9D$`SYCoIz?nP1nge@UA5bpPwNMQt<@DR69PX_z7 zo!)Pk>OY|b)(F~#+$u=+J@86rAVeG0z1hsf85`zcZsL&^&Q$8XFu^1lWKoiu(zaqNVqjiFRV;&$fk zAUxp7OwE@Kywk@~SMd~}ajzV_sV6jWzD@cbkd3`0&txx-{&H-56nsP_Xfu;08^)_| z(g>0TYazv0@f6+ggSjYDVA;uDOI0RC(hOi;1#3Q07UUPE>?3R%d-~ClcV3E@SaD_Jw~7b8?!s;uWW)r6r~v7LWikJ)X~+^ zkPL)S0V$IU*kS3&^tnG(b%|BFlgLz`l~#vfhBRpWxQ4AXAc-=BJwFUy$76P5XiI|& z=#6FXiA2?pArSbVyN zNmBR030rt= znFuWn_r$CH?-6T2?^IH+{wB<5c&L>fW$01%aB-Lk0M@CR@jtZ%?=1ycSaF76dmTqU zNq2z?Zy#C#OGT;pA{uPWqBHIB4Sq$Vcra`EyU+@CAI3hnn-0|?D*=OBbfB_yVJ@QukA8SKa@Q??g;FVH2;et``Ra-F z?1_QYaYCHmpE9H`CBECWE!)8sMKB{a11e0MOxJ!wDG(wpILQq(EWnM6;E&*RQyAI1 zh6DL0#HRBULUNik^%xYOBKys(p)U3vM55Vk6H!})IS3o5d2?~j@Veu%5p(0Rf&d^x znWDO>zlotV6VNB2>L8#$MP+G}>gfR+_CCOo?6o;qAa<7(j_F!3Zc2Fa=AnlTy+R@) zqtrN65>4qKCoiR8w*dh3eb+k?Wk6^d?|@t=kh@p`X0jRZ}eAJihFZ zq6>-lU|%L91zGre!{(NQl^BCC_7W9f>vDSwY5`Vf+i`VQu+gd(n|@VQde8J%1SblD zEb2Dl3X6YX7D`(1+L(oRR{^Xi1shBaBMv4+ zg5p(k)mdThe#5FrH!@ESZtn+L1o3|qG9Cu=4_nF|Nk`7#{lH@eQ~_|Vgpo4R0z9o~ zCSG(g$3NAzg|pi&+eq#Ff}7ud19;3Wh{YQilJK)AYM}3cR?c6`rcz!BK{V!BlFAFW zl*4;}PPB$&#W!pIbK}S>trG1_fpmo1BxHD)f=A}3;kDRMwd9EfR2NeT1{i^#n${`~ zEtWzAmksO`8v%Q&YQ$krF8~Qr@rmvSmw}>EgCHo2g^sRRYrM%6k|Kg|&xFz*y`w$oRSkKfw2l3E#kACNN`0buD-jhX{zznxuA1;>ai#erUaNA2Mv#t(R%Acc`K zfN+AK<}zQ=vF+r@22sw+09P95vYikg?8+DtpkY1gJzFx~V7wQ(kw^3U&3V|0YqG*! z-&8ueEQKRM3Bg``<0m@>eG&2U2p@)GkJv!X{2W=bS+R~oIzeA|1J-pCg?uPs2-0q? zm@iEhNA(#jcfhPDhu>pgTIG(ky=YKG^Jo#3@SlNF_ka-&&k^SF(#y0>@3##x`Y&5x($bG*@1p3RI%vD{k6 zn)=#nQU7$`a%w6LXy36@)&gON&`tzsdhgyFFs8dWfAB|l~Gu0Hst{O45o2tZ%M)GuP zaGEJn!YVx>Rs@yAy(qg^&`1 z!#Obr9c3kLr9lb9;!$434srmXSOOAzP=8%WluwLsu7u$TLbdpSd zNb`z=WSaC!E(>aOaEeS?`+TiqqJY%nc&EY*>E@gK0EvmyrWpi`uBq_k9bIoqO3*)I z9&QkDF$Igpiqeg&9HZu@G`lY%;SEAoN{4}>lAj0o0{}qg{aJh;XoBu$f`l z`&iII{3uO%5Vd(Z6|K`G^1C%)v^WL_$Gyl@9rMvwtd`~W+>SEz-ZSGw5r;7waQclK z0JzIOQh1>(pr|VBD_}Py1fiB-btxd*M8aVQK^zb{8SqinQwwB8U69zQv)-LAp#V@x z3fj&n{9+9PBW!RYbI1#FA;ThB^uaMQ-k#$1SJ{8Ho!j({rq1g?Fxv(PF}%qX#UHLT zXC}wqTVWMnN4hk`>#Laz?i1wgVG-hJ>=bn%i!iV^_m4q`V7+k|5DSQ7Nlo~fvh7x| zw%FWA1b2frvCBfypl`5$wYGs^yxlV)!SFM;Y=^k60wQ;+T+i&g*6J+51z+lo6p(Z^2jSn~! z`W2IJrEVC<7%m*Wo!rrN+u8vy9WKlDBDn|oJ^Yzv2o7%LItgN1E9KwxJ{~O70LL&! zcC5^`YgRAGdI$?Ox&i2p_>O^h5WvmSWLSK5RQtQC-{i_f0d}O_|8{_4msD$!aNh`D z&KjNGu=itZ336GZ- zk=0_eR!vlJ4)wy+#sYL?8~1`JNJElBTXs+~m<0CCIpr51PYA9Ask_4nLry-eT<@vQ z5=60XDx8U!3$(W*<*^UPxfK)TWe@aAIB!N=10a3OO~+9qQjkM4>n&(jx7)02wzyPG zn8KJ0-4cDFn9%?xSZqK0srhz)N<->dg??!!p#reXPPz*3n`O$ZD9w);gbK#BWJ6|@w{e`8Mz7eg-CUdP+XClp4%c`WSi6&|ccY47?>h_lK#O+b;J5@3 zYH)&ih$VVB>#tPy=kjodW%8&!VL>vbzQ)ymHn)DxN3MuZW+yBJ0HE{ygc>;iJb_{9 zQtODnWUWGwU-GxD|@&v}p;n~6Y5&U<2CH3>aP43Y%p3nNtK!NP-@NBV%8 zRKRa}21z_RdKze)!;sAmL}QRxP<%KcXh7S%g6?z*2F){`Eh-z!Au^a!?0Qjis>n^N zKuWZH-5OL3Y=!iGkrGG#gn>PGou_Nqf1d8M{7DF}0!KZMs9r!aT3kvXf$x!FI`+lj(}4B zPX0g-F`9zoU%=E%etcHlcFR0^To3=^FM$(cXa=>I2p)3y2TFb&yieiyz}P>r7-$E+ zGCZ6e_>69t(rj*AoN$8Z41G%|kGqkSRbW2a3t)EAvi-o4$QJ=MnIZ&YPy|CIXspYV zxDbj@$w5zH?fd22kg2oW z0?mEorjRfzPT(|dWP~MCul_YuA7QoOL;#`ITHGk8pvasLR37^bm(WiVv=nG8wI};V zWH%p!tAH}vI1wrC@|2OWxI)x$Qd=E+R5lnp_GM*q*Dv_^3jjp{ zHEhU50edqq)9???Y?=#!;$%)FSr>*xi&P;o4+=z#)Nne<1U!r0W$8h9Q=6Y8nfP$` zQ*!WK|lkP%aAAn2MN1>Ur=(A_!9p%+W&$HcCi*4 z&rj|I6WG8_!;3BibF*QffwMQeK3Q@`gPPqK`;)9?yd|-5Vj=145tO_%tWHW^SHL{( zc2F$iZ^wif;2uJ_BtAoqC*3#Vr#^7dW@nnY8|E7kIEcBxx*bqf@3W_;-9*vMfU(~%OkfheDZxww@j@Z+{Hm~fq1}mI)lPHU?akWQ zn|hdRe0i~#2crD&DCm>1G_4^rDI<@>L?QHm9>AIOt>1u?fvjH)^X+;(>C3`8Tc}yW zo_LUydSMMDDC*I9^KcM2nfz;gFE@9z)Gj#Opb}UHHIa+-7#zZ_DaZ7omkW`yQ`$RIaut1)NG^Z#@swT6T)>r|3 z0|0%4t))qGK(GO6t!8P^E2ZKHE)vW}BY}wP{`6za=apex0U7C}rg`qKN}1$8^rQ$0g!a*?xb6j(75-Q?I@za{{I zWlRdqffSS z+n&|iwbW*S(8EK=xfia>ubEa+@+v>)u&t7V*NA>O!1-|q9XM0X<;)*6cKezse9hFj z-`=Jd4iG2ZnoFtZJF_EZD0v6xA9TP>JGZol$I@|?qXnADoI((QyiiF;fQKoJ9DU6@ ziD;5p7%dX6T_iz*S*Q#Nk~dJeNW>=NTa4rlfL2^O*++`2MIj7Wez|A>ju&`vW7HHB z)TTB*!vBvB9ml~V*E`I6H*U9k2A!yTPcq7r;Qanblq4lU8sWjPJb`r$eY>Gg>_BfV5FI(J;wm+_n0XH2jNhpW`FnQ`{WOT%T?7Fgr-(@) zFbWMreNMk*))T4jX%i94u_COjNK~`_0j|=+CTy>Q$}*8ZAO}zy0>7)cA4Zn%j}a~) zDfQE(t#|D$$mc*HO3Xz9)fho?RAS&0xX}4{B56_;r^5Fko3orMK08C3YdC%l2TM$y zymu>4(x&5{1$wzpyqNsh1y-VtK?JY#Sq;e`%`u-l+cF zY-VWY)m$uUE#nm>!lTZ_^64}+GpP*$GvGRKDa1yn=h~w2%D+7xPG=odq7&HRKPpTw zfiWz(XtAL*2F-0<)~r_pjP0FLi;{vUr!3F+B6OCfEW#Z!X|sW~pR~D_*-)SkMY1j` z#2rg3g$shT0KL=dVU|m1DZ6?Cf6NAIUr$ap}sWEZZvyJiB%#1E;sFZg^^E zh(UQSzr{jaJs`l$VzCS_uqn_#l~qu*M{~79kN(JVzFK*;2bc~_v(7`e#~QExL%pm^ zm8DV=^aLPYaU3o-S_3@{@P&b3&hkUN8NwPIcB$Q}1VZm(3&i6M0~`^piOE6h**|O%aY{#x5)o?_iu7EL z;l-P_MYt=pEL5B&Ys0ej<^XpWK2$N?A&ETh@381_9SvfAys@Qa3cI$Ucw&AdqEjp?Z)+7s1*JwzzBwG=$Ok z0!PaYK(Gb%ZgQjwj%bVEWLoGRsxO`neggs_IEJoXA}4*cI#XMO7IB(@<*zNXYb^2)cmH=OqTDv;gjBB?n}5@h|- zL5Rp@|M46Yq@4F_eo?Q3s*d6f0aMQ~APIYv<^LRU4L+j&E3T1JYHBA8MP3@N%8|y4rJt9MEnqr_&|U?h$lHLJj1!p(SRrNGjE8MTdMGT4o97K`+6_{ z9%-CKX(Ry-~XllwSijWppX4@5>9RSyfQ6L@e3-lZTUx!D$Pr1lAdZB+yu-s#lPByaDTnn$8d7Z5%NNY^s; zKr!$xS|9-Wsk3D%j7g2=P0YW7VQ8{Kc}S@7KPT8;Fe@Sr+;=LP#;MGn&*{wTMUcS! zjf7-|5LW^4EUXt5*ZluIb5RB7P&}SqRg^%fp+K$@1vN7lpg47Bn8a(rQ(SZf2?exO zasD!2zr*-xi0$^{T6ke!2$k|6yaQM)Q9auW>Oh1CN3jDv>Im-d_o}V79m=o9 zM@uBP2hVf&b=Uz@BigigA^G`>f9!m<7|uV;7s`PaHYIsl;2<9C^?EE?G;y>6@T=Vw zT=et0YaIRv?1}#K%=-Dy+bqPnK1sDTLf%A4FA_P60(b_+xU#>n9yxLE;GkbUDCVz{ zz#$~}zyTncQ;+Z}Ishlkyd@Nh(u<$ELvNg;`_YC-?;_Ly#4M#y)vu~U{!a?uzu;K; zV3?rL4J4~u&sl`+c;N2)2&dX zM#wtUOO64v4B>i%BL$D3^nE1YGCA96qLW0XP)2F~jNY1B>ni+&kHH0wVslzxge#!@ z_i^G9yZ%_&zGB-lhnKILasTf`XrUuLUS2Q|U@a(u-*vwg+yD?j+r;n9G}x(@+<81ct+KSh#IX%3Hl|VqHYI+ny7F zdBBSiR%Yz{6?Ecs@XL*d#G@~dVgyhX8{l37Z=6lED5Z?4?aH1Pa!c~QJ#7_zlZ8U> z>~mgL%!X1HZE*CZmoYoVBk&5e5l6;enyE^rRe=Oor!y)yI)8-u*}Dopyv>;~vmW*% zC5U)D7rcl9judw3{*cuokW+S&5cp_AZz{KFvuySM>B4r4MU}B()!y$NW0|qXgwT$FhLkO%g zeS)N5yp;!;RHU#)A$lw>q*#XnY%EC_Axa$l7WgROut-LMIYqF8=;ZuuV10Dj7RV*o zwHFygfnp>J5kkiuK|M=9u_j^zKqj*}A&!(phR5azvq344V?E4{snW5SvXT{m>T)uW z3WQmSg*YNE*OzG_St2`dR2XCeks_@GcoLZ)uK+N(11P}?fup#kL0%0Tgw!ZuMeD$V z5xKKzth}W_US-*Jaa>s08;CHSCq;{#6ulG(j7w_FENu8DYFh$GFd_o618>2jC8dpsv8o2jzm@om3cZ$3D#*Td z5ppg#y8ZxRYcTj>I_}A9(?H^VQSj&(QDEWE!;^%^3-%a8Gnf^{lg5Asj4d5pI~YJ% zZ;J)MdJBNz_AzAeTs{RPd^jQZrw0g#U8&F%bBTJv2X}zPy3!!@Mmc~Z4NlI~BY&9= z`$^#u5?S%H@m~mt0AL^$MQ9^WL>!F9hgQ}Z#uVk#joCQpCuBWhm%XCcy?HHUURcm# zSdJ@RD$zAq>L`S2(k0HqLvN;kdH82{gId#pj9%MS4h{;*BicEvSghqh{?-~G&3y|t z{6-dYtR~}-OcSa6+jv_dLn9OIp-M%g0*g};{{2&ZEJMN&oU&|IDwPc2v8n?xGJ;Ts zgQ4>igPHS^MQ-Tua~QEIIWJVD4jOopJ&>aXa}{N;0bv4m>mazoZC9_d@OD-ffmb4PejWM=>G<(xO;lYi`YH1POXMdDnyiu zn^J+zH#|xl-?GwBGMSBfvSJ?iqK*yCEQ+n`Y$HXlDcXdyv_fsGq|+)^drCwh(*g<_ zD9)mh$^g`aIf;W3J5jJ2(YhGI2$C>o&@(=zJJ1D2K~N0DU}y;e@jb@R$#jk+CL@@UuV!dQ&a}ag~QNJ%S%oN-70srd;ks{&E~|vcqCZ^nh8 zjbZ6;G%*s;$c^A53F}^9vD)~Ik{H(LA|aUDS*={hNenT#sQ3fU7>*sd#?|pn0pJb{ zyZ8gHV?71$D(&WK2k?Wk5k`lO)I14U08Uj7ukg(?b*(M_mXVDI96Nusz{*V1z>}JK zW09WWKdK&1WE;{@ivxX_sY+`;ARL*{2&!6cr~+xef&hg*jyDM3M&uh$W=4SrNmypj z8l>C`J=$D8CXg!xlUt{38+PdjZX3Zt*E_4Z8jzEVJ#X8Fv~r?*i`6c*n%_!AqTsmP zl1#2JE@~F|^5Y#GV6;ots3^ExHI&&DqT6YF3Cr<78%XFfpbJNfUEWDhW8_9$$z6c7 zI@|~8$nuYrN71248A(1gUH2^*$I?jrN^RW`G0UK=V28pa`FW%0CFrq4!jk+zEy-rw z6K%8Y$U!81TVG))N1_%0$hvP)FKZf=uNkasH&l)p^c7wKv)Dns)`|25ZE;V=+@r^1x}A)@<`(Q!>kU!+7{O+n^xfY_QJ zKZl$u`!>=>(p~1)l`FY`y<(N9p1%O-A=5|v5$(s~lgOE1eqei&7d{|A=6djikACWa zCAwFK<48I%&H$0TKC@yTd*B%rW`Acu4UQ0Hm5Wq`HPph-E+Wd$m-;_V0` z|EnP2**INjQreNTzc71YB;Hbz5!q8|x&s8xHC0EtmOM4+#*mACHPS>V{=)7l4@N;;CY6wo}=b4B)Gva-Mu9r;IWB@3J% zp|ZfJORe;1iEBhiA4Y(Yvfqa>3@KQDWAMg<3!qg2^m!20g;sqWL*3ZYyyDMvJ#$pF zf?F@B^*=50;2<~esPO28k&p7Zaiou|0?txg{VkUIpb{f5xV(`LV@nPMb(dDc z7X#8RuF?<3&o~C5aVr31K%2kwhcW!IbDmO2MU-Bjb9LCfsQf#1d8*(b`l=d^g&nIM zR4T|;{0%Qw`L804wKeN8P*obwzMR{m4xBgvNrzJqOo2Qj6jEoT8T(Gq2NanhJ~~2E zM>MA-R+|oE>157L1Bnqa=`$Ds1ubG$RE@}hXz%8BY3z-a6=E$!c_`A%zR2!)Mh39^ zgUG4icZVVWQ_Z>Q*+%DOWH+MjMW}uwp`7g`bcYK$C%FxZnB+q!dbX3h5k5?3el&x= zX$frQM3HGC+LkcAFV*uZ895syy*mp3anhPC9I|J4ZL-^OpwK%*r@029*zdq(137U@ zfBYnnZ78IJX)2SFoWHNBTL-h)50C011xRbUsg_;VO456)j;q+(w--39W_EY9h zNIXvctI`hzo|1TK@Fn3h!sq**vww}|m%(2&e--p+j~)bjnc_Fj-`wq6kFGHtlyQpa zqq462yDH_L<52%vGl+L2t}k2{y9DS>OlAq(6g8J_5u=+~jpj4}Y#44+ap-&k`BaW8 z<)z{aQ9%2i+=j(Baj-2{)+l5on^9XZ+!pSgFfxGYfEN~mnjm2Lm@Iy0lRa2Jk)~_) z^E>1Ul?j8X897Nm4+1=J0~JtfthdQC%@m<)lEaCD9Y}&t2*hN`Ec+6v-tjn^ zI1R?_hC7aQ5U`q3a!bh4( zhb)s0H9%}#AaV(#4Ex%B52Rp#DCU^TfD^sQis6#fp% zxONGWG(!F-a3BO?L_F!&OhCZ&K; ziWacMAOsXo0xGldW<(+PS2$paDWdp_K#B*{H5Q)e31PocCKMU~V2UV~2pYC4N_V{T z1+c^WEOpH^uVU2;yf}ki#b|qVo&^L@fN0^_5VLh7o49Bs60AJjFi{MjW#$R&NhGgd0g)2iW9gUX5}D-i<}gn{f5xkyMl(Enfd4|qr(m7umZ2tqNx z1rJ9By00XHJ3bzduv8+z-AbpY5~Ji*f~aB$T+qhI=I-Os=7k6YC$HjT5M{%{2FRRK zUav4w_z*g(3N$%`#)^!65O@bWNJqt9flh~`#DPr1Ui}y~u$Khz)`2g=z`T-Ufynn` zfynm-fk}Pe-#Gwc}T%)9Ze&l^F;Z?BET&7Hd;uViA76LL(9>$q+Yb=XmT2(WzaO=dMOWkEj z9JCJ=Mu6a{&q$@OoeQ|aAOjpa6X|7J^vP|otmTiwL8H!=2R!;=DMIZP`2`=@vOb1^ zn`tq&je-Ve+>hbXsYT$C$Vo<69->}7(NYvw<>a7R7o5`slPE+8L*u}7pwvWsTX_W* zheHzq-5Mh0Xb>KBsH%(S0pcU%G0=Y38bx(60xS9;$A3f_VwxX+(GL*HHXZd)m(?mr zDu)diUr1b11$Az9E`(RFQI5bxcz;zsIvG9Y>K*VkE~u8IkwI2e5iYDAt0N`4^HgoA$Rh^%}`9{A|MI7katC)-b7o70=o>5f)w4s zO#}$(Fh^!YcOXI#A6ggj9)~9b_2wW9dFNaS<3@qE(Z9Rs(S0g;-!&HPnu@l}!cDYr zk6g?d*Gd-CN)Xdazn+(GJuaSlT&!^2&t$;QWVFjPH*iuP=YvkeYEufXoUBf9jA>%udsD$KNrXGI{{?*^6ig6j05y7Z!2 z^rA}iq6)OK^=V~l>m+K~BWl?rD%m0`*&!*}9;n$KDA^ro*)pfuF=yE=X4x#J*(>JR zD`wd$=GiF%y$L^RVF5BqAuNcnq(iN!Zi0h0s4r6jcUT*D0kj$mu0WeRDCFQ|Pt3X! zTE=kK4VQnX2PB$D6(AF<|dlv0jNFojPqO6Ex1UEzg zv&|xdyED52yJdO-Q5PYZxY}aqK_K+HWIQ?VLuqO6OT25lE2i5opw#NERO9p64wR*? zjZH8Dg)b!pL1i596N88olJ}NR+l@q^%3v5xD&wvKieC{*Qc*%kED8{*t((@IqbLyK z7zu;9#z3RPG}E5W`1eZrhH`neVbIZ(jsX0U7INod_dia3s(O{@>0{0aKMg@#1JWwqQDKePe_hP!2Wr zZNcs<^gJQmuiGJ&+g=TA0swdL7zxe*us3(gNWarvKYD}v@7rp2;4YBsmEhy%h+B_8 zZm|Y}EugXbX(FF?5f4!g{yTK`7B>ZD0jWHdyjlfM5n;*WTqE7-);*4Oj(aB9gMaep zt&-UHf2hsE^4dkV)1umG2Ve*Q4B#ajnaw5?g7LzMnI$GvA)V1%Om0I&s#7?dkHuuQ z>}o5lN@5lyrLR6&TB4y$-iAUYvw6!V4;$Fipk~~rv{Qx^0;5U2BwIJkCC^MmH&eDj znkdL;m07jLWKs-S$)*z8E2KDTiUZ}=(}N+|Dpg{>E!ed&&DG+&a!GIZ53%IDFqCt4 z>`h;{c8qf&SDd+<@h|`G<~7%$i(3Ax#$iFti2(D7OAG<7w@h~nHz?1X_2gOX2VWNJ zZm$qAvD~Np8YRW+L4IKMwfJsggOnKJ6c7C+fOXgC3@;o&Z#1sO()sw+r^_jwv?=xG z3>f1L)V|xem=hpySDBx9ccE>hAQ1u>o2L(87!3Bmq#SCLkhsaBdVoSqD_ESWf1Etc zY7j$N;h2gFe|(r`jAm7ZAb9cU(y)y0(rO4Lq{-}%2J8}%Q?3n`A_e7@2~t-k9(wRT z!y#3WR4_YI1vL=SnZ_JiO%nrhKqdsqO1B%N=y=gSGsZf!%TXyH5J?FKwrk5Q|~0!(fG!qM5Kyh~k&AY()TUTVk7Q z+tze-Vynwawk>UNw>s-%B$k#rdU{BHA6lId2V#AVp!4V$#q9Y?eHGA10L$21z#Qlj z|1`rZ5qH9Am8|UgGcGagfAiTlTuQbK?!s;t@^!}mHGdxkoQdG>zDJG(8_DIvfZOtz zC~Ai40W^Hk>oMD+%lA95|LJ0*|6R1yd#d*&)iufD9cy!o)@Y8q_xgZ zNi2N}8NUsK+&~R@Yh4vskh3|9B2TKeP2DL;!uW1DOW|y{j%b0E(YFLQW{S`i=_&*> z4va3cTBOnA`@6xBekQSB&FB@u05Bm*Y_($GO~8A6A!7qta+p4BusZAhxMc;C0fz)7 z#4rYREsT9VFXHjI?vy}-yn-Guz$g6Ru_=T%ikX}O7E&wc=opA>?Fj#IxYg2Fu3DX+ z9c~?xLFPqA>EB1YY%j)Y5rFFCO{u{}BF>GE@JrDftU%hel)EPt*TT#cFhCF%Q(o`FW}txRpD?FFEfWnKYu*t{ z;;chd3kfpsF18IypfS{t;Pe8JIG}O>F3iAnL*DHFNSc;oQ6N#A#`}tpg+2y&yb!pk zVkSg`y$NoF@hXr-2QV3IHSd7@JTg{{;`moxS7ZhO(k@)&AUG>INUmGjl@pdz0~kiy z0G%ovU^F#Fj1bnmqKb>%XA=@0+X6$H$fozbJ&JaKk)(LJk)-#jR|@B)7FB2KnI(?K z!y8W@O7RV!P@{*WhXzRcbtL+Q_O4;$^B3SBX{*4xk3>Uop<@mA@_hZwsA|Kam-Or|| zVn%BUrE3vEKZg+J0)m3^0n{t1R|PWDx97PeP7DG&tF~52!V*eW$tqh`fJx*SvWhpG z)l4}_KaScmRt?iKEs>dojz|TM|x@a8V%}3~}$dqd!bkfB1z|381r~yo1!NNN5<@ubT zTeKF6$wClC4fm=k!FJ^umM{BA`~;_@7#R3#W37ykn4Vka!iTBl-5IN+UX zWH2bkil)GaD}>9(IW49!WEQj>NUeC_e$+--5dCe&&@@6ODN9<*Y!eA$`zX@n=;Vck zJr>Y=b&%ih%M=)ZN&&G+!RsKT=h%_!$e%}DnvgBZ{fj00h1g5!Af_-nAvS!89;QDy z1_?3vl>J?NWzgn~zv1S1ua4Er8bM`w#Qr*f0Iur!KaoL{8s^+au-t8r$3@YXaa>z7 zUVSVehzBD(gQgBQNMwlTD4~Jd^Fs8rJzAA#QUEEor8=WJa|O|AbKPZ)qzhAZHJYY z5wU$HXr82>Xk$#iC2)QP!4=B8=GZ7z3FZ|hVX!^33aGXsL9w*Q4yfMgR4PFm6KR%s zBY54GCk1du7zPea7%iFf+H6=W`6$AJGE;dLilzCi5eZGgDMApBOduWP&6oufW;^N^ zM>-Nb73jq-ISxDmy5lF-K@V7d7PsGTE~*RPqj~axU2^?E5CdrH`U2NLMFfyKAIhDC zQ72q^whs?7IV8v%@;&*2{22obb&1Hl0K5-3;XfWQBlFSzqhS3~I;&^~d}W9`OKFV{ zhlGVEz|@8~o^OgkSUr$dGa?HW@QU&$W*&IKECqlQkfEX(h+gSKTKJVBiz?bs_=761 z2y&zieUU=TQN#rlG>*%B%dYUr1ljR!fh5RPzzqV}4h9XY7TpniR``tv6wOa;CQF$! zFp;nVuNX(Du_S@z!4|Ct?7MDIXAuKfkC}#iU!D?3`G#H6V2dNp!8kZ=+yqf!45UQ} zGAR~b?vIS9x-?cju04-I$oHbhRvif09MPL?sB2Xxf{WO(Gn?IHADde@QUhJAUdaO~ zvdLzLIK}oZbrk0Q{Fp$3V(?@cW#@GX33eZc+KmG(wGudFD)zX0@T!O~qEL|}0L`2f z&_wtHT{lfPYfuZ0M1l@I1$g(zdy?|_2Kg1Dkt5`<0V6+yMap)@aR#(aRUtML_Mhyk z4=|SCS3iAJLn0jMBEW1BU^Yoib|@*fX4JGO?4ap|u*}kNbHr{q7B}f;i##>D6fgnW zy97=iU#qHGyq7^!?!d$^iChRcI#iXVi&~<7NQoVOcraWb*|=BOCnQVl#S|lluA~Rw z6habKmUYx5fLpU2dRP$qmXbOJ$N?&jTmhiTv)}R1)ixz z5#-dh^o1V@fICH{Nj=d}Sjw)Uit&@OXEky~bwKRzfGfcV79xuj!Dg|A$IK;0ELyZF zvUcXPnI3h9EX^gP69RDLGuB9+!eXX?JLeJhFgd@bQJMHEY59`4q6$U360kYqy7k%_w#(ecDHMoGYQ5tibP7ZM=J zqwkVPyMaxRVpGLk0aEOGO|hFxmQrtHUe~C(#w#4S&&@H}-z-xX#Y9y>-w7g9mK9beXJ7@u7_i zAV+J)J_a@S)Y+$uEYTu%6SmPjB`AoCwpwX{GLglAtmV+FSVmZttQnPSKGiF@{5s@M zDR~jeWC8M}8l^UD%_t?KT*aAXT07C%xkE=M2&UYo{mL|j=oRLOGNW;FJod|dXWD+( z?>chejGx3S1OyH;#28sbFj6j((gfPpGb}iDn<1on-6z1i#%Xl0Wl@gXI&k472In3Tm9-8%)GDfx7nUc{9ZI@WiUsRcS57?blesQeqg)|#Rc(eBCtpgM;lnGZI(t9b<0M|INg-2VBwVD zl{p~QwYW3E8c0%O;L-7q!2q|s)`e)at=ttg3Jd{Gsnf!{axsMS^6Nwt8tB15*9aq& zoHBz9&vD2O%%1iD6o;P|fBYIUWTB?wd_ZD&EBT1doDFN;b}od0aI6`e(K*~6P>cp{ z+Utz>nhO!*8-i(~$&TnC5F#aqf%`POxYx0+V&r~Xr(cPy1CjS9a#c;EXs%Y_h0Dzj zqG-^B2rn>#04O<=#W5wXNnz9@^m zH#VKF9V9H$yc7ec$JSZ78e#uz+{`6tzEm?(%Ui+AHnbxKp9QW_V&{?o zsXU|2X$w#-3UcFht>SF<2UA4JWW(Cd^!pCH5C;Z|3N8)j*IV$`oU7SSo*R#liwA+J z?<&W%TcY9xo_h+kdD!3^ImU>6Xa`x%3MGK^=^b#&Y-Q>q85uo7FEBd z*v)3OU@#CmMEZ6%8o zx)MW*Q#ELGl1T&x1R=Ucb3s@dO~{r^NSiqy-R|-b*LDJxGRVkcWB*C$RHX6J^pFf= z7$2M-y?P=c{_~bS!^G$+7Sfh0s5i;g@v>PPWdfJn{|eEc_fmobq|*V@-l8Y;06YKloBZi}Ze>27n^485R}jFd^&MNi@@vl|FPoCH!C%GICd01dHR(XK*F~ z7*J6&g}wrW3b`>;l#DV^(IwfFJ{|mTJW+Qio8c)-)|{bz!7iKk0ILUAiI^uMhPuHx z2obfJE@4Rgb2cQNfIRymhRDNtT=<&Zq4+fDX)dZ&H6}=CM;pAhV}@#zx4iJt1Ks3% z*XdIBJ>yN?NU z@WLuk3xEI&^NQNO3c+!aX(BF*fWSkg;>@_VGtl7kTcCzB?>k@7PC{fu@zdop8Ysw7 zs>a4@wfwTtZ^#*@#QVq7M)X=w$OJHQ#>3gQuz=gn1VCH*>4FXvD7sx?lZCG zKGalc%4%%Epg@+=Xq_PN?Hw7)$u#=%Bm2B~7Ci2oi1c~SKshib4&@fL+Il&fbd}P} z!y6}!etGM=(TaBGJ0VL%HmrV;JLHYNR$GvFm9y{PtDAs4Vf~P1hXzQg_HT=s zVQe@_>Nes3?|Fq}=qIqT0Tm;lf0gR$B$qnKrMy51Fc6-<>`=&QHDWSa>i7h0jRMXQ zJ_1S_V!#8io8T4ranTJ}j9xHE#uZK30y_c=fuRCL?q|q-c$$2-_A`Q181uCZG8XP3 ztZOxT-X*&*e9sIq4mQsI&ub%tD#^wLR~o3_F;3_<}5gEuF| z=Adsog2^EVCj#*)3d~SqnopctWMD`nI~PDq#cM&+-KneqY6CC|76Q?$B#E~-qShtz zBvuJ2Hgu*a8c!Vgo!fAUZ0H+laXAnXd>d}X<)d>g7;N>&C5w?zwA{%a!QciPP&0QR zjej1nF7}0&*&|7}V1Ts4s~i_zuA!+8d8*C=yNr zxFE@|7i+gEwQ(Xa7c1J(45;6Q{)*hBPf=w>5m-ZFnL}l(70Yr|hsY2~+T}%}O2AY&SaAY!cfe^+PdRX4(ui1~>f>(IBxX;o0)pNEU2X;G-bmj=OQJB*Z4r0!xf_&s(npT+F%5kZ=wFmL+g< zf!WtV+vtc0Du+9|6+q{^*`a(?DK5j*35{E1R!AP30}r0zUMI6Z9F^$Ef|A#I*0%CyHEZt@`7s(G270F)t4Qw-TLOvRWo2dY zsS{cge=v!byRH~hTeg6PO&-~%B%Ug=7x>wHcjW3M1^DTn%h^o4m|D!%@MG%2E3HJX zy<86XP>13_elAwidj$7%=Btuw+TDa2{8q7p7sR?k9;{`6pJzYhm^i;Ks2EHse7z_d z-s(dt0bLwnmY0J;>NPs(XPA_1-iaKN$(g=GHlj#uc*eVuD4-01I*ze6s#Z2eUEZ<_ zvqMB2;}KG!6EsBG)K%gIrJ-x2&~(oaQc!C*2r(eNONyTY&k9bp^jH3LgPOA|lR?ii zwjO<>7Lc@#J$503CPH?z-Zu;r9$kKPF~S7u0uu;*oxCX;4xC7pRh2zvS(o81H{<+- zYW${e(r6j&j>yE*8Q$sa36asjX4p~vrMe}3&mrq@N@EgG1~LAL%d5c|5D9JI9X5Cx z9@`}kWvp9M|E3HOK4h{G1W?62+XKehXr~TDBjhc%T142d6V|>s^im9LmA;HNoQ{{5 z5fO;WTSXb94gg@C0rYp#&v&xzVJ+cnj)0T=MNz@Y0M(Wl8C4GglPm)zZ2!9@HN;jJT08X0#08mQ<1QY-W2nYb;CjU?X z00000000000000i0001RaC9$bZ*FvRFGyuU;Qs(d0wf^;0B8U;&3Yi9&;$z5 z1pHro1^}S{)6Jt4TEPDoH~?*c4ZsfI18@dd06hPTkOK7oi+BRO01f~*fa`w+cmZ4h zp8pI0padxWmvaXA{`Y$Sm$&-w4-}yBpXc)*{|8weMa}=W2MF-w4G261B<29z#30kh zapUHzb`uUd0ihugOnPF)0>Y(sWQ%hCk=lT$P2X|Gg1M~H_-fi?94`$G;}GrmRjnJu zemGy3&Ks6KblaYBj;x7cfgU=D&9m}1y`I3|wAi-I5!vN}YstG(CDljd$_2+Q+g|DP zns!2DwA3_MKCE3Jy~xz#~IthfhOrSL0mfLq*2RvD$Z%WmV+JNd4Nv6*KjeN8FZ zJcYo6{I5FKpXAIx{Gd_UtS{c@O+s6QXXxUt2UzDk2R}@V-QcW=tLDEt%!aZhv&(%^ zhK%ULX&sRK#IYifHf8LTD52HA+e+Cn>z2;VY?&KI1F-%!bbqjT=@57X_#h|I|&!o|)j>JRH@D8?dYR&WK(hf~OzrG2GaYjLs%Jv@u4 zW5aj82#gkMrsZecXGVOTRJs?cRSovS;E-{I_Dc!=w4?x>5^OS3{J@*`HTyKxvfZ~m z#W==ef!~OnwBhi_x|qYs`tNRoi>KQ7(gf?Ogu6|pPd;W0t@3Y6$ARDI#V|^S!Bhfl z_IdW}_~rNSfhmWB-KC83Fs^WAGsDif?vU#JTa8o!G7=-Pl6_v)s4^^F5M8yVywam^ zB}4$SIfI(--yC_llAy-9Ylg#HmGHdND}vSMO2M-ps*aPUTT<~6R*m{*0$3alv3GgL z`qrvSne%{w3pRn`ZkTWZ4aO8CNB=BQEn9?$N40=++Ud~qTg*1hMH|)5h&77#2qsPo)W>SJ<3Ja#`8V`4kne#HXEj+XCmSG^|s+H z+rPY;5sEU^j!r$hXj*|CC9)>m%IPb(P>~D0WEvLCd?=g{7GB_G-LsSZ5@#P7qWn_x zFA1eG8;KHzb}!Z!4iIk)KI;4}mN4o%_IqjZ!{Bxc+#B%$#Es`nC5BfG)+W-i**hKB zNwdiF{ovpHRa$s}J{p^>MBE#9)6T)h^D^HS3VEpS2t`__}b zfs>N2G@xjAZp4_U&HRnsNp0!KX_2w8t+RMLxBiGI;ya!g5Kuk6Ww{_%>0hEco#@L} zj)pYa6MFY^vyyH_zjg3f=$azI>J4Yy`{CN?Mt>o$_l+7+p%wkW;)_Nbtl=4ysYLbF z710qraU*lu*L`#7r_SkK?Q{w0^q{DdyyY*PPBB(GsMv|+toO!mRHjS)8Xhg0_eZn- z_5qtoI}@ui+UA>MKfhPC0>DAP-^_nQ9KNM1P(|=o5>5%5caP<-GUkOmX#f5GDPtQ9 zfDiyZmjz#TBb+p+or(Xnw+b3Fmlu!oUko5nA+OaEZag)a%@{*`7_~K7$?X{}s(MiM z<_9QYSb53a@*7xyZp8jN{PTy+a|h1-C!YC=Eu z90z?m7K{l*nar-%7H0lJ3nnrQU&Cg+@{8gvK3%gEd1jrCHX8&HRmy^80p?J+BaEV? zu!(^f*9})buuf!Z!1XYf+IO|y?RFyEb%K*M*iNGwdHuGS<;l@RETa=F8s($-AyROU zm{6tbN=+Z(S9(CJR*O2uMB44YU*pX?KT_3q@>NuD4UytT15k5#7UzzAEQA->eHuOZ zVyKCo6H4go1=M`1l9FS(Kxh89TWR+vyYT|N% zv=Tys5DhfM>wb|q?97;mE%j!s-o>wx=od4f*bAG0TLc5cNZ=#MsHA4}vKbcxbW;nx zz?za3rOnLPFGThu{V=WYgns&RXdgF9f?l21yGLm}ykEbKQWoLN;db@5@>QBl4 z0Hg4F(L#6som4z0s(zeddl29|z_fkS6VAcK+G&Y^C!zIONeZYXeaki(19qZT4U@MG zjWelJ3XL;_2I?1Pg6vD-$Bn=J8MHSQMuD_?X=E$5+Va z`K#Xfu9VocKSlGnrf1jUZG__~-)LTvIllBhUAv{1t2&yP)Qu5F)a9P(SJ6{y7?SKXH641I)Z8rIU>TXZm z<58C2o~qwKeC_8`uJ5*?vz|T1>s!ja>*W+=b;R1w1~3I935hbX<5~^>c!vqq%e>2n zjad|XrTg;N#2K5g-Qrx;n8$_-#aMn3?=kw#vGz%oi$d#%84C?U1a?SdU_Gblqrzgx zTbNtBD4;V+B*Av&imwym=t?|qy88R>U_=636-Qkhk2cD05LzE@o$zlM9vXx(OUk3Z z#S>|{d+P5ArCJqhA`j`np@@Yu?e!j7tjvqg)}~Ghq7@kBIh30znRcVzsfFy_9{z9UP(ZkXIm7NDu8vyt2XRBbQy;p zbsXIhJv8z@pCA94>>6!EsDz@N1j{zG9V3I6&E{fyibqgBJ60n9p;Y6oa3T{qoPzlq zh2IaIAM;g5Chh^WokI}H>#bB19Z~Pq+=5g&;(OuRtFizQa(VU?qw^(st}jvPdEGOssHI zb|P++_v#a$s2{Z^3YacL`fu$EY^$Tj!5NzRK1{_)XR^U+J_=Q>vs8 zW$Lpi&mzT!`hkY@%@zINH~HTYuUJEy zb@HVRht{!4^cz4@hr;%V5fTVs)~qiM#fda4OItvouq$K$Xw5NZV4g%A(n*u_g6PON zOSOxh|4gQpIz?vP#Z4+;bIP6NIC;UsP_T}ZpYo$&vc}!%)g-Lp@-2?4EntXa7(r5$ zBSbdUo-L_g^EB|Tj*;x*gw)`{haOi}#MG~eo1<5=Zy6d4057H=?4Phkb4j(k>P$L3R%fFQYe zk%Iq)2jz0$h`1UXHOL1Orv6cDxfv%WuEh=HTb9YrXvmoA;yc?;W>5sjsfZ5g;x|wk z^ws*e#Nf~&Lyo>s>`pL|gx7MDH}qgJlDz!4p|b#9>{4*iBxa8)i%Ua)pq5e}RYP`e zLU(aH<8D77)&Ta>uq#dqAU4CFgGQu%il|C@-)J#G0&wLvU5&nc6DX{maU2=lC`B%Q zU(qkP;XaRuAh{uhH6U3TY6OuL9Dhsi9zTm15cX7t1k^(XGQf>c2a50Tv0v^V$9_7Q zaQq7XE_1p(d(K;XT+sN&*bZFRnNlA@M;gWnHulluF>a^bbKOe9$?*#Lvbw_!28GSR zJ}mqpRynuw`S@*2)euElmJwHRG}blt36CQ^M`Fm7phcY{{42T63Qqqr(}~5&0+|xN zGgl-QWgBS>#otPo3C@NzVV73EbH-4H>$-SO%aziO#11P$Ei3YkP=OZtGy8+)mPjsr za!Z;oeUSn%iPbF+HlbD7_-vP`+Ney~6w%PxFVS-35oSr0(0f|5vQEqchiD829B6BP zo;8wEe}P16U+M5~UO_r}aqsg^SE!>Tt7aN?ux98*Na^M2%{F&*WU=NhC#8c_P|P65 zv|T10Q%$9YTgp!%i?bI?>smsTbUQ^Uz{T4%g86;e#NNau6J6dh zg$6v#IM-{Yooy}k14Lm|3yIWYeD1>~s_4A|7>6jgqIU=ZV%gSpg zzuIqh>cR5bzcVqysdM$qHvS+fc*sublA<^B3RbjRixdC)4eB#!(k~&{{WC1OTL-Dz z4FAJui11LJKSkg>H;~(a%XZwJI4;V+Nxiu2+F=%5#&IhbmCl@5-+9jUu zb8Lw>f&r?>%>-=r#+Kx4Afn`zvBhf&rSh}?lB-y9i;}AxqV4+@OS^toE1cK*lc%I^ z&k*XC_&H$;#QvG2PTr`eHIF)TgC@sZTfYw1ME(qMF@$!!x~8T7OZA zV{Tc!$YHbAHoePo0KPr8D|u=iJU;`itHeEJlTq~BSS}bYH*Kvv3Zl_2ju(pObMwR%hLitz4 z)_7OFczf1(dn5AsDw}fPBsNh7(YQu*^t08E)S8rhPUZuS%9~|fn?YtZ!pAJPd3V5& zh_qLAo7|E~v4Tb}E<&YxVvqSV<}zhR9A(EtM#s)4$&&qg;*N}FR!-a4WHY*kvO=t~ zB3?`tq>Kze9lwmB(o&rkpJLh)0fYZT>XFuNnbT+1aWRM;WH4ej>xq26Io#<4~w@iCW2me(<`ZQW-cKt#Oon&=*6CXbp z`fy`7GqGZ~21jCs24(!Yl(OdDex_1jO7VXZ7MSugd`4P3rMrv@C6oepxGt%yJt8zc z$`N`c#77PLic4rmI8%a~mwdS%i5t7(v36kA(Qu}m`DJ?6S?@l6>1_YN-;>&dr2vQ3 z?^}%4zircpzu*rj^avvvP6am1isK7}74-_)2-gC3Igb-TM=EZVIiokz+0|JVQ9AL4 zH?+*bR&hm$gA=*lbcT;q`>8qm0g!Q75Y@#zbFIl$iF;tEZhXOI%g3gfb`zWuzMWk) zl7;46r>j!XY9>rg)At6{5UBZ<;qPO_?;weOx|OB$gLwYCqu`D}{-JY1gBjhdtmN9* zA4TD@Gzq`MP2BbFvhR+ROywg;3MJ^m{1t>Fd8nxg zO5^Zn#V$+tTF@yrix`{&!?2b&Bv8p^rU=WdGHLT=@H}frpVUr{cTZSY6YN(}*un&? zD<-SU1p=$Rc4d+Be_5X6KiG2qcvm`ug@r2Ckk1I{ zisfU)nNKYD8GgL>WJ+-F{Neb!et=J&b)8F7hjA^Fs$Ny_PIO$xO5S0Cc0uKRx5#-T`F});#%T90cHlLAQ}Cc1OxY2V3W= z#p5F1v%GqbhXe$Jk}mb=NU0Fa{6|T3A=eJqm+^fzLRp4R<{)efZP(tv5cD9WmD~^{ zIM&3=cSKeHcSM1L0#6iL;SPtI$l{#{JB5rtDqm|?LOz$-B<9DMfvWuh#00YkN41iz zA#ykMnm*y&q~8mFtQhhPoUHK(Ahjjq)1O-vA=G(L*fi03Yf?hZHB4U|$|cP%Fhy9% zTMHKA5J)ug3%nLg=Oq-zzd9;}=w}2U50Ly9 ze{e4cw^IaIpvJpAX9T8*w4A3Wr#V;U-r}-mW%oH+eRIWG{X0mkpp3{HDf|-=T@x5V zm?QVbo06i8IgF~(Tlq@vH-CKM0DN6yWY1;0>+wtCx1sNY{-nUh_KAb1H-Tr>4!Lf4 z(n1J=h6t91<9M}n`#r$)g>YjMR;@Y4Tmyq{KQX8=y#VvdV6BMuK;@>VI$d)jB2B31 zHdZ^PH4MgYkt8_+`92t^isO49k;ya~Ht6ovu*5dZ!~6aUkv(M zx`Z57?ajh=!)P?nB3P) zYC&>0h^#Ffny{%#CY;hS9_dg^s*ZE`@MSi&+IEz8+4-j@z*pI^=gW&!MZ;(DT}H&vW;@GB+3SLZ&W@xq-m(Y3 zRwLwt%8A|_7skE4i3+W;7hbL74SwA;m8MG4zwF+iT*H9nyuM>)Hn>I2sH%(QXe)yR zLBa9xPTh(|Fv@wmcoTuu_Qlg%eg)0aNqSwTwg}}qK^_E{Ib>qsG3B=dm2kL_u9YyM zCfdtUCGS_4@9!m%p}Kaqvb}x^-^vC|$fi?4&WC+G$K6yIr7UPhL!+N0!&`%uOrd^A zUU^(-)fnREfT=d@V88t)CJ!oMJXL zb8es~)+B>PF#~1I&ApY}H|pB%qRU7-(F@s$Rxs34zEugV7iKkubkdi`5Pt@v=1vSi zQ3er8rTRtKR{gX|kw&fB_4H)?0>?brg2%u8Z2zqnk;B9o%<3&as(E>R%IA%Sw!;ZS z+w}NLy|-U6l>c>=>22bJ9QcYsCZ^L(I~_qN8%iuuDUa6T(-dR^=5g4x`wSW&avY9u z+^_7b{pr9d3ns_LwJ^#J-S{6ceM<$}vtdcKd8T4sht&>NcB5$d{d`GEO>jYT|2)8hU~+y-{K5ay zK-wl!f&m4hO{b&CwfGSN$e<^&tWdCgm2J8UtHkn4&v>vbNn550_M&QnPBl zMi7;$oA)}sHxFM4Ph=%b@1|TX`XUy8u`)}JSxK_p6w&uC4p}x6X}xw>!>2|5>u6XW zY`FAbX=9hy@D-U>`RAaep2T#7$@tt$=05+0`SVAOlxd?Db=G6%G#ZI9U5|sL*uC4-+ZzAM3!9)0O-r;a&qAncat$j z6V2(bNJm;GdLxHITpqykgKv|R(ni?ml*d>=v*($@jzajS-}Ax>bqZ7N9tm59amedCNx**96W*TPr`vpccu2laFcfCv?th)bKlHcc? zLgW7=M(US$=vy3Qz3xga^^YEZuy&~00j|hSRyTMHi!vqnkAE zER!gfW2I4*u40d=BF_5HFPQcsJDI7ZtreL|k!{stf2f7^PRv8S9;#`whq(kvN7 zZWI{*oJpQPQ|!t7^y=?5Kch|+#Sb~9bg*x1rmY`JE+w5(!Omfr#<#%=s||>o&0rmm zL}i`6QtUpYIYG5xO^((vM@sl4p@dVK{6?s*`-_c^3GF%QDDAJ3YNukgrg9k~&cNMX z5lsq(^Gpjjjs8;ekmY8UFz8*?wqi6Fc|tH~qLJzBBM?Z?$Y_X!71(K7as7%_Eq||! z%LkCugFK2`tvpkQ7dVRzXkS79~H_*JQf&c> zeYqjy9jVZ(tm0{i4fI&d%3WkRT$a;uIXWM}a6^yD4^-WstY`q5=Mik>y?^7w0B$pv zNLRlTPtA@7DpQY|)5&6nO5E^F3l$^hFs}*x8$by=fW0wR4|YO5^MX2yt)g{H_$5fp z52cuV*w3PPrq4Md-IOiwLWnXXRV^sSay}u|B9(qs!DcL%qR9CuW$VFJ|&wH51V z@pkLim5{n7_mvL1(IvF+ov~XNz*@;U>jc0-OEMb1Xo&rh-J@^>4@G0~6J?Hj9pfzT zeWiYm6>c#&Ab7?AMG>z7G(0c$U?X7Wrixue9S~YCHM5^L{Eb3TCM!bZ^yFpeKRr0~lWTY&i??2jZpKc(44x3#`X2B;fOF z62QA~fgry2ZE@8U&l;GZ0~cJpcOmyH?uHYE7beOSo7SNXj|!I(!qI#&{gPOW+wxZL zTj&F|eNi+b!7dCM;lqQkX}j!(BOmS37k^qmSjywGP`mL=ZNh6!IR zc@LGzCn$l*R$`F=J!rk;w2}Sqwx}A{WO(^NE7N3ce=A`@G~2I0)tarr+6Vc58}ysg z0$)Tf%MUVc`ue=I@L&fqeq2}61bydPDoAraf;`bNj>)!k+_(dnlPp`j224y{qN3!;`X}Nfv z4p!#OBvx#HCki-qNIL*Lw+#VhK80updiE3|c9r<`$J@@QHyU}3{n8rkyYWmC zi##WB#Ho%sW1Xdi0)?kw74B3HFGU-@n0HGc_H0D9I5g20faDZ5P~XBqpCH>z@(Quyjx&V@44_v&B|wJ10G_iT8ntJr$UY8__@9@2~H!#{&qxk`w+ z>kLnF!t>3a|4Gb{Dq!Q%KVzu}4}*zW9!M&~Kb-hmnsjp{w8wm7&i6hcKqY;;)_s3I zlHJ3AOc9oc=id52=BCl7!7F@EKu0e1yv_L8WD}`~@Iy!NWv8(!y*;hcTgrJE`1FlE z0krZgLUY)YiAS9{wnc)ekHu2pk4mIAc;@>CIDq%T+XIQVNWAw=$G;}=lA9-rpJrp) zWF76N?NmjrRZ}+jWOn}Q*Yb8$u=|LA1+7uUQI%L`sN_CGhPI8xk;4`G zF;5fTy)5*Fl=v9>r^8U1k5bIQR?vJ%amg_=izn9;na(zL*n%7VqnGSN%I{tL#8Z zZD!DWMBYhfT?-JfSFh@XCFFfqgAb`;9c zwze}cmY-~F6CS<)mVL=95#d#GT6l>(ZLV|bgs}_Y!YjLyAUA+3zFMqrBU>`!X;+?Y zQ{^-d_UY(yt}sJb$uLIu@u>CDCiL)iG80sWoBXi1N+6v_j*fXoH~4!xf7iX@lO-U0 zW1Ha>NIPKeH4^#jh=A4nxEU#U#FevYd6%cqb)gCiJe>dfrQXG^L~)r{K!RZ?{+K=v zddb`RIl!t1L;aw77vgY@pe$Nb*OBV@u%+vgLo#L4%ozM<3VtaTsZA(@BCM}U0A}rOfVgwz4=}d+Oz^My ze?Euu9jFB^urb|9ufA5>n~W1KHlp;P5-D+?=$3!LkP*0CCuTr!6e{yo!p`rfJ(=`P zTVO-F_AZ6aGi2LiGSl_lGiP2L)aO{KBPZm0e#NL%KL_p^>~;HcnsM zkMgec2hT5(JPX&7#fer7C6`ouTQ$u>NR}?Haa<>#CzqZHTG7WN5*2rwN6Ts6q#aD>y zb`a&UQ>3k(>zqEs(LjRJ;U!T@;5N`*h`SP#(!_8ouEAaA--PKzpuoPCQDo6|$WApX zhpO|@!vxeP;uh%LlwxGWrOQrTcTRLIEQwm~)g$UHVB?OjTNx(Xi|QL-XI)h+jSvg6 zY8j)p-AS_K-{7M>>BP7R>PbNckjYInX%z_9h6bjwM@WT7^%G^m>-0Gia=eQ!6qCaA zE3uoLMS)xJAZzR{^+;>~&y_53(&dz^=kIkp9=j(v=+__b$=X}bmv;w@eAK711N{30 z!BWgEmxNiE7Y#ep*RW^`62!jTkfCp2n=D~g`S$}d2h07B#N7XKD3ke zQpCRD&7@4@MqWf-ckGTw3RP?~@y+Sm$`y&-{vm63>2d+BQq>8hiiOO0 zg@`~P35~c4L>fe29AJN225Oh-QnMb-#nzFIL-4P^8oU2gDq@GpMKKbY!;G^i`pUHi z^Z3!&pT3bZ)Dq`hFl!~NEwO02Bj6Y)aU`)%CaE}OY^2W+lu3D7D@Sb}Pb(;4%hgzM zx8%Fp;X|4t5fKm4Ks#Q)LS;tOWV*h3_NjCfduq*go0{#cNA$=h4OqGgfq~;O<##JZ z4c@<0fnL#i#}84KlwWmA?bpaKMVa)>DBo4kh}H;`luTP-#gAFM&czs1wYQ zn&Ay+n5L;-?hG?JU0P3h5q@#h{#?u9=rR{lSlJjAQZzCI(&{_T%siA!`}mC+d5(u) z|7l|0i zs2;^3SE;9`kzxEsXE!yCmyWK0{Rj-Y^auS3RmxS^C$crCj!}P~sJ%Ov`IQ5v2J+WJpHSG32B1m)MGE7$jx@Eo{&gMW zXFSG0bT7HBpy94YHVii(JAcBHsE1Zp$#K+5iK4YxIP!nl*zNO7S_+nk<_Yuv%R_~i`CLlnPHtKjdI2Qv zq=iyBfhZ|N+>G=#OplJmG)ylClBy$io&qi8!)^{OEPPxeCs-fAoT2+0 zi}OR9uE?yy_9)%Jcz+gyxRZy->RsRhZT~R8TRk$vTYvr=$dE_`Ku=DNS0R#*Qjv5j zlHuTD%N-KMta9buJI2BC4L>V+w|%_(qZ{0$AT;q^Y%GkdnLiQ+)^Ic=iFm~6@BNYP z(Oy?HLGhY@yd_%HfMNM5sU72F@3bM(E{hI+8 zI25;;UDv^U#;QVF!qNw2?VCp)Wq5I)Fe`?58Nf@Eq-Ke8Z&$pnf!`{GSS>3n0WTbH zF+704h8Od5(roJ5>(6$6t>4aC#XUP>&u-f^bdQe*Ov>{%JO*WYSb+&~w>(aCAw%RL zHZ@U7^ZrK4KXC*tgOpY#Ewd`M)DH2OT+A!Ag?(Uwg?OLEBiD(_-+Uj?TK%Yj*IqAy za0m+(tQ0(@+fA0=(4G;VVBZ=#U#k(4)977;p?0qW^XwOu?&`H6ZWHQn^~ zfS?%@g4tsCD^k(p<72`tQK~mN>*T3;HR?-iXc5%-2hpBpg0Mm6{e-a+aFLgQOzwdY zjPw&<7@P9Bt~o&i9IWzhzS=7M(^|0qE$1I7u758>b8x~3`TbjDu44)QFLT4F?4#&v zzP6>^oTSAJ;W}Y`3zxF}ES~&Q3FZAPOGWjO>Lx*)0U^P#t{Hr27y(I$2`R0WhrlHG zS)h!WQY12gR6CQj&GKWhfShPFw?V@Lw_HvKA-2$$ZQF*Wd&=V;wu}lr@}=TG7@-R~ zYnKT${a==M2chj)xO;@&4g}u&f*PoA92;Fk4QwHXX=A5ZnT6eblhO{QvCJH`JRlxQ zQ2VwWI$y;w*}`$f2fZK8k74HK%7C6G zN=BOW4g&5thN6!d*gZ0dqZPgFp&Xi|Q<>~^Z`eMzVgJ-_r)xl*p^_@Q9r2{EW2gAJ zCPZP$;+ukgEUgx^CcYnw(2O>I|HOu+D(i(0pUuML2lIdY1HQl7&i|=0?!wt<6;nnd zn48X-XBYFc^eLJco|Hh8xutJq)@|_5M#=<^vrIimCuDT6LQKBEtA*q=tXY*vO6y1> zhoTMx_Jmqm1;;zBq4UhRKbX;V`J6+65~o|}?qY2DcB z)9DO)Io`%5$%ZX(Tq^PsHd*1J24)}Vq9@>|J&EOrMi9K6f72Qixr3zkLWSJW>uIY- z-yXVz)NsSc@+Tbvt)ucg76flNPsbTwIah`jXsAy%z8w%l){72a3~p$V~CppDJs1BAkG5nv$3<+ zQHQ~&vh~sLlcd3)542||X73K86TB~T)IB701Bh30!Asx6^(y#hxpktc00RRX5t0;1 zO)OTB$`cXN`U@wpcyuV0Hg;6el;pLr?6^?ChS==5)goShns)gXOWtZ07)iBIQS#>f z2uCKb@#yo4Z0q4C48}D>9gCXo^c3<Lor&2P{4y8$w7NL zR0ocVlt;80KUR7959SUL@}nkJWv%*Ani%vg#PS1f6^&fnYVu7Wp_(u`?Ya%%0Jy^I zI}B-wQ@Bs$RKieo+`kl2TE5!Nc##j*{ZMg+_hq#8Wpf}dnz5>cT1Duj5?O`dver~Y zd#pw((c8nkGMM=lH6Y==RUZ0R?ka_&o9)M;z5!ctGg-7atW14m{s27 zSGrwG=z4{{8_pbcFZ4rm<_g<%C`^5*G>K{-CWKlm>naD$QDct8s)c1C+q>{wagsoa zuClsR?W zqqG?J7l(AJGG_OH55R^@D&Yw!DjT&Wi<;`0rSclcGkW6MZYN)La;9utvHFdnP8EQO zUK+-LL|9*ZkNZo`gt&Fl+a=!LZTX+o1sXRFE_O*L-xaxI@f&nRsbm>fI9(*eExemS zq|Ctx=dSE0kEcwR%9{=Znj=H>V>Vz(XgkojB^aZ0dI#|a;u>l?EG0bjzB5R4gDF)h5TSO83sdHesa zEPUlEY(dLGAJH$;Qwo0uKy1|E{L-rvn ztXS|(UD*rPhcaCOZWkifuBfCtqidhl-r(j~&g#6grGU6~&4l0l6G5P}y+$H<+nd2f z;>RuE!``2DEi2F#dH^X;U~^cYQb zI~bIdVrQXY>LG%_Y21DIXUqnbC(A`qpi=UtoeP@ z`}$ijXbaagc5KhO>*hV?N>(aG!>eA~?YJ~Kbl3DS-mG?HFIka;uq@;l!#D~4HvW9f zqv4jpSQh%otEF2+I?5Imm9b4wnVWhk#OJjp?NWBBUTcg$p3>;=R@{I{&UTxgXO<7z z-}f5IDSmL2TQ?HuWkG?$qQ?f8031B(l%16Y)fbgoRB@-P9Y@{XUWl`hT=NiuWUQcr zBE>GlqT|*!s_CW(B`qsrt58vU-muyr0!voM7J46ghU^vUFc%SRkF($+G*3p)cU~xC zX)=st$wnRAD;WZ&W9TW**zP#cm7iRaEYEGpF`*w_SZ=^4gTv1wa2PblYVT<a?@X|>1H58Nv+x2+n2^q z$nZcaT=P|Us+CrYH=T)@%6_>x7GDC-?YgFra6n3C2V~-GTodRV5@^#{NRlL>cW|oW z#7)ZDQ!Tw9?ARf@I_i^Y_ovOzCIEHpz5#M(0mO`mSvDhHEIA=o3D&6WF3y*4fX6eC zF^WaQ35A7kN>BL4-NncUY^b9C@mT~BsVDO?4`VA0aTkU`R4ihD7Q~fXEs7BB_y3~P zWPB=%9{$u)Qn&go&z@|Qj~2nC+(mcNHkCN-E?IL}mC|2_^gEdD!&fhZN*zPX92U6j z^HAL1A3+&q+75Ys3(woe;|kgq=Le*eTOU3Vm^SqYK{)4UuBMY+k%^I93+;Pki84(s zJ0>o!TzIc-&CfJbWEFDY!cRCN>ers9g(Qp>`6{_-NY1>F%RlA&WX|a7th#?ej42YW z22%V=aEbV=;$U#GzHw|@9KE17Mxklm(9Uj-pPG&7|3$z!rapDu=p}4(*qp8aj^7A0>kKr3t*0z{!de-68`7TIJm)IrjY~T1 zQB|HjF+UIP>bq1@u6T-U;a-raMzjhp^1M>Hs;Ugt$Bbl>{T9e~X+rPc{{#&|!0^l8 z&{-!ADog}o!va2Lqd1W%_nxex53WIPrL~1bM%zGdd+ub#`6~8sP*yVgVUtrP8t~Ii z%`>M3p}8|?U5)Hd^hZSB7dZs!NZa$2)%Fwj?orH|?dRyvI>;I#Y=H%zga?Gb8&}Uw z@VFblQ_3Z$8)G48Cy4|VEZ=Z7{wh!W3;PQykPmjX6a6GhB+YjTUkZ#wgrf&~d!t8^ zv`>qF$7DFH#VXpazJA0BWqR6FYP8iGQG;IIU(WWEs?_I};1hIfhbRksd61`Z``y_p zUd!~@XgTU-v9P^On+hW?8+3*9&E06F&28f4?xClWtMWgf5FyhMv9y)C-6J`dywp)+#A7xGL!Jc&j}rb68W(KH%%5CjQ|#`pikvJFQr)!0g?f6V zqYpB=3I6hb_Op6?C$#ZIcppiaZCFxSvQF5|%mG4A=4Gr)X z&_ua;=J(HLc~@N@%3gs%5m$=SH~04!be9JwMl<(aB(6>wt*cEpBAVnZ#Bny5>I7YV z^~w5f$x-syq_z$qF=FczGjsWZrLqxHAvAPqJD!a1p{7cVYYU=TI~6D`e$;TsF-OQP zgM1U~F)N`1V`a;rSfx!B#O&^w&@$)V4j>lfKH0oPa?3wA;ZD+rD%xXYul@s=80^}XtC?=n+RXL)i;(Z z=CKB;3rL$pcaaVKz)k`#1nM_YRD33sYYrN+@uHQez)MK9eeC2P=gdcc|7J z@bXRHv?DFt#>cJESR^0L{Q3#XBr8SPwR;m)uf-&$ug^CAD~t++rc&MLNjtO6H_cr3 zXQ%23n_O|#kqZ7ro+j>o-lfa3zMs30JGF|7;G?!zZY<^HJIvS-6VtF5D+NVBZ!-<= zgKk-iGCXL&WRJD_7gi4}D9^@jaPE9O!&*AolQNfn9)~HJwI>Lmt!vI_zI#`=7J$1G zS2R{ud_F+kZA|5kr-xz?XdahOW@&JOU5dh#!yp=Gj{x^GpGE_wdFilPyN{jqow$kX zHGUF8oh%zW{F|1GO->4|_SUb<62cT|NOoaV_(D?r5v_OFCyTKX0Y0p=GnaYlOclFB zOdbHs97;Wlvq}l1YDk0h(=7Fa6w2bZt>yG4-)8YA!=0&p%y$Qo1Jm+$=rGV8cl79R z{JE`wz&)nGFD?d8$85})Bb((5QikIwXO{)x84|l69(9`$^ES5F6+e z8jPMwgA@73&VIsgZg41imS7C?P*S^48gw>`?D#%L)KO~{yB(!(Zxrb;CLyT{e8 zKA2vD0ILwy-!*{(J%l2{*iTXC1aWpea6)MIud^tWK!icg%G8Nj77EN(as=%5sT(Xz z#blk9%o~L$Apm2f%;~IWK#w3Ah)D!k)vw;18T3Xi_W?>@6MhKtH!5%m?;L1C% zK`8Gk0;L4~9rPGx& z+LM-3o)5(xo`v3*3YUw_9-~A0KntwZexd$#hn(RDES3<10gpWgf1;-fmg zIcfmmNmow} zNM|*#rd6MSdafDc3H_cxnLT^|MJDPOd(6u{!Tdxd<_X^X51i`FPJiZiAmkTOQg+VF zY`!b1HH1}Og_brZQ&tBco&FEuGW6sJbFQ=OvWVGu%33;8iwK@=^Z+7h$ov%~`N%oB zdR&0b^l%$?4tsfiENloArB!Ceow5i#Ut-FaXD*)(^m7J{rmaRvy3a5A0_hpnTv;AT zzu>(gTC2hcJszx)5}fN444O7TPTP{W7i0qyDTM$9{YQJBo6`Cd9Xmk^efc;YHE4Xi zJp#}X%TFSUx}p}r|8czUxEn2r_9p~!$P6z`Bip(|V^Rq%!h+}(+A8u3%8^rhDpzkM zBjc}hVSW}iY(JqCmb?92UXPSa&%d)u=ymGj~_Vyf?|d%m8^vbnCQyc z0{SJ4iMwmQh<#_CuMJNvXrPw1*&_pN4Ac@#5zdEz3qu1+nQ1}^5NR2U*+z_L*Af*Q z(Nz!f&+F&ZafDN010pU~7Y!GkAH`W4h)~5XRB^w!d@{Hm7i4JSxH2?F4>TG-L{>mU z)WIra0Nlo2Qb{Tla+PFjq+495ct!(9r_M6oi#!Tc%IyRN0T4UXO?AB*pjVv$y+Ie+ z;v*+8{?^R>5z5r*o66V%K_t6?T*jj^3BiLjk@s$*@vh*~bkki&$0VYgKfH&op))Fo zJ>d^t8TDJBNp{-B_^P_DddWdk>W5(J;P|}zUE@!HoZgdqOG7E~A{5deWPH%&+ z5IcNBY12F*21{9QlH(5bw}`jz|CdIcrc>k9QE+VmVLcwKHRY zP(a)C%bD6xRDK7;oi8@kUVs~FuGswPs~%#lppxkmtyFYuWe(#2;7Q$_fnk{TL1H>y z2I!i9@!gv616&e;LFW$IR4{H8xxYY58wQa=Z+R#Z26fDQq0h>NAuJv3v#`SSgCTle z1wuZcJ*i~6u7_e+e{v=Vd8vwkXUZVJB7-UfUZc!xiMxc1yGKE@3v5x4lC1h<91W55 z;6T`1uQvERn^J6IMi^wCYMz7nN?mY%_7X%~PAFa}rf^zWo)x_g83glAH`wT^18dx% zJKPIv?CA26v)3v!+Tf8s_+cx7vx%Z!1B!-uy7X5yN!SxxJ!EOv@|Ht)a?dbK_5nh+ z3#rSyM671E4u=l*rULP(JSrr5W)LBA0^`A7e7h3cCsP{lWPDQy|T_QYxIGlZlnX; z1xe!99O!w5?O)(@E~QihB17tV65BiO0`!DMEG2CP>$xLXK+T(lq4fEkMHa4KdPw$Q z0!tj&%;Bf6IQUb(k~%PPd2%VMmmTZY8LPw|_&NGP_5|%Fh9sojY@pf67Ew$o=~vVQ zq!axFx@WkIEv!2sq@9Rwvo@)*n1wj$n+NrAT?rnaHiSuPWnMZ~WsJIp=8|+TcpAqB zhwd42#b^GSd@d7BSWodvPAI8_gAB(ze5<<-(gUvPh1O-l=-kn`J_jcYTso05&&i)1 zw69YJJ=LY?r6xyTcYM`#wHrRXK2G5Q-%=>{=DY-UQoERu_@DYs#&+rxrXX_IPH;K+ z%GbW6c3?=uK@LAvqC}nBhME>NwRs}Jo&hH~B}zq1-a3k^jyI%702!|DU%N}JIQ&c% zTVE#}XT}B7=!v`D#zaU*VFXXD5S0dZu_Yd}g!7t2=tP6TajE<#iGT#gP)`V_MYo&< zv~uqM3$el2Y>Z%ZB#`GSM;pW%>%=t&2G#BVB+1n=)3$a>pG_l&BMaArNby;`AtPq) zxV3^*@mlY0Xsny~Js@sZ7>mIebia#ipKpH{l9FV#$K6W(C2e4>lD%f5ozlwqkrF(* zNZGL(B!n%_=RSyd)#($IMBc$RJLSW}) zmXE4}hZ~ANuMIaDXd)wAusg2-;Oc?=R)}2O?I_c>j%f#J>9ZB8R(86;DK+J215Hgg zw$ViF(Lo+J!Iq6U^nV6(l%JzW)Pk7x+F>68(C}>_L#8M&y195_RnbDDR8-VgTo&j; zDzf89W2a>JE{s}I673rB{W`RIZeFMuAZqVHoQMOlgta0u1H_*G_^vz9PCy^v58yHB zW3k@N;2f;BM*kD5FONOpnY08^6>#>dp@H5w^^PraS{=zBWdPW_&v#LmXNGZxkdnEj zdyu_Rl85r}8RlK+8C?V=6{FD9bTCw|C!CRW_aKn$RfmRcOvuMSBtK8`6L#cYe^Y|0 z7V>LsNkE=Th=H7d;v-)OWQr<&Qf6LnjmwrDpuMT$TROiY%%vs9MN?u`*rVboLz4*^ zZ&P1CJW@U?-Rf>5t8Nr9Gue@-qpm45vUZG9A0=EjhCdGuQg0$!ZcBfm!PWoZfzA1! zZ^HTUfqmbjB_PVRx5XJrN*vRe5b&8Gk+<^FJbu%A10gg3vz@x}a~nKRYH485L?xM_ zWJ^=nEDVqy#{D|)P-SXbXCQ9R=I12VBH((0^mGT@5dCCO9Rw;2==`&40d6yw>tK)t z811}cOd_xa7E=?a?RhGt*US;-mQ_EC6I2cP3EF>ba2Vx|k0g=DkWteg7hIl#rsn-G zLhq-^zO9|aT`Yz*khp1KVnHMq>gsr{{30N+kVe6AIRVg-wM*5mkCk5{6GL_XGh>8zSG(d;@ z>tLgahcb0X1lp}?KDKt5T#R&qg5ceA=;jE!MmEl0(O6HIU}0*BAdvP@S=;5)BM3fF{mCnXftpZykF|j3`y3d5?r-ni|FoT3p za`DJr_K9niKF*2oLOyeQc~fw55LB8*9XpeZMUw_{e5NLZM<7N0t|}5LS;ikv!UV$( zOVyMFlJvd@RJfDQ0KRaps0s*F zy}JW3$Mr$$#1l7>pu)lgAaY(N5kWX;dxKm}7n~AD6(XZn#3$zdyu1px*@OO9)h633G{}3 zU8~*+<1m;vJDEC(L%ToDGuX7^O!9r)oZ zV9$n!f({HEChp0ArlQyy;%#Dj&h#b*QZh(SqoOES04T0KHv^=j^haDluvW80L7C?e z(#fSde4vuY-^DeA$76*G{!y&W3)JCj39ABlQIv#P90#zE!;?*RCJ)gQudSaT6^vkD zNmOj2t%h?^e~4X6I|f!K(&T;eYHZ}DCH$(o+x4Fk)aDbzd19M#^~i<5w+kCFmY_wAb=4>+VKddiluboo##yetc(wH*<9hI` z5&Cb)_eQaqM#r=w(FUV%n#c;0!yXY*-!1qMr(@jzeN(W+`7rF9z3M`F^s5$+QXszoUUpY3C%S60Pv93z;oC$j_Q^Qd$$x=Qna+rqNQ3mzz~&Y zfwk0-l)43*xdHd$H97A}y-b-{FXs^{79>0JPC(T5YG)8s9P$1#G#bAD&MfEw_JTh# z?eNOzJw0@|T=0RHOD_{5D;)^K*XS80OSUEo*bC2 zta*~e)?+bK;&t$DHCgFuj8+X9Nq5{Nx$rebv0lH-Q46>t8Z+DL#Q`{jcbzuFoI#;*p>4=uh=FLxBp?$y zAt)vp{ei$}kf7KIFLOgAT1rLSrqr}~cW#-}gRLalsA6c~LmoIaC^iZhL*f?O#pjIe zjrC#?MRi3pu7|nCOf-CC-$XKh_EiyPjtET>lFVMtA$2L2PIUC7V8Adr9R~=u@!Itv zZqL9DQ*9|yTeWN1uvD080^8TaIjKdaiZSf7s50vwvr8rM2enN2@OG#5g8?&#Dd7ug zaDdpF1|4`2aszfNj+J7yyme}J^>vZFP&$-ZKC8)VpH66x{nrM!p1pPkDBcn{sq(Lf zTxOChWU_{&U`-g)BOA4PWhjaOXP__#P>%?cO=)ldpS4z`Xctwmr&2}Z?UD?X7`Ljd zef^CNwgm(8Akl<0+z>Q;I~WWLZN)-~qYdY`FC)+%r&(Jr17rU0!u=1NEWy`(G9qqqps9TRMla`qFXC%bCApKPU#%RD{L66|kT!i_Sb`u3Jc`KHnjmM4QR)FTYTB^aeZ4%JWt zp4`&gIgS-;g8O%5{&l2!?jwWYZwllBu{Wwf@ zt0{*A8=uAhkIEk-LS&(aszO~yQhY7d%w|XyfTwIN9dMS zT-D@dP}@eueR;-3nA=2$$l3CSBr^&S%qUon%*^b<1|uz?4B!DRb<0p1Z!Lr-&=Bb* zOn#?o0R#win@_L9_p1T`lXuJ2*==bDg~__`by?Z2Rb9QAC+F@uC>tNgz^IvjMBUA2 zX-Tju0(JFpG6FG(5sp>4W#W64M$WuMqt3>Tvc-hFVQFJN?nbVDHeIQ(KMLlASnE;c z1xB()K6HL@g5~**4^j+4pw}|$S2O%Fa%^9mx$yi;jesIRi*`E;5h$qQb2-KHNE2Mqkiz~`!6|WaC&&oMrnAQ(afDjR$1!dhh%|f* zxS$zPr7d6V$yoM4h+CbF&76RghzSbiFoM`8HOx}lL2qCWEmN}zgurHS(y(T}SHP>6 zo19E%o|V6n%SSA6)nsRW9^-}vUx&-~$_|~9HKzQ*`6L}nDTyKQ>6t99DQ|`JA1r}F z`E~H?9aI|yC{#Kzw(WN2)F$PWhMZ9ZoZ17SC|4QJ%5J6rV*pZUKfp)vd8*n4oz|%% z6O8VP&mRhPDuyV0!gV@oN;s~{4Mb(lU4tv54|^Z|6^j=q7vK`q_cDM3Vk89Z;u_cq z+S@DId^BE_inKU{G+R@e#O_6N8r>imD2b5UAX&Hs%pst#)HY#WkGdhJp&?iz&hvO_ z`jzaKUk7t7wG92gWs>|T^#FcD;)MlV=mtPxG7znZpK~3y3Q`jzS3MW0p@$WbqJWZ? z@PK{b!$_N0BcI{_Z$~*BCRl+Y923JpE_txc5aEWZWuy3_k9Mo5PVeM{l*{0@99DC- z#zxNA?+%Y9B0&HfoX8jMRc8t1K9DI5G~rXA-X+9HZxe|}1(RA8ZCi@#oYY!GD~R)u z-YBB^3Y!8>7mUObSr+@d_Jvzm;Cb{%1IYNKdbB|#*$hB5R`wT4Ei`(z!0Jk_$Ay}(qrxMWtctG0*Iv%tppaylaJC-y+b08$NHn3VGRdvHhrvO6C zw%hLu-vMC!;bq>;CifE5dEr$d&{?79c!e=wC`IpcsQWo6E`|kHpTCyI=tGvF98W?W z=-G1(;0|zk4C+8PU=Z(SXfehDodh63)#fj}frasX@YjTnxcg2NhC|yPhfh^Xd{YFk zwyxy>?VZHs3XT!7AT4|BoZGT__h}CF4(=JyVLE}YQxh8voRD{$;(VK6f^*CH+g2*P zBRPcrs{QJUqz11^P3B96>fq@)%cM=yuZy}95w?%EdZU{G)UL(MGrOY_H0+Y<8R)vj zFDywMGO|VT<4vJa;JLj~-(?E?GaNQHWb7J7m*fy-F{oEzPFqzGM!E9R4h`!tp~DKi z9%(qtxTqmY2d3hx8+^^|Zn2*9JQTcBtLhj%V9)CUP`uSjOWC*@K9>;ZgYWAG0!Q8M zD#fiZurWKH0G7~iSd~181p4R>uZIkOv>|_T>fU$_Kfl7rHU_A82hiC933H+%Y)c4j zoL~?+KKxoXXakLyfJ0oO8$N}_4W+J)o@Q4jfV2>zuqFso<#w^1suG2ahxAeNV9Owb zlHi5)IS4N`y^bV3M+6f>Mps2ceZ|A3T^m%KR83Aom{Wha-m%UbcQZw{4X`3ie)=VhR$XQma8NS(JAEqxXkvzQ=4(ab9Y+a z0y9-S0SnZEAbvh9f*vrH7%N-x%|l2DVh3MM&Lo+p_Mi=sfdx|#vlc$96bC+>7XkTj z6?;b?mj~^ZgxyFsQ``7g?zB21=pz?o6RSNW1y=UMLSn!qq$4d&lz9q)LR^0~i2#s@ zw6M_4fg=$TU~0MOQx`&FLEXa5K}hmz=m6JY{zLUDCfoZZ1@;gAc3VcVgwkmZisBCB z$GtKb>MHUqlWks^Uw3m>ZM0j3j@fu*IrA;x0>h*&lD6&-tf}Gho0)`gf5=Gt56-Yk zopH^4vXr)>Ku<7s-qIc-EO2rlP+<2=8DOEi<&mV5Kp|DRd|{bLC2X&bz8V~`Cg!1! z?fL5}$H`j;)4N_*QpUKqJqia-+% z7enRjp-42piV*ZC#eA+4?}tBeuxV9R=s#qKTvO~>vW&bB$(2JlLF*^Rz}iUxkbbLz zR{>m5g~xbHz*Ku%QvTdooFW?%7u-JyIgC>_UAPS=ArLde@tFoRH%AEJOBB8BM+~q_ z$n|X}n?H@>9l>gQLckdOt9cd4MH@ z!hXN?suN3jO8i#h3BE)11y~zf+o+v{;O-8=-6<}?-MzTG zLyHwcaCfJ;TagxMk>bUrKxuKeQVNtpp~A_#_xZ1{dvEsto^Qc$k-)m=p66LJv)0Na zW8JN8WrmHc*-T6%o!9j>`!yiDqq-;^bJ`li{RsAx`s+T9x9iNi{FZsTmsIBA1Rx&J zaXd`rwjto}{yGKbXLujGM9HW$C7F=O0L@`5mqa8cFj7n!Y#k^@IeUbUV;WIJ=E z-sMtHhsZk!Y{!W;oV3fai=gmtwH*)5zJ96OIP->??DWAXn(%y8YAbB#O~mtG$mCtu zWbGsJB!edTbzI?$GxV9n0eDVA0+It=iJq>woqPBJa^wEjy7M>$v^`nvjQOGf^b4V< z57gNEW235+`_`Xn51gD12;Iqu>7vDRrj72ov1%tfPq;p3_!Xr&9jk)B7?uM@o8T*X zJ;ISF;hUAyc5Z9ZQdjt)>TOezXr!=rCx&I;SL7kgmmG=DKOXTSd50DNtovVeUkZOf zA)$H%r#?_BDT^})JRe2OMtvCxe8gjgG^aRzD~}q21c7gtalQ-)d1dqRd9Q_a_+X}; zssaxjAcF<5QH;15VD!|E{vwFUS!eoVBdUv@Hx*6Zis&iaB2PD+6ZvtZ-p)*@u5QL1 zx1lSzMvB6mK9A+s2+~+s)h>xzsH){P@9=`ih{zcvz!IJVlxG<*o(-+78HH~wrTqFa zjzW^({oXaO(W5bNOy+_cQv`Edh%pQza`mg%__Kr*)7jup*M|~atyg5B!m;pAY`VWh znBZST+Ky9&(j+dv7c@Y~8`Xysx_ z%fK5F5+lhd9ThW;;nt!&YM&?uE3?1$7DN}*jeuD+!}C-)_r7dp0tvD9(#Q-nvN^M& zneoWC2j>QMhpZ1zi`C#cHgP8gt##T;%HA+czAb*nSFwz^uGzgn?~{*YEqG7LgqLkUG^BH%}3%d6+rL39yW5L%B;~LDiuqN#MyHbgu2CcPqGOtcYUw>* zha>O^U2RnsJw%!$;m33F(2N7es;+FaJiaVKKa1B=kJ7Dp zI$yByaeC`^K+pyF1Ny7D^^5XCL}+1v*6=UQ45ez&{+t{@4~!k$QZDuw(Sjm=h`ng? z+X4!D&81rAkQ~a7*}`6;H``j#rvh`r_$~L$4JD&aV$rUp@$4#MU~}HF<#NWlL_+ zP-CvD6tUD}+rM%gL7=QCw)L!Jz1I{q_H>3_?^C1Ci0?>2muk2P5N8E zjGHQnnrFgvV&u_ZhPn8M-owLvCXrS=5d66F7C(6bR1|iSl<@ zN~PCe46UL{de-(DrAsGX9r=qFCJg?_J$H$OOmmPIAGpfPiS50W=LFgxtQH&yV%ds= zY4q2=yz~_5cbm02IHHL4Rr{#->&n|odFcEOJ;VXeSxE+r8N*VxIZUD-QHM)Ge#Ys6 z^6&ZG(G~QM6hplF>?=wc}4UI=|(G7C3J>dh393l zvZ-p8iU1587#K-~dKMU_&l?%p&uh#F#brr(2~LvN$7Fe#tYgU;M&Hf>9R>P^V|2KS z$WSLoQ-R`~%_b@DLg5dVjoH1OwinOw6fK^R=u5(Fc6BHo>d$4ZZGzGUkKP`Zu?t?b zfw@VS$D}EeeNO3d6Yn%=o}>=Q9&Os-~ahNmCPWKA^H&A@J-@~ST;|$MHYQ`tR&i7Z)TP~ z>7+ZWzO@2$lm&%&g*;kIALS%m#gHI8*K%k;y@y4#DY7^EbyQ$PUrp{2ABe}XPLs6- zc826a)-Kp=UhqWJ)aC_Nd&6d&Mk@%4w}7nB4qyZD!rnxq2^f1(Cl#1eoN=my=s{1E zXEDjFDd?~Zu8Q}~@MJoqUQJ$rgKX0x!QmNS+I~bA3_z-CKIzX-iBnf;N5;`(ydi~+ z4ToIFz%N|%BM5Fds@XN{c>Xv(oYHLJ1eeZ z_9REVwGA{)c6%jcJkUJgNneuSH{fTc%Z>@9`#7Mltx#AU0J~KfzN*N*>`joW7B&9) zL(Xpc=L5U8@wC_qr7=3RI}ST0787j~zGk0`_puMcv3TNXYa@pkN?60c#r2<4^X4VF zmK*F@5DmxJ6y0d>3i>@3A}E;=%=^qG&QFLQ7bB^T!bXQu6f%6dtMtRwiQ@8nM+ARZ zN!|=k-PZ9+3R^28l8HlJPyjE1{Jm<;Bs#aexf>uSNdSYM+caagt!c7ZxY3OBV=`EZ zp-(Ts>5**d+{@V=tDkRU<+v&2Dl%e8h#<IlRgJ&*$&I5zP?W zzd3kE>p?!)(W>X~>x253NhvM#L2r+2OROwYVL@BY+oOhhY^q!$2?2E08K6ps23jJ~ zEpCP~M^J9!$n2w&aqCgf5}(c^y>gjo+dfxaexA5jSsos&h1y9>xzftEebH3f zc{G}M`>d*YP}NG6!s6;v@j3Pnc?M*5@-a5dq6hgHeoS#lek&?Jv@{v0P6NJi9Er)- zOMZHU!27mHSEr|GMz*CE%VMuFdc}hn21_NUXNw>I^xZD_2>PW|+qB5fnNB;T@HQ1k z22Emv3$^@h@I#|hQ#zGdhY%61>%`HD*oC)mHOWjEK(CTx-+33{%30`g3`HW5sS!CF$x zA>|*{FmTiKLgn%!UCO{ejd)KVZ!XMi=EptIQ~^3@3WaiwZYLq`Zj7NHHTt~HO(p{} z1sdPe;%+op@XjN401EElxTbS_@I{du8Gd*It44?J{TPhZiTfJ=jC7r8(xS7`ThgY) z!4*Gs&8X`V)*69yrSh0XHUi4`ov{Gz#T?Mg4i5!|90^hqmg~lmd^9>_t>yDp>F?~osh!G-6zf5|u8hrGNztxrAC=1I^OVpZkC>{HM9#1ptIW0~iQyB@ zCqac&bf~^F6VpfnH`2Kx2vro#?}$0gp8x>dECxm)0ZWRtv1rl3%=C=|C|d@V<;b(G z;S2|P#7-}qnya>44Ina_*`eS=42LeVm>oHmM@fyL-XgUNRG?QU*iBP9TF)1&5Fe=_ zMjk}I zJ)LiG$f*c8ilFV&+Sy}@*Km?dS8qeH1{PB%UN7X>=-GPtQFz00V>x=Dyv`^-3!@{N z`XRCLifMY<17;VUz`jilDRuw8bgqnCrA%2KX!Nuv@V*8uK#w zu$>a{`Cb|`*2b9{DaczR$s9`fa7A6Z%k}WY5NOgQ>0oLw!s212qNM?gLi?SY%xj#b-PMOf*7;QZB{Gg0%%YkWP#IHJwK-zaS^f0g&bt2-bXEiY5wklQkez|6b#5qT0%_8YRg3nN{(&; zO6F*!zZYSjvwoP!1*8<|KM}v%=;xb`ONo3Q*cEnoBl%N=fXB-`)F265$XxKsuTQr9DrsF7wO$4IgwLGhlYdf_v^8+)LW#XL| zz)#%;)#UTTXkrr`g(9tMU>?X_2CJa*^OHxtL{@|iO4#BbO^PGQKVQk-L4f)XNr!*} zxWQQ0?dJSXQYL?rB(mNzb#A22t@9T8EoYSDBy!1*=VA?AzDUpFedL-(+TvWv)^Hja zyX_CtpUSjhKKzQf9xTQoK|iRSN*L4RF#eG8D8-mpq8KtH$PivP5X?A_V_NpQ%$?d` zt}$MpKp^OW9!(d~M|~5+Mi`2AhkkTF4@M2W;PEoZMw5R6iTRy1qkbf-Jt?ryZnOw& zrC!Y1%>T6Y#yalY)SCxO)-C3$$|UYx|HZFKe7D($9TUj|%zf+q^^9U@sw+ZK%%MQS3XjDqi# z!PB}PNbv0{Gh`crr+7H1rYPBK#ExGqCeT&gL8*i%CM5dM%Hy35fES1K zLbD%q_#4T^n-MJ+3F2?`&ac-Hvt;vR@+bww{?gm#@Ql1mBC3=j`7q zVN#d;EMl-SnLgNjFCX)W%vg%zywcKZU1o*GMG!roGTHK+TGh?r03L79KaWyD;Y!Ud z^@{vepjGp?cJHaz`Z&*tC8r%_ef_hhL+r|uI|o?_aF7Ftc!}2!q)VctIi5y9&lpKL zS*h;CoX0!pifuaygpEA=DnRp5c%P~tyt;HOGR)yT#w>+^NW~k+bP6MQBLb?Q`+xl8 z)eOWEf3oO%_Ab1yP=uJo36CMw78>;utrUb&xg0}guehh>Te6)iyJHRltE0`WW$}WB zHjt?xbkHO~D3UU=o}NCD^oy^o`Z>C#vQoJrU1xkDjX$lx0V80(4>F^< zkAKovF>@Jbjji=b-}7H2HyqXjGw}kbV0vgdYEirZPTI;y)y`ZBQ$biQuO3XpfVC_D z+)mJhcG9;f_&%%5qN+~xW62t8t(7J})Y-K^5pS$tIIc}4kOBOqlI$t?V+Cswc!Akp zbPT)nVk0S;0P9**)LS?|bv8(H&+=|Y=gxKNZ!YWf1~?y6Yp7+U_MUj_C&{Tcr(xH;<4p!GK*#Hpgdcqe86Ux_$L3; z3?Ka6kIMT3uNC;e#!+RD#5aHHgYdv%mg=8B17->zpC6#qt_fAQVQ5PZJj@tk)t8>l zM2c^|(+gp0UY4z?8{>;TE{!~_#oO(;={P)LfbjX?_;MEl zB#ps(EiA`mCF)!v5p}L)0lJj>Tr+vG7UT3W@RUg+;nYq0;U0y~e0>>!(*CFg37uk_ zAE4U;5P_8N$yW;()57bjq`B0iZw^<>eXIkWDr*0{5;ym9_AIm#^Rc|la?2@$ z$*1AX0EdjK1C9=D8RHH|q7AP?&BH1^i&f7bD*AS~QnlkJz=KXD$?g?8?r@UbfZ5;w7@=)-*RSgNKs!*poJDA)R8o{CK&Hc63m; zGC&a@Z;l*z@C&mocKa;=H&I&tYz&0ceEOi1^8H)PNYLSk#h41m=pLN)**;Mf_4lqo zGs3wu$VzWu6Q`s}C!3mbK)AIlq&1XwHvLdr3uKK&9z;hYxBEg-LY;(E!I&c0@AD

          kgA}-F2v3?QqPbqiqRZb40fG)+p z4J_=#^J&UM^+l1kQz9vowW=+#dbr`eRI$&CvahSQ-}9UgnaLD@DF&NlOF2Zv$=&*e zx;azMOKcA&UQAU(-!zX8W1sk(xvX*nc66=ZF#AxdWKYe97<$fe;O_{X)+vS&z;;9bJDX;80FL$r-rx4E>>YtWOYnDuNmP|87gs+VVUm6jT*2hKb4I$o2 zVv%{y%`TFq%{^5fXR9;R#JX(|FNb@&f7g}2erj{`(sh5A^EPk&Nup2jYQE#Dn(sw? z#zmaoMQnyow17|SyicspDT7KtnqaqbkOdwd?-_AdDRI|-d|#%-9i_w_ zCdBO~#H}R6eIR(N;i34vsQVDn%MY5W@Cl&rBVmnG|;py;??)p!Y%{LwHGSb zsK5y0DqNdfIvSheEzkWip@GO@%2OFUq}z^pNym`xc<Q)){aHsujUW9#Z zuzet*og!tlP+Xj`XH4pGI^m{;i30ZrzB+#qr*OJ?FP#}YP7GAA#V0MN4fGp`3Q)$R z2PkUN35b62#zoZGqKOV?3&?wNBiBt40E z9qHzhN$oNfK+0|@ZL1j9DVP*5qb$kzTC)OreMPuAVpZu%&U;Z7Wn-r6g4e5_G_#V#h5@j=Wl~dxD*E>GMHLH2v6jjpfk+=Yti(=ri{!K~zn?8QhD7*_o@ z9bN^Y^#YnVs+K2lbVoJ{(wNfPYop4%j9~ibBYOu0@4sS`c_gQUj4$?7ZS6P@rDndw zzWTQ5pwNo5=qWn0KT!dkyRg`I&9_Brb7eq)BJi*IJt z6YG&r3on+ZZC*-N`kN;9R=*DT%6`SAa{Z&bTESA;xnE|6w*B^Jv5SIYl;B6Y&`~m>WJ+2K&%1YNM&dQ)?PW%7I!}*w!ObG zRFEx4WXnz|Gqca~_$QVHo6X^ky*NHJrKCnk7DN=Kg*FF?=nfu;L=onJ6TLLw!SEJ^cbqf(evold}H9k6HXUHn>kqYMRME#iM7Bt{WSF?!cBe0 zMXckRjPh)XT=;`fs&L>NbV`&AfRgsxE+Qx$$Q(s_n|%MUvX*6U5A@`=uCFvtuPLCJ zqL0GAEH)aQA3hClp?H^yO`*&s|HNBX5wFS@WJHaP`X#U&%0D}$kw=rJh`ElHLe&2owh={g1VE!?>?T>>*VQ-D_$f&yhG8{`TiSP`0 zyYvSleo+0k`7BiyS(USW9_TX^()tf(MVOOfhBJZ7)HW<4GNMz-u7}2k6xsXntUGiO z;-yT{wFL2VDn#ZZkI3E_NH#I<+2SsOQq(X7QrOo?m`Gh?6buzV+kSef@TB!*WI~PW zU_aAkY%!yz@d0x9&d^uutJV-pEndo?*6-ze1k7tGSH{gYjg6&Oo(yXIemA>UPx`ay zL#=Am-;0NQ@O#DKqZvN(bLH*GsqQTtX+<5H`pXPYp36X2%pvqi*YxUE6p~&i2X8eo zd#-@>Jb@Jp47`YR^dyX3AMOGQhfc5w9RCWMR1JmK1PATTjfOLow5E9?psO z;U3p1ClOHUdB~2&aTD55JO2!>baKZedMf2Oos@PK=-tPFO2dSq5hO%+{CRx=i@xvc z9-O-{@mj`hU)fGL@9U#-aA{O$o~>)2=L+}iaeK*G@1CF4W`(Ww^c~ql%JHR3*ZNsr zXw+=OF>n#ht$DNonFuzCA?Q0_|5WePId0i+8!6vKUt&jZJekq^5_z$MhJwAFcgsM5 ztQ-4Vy6J?inEaDnE7WjuI&MZqTU?LT&BSFpG#Pt|hXXzIz|wN5*Z4{@wle4V+8#0K zz=IT8$)bjlQHQ20_?DB!UQY#S5sw&%E+~PQo};=$ndZT;4ywe5q5fHZW?SU+a7;#l z!ALuT)fPvV{c{lRxx%utF73)9BY9!`LTghj>$T|hG0kO5ajH{VTI##tC|oN5Bwy$@ z?vr%8&!O-;V~&RS$Uzl)4~6Lo8hTcqFoi|o+C~m(HWJ?JH^PHPnQ209)@6e%uy->Q zL&2_|eLTc(q#A;2o)gRNEvPMo?Z}-%Zd{*^b z9`!`CF=v=m^YMdhBgfCm{4tsNbRgSoiNa&`bS3zWlMqH{$uar^muwx1EWPx=bEP8g zE{UC4rgb!n4Dujf|F$ptHx|?1)~7!?QF6wq(!cTVYENtVTn76R)OXk*^tI0Jc?2Ct ziU?}&V>hd*7^2zTv$Wngu-Q2;bS?rnOg0ltu)1>@8;rV|GaHH0X$#D0pJij-9uRIv z^4!TeLJuE#G@uEL2Oq(AZSPL`P&lSFb8=;fN4%5`_VYgIEzcHnXIlRP&9)L71LVqt z9fTe!tWbtmp0JibwDSmX{tlYU<$vSM=O?q`N_4Azxf2d-KM6Y4If31s6@J&T0ntX(cwPM3DiYLc94}`xy zDc3qs6p$_u(ss-MQ&6Iqz_9{poBRBjFE%3+G7>&3E)862bSWxli1kISUS(-~Gx^-p z<&{_pfsI-`yZ8qtJ6B?Q4w7(!x)$PSf5E(coO1pd_M;Ty?uy-!3?}$P<&EhcA@gKX z4D{h^r8ukZg|d(>lESQp!4VjKvzhT;`na1ewXiKh&Ol`ocObI=l#_b)U68ESAS!VI z50yEcs7LC+k;*vx(-Fk$Qh4yPk0=y-+AQH%H9Z{ryN6_P>Xd|=gnYx~3MK;}Q$|b- zPH^*wMknf%%vrSdWt@FB4>~t~O}rQ^Hie8w9=3Q^oTx4$=eQ4I2WVA+40gLp#V#Nm zYT5Lvz5+YL3w@D&2SQGxNzxdzETtD%;?`mWZcHnivqrFYsFCF}4BykpocjlTgy9nr z`Lrx|oDqqM68^sm{2Z=+$(NA|q-4m|ywjBU(2UwrjZTPe~({;Vu-wCNC(`-viX^N&! zR~h^|pkYEy5oEDOodnsPzDadr^7;`FWtcRZ{!PJ3EKUnr0f>xZz(Naw2H=RLmIuVz1>pP8`Rf9>}nnGSyZIGc*q!YG2z0qr3o9yGN6Y%ob|@JV@dQKB_BS zC0jryVX9uJW#Xg+|B|b!qp0M4Z!X83_LBAMD% zONuu)+#3jNemuLUS5=Wn7x`M;q0fdDZDObn`-LGL7U`vaT6tw|^ut);I~TdRYP+q; z_E@5hj;wV3KD8ICgm)y?!O@z`I~G4}KAVRcOH7n0C{OQXaQHEqDEryxrFsz8FruyU z#dx)3>M5C%k@0cdyvOH-%Q`uIF!oK4fOui1L@=Yf@2RwX`-EQH>nW;v2$B8KS8Rx* zpGZTA;n;S`c(XY+^kE{IAU_^kn-tnZR1hC_m9UXbI7NZh)R36I$5498ix(H-5WBRg z{4yF0IOg_ENER)j@wE`sj5}E(*!6;NhuJ+w966QVoiPanvBSdVsZR?N3=FNa2%#q9 z3zq)q{ipHoO|db@-Rfkoz2q})l$oPRTnNwosMiVr7*BX+_HBJ|;jvWx(UlB&sV1mu z1}4h0(QIO$ppQ5%^`4SC_-8}!6r8MEavmvy@%2-jD=AhaO3hC5Gjdtncnx?5iOpOM z6dME@g!wF$v2nzxIl}vg!nLA*MK(ROY-!>hq%U|fa-ky+oOo1o5_I-xYej+*J?cG6 zHeA}(9)PJBpSIsGYNlQj`>{U&7EJ%y%K0RdQ&lnXEf@o;_`vAfqXUh~lV9*J!aIEl zRB8U|IJ6ucZLcjU%JhSwY(CVFHkz4a*V+t?Wo0P@X?Ab=?_>{FXmqAvAtB9|APpP{sPxIL6MiAi0LR??m6OBRej&c*J?+p#|}hXEF2TrYG;ivS8FpK9g<i8x&vCaA(9^#W)NEz_dKGVFVBH&uZfn<b%49Yi;*1o{6PL~Iq1<9 zDYL2>GAFjWkg8viQ#7?k(e$gc z+gOZ>_9oT9Lw6jjy6Po*2A7}`DZz@|9)Y?V*B7Z+n=UfCiqJnEv26=jb%QDRM+|ROag^ zKR=h&VQ_WfDi%#-*Htw`?Z+fj%59#!m8+r(Dya}9Pl)*y@Z#b~R9QPTCY6hEY;w0T zV{lay*B*|l3Rro?Y;0cA6O+ekNA+{qwoGnDOR6uka-#4W?X23vJtjFJHT&5)Hp`$K zMGcCkUk7T<`06~>Y8@L@J0_h-Gx^I}EREXW8&V?n#&L?62lI4L0N|T}^kxaQNYb}H z=iMnNvPxtI#f=%|%|1n5=!yN8mtL#D-c~0OK~&tncT^vk#;iF-&w!RKtmd;H*@nZu zFK|6Xin|uQJ&_=-*^pbZYg44!&{>%#@%Z=p zRx0WO84lxvk_Pw{35{L*Viz2jQIj{k^3uot%_d*zn>EGs>z)F({VrR!$wgGjwB* z)2%rd)5LgK-;aY62^nR+o95wD<-_3UM-AI$>;zI@OYlWdvK4ogUP8b=*k~BBg}Siz z+vhXiNREYC-6OsXMk32WhfFFsB^zN^{TIYMuzaKk;=nf7@I_Ah*Js>XpMY}A_>S0- z=so`LQAeL0#uPKkjh>xnP(fq;tx_vSGxuk55fhCrzPvwHYpYW@n4}3@Q8d-IWo4%n zSfwN*ZYks+JIOPk;yHy-;Fue*h!YX)x*7`r2W$50WBpf8h~-WOqG$pRxv`hOr|Q)x z> zawE%@MJN`J4vH;`I)keR7WLlaH5L zfGy*9$U@rUXd*w)jA#fD9@`t#>;YI@dl;_Kx3~G{-E?gpUY2iO#S1A2yj{;wxB=k! z7HZeSiKWqmFnE=+p-HhlD_XD2u=BF7`yk9;gs^xxg;M1jsBtI|w+5CAftUx-rWG3y zU-g+|uvroZo41BH<<9~92Z$7S1t;Ua>F8|ic5AzMXw$$$hrm41v9N+cKb!~PrV}X# z02xqwqMuRb@u4X6t}uqWA1!WEGlHs}#(W`aFkUGPk0~Bd(A{qv95c5c+<09~fIY3} z0=N#U5`nMN?0I4X`rpPXP?;hwOiSP4Suf(CTIu!NKEZkjK~=GGFXB8D!=xZr*Ow3MEW{Ek zuu6O>n@#hlW28J`E?w0I?$aXf$uUaaqL zmBp3Sz*Yv#O$LTL&5wDQ)GbGyEpvMt%ee1d5wt+d&+$lEhJ4%~>vzyu1=TwHk#s?X z;0JmAX+*82UWHCy65rF-dXYTP`T-_#{~o>1>oqVNqD1Ev^vNG4UyQ^t(&dKPIi`tSPO=8j?IvawH(hZv0E%2(I$-#(F7}oAYqnNx7x0D zi%MrKoDg}M2DwIC7J}<0sgKzdgHTI5O#>IV(htBY z{L04ByU~wqZWy=Tggdk4hD&{jT}8y0Q(H*{Y1w?N%Q#5kYt}6ImMXPubIL_U6X4Um z?xqK<2x?CGxbjPM(-*eaA^3_!6-iQmaz_*U+2Unjf5om_3hcEYQ7 z=1RhymOF-R0Id_wnG!AWbDB3~4#ra*d)7ZLiPSf*WSFRgkzrBQRPD*M_U!!PJ@&W- ztJQH=Ex3T@h~MuHpJm}Fcu_7xVE)zCHy@YSFl^y>VmO{^^|8%n%KlQNRIG1h@|S8Q zOcV+~<6GrgK@^efWqBMwvGFPdC?ieH9-7c+KYN*LDoJ}G$#Bzz#G>lm!boy1@3`-p zPab;HM96D18o6f2Hro5+4aMhOxQ@g6WkJ}4mo)(^IUM;bzgmBp6NLmd_L8(Upa)Xl z$tZ^pEI|>A^E?NJ*ji%QIV{;C?s_xQpuaHs{fm@2svlIF3ElF&j<46|N53djfns5Z zs4=COao;=KFq*7j{EWU89eU4u_G^gZ#;0 zq(u)VmZ7T@iEi@hDMTqH!&}$y#RzbKXKS1*36w7}(=l89&b8H0(X4{pk*Bi(fEmQ1 zt%d|90-&Lx0k)a%v;n^rLd5T;a-JT3_8xwg`hi~dzUJHk?ru{#E1EmJc+thM$FU(@ zs;jM3Mj9`zayT^K4j^HTRlJDBy-?TrekGz}jr6>pK6bZ&aVJdT%hagK$=CxtF;DU6#7Kd`j4v zCNa#cSzjqcazmM^iBfpwUP@C?j}1^L4zdaJ4f$PAau_cZ7Czq5I5dO#zF>zT<(O3& zq^ersjTPIU9XZv1&0S@RY}%M~KQZ5z%tO3Jar7#C(su23)=c#FTuMf}cFc<~|D!=% zv8r{wDzWjhP3d6fWMGAt=bHah@(Jazk9f}~f-sAo$;33j70tf0JC;mo^B+S3=wxgX z7(VK!j0+O`npBL>Pc1*{M@Bav8mYcRMtUPMt~op^7>{=>B3o6CmZwQ?Cq|urgb_RE z>3HmSOj#UoPLm0IeT!$pFgVffSToR`^PJmZ^s`cz(LTh>k1OQ~P|C4VZhzRgM8Fz7Pw8%vU0Bk7%h-d#^yuAE){G9CF?f?AA z_n!vv|4vv&aQ{Z&zmK4}b_lRV{4a#l{X8AwJdnGMcv2dOuqM6P2_h_n<{>k;rV&`|LgD3#Nh0@`WBVB*qQHMd zG*R##@^97;{&%ABqW9qbA^HdOUeRC9o4*neJJT`m-y@nQb`SYCQKA2xs4w&$+&@JB zfZi+ml})U<50QG|f21BRa}W79QQ`lcXc}S#|7c175d8ysuP7r52>^nK=-NM0U!rmk z`8QFK|D7nV);+j?i2eb+S5#L*CkhD>QEc>x8m9ieJ~ z*q#2Ws%~))`ghg;dUE=!>Y?pDynm|xLA_TMLJWt~AP!M?4)yE4pIkvj06;imL3E{lE4?v))^tD&Kz(_X?cE;P`#IZM zYufwUb2W!`Lq>Ea002J3S0F9`ux)}k4+8!V7)$Q^ literal 0 HcmV?d00001 diff --git a/python/docs/source/index.rst b/python/docs/source/index.rst index 34011ec7c5573..b9180cefe5dcc 100644 --- a/python/docs/source/index.rst +++ b/python/docs/source/index.rst @@ -21,8 +21,44 @@ PySpark Documentation ===================== +.. TODO(SPARK-32204): Add Binder integration at Live Notebook. + +PySpark is an interface for Apache Spark in Python. It not only allows you to write +Spark applications using Python APIs, but also provides the PySpark shell for +interactively analyzing your data in a distributed environment. PySpark supports most +of Spark's features such as Spark SQL, DataFrame, Streaming, MLlib +(Machine Learning) and Spark Core. + +.. image:: ../../../docs/img/pyspark-components.png + :alt: PySpark Compoenents + +**Spark SQL and DataFrame** + +Spark SQL is a Spark module for structured data processing. It provides +a programming abstraction called DataFrame and can also act as distributed +SQL query engine. + +**Streaming** + +Running on top of Spark, the streaming feature in Apache Spark enables powerful +interactive and analytical applications across both streaming and historical data, +while inheriting Spark’s ease of use and fault tolerance characteristics. + +**MLlib** + +Built on top of Spark, MLlib is a scalable machine learning library that provides +a uniform set of high-level APIs that help users create and tune practical machine +learning pipelines. + +**Spark Core** + +Spark Core is the underlying general execution engine for the Spark platform that all +other functionality is built on top of. It provides an RDD (Resilient Distributed Dataset) +and in-memory computing capabilities. + .. toctree:: :maxdepth: 2 + :hidden: getting_started/index user_guide/index From b14a1e281639f3c57116a80142e6a04900297e84 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 5 Aug 2020 11:57:33 +0000 Subject: [PATCH 369/384] [SPARK-32402][SQL] Implement ALTER TABLE in JDBC Table Catalog ### What changes were proposed in this pull request? Implement ALTER TABLE in JDBC Table Catalog The following ALTER TABLE are implemented: ``` ALTER TABLE table_name ADD COLUMNS ( column_name datatype [ , ... ] ); ALTER TABLE table_name RENAME COLUMN old_column_name TO new_column_name; ALTER TABLE table_name DROP COLUMN column_name; ALTER TABLE table_name ALTER COLUMN column_name TYPE new_type; ALTER TABLE table_name ALTER COLUMN column_name SET NOT NULL; ``` I haven't checked ALTER TABLE syntax for all the databases yet. I will check. If there are different syntax, I will have a follow-up to override the dialect. Seems most of the databases don't support updating comments and column position, so I didn't implement UpdateColumnComment and UpdateColumnPosition. ### Why are the changes needed? Complete the JDBCTableCatalog implementation ### Does this PR introduce _any_ user-facing change? Yes `JDBCTableCatalog.alterTable` ### How was this patch tested? add new tests Closes #29324 from huaxingao/alter_table. Authored-by: Huaxin Gao Signed-off-by: Wenchen Fan --- .../datasources/jdbc/JdbcUtils.scala | 64 ++++++++++++----- .../v2/jdbc/JDBCTableCatalog.scala | 9 +-- .../apache/spark/sql/jdbc/JdbcDialects.scala | 47 ++++++++++++- .../v2/jdbc/JDBCTableCatalogSuite.scala | 69 ++++++++++++++++++- 4 files changed, 164 insertions(+), 25 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 816f5f45860c6..5831c35c7e301 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.sql.{Connection, Driver, DriverManager, JDBCType, PreparedStatement, ResultSet, ResultSetMetaData, SQLException} +import java.sql.{Connection, Driver, DriverManager, JDBCType, PreparedStatement, ResultSet, ResultSetMetaData, SQLException, SQLFeatureNotSupportedException} import java.util.Locale import scala.collection.JavaConverters._ @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.connector.catalog.TableChange import org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProvider import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.types._ @@ -94,13 +95,7 @@ object JdbcUtils extends Logging { * Drops a table from the JDBC database. */ def dropTable(conn: Connection, table: String, options: JDBCOptions): Unit = { - val statement = conn.createStatement - try { - statement.setQueryTimeout(options.queryTimeout) - statement.executeUpdate(s"DROP TABLE $table") - } finally { - statement.close() - } + executeStatement(conn, options, s"DROP TABLE $table") } /** @@ -184,7 +179,7 @@ object JdbcUtils extends Logging { } } - private def getJdbcType(dt: DataType, dialect: JdbcDialect): JdbcType = { + def getJdbcType(dt: DataType, dialect: JdbcDialect): JdbcType = { dialect.getJDBCType(dt).orElse(getCommonJDBCType(dt)).getOrElse( throw new IllegalArgumentException(s"Can't get JDBC type for ${dt.catalogString}")) } @@ -884,13 +879,7 @@ object JdbcUtils extends Logging { // table_options or partition_options. // E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" val sql = s"CREATE TABLE $tableName ($strSchema) $createTableOptions" - val statement = conn.createStatement - try { - statement.setQueryTimeout(options.queryTimeout) - statement.executeUpdate(sql) - } finally { - statement.close() - } + executeStatement(conn, options, sql) } /** @@ -902,10 +891,51 @@ object JdbcUtils extends Logging { newTable: String, options: JDBCOptions): Unit = { val dialect = JdbcDialects.get(options.url) + executeStatement(conn, options, dialect.renameTable(oldTable, newTable)) + } + + /** + * Update a table from the JDBC database. + */ + def alterTable( + conn: Connection, + tableName: String, + changes: Seq[TableChange], + options: JDBCOptions): Unit = { + val dialect = JdbcDialects.get(options.url) + if (changes.length == 1) { + executeStatement(conn, options, dialect.alterTable(tableName, changes)(0)) + } else { + val metadata = conn.getMetaData + if (!metadata.supportsTransactions) { + throw new SQLFeatureNotSupportedException("The target JDBC server does not support " + + "transaction and can only support ALTER TABLE with a single action.") + } else { + conn.setAutoCommit(false) + val statement = conn.createStatement + try { + statement.setQueryTimeout(options.queryTimeout) + for (sql <- dialect.alterTable(tableName, changes)) { + statement.executeUpdate(sql) + } + conn.commit() + } catch { + case e: Exception => + if (conn != null) conn.rollback() + throw e + } finally { + statement.close() + conn.setAutoCommit(true) + } + } + } + } + + private def executeStatement(conn: Connection, options: JDBCOptions, sql: String): Unit = { val statement = conn.createStatement try { statement.setQueryTimeout(options.queryTimeout) - statement.executeUpdate(dialect.renameTable(oldTable, newTable)) + statement.executeUpdate(sql) } finally { statement.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala index 5d64cf4ca896e..0138014a8e21e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala @@ -129,11 +129,12 @@ class JDBCTableCatalog extends TableCatalog with Logging { JDBCTable(ident, schema, writeOptions) } - // TODO (SPARK-32402): Implement ALTER TABLE in JDBC Table Catalog override def alterTable(ident: Identifier, changes: TableChange*): Table = { - // scalastyle:off throwerror - throw new NotImplementedError() - // scalastyle:on throwerror + checkNamespace(ident.namespace()) + withConnection { conn => + JdbcUtils.alterTable(conn, getTableName(ident), changes, options) + loadTable(ident) + } } private def checkNamespace(namespace: Array[String]): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index b0f9aba859d3a..cea5a20917532 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -17,11 +17,16 @@ package org.apache.spark.sql.jdbc -import java.sql.{Connection, Date, Timestamp} +import java.sql.{Connection, Date, SQLFeatureNotSupportedException, Timestamp} + +import scala.collection.mutable.ArrayBuilder import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.sql.connector.catalog.TableChange +import org.apache.spark.sql.connector.catalog.TableChange._ +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils import org.apache.spark.sql.types._ /** @@ -184,8 +189,6 @@ abstract class JdbcDialect extends Serializable { /** * Rename an existing table. * - * TODO (SPARK-32382): Override this method in the dialects that don't support such syntax. - * * @param oldTable The existing table. * @param newTable New name of the table. * @return The SQL statement to use for renaming the table. @@ -193,6 +196,44 @@ abstract class JdbcDialect extends Serializable { def renameTable(oldTable: String, newTable: String): String = { s"ALTER TABLE $oldTable RENAME TO $newTable" } + + /** + * Alter an existing table. + * TODO (SPARK-32523): Override this method in the dialects that have different syntax. + * + * @param tableName The name of the table to be altered. + * @param changes Changes to apply to the table. + * @return The SQL statements to use for altering the table. + */ + def alterTable(tableName: String, changes: Seq[TableChange]): Array[String] = { + val updateClause = ArrayBuilder.make[String] + for (change <- changes) { + change match { + case add: AddColumn if add.fieldNames.length == 1 => + val dataType = JdbcUtils.getJdbcType(add.dataType(), this).databaseTypeDefinition + val name = add.fieldNames + updateClause += s"ALTER TABLE $tableName ADD COLUMN ${name(0)} $dataType" + case rename: RenameColumn if rename.fieldNames.length == 1 => + val name = rename.fieldNames + updateClause += s"ALTER TABLE $tableName RENAME COLUMN ${name(0)} TO ${rename.newName}" + case delete: DeleteColumn if delete.fieldNames.length == 1 => + val name = delete.fieldNames + updateClause += s"ALTER TABLE $tableName DROP COLUMN ${name(0)}" + case updateColumnType: UpdateColumnType if updateColumnType.fieldNames.length == 1 => + val name = updateColumnType.fieldNames + val dataType = JdbcUtils.getJdbcType(updateColumnType.newDataType(), this) + .databaseTypeDefinition + updateClause += s"ALTER TABLE $tableName ALTER COLUMN ${name(0)} $dataType" + case updateNull: UpdateColumnNullability if updateNull.fieldNames.length == 1 => + val name = updateNull.fieldNames + val nullable = if (updateNull.nullable()) "NULL" else "NOT NULL" + updateClause += s"ALTER TABLE $tableName ALTER COLUMN ${name(0)} SET $nullable" + case _ => + throw new SQLFeatureNotSupportedException(s"Unsupported TableChange $change") + } + } + updateClause.result() + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index 0eb96b7813e6e..b308934ba03c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -22,7 +22,7 @@ import java.util.Properties import org.apache.spark.SparkConf import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { @@ -106,4 +106,71 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { Seq(Row("test", "people"), Row("test", "new_table"))) } } + + test("alter table ... add column") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (C1 INTEGER, C2 STRING)") + var t = spark.table("h2.test.alt_table") + var expectedSchema = new StructType() + .add("ID", IntegerType) + .add("C1", IntegerType) + .add("C2", StringType) + assert(t.schema === expectedSchema) + sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (C3 DOUBLE)") + t = spark.table("h2.test.alt_table") + expectedSchema = expectedSchema.add("C3", DoubleType) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... rename column") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table RENAME COLUMN ID TO C") + val t = spark.table("h2.test.alt_table") + val expectedSchema = new StructType().add("C", IntegerType) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... drop column") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (C1 INTEGER, C2 INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table DROP COLUMN C1") + val t = spark.table("h2.test.alt_table") + val expectedSchema = new StructType().add("C2", IntegerType) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... update column type") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN id TYPE DOUBLE") + val t = spark.table("h2.test.alt_table") + val expectedSchema = new StructType().add("ID", DoubleType) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... update column nullability") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER NOT NULL) USING _") + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN ID DROP NOT NULL") + val t = spark.table("h2.test.alt_table") + val expectedSchema = new StructType().add("ID", IntegerType, nullable = true) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... update column comment not supported") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + val thrown = intercept[java.sql.SQLFeatureNotSupportedException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN ID COMMENT 'test'") + } + assert(thrown.getMessage.contains("Unsupported TableChange")) + } + } } From 3a437ed22b96f9928c611acbd27bd7c3352dab60 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 5 Aug 2020 12:03:36 +0000 Subject: [PATCH 370/384] [SPARK-32501][SQL] Convert null to "null" in structs, maps and arrays while casting to strings ### What changes were proposed in this pull request? Convert `NULL` elements of maps, structs and arrays to the `"null"` string while converting maps/struct/array values to strings. The SQL config `spark.sql.legacy.omitNestedNullInCast.enabled` controls the behaviour. When it is `true`, `NULL` elements of structs/maps/arrays will be omitted otherwise, when it is `false`, `NULL` elements will be converted to `"null"`. ### Why are the changes needed? 1. It is impossible to distinguish empty string and null, for instance: ```scala scala> Seq(Seq(""), Seq(null)).toDF().show +-----+ |value| +-----+ | []| | []| +-----+ ``` 2. Inconsistent NULL conversions for top-level values and nested columns, for instance: ```scala scala> sql("select named_struct('c', null), null").show +---------------------+----+ |named_struct(c, NULL)|NULL| +---------------------+----+ | []|null| +---------------------+----+ ``` 3. `.show()` is different from conversions to Hive strings, and as a consequence its output is different from `spark-sql` (sql tests): ```sql spark-sql> select named_struct('c', null) as struct; {"c":null} ``` ```scala scala> sql("select named_struct('c', null) as struct").show +------+ |struct| +------+ | []| +------+ ``` 4. It is impossible to distinguish empty struct/array from struct/array with null in the current implementation: ```scala scala> Seq[Seq[String]](Seq(), Seq(null)).toDF.show() +-----+ |value| +-----+ | []| | []| +-----+ ``` ### Does this PR introduce _any_ user-facing change? Yes, before: ```scala scala> Seq(Seq(""), Seq(null)).toDF().show +-----+ |value| +-----+ | []| | []| +-----+ ``` After: ```scala scala> Seq(Seq(""), Seq(null)).toDF().show +------+ | value| +------+ | []| |[null]| +------+ ``` ### How was this patch tested? By existing test suite `CastSuite`. Closes #29311 from MaxGekk/nested-null-to-string. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + .../spark/sql/catalyst/expressions/Cast.scala | 48 ++++++++++++++----- .../apache/spark/sql/internal/SQLConf.scala | 6 ++- .../sql/catalyst/expressions/CastSuite.scala | 30 +++++++----- 4 files changed, 60 insertions(+), 26 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index f646c09438ad1..c7f6116b88f87 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -34,6 +34,8 @@ license: | - In Spark 3.1, structs and maps are wrapped by the `{}` brackets in casting them to strings. For instance, the `show()` action and the `CAST` expression use such brackets. In Spark 3.0 and earlier, the `[]` brackets are used for the same purpose. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.castComplexTypesToString.enabled` to `true`. + - In Spark 3.1, NULL elements of structures, arrays and maps are converted to "null" in casting them to strings. In Spark 3.0 or earlier, NULL elements are converted to empty strings. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.castComplexTypesToString.enabled` to `true`. + ## Upgrading from Spark SQL 3.0 to 3.0.1 - In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Since version 3.0.1, the timestamp type inference is disabled by default. Set the JSON option `inferTimestamp` to `true` to enable such type inference. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index e93dc588819b0..312c78a2ef1e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -297,9 +297,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) + private val legacyCastToStr = SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING) // The brackets that are used in casting structs and maps to strings - private val (leftBracket, rightBracket) = - if (SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING)) ("[", "]") else ("{", "}") + private val (leftBracket, rightBracket) = if (legacyCastToStr) ("[", "]") else ("{", "}") // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { @@ -321,7 +321,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit var i = 1 while (i < array.numElements) { builder.append(",") - if (!array.isNullAt(i)) { + if (array.isNullAt(i)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(" ") builder.append(toUTF8String(array.get(i, et)).asInstanceOf[UTF8String]) } @@ -342,7 +344,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val valueToUTF8String = castToString(vt) builder.append(keyToUTF8String(keyArray.get(0, kt)).asInstanceOf[UTF8String]) builder.append(" ->") - if (!valueArray.isNullAt(0)) { + if (valueArray.isNullAt(0)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(" ") builder.append(valueToUTF8String(valueArray.get(0, vt)).asInstanceOf[UTF8String]) } @@ -351,7 +355,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit builder.append(", ") builder.append(keyToUTF8String(keyArray.get(i, kt)).asInstanceOf[UTF8String]) builder.append(" ->") - if (!valueArray.isNullAt(i)) { + if (valueArray.isNullAt(i)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(" ") builder.append(valueToUTF8String(valueArray.get(i, vt)) .asInstanceOf[UTF8String]) @@ -369,13 +375,17 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit if (row.numFields > 0) { val st = fields.map(_.dataType) val toUTF8StringFuncs = st.map(castToString) - if (!row.isNullAt(0)) { + if (row.isNullAt(0)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(toUTF8StringFuncs(0)(row.get(0, st(0))).asInstanceOf[UTF8String]) } var i = 1 while (i < row.numFields) { builder.append(",") - if (!row.isNullAt(i)) { + if (row.isNullAt(i)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(" ") builder.append(toUTF8StringFuncs(i)(row.get(i, st(i))).asInstanceOf[UTF8String]) } @@ -895,6 +905,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit """ } + private def outNullElem(buffer: ExprValue): Block = { + if (legacyCastToStr) code"" else code"""$buffer.append(" null");""" + } + private def writeArrayToStringBuilder( et: DataType, array: ExprValue, @@ -917,12 +931,16 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit code""" |$buffer.append("["); |if ($array.numElements() > 0) { - | if (!$array.isNullAt(0)) { + | if ($array.isNullAt(0)) { + | ${outNullElem(buffer)} + | } else { | $buffer.append($elementToStringFunc(${CodeGenerator.getValue(array, et, "0")})); | } | for (int $loopIndex = 1; $loopIndex < $array.numElements(); $loopIndex++) { | $buffer.append(","); - | if (!$array.isNullAt($loopIndex)) { + | if ($array.isNullAt($loopIndex)) { + | ${outNullElem(buffer)} + | } else { | $buffer.append(" "); | $buffer.append($elementToStringFunc(${CodeGenerator.getValue(array, et, loopIndex)})); | } @@ -970,7 +988,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit |if ($map.numElements() > 0) { | $buffer.append($keyToStringFunc($getMapFirstKey)); | $buffer.append(" ->"); - | if (!$map.valueArray().isNullAt(0)) { + | if ($map.valueArray().isNullAt(0)) { + | ${outNullElem(buffer)} + | } else { | $buffer.append(" "); | $buffer.append($valueToStringFunc($getMapFirstValue)); | } @@ -978,7 +998,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit | $buffer.append(", "); | $buffer.append($keyToStringFunc($getMapKeyArray)); | $buffer.append(" ->"); - | if (!$map.valueArray().isNullAt($loopIndex)) { + | if ($map.valueArray().isNullAt($loopIndex)) { + | ${outNullElem(buffer)} + | } else { | $buffer.append(" "); | $buffer.append($valueToStringFunc($getMapValueArray)); | } @@ -1000,7 +1022,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val javaType = JavaCode.javaType(ft) code""" |${if (i != 0) code"""$buffer.append(",");""" else EmptyBlock} - |if (!$row.isNullAt($i)) { + |if ($row.isNullAt($i)) { + | ${outNullElem(buffer)} + |} else { | ${if (i != 0) code"""$buffer.append(" ");""" else EmptyBlock} | | // Append $i field into the string buffer diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f54e0192b6df8..a69f09bdced17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2693,8 +2693,10 @@ object SQLConf { val LEGACY_COMPLEX_TYPES_TO_STRING = buildConf("spark.sql.legacy.castComplexTypesToString.enabled") .internal() - .doc("When true, maps and structs are wrapped by [] in casting to strings. " + - "Otherwise, if this is false, which is the default, maps and structs are wrapped by {}.") + .doc("When true, maps and structs are wrapped by [] in casting to strings, and " + + "NULL elements of structs/maps/arrays will be omitted while converting to strings. " + + "Otherwise, if this is false, which is the default, maps and structs are wrapped by {}, " + + "and NULL elements will be converted to \"null\".") .version("3.1.0") .booleanConf .createWithDefault(false) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index b8b93d929d39d..0e817a7652473 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -691,16 +691,22 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ret1, "[1, 2, 3, 4, 5]") val ret2 = cast(Literal.create(Array("ab", "cde", "f")), StringType) checkEvaluation(ret2, "[ab, cde, f]") - val ret3 = cast(Literal.create(Array("ab", null, "c")), StringType) - checkEvaluation(ret3, "[ab,, c]") - val ret4 = cast(Literal.create(Array("ab".getBytes, "cde".getBytes, "f".getBytes)), StringType) + Seq(false, true).foreach { omitNull => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> omitNull.toString) { + val ret3 = cast(Literal.create(Array("ab", null, "c")), StringType) + checkEvaluation(ret3, s"[ab,${if (omitNull) "" else " null"}, c]") + } + } + val ret4 = + cast(Literal.create(Array("ab".getBytes, "cde".getBytes, "f".getBytes)), StringType) checkEvaluation(ret4, "[ab, cde, f]") val ret5 = cast( Literal.create(Array("2014-12-03", "2014-12-04", "2014-12-06").map(Date.valueOf)), StringType) checkEvaluation(ret5, "[2014-12-03, 2014-12-04, 2014-12-06]") val ret6 = cast( - Literal.create(Array("2014-12-03 13:01:00", "2014-12-04 15:05:00").map(Timestamp.valueOf)), + Literal.create(Array("2014-12-03 13:01:00", "2014-12-04 15:05:00") + .map(Timestamp.valueOf)), StringType) checkEvaluation(ret6, "[2014-12-03 13:01:00, 2014-12-04 15:05:00]") val ret7 = cast(Literal.create(Array(Array(1, 2, 3), Array(4, 5))), StringType) @@ -713,15 +719,15 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-22973 Cast map to string") { Seq( - "false" -> ("{", "}"), - "true" -> ("[", "]")).foreach { case (legacyBrackets, (lb, rb)) => - withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyBrackets) { + false -> ("{", "}"), + true -> ("[", "]")).foreach { case (legacyCast, (lb, rb)) => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyCast.toString) { val ret1 = cast(Literal.create(Map(1 -> "a", 2 -> "b", 3 -> "c")), StringType) checkEvaluation(ret1, s"${lb}1 -> a, 2 -> b, 3 -> c$rb") val ret2 = cast( Literal.create(Map("1" -> "a".getBytes, "2" -> null, "3" -> "c".getBytes)), StringType) - checkEvaluation(ret2, s"${lb}1 -> a, 2 ->, 3 -> c$rb") + checkEvaluation(ret2, s"${lb}1 -> a, 2 ->${if (legacyCast) "" else " null"}, 3 -> c$rb") val ret3 = cast( Literal.create(Map( 1 -> Date.valueOf("2014-12-03"), @@ -747,13 +753,13 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-22981 Cast struct to string") { Seq( - "false" -> ("{", "}"), - "true" -> ("[", "]")).foreach { case (legacyBrackets, (lb, rb)) => - withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyBrackets) { + false -> ("{", "}"), + true -> ("[", "]")).foreach { case (legacyCast, (lb, rb)) => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyCast.toString) { val ret1 = cast(Literal.create((1, "a", 0.1)), StringType) checkEvaluation(ret1, s"${lb}1, a, 0.1$rb") val ret2 = cast(Literal.create(Tuple3[Int, String, String](1, null, "a")), StringType) - checkEvaluation(ret2, s"${lb}1,, a$rb") + checkEvaluation(ret2, s"${lb}1,${if (legacyCast) "" else " null"}, a$rb") val ret3 = cast(Literal.create( (Date.valueOf("2014-12-03"), Timestamp.valueOf("2014-12-03 15:05:00"))), StringType) checkEvaluation(ret3, s"${lb}2014-12-03, 2014-12-03 15:05:00$rb") From 1b6f482adbd5e48e6376ed6896ff968dbe75c1d3 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 5 Aug 2020 13:56:11 +0000 Subject: [PATCH 371/384] [SPARK-32492][SQL][FOLLOWUP][TEST-MAVEN] Fix jenkins maven jobs ### What changes were proposed in this pull request? The newly added test fails Jenkins maven jobs, see https://github.com/apache/spark/pull/29303#discussion_r464729021 We move the test from `ThriftServerWithSparkContextSuite` to `SparkMetadataOperationSuite`, the former uses an embedded thrift server where the server and the client are in the same JVM process and the latter forks a new process to start the server where the server and client are isolated. The sbt runner seems to be fine with the test in the `ThriftServerWithSparkContextSuite`, but the maven runner with `scalates`t plugin will face the classloader issue as we will switch classloader to the one in the `sharedState` which is not the one that hive uses to load some classes. This is more like an issue that belongs to the maven runner or the `scalatest`. So in this PR, we simply move it to bypass the issue. BTW, we should test against the way of using embedded thrift server to verify whether it is just a maven issue or not, there could be some use cases with this API. ### Why are the changes needed? Jenkins recovery ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? modified uts Closes #29347 from yaooqinn/SPARK-32492-F. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../SparkMetadataOperationSuite.scala | 86 ++++++++++++ .../ThriftServerWithSparkContextSuite.scala | 131 ------------------ 2 files changed, 86 insertions(+), 131 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index f7ee3e0a46cd1..5df337044480e 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{DatabaseMetaData, ResultSet} +import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, DecimalType, DoubleType, FloatType, IntegerType, MapType, NumericType, StringType, StructType, TimestampType} + class SparkMetadataOperationSuite extends HiveThriftJdbcTest { override def mode: ServerMode.Value = ServerMode.binary @@ -247,4 +249,88 @@ class SparkMetadataOperationSuite extends HiveThriftJdbcTest { checkResult(metaData.getTypeInfo, ThriftserverShimUtils.supportedType().map(_.getName)) } } + + test("check results from get columns operation from thrift server") { + val schemaName = "default" + val tableName = "spark_get_col_operation" + val schema = new StructType() + .add("c0", "boolean", nullable = false, "0") + .add("c1", "tinyint", nullable = true, "1") + .add("c2", "smallint", nullable = false, "2") + .add("c3", "int", nullable = true, "3") + .add("c4", "long", nullable = false, "4") + .add("c5", "float", nullable = true, "5") + .add("c6", "double", nullable = false, "6") + .add("c7", "decimal(38, 20)", nullable = true, "7") + .add("c8", "decimal(10, 2)", nullable = false, "8") + .add("c9", "string", nullable = true, "9") + .add("c10", "array", nullable = false, "10") + .add("c11", "array", nullable = true, "11") + .add("c12", "map", nullable = false, "12") + .add("c13", "date", nullable = true, "13") + .add("c14", "timestamp", nullable = false, "14") + .add("c15", "struct", nullable = true, "15") + .add("c16", "binary", nullable = false, "16") + + val ddl = + s""" + |CREATE TABLE $schemaName.$tableName ( + | ${schema.toDDL} + |) + |using parquet""".stripMargin + + withJdbcStatement(tableName) { statement => + statement.execute(ddl) + + val databaseMetaData = statement.getConnection.getMetaData + val rowSet = databaseMetaData.getColumns("", schemaName, tableName, null) + + import java.sql.Types._ + val expectedJavaTypes = Seq(BOOLEAN, TINYINT, SMALLINT, INTEGER, BIGINT, FLOAT, DOUBLE, + DECIMAL, DECIMAL, VARCHAR, ARRAY, ARRAY, JAVA_OBJECT, DATE, TIMESTAMP, STRUCT, BINARY) + + var pos = 0 + + while (rowSet.next()) { + assert(rowSet.getString("TABLE_CAT") === null) + assert(rowSet.getString("TABLE_SCHEM") === schemaName) + assert(rowSet.getString("TABLE_NAME") === tableName) + assert(rowSet.getString("COLUMN_NAME") === schema(pos).name) + assert(rowSet.getInt("DATA_TYPE") === expectedJavaTypes(pos)) + assert(rowSet.getString("TYPE_NAME") === schema(pos).dataType.sql) + + val colSize = rowSet.getInt("COLUMN_SIZE") + schema(pos).dataType match { + case StringType | BinaryType | _: ArrayType | _: MapType => assert(colSize === 0) + case o => assert(colSize === o.defaultSize) + } + + assert(rowSet.getInt("BUFFER_LENGTH") === 0) // not used + val decimalDigits = rowSet.getInt("DECIMAL_DIGITS") + schema(pos).dataType match { + case BooleanType | _: IntegerType => assert(decimalDigits === 0) + case d: DecimalType => assert(decimalDigits === d.scale) + case FloatType => assert(decimalDigits === 7) + case DoubleType => assert(decimalDigits === 15) + case TimestampType => assert(decimalDigits === 6) + case _ => assert(decimalDigits === 0) // nulls + } + + val radix = rowSet.getInt("NUM_PREC_RADIX") + schema(pos).dataType match { + case _: NumericType => assert(radix === 10) + case _ => assert(radix === 0) // nulls + } + + assert(rowSet.getInt("NULLABLE") === 1) + assert(rowSet.getString("REMARKS") === pos.toString) + assert(rowSet.getInt("ORDINAL_POSITION") === pos) + assert(rowSet.getString("IS_NULLABLE") === "YES") + assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO") + pos += 1 + } + + assert(pos === 17, "all columns should have been verified") + } + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 196c093aafddf..102fd77c06f3a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.SQLException -import scala.collection.JavaConverters._ - import org.apache.hive.service.cli.HiveSQLException import org.apache.spark.sql.hive.HiveUtils @@ -104,135 +102,6 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } - - test("check results from get columns operation from thrift server") { - val schemaName = "default" - val tableName = "spark_get_col_operation" - val schema = new StructType() - .add("c0", "boolean", nullable = false, "0") - .add("c1", "tinyint", nullable = true, "1") - .add("c2", "smallint", nullable = false, "2") - .add("c3", "int", nullable = true, "3") - .add("c4", "long", nullable = false, "4") - .add("c5", "float", nullable = true, "5") - .add("c6", "double", nullable = false, "6") - .add("c7", "decimal(38, 20)", nullable = true, "7") - .add("c8", "decimal(10, 2)", nullable = false, "8") - .add("c9", "string", nullable = true, "9") - .add("c10", "array", nullable = false, "10") - .add("c11", "array", nullable = true, "11") - .add("c12", "map", nullable = false, "12") - .add("c13", "date", nullable = true, "13") - .add("c14", "timestamp", nullable = false, "14") - .add("c15", "struct", nullable = true, "15") - .add("c16", "binary", nullable = false, "16") - - val ddl = - s""" - |CREATE TABLE $schemaName.$tableName ( - | ${schema.toDDL} - |) - |using parquet""".stripMargin - - withCLIServiceClient { client => - val sessionHandle = client.openSession(user, "") - val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] - val opHandle = client.executeStatement(sessionHandle, ddl, confOverlay) - var status = client.getOperationStatus(opHandle) - while (!status.getState.isTerminal) { - Thread.sleep(10) - status = client.getOperationStatus(opHandle) - } - val getCol = client.getColumns(sessionHandle, "", schemaName, tableName, null) - val rowSet = client.fetchResults(getCol) - val columns = rowSet.toTRowSet.getColumns - - val catalogs = columns.get(0).getStringVal.getValues.asScala - assert(catalogs.forall(_.isEmpty), "catalog name mismatches") - - val schemas = columns.get(1).getStringVal.getValues.asScala - assert(schemas.forall(_ == schemaName), "schema name mismatches") - - val tableNames = columns.get(2).getStringVal.getValues.asScala - assert(tableNames.forall(_ == tableName), "table name mismatches") - - val columnNames = columns.get(3).getStringVal.getValues.asScala - columnNames.zipWithIndex.foreach { - case (v, i) => assert(v === "c" + i, "column name mismatches") - } - - val javaTypes = columns.get(4).getI32Val.getValues - import java.sql.Types._ - assert(javaTypes.get(0).intValue() === BOOLEAN) - assert(javaTypes.get(1).intValue() === TINYINT) - assert(javaTypes.get(2).intValue() === SMALLINT) - assert(javaTypes.get(3).intValue() === INTEGER) - assert(javaTypes.get(4).intValue() === BIGINT) - assert(javaTypes.get(5).intValue() === FLOAT) - assert(javaTypes.get(6).intValue() === DOUBLE) - assert(javaTypes.get(7).intValue() === DECIMAL) - assert(javaTypes.get(8).intValue() === DECIMAL) - assert(javaTypes.get(9).intValue() === VARCHAR) - assert(javaTypes.get(10).intValue() === ARRAY) - assert(javaTypes.get(11).intValue() === ARRAY) - assert(javaTypes.get(12).intValue() === JAVA_OBJECT) - assert(javaTypes.get(13).intValue() === DATE) - assert(javaTypes.get(14).intValue() === TIMESTAMP) - assert(javaTypes.get(15).intValue() === STRUCT) - assert(javaTypes.get(16).intValue() === BINARY) - - val typeNames = columns.get(5).getStringVal.getValues.asScala - typeNames.zip(schema).foreach { case (tName, f) => - assert(tName === f.dataType.sql) - } - - val colSize = columns.get(6).getI32Val.getValues.asScala - - colSize.zip(schema).foreach { case (size, f) => - f.dataType match { - case StringType | BinaryType | _: ArrayType | _: MapType => assert(size === 0) - case o => assert(size === o.defaultSize) - } - } - - val decimalDigits = columns.get(8).getI32Val.getValues.asScala - decimalDigits.zip(schema).foreach { case (dd, f) => - f.dataType match { - case BooleanType | _: IntegerType => assert(dd === 0) - case d: DecimalType => assert(dd === d.scale) - case FloatType => assert(dd === 7) - case DoubleType => assert(dd === 15) - case TimestampType => assert(dd === 6) - case _ => assert(dd === 0) // nulls - } - } - - val radixes = columns.get(9).getI32Val.getValues.asScala - radixes.zip(schema).foreach { case (radix, f) => - f.dataType match { - case _: NumericType => assert(radix === 10) - case _ => assert(radix === 0) // nulls - } - } - - val nullables = columns.get(10).getI32Val.getValues.asScala - assert(nullables.forall(_ === 1)) - - val comments = columns.get(11).getStringVal.getValues.asScala - comments.zip(schema).foreach { case (c, f) => assert(c === f.getComment().get) } - - val positions = columns.get(16).getI32Val.getValues.asScala - positions.zipWithIndex.foreach { case (pos, idx) => - assert(pos === idx, "the client columns disorder") - } - - val isNullables = columns.get(17).getStringVal.getValues.asScala - assert(isNullables.forall(_ === "YES")) - - val autoIncs = columns.get(22).getStringVal.getValues.asScala - assert(autoIncs.forall(_ === "NO")) - } - } } From 4a0427cbc1b557a3e08135756bf089d882a0994f Mon Sep 17 00:00:00 2001 From: Michael Munday Date: Wed, 5 Aug 2020 16:11:09 +0000 Subject: [PATCH 372/384] [SPARK-32485][SQL][TEST] Fix endianness issues in tests in RecordBinaryComparatorSuite ### What changes were proposed in this pull request? PR #26548 means that RecordBinaryComparator now uses big endian byte order for long comparisons. However, this means that some of the constants in the regression tests no longer map to the same values in the comparison that they used to. For example, one of the tests does a comparison between Long.MIN_VALUE and 1 in order to trigger an overflow condition that existed in the past (i.e. Long.MIN_VALUE - 1). These constants correspond to the values 0x80..00 and 0x00..01. However on a little-endian machine the bytes in these values are now swapped before they are compared. This means that we will now be comparing 0x00..80 with 0x01..00. 0x00..80 - 0x01..00 does not overflow therefore missing the original purpose of the test. To fix this the constants are now explicitly written out in big endian byte order to match the byte order used in the comparison. This also fixes the tests on big endian machines (which would otherwise get a different comparison result to the little-endian machines). ### Why are the changes needed? The regression tests no longer serve their initial purposes and also fail on big-endian systems. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Tests run on big-endian system (s390x). Closes #29259 from mundaym/fix-endian. Authored-by: Michael Munday Signed-off-by: Wenchen Fan --- .../sort/RecordBinaryComparatorSuite.java | 48 ++++++++++++++++--- 1 file changed, 42 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java index 564e76737ecde..6cb7c40e3332b 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java @@ -37,6 +37,8 @@ import org.junit.Before; import org.junit.Test; +import java.nio.ByteOrder; + /** * Test the RecordBinaryComparator, which compares two UnsafeRows by their binary form. */ @@ -261,40 +263,74 @@ public void testBinaryComparatorForNullColumns() throws Exception { public void testBinaryComparatorWhenSubtractionIsDivisibleByMaxIntValue() throws Exception { int numFields = 1; + // Place the following bytes (hex) into UnsafeRows for the comparison: + // + // index | 00 01 02 03 04 05 06 07 + // ------+------------------------ + // row1 | 00 00 00 00 00 00 00 0b + // row2 | 00 00 00 00 80 00 00 0a + // + // The byte layout needs to be identical on all platforms regardless of + // of endianness. To achieve this the bytes in each value are reversed + // on little-endian platforms. + long row1Data = 11L; + long row2Data = 11L + Integer.MAX_VALUE; + if (ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN)) { + row1Data = Long.reverseBytes(row1Data); + row2Data = Long.reverseBytes(row2Data); + } + UnsafeRow row1 = new UnsafeRow(numFields); byte[] data1 = new byte[100]; row1.pointTo(data1, computeSizeInBytes(numFields * 8)); - row1.setLong(0, 11); + row1.setLong(0, row1Data); UnsafeRow row2 = new UnsafeRow(numFields); byte[] data2 = new byte[100]; row2.pointTo(data2, computeSizeInBytes(numFields * 8)); - row2.setLong(0, 11L + Integer.MAX_VALUE); + row2.setLong(0, row2Data); insertRow(row1); insertRow(row2); - Assert.assertTrue(compare(0, 1) > 0); + Assert.assertTrue(compare(0, 1) < 0); } @Test public void testBinaryComparatorWhenSubtractionCanOverflowLongValue() throws Exception { int numFields = 1; + // Place the following bytes (hex) into UnsafeRows for the comparison: + // + // index | 00 01 02 03 04 05 06 07 + // ------+------------------------ + // row1 | 80 00 00 00 00 00 00 00 + // row2 | 00 00 00 00 00 00 00 01 + // + // The byte layout needs to be identical on all platforms regardless of + // of endianness. To achieve this the bytes in each value are reversed + // on little-endian platforms. + long row1Data = Long.MIN_VALUE; + long row2Data = 1L; + if (ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN)) { + row1Data = Long.reverseBytes(row1Data); + row2Data = Long.reverseBytes(row2Data); + } + UnsafeRow row1 = new UnsafeRow(numFields); byte[] data1 = new byte[100]; row1.pointTo(data1, computeSizeInBytes(numFields * 8)); - row1.setLong(0, Long.MIN_VALUE); + row1.setLong(0, row1Data); UnsafeRow row2 = new UnsafeRow(numFields); byte[] data2 = new byte[100]; row2.pointTo(data2, computeSizeInBytes(numFields * 8)); - row2.setLong(0, 1); + row2.setLong(0, row2Data); insertRow(row1); insertRow(row2); - Assert.assertTrue(compare(0, 1) < 0); + Assert.assertTrue(compare(0, 1) > 0); } @Test From 42219af9062d4ea524a13c2a6ea40d0d99f96c66 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 5 Aug 2020 10:35:03 -0700 Subject: [PATCH 373/384] [SPARK-32543][R] Remove arrow::as_tibble usage in SparkR ### What changes were proposed in this pull request? SparkR increased the minimal version of Arrow R version to 1.0.0 at SPARK-32452, and Arrow R 0.14 dropped `as_tibble`. We can remove the usage in SparkR. ### Why are the changes needed? To remove codes unused anymore. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? GitHub Actions will test them out. Closes #29361 from HyukjinKwon/SPARK-32543. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- R/pkg/R/DataFrame.R | 7 +------ R/pkg/R/deserialize.R | 13 +------------ 2 files changed, 2 insertions(+), 18 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 4d38f979c4103..4eca5bd23c3bb 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1234,12 +1234,7 @@ setMethod("collect", output <- tryCatch({ doServerAuth(conn, authSecret) arrowTable <- arrow::read_ipc_stream(readRaw(conn)) - # Arrow drops `as_tibble` since 0.14.0, see ARROW-5190. - if (exists("as_tibble", envir = asNamespace("arrow"))) { - as.data.frame(arrow::as_tibble(arrowTable), stringsAsFactors = stringsAsFactors) - } else { - as.data.frame(arrowTable, stringsAsFactors = stringsAsFactors) - } + as.data.frame(arrowTable, stringsAsFactors = stringsAsFactors) }, finally = { close(conn) }) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index 3e7c456bd548d..5d22340fb62a0 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -233,24 +233,13 @@ readMultipleObjectsWithKeys <- function(inputCon) { readDeserializeInArrow <- function(inputCon) { if (requireNamespace("arrow", quietly = TRUE)) { - # Arrow drops `as_tibble` since 0.14.0, see ARROW-5190. - useAsTibble <- exists("as_tibble", envir = asNamespace("arrow")) - - # Currently, there looks no way to read batch by batch by socket connection in R side, # See ARROW-4512. Therefore, it reads the whole Arrow streaming-formatted binary at once # for now. dataLen <- readInt(inputCon) arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") batches <- arrow::RecordBatchStreamReader$create(arrowData)$batches() - - if (useAsTibble) { - as_tibble <- get("as_tibble", envir = asNamespace("arrow")) - # Read all groupped batches. Tibble -> data.frame is cheap. - lapply(batches, function(batch) as.data.frame(as_tibble(batch))) - } else { - lapply(batches, function(batch) as.data.frame(batch)) - } + lapply(batches, function(batch) as.data.frame(batch)) } else { stop("'arrow' package should be installed.") } From c1d17df826541580162c9db8ebfbc408ec0c9922 Mon Sep 17 00:00:00 2001 From: Yan Xiaole Date: Wed, 5 Aug 2020 10:57:11 -0700 Subject: [PATCH 374/384] [SPARK-32529][CORE] Fix Historyserver log scan aborted by application status change # What changes were proposed in this pull request? This PR adds a `FileNotFoundException` try catch block while adding a new entry to history server application listing to skip the non-existing path. ### Why are the changes needed? If there are a large number (>100k) of applications log dir, listing the log dir will take a few seconds. After getting the path list some applications might have finished already, and the filename will change from `foo.inprogress` to `foo`. It leads to a problem when adding an entry to the listing, querying file status like `fileSizeForLastIndex` will throw out a `FileNotFoundException` exception if the application was finished. And the exception will abort current loop, in a busy cluster, it will make history server couldn't list and load any application log. ``` 20/08/03 15:17:23 ERROR FsHistoryProvider: Exception in checking for event log updates java.io.FileNotFoundException: File does not exist: hdfs://xx/logs/spark/application_11111111111111.lz4.inprogress at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1527) at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1520) at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1520) at org.apache.spark.deploy.history.SingleFileEventLogFileReader.status$lzycompute(EventLogFileReaders.scala:170) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? 1. setup another script keeps changing the filename of applications under history log dir 2. launch the history server 3. check whether the `File does not exist` error log was gone. Closes #29350 from yanxiaole/SPARK-32529. Authored-by: Yan Xiaole Signed-off-by: Dongjoon Hyun --- .../spark/deploy/history/FsHistoryProvider.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index a73a5e9463204..bc3e53c4122e7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -530,10 +530,17 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // If the file is currently not being tracked by the SHS, add an entry for it and try // to parse it. This will allow the cleaner code to detect the file as stale later on // if it was not possible to parse it. - listing.write(LogInfo(reader.rootPath.toString(), newLastScanTime, LogType.EventLogs, - None, None, reader.fileSizeForLastIndex, reader.lastIndex, None, - reader.completed)) - reader.fileSizeForLastIndex > 0 + try { + listing.write(LogInfo(reader.rootPath.toString(), newLastScanTime, + LogType.EventLogs, None, None, reader.fileSizeForLastIndex, reader.lastIndex, + None, reader.completed)) + reader.fileSizeForLastIndex > 0 + } catch { + case _: FileNotFoundException => false + } + + case _: FileNotFoundException => + false } } .sortWith { case (entry1, entry2) => From 375d348a83e6ffa38dfaece5047633f67aee1da5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 5 Aug 2020 16:28:14 -0700 Subject: [PATCH 375/384] [SPARK-31197][CORE] Shutdown executor once we are done decommissioning ### What changes were proposed in this pull request? Exit the executor when it has been asked to decommission and there is nothing left for it to do. This is a rebase of https://github.com/apache/spark/pull/28817 ### Why are the changes needed? If we want to use decommissioning in Spark's own scale down we should terminate the executor once finished. Furthermore, in graceful shutdown it makes sense to release resources we no longer need if we've been asked to shutdown by the cluster manager instead of always holding the resources as long as possible. ### Does this PR introduce _any_ user-facing change? The decommissioned executors will exit and the end of decommissioning. This is sort of a user facing change, however decommissioning hasn't been in any releases yet. ### How was this patch tested? I changed the unit test to not send the executor exit message and still wait on the executor exited message. Closes #29211 from holdenk/SPARK-31197-exit-execs-redone. Authored-by: Holden Karau Signed-off-by: Holden Karau --- .../apache/spark/deploy/DeployMessage.scala | 2 - .../apache/spark/deploy/worker/Worker.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 58 +++++++- .../cluster/CoarseGrainedClusterMessage.scala | 3 + .../CoarseGrainedSchedulerBackend.scala | 10 ++ .../apache/spark/storage/BlockManager.scala | 8 + .../storage/BlockManagerDecommissioner.scala | 96 ++++++++++-- .../scheduler/WorkerDecommissionSuite.scala | 19 +-- ...kManagerDecommissionIntegrationSuite.scala | 17 +-- .../BlockManagerDecommissionUnitSuite.scala | 139 +++++++++++++++++- 10 files changed, 310 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index c8c6e5a192a24..b7a64d75a8d47 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -165,8 +165,6 @@ private[deploy] object DeployMessages { case object ReregisterWithMaster // used when a worker attempts to reconnect to a master - case object DecommissionSelf // Mark as decommissioned. May be Master to Worker in the future. - // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription, driver: RpcEndpointRef) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index aa8c46fc68315..862e685c2dce6 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -668,7 +668,7 @@ private[deploy] class Worker( finishedApps += id maybeCleanupApplication(id) - case DecommissionSelf => + case WorkerDecommission(_, _) => decommissionSelf() } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index def125bb6bfb6..55fb76b3572a3 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -64,7 +64,6 @@ private[spark] class CoarseGrainedExecutorBackend( private[this] val stopping = new AtomicBoolean(false) var executor: Executor = null - @volatile private var decommissioned = false @volatile var driver: Option[RpcEndpointRef] = None // If this CoarseGrainedExecutorBackend is changed to support multiple threads, then this may need @@ -80,6 +79,8 @@ private[spark] class CoarseGrainedExecutorBackend( */ private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] + @volatile private var decommissioned = false + override def onStart(): Unit = { logInfo("Registering PWR handler.") SignalUtils.register("PWR", "Failed to register SIGPWR handler - " + @@ -214,6 +215,10 @@ private[spark] class CoarseGrainedExecutorBackend( case UpdateDelegationTokens(tokenBytes) => logInfo(s"Received tokens of ${tokenBytes.length} bytes") SparkHadoopUtil.get.addDelegationTokens(tokenBytes, env.conf) + + case DecommissionSelf => + logInfo("Received decommission self") + decommissionSelf() } override def onDisconnected(remoteAddress: RpcAddress): Unit = { @@ -277,12 +282,59 @@ private[spark] class CoarseGrainedExecutorBackend( if (executor != null) { executor.decommission() } - logInfo("Done decommissioning self.") + // Shutdown the executor once all tasks are gone & any configured migrations completed. + // Detecting migrations completion doesn't need to be perfect and we want to minimize the + // overhead for executors that are not in decommissioning state as overall that will be + // more of the executors. For example, this will not catch a block which is already in + // the process of being put from a remote executor before migration starts. This trade-off + // is viewed as acceptable to minimize introduction of any new locking structures in critical + // code paths. + + val shutdownThread = new Thread("wait-for-blocks-to-migrate") { + override def run(): Unit = { + var lastTaskRunningTime = System.nanoTime() + val sleep_time = 1000 // 1s + + while (true) { + logInfo("Checking to see if we can shutdown.") + Thread.sleep(sleep_time) + if (executor == null || executor.numRunningTasks == 0) { + if (env.conf.get(STORAGE_DECOMMISSION_ENABLED)) { + logInfo("No running tasks, checking migrations") + val (migrationTime, allBlocksMigrated) = env.blockManager.lastMigrationInfo() + // We can only trust allBlocksMigrated boolean value if there were no tasks running + // since the start of computing it. + if (allBlocksMigrated && (migrationTime > lastTaskRunningTime)) { + logInfo("No running tasks, all blocks migrated, stopping.") + exitExecutor(0, "Finished decommissioning", notifyDriver = true) + } else { + logInfo("All blocks not yet migrated.") + } + } else { + logInfo("No running tasks, no block migration configured, stopping.") + exitExecutor(0, "Finished decommissioning", notifyDriver = true) + } + } else { + logInfo("Blocked from shutdown by running ${executor.numRunningtasks} tasks") + // If there is a running task it could store blocks, so make sure we wait for a + // migration loop to complete after the last task is done. + // Note: this is only advanced if there is a running task, if there + // is no running task but the blocks are not done migrating this does not + // move forward. + lastTaskRunningTime = System.nanoTime() + } + } + } + } + shutdownThread.setDaemon(true) + shutdownThread.start() + + logInfo("Will exit when finished decommissioning") // Return true since we are handling a signal true } catch { case e: Exception => - logError(s"Error ${e} during attempt to decommission self") + logError("Unexpected error while decommissioning self", e) false } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 91485f01bf007..7242ab7786061 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -136,4 +136,7 @@ private[spark] object CoarseGrainedClusterMessages { // The message to check if `CoarseGrainedSchedulerBackend` thinks the executor is alive or not. case class IsExecutorAlive(executorId: String) extends CoarseGrainedClusterMessage + + // Used to ask an executor to decommission itself. (Can be an internal message) + case object DecommissionSelf extends CoarseGrainedClusterMessage } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 8fbefae58af14..d81a617d0ed7d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -442,6 +442,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case e: Exception => logError(s"Unexpected error during decommissioning ${e.toString}", e) } + // Send decommission message to the executor, this may be a duplicate since the executor + // could have been the one to notify us. But it's also possible the notification came from + // elsewhere and the executor does not yet know. + executorDataMap.get(executorId) match { + case Some(executorInfo) => + executorInfo.executorEndpoint.send(DecommissionSelf) + case None => + // Ignoring the executor since it is not registered. + logWarning(s"Attempted to decommission unknown executor $executorId.") + } logInfo(s"Finished decommissioning executor $executorId.") if (conf.get(STORAGE_DECOMMISSION_ENABLED)) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 47af854b6e8ff..6ec93df67f7db 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1822,6 +1822,14 @@ private[spark] class BlockManager( } } + /* + * Returns the last migration time and a boolean denoting if all the blocks have been migrated. + * If there are any tasks running since that time the boolean may be incorrect. + */ + private[spark] def lastMigrationInfo(): (Long, Boolean) = { + decommissioner.map(_.lastMigrationInfo()).getOrElse((0, false)) + } + private[storage] def getMigratableRDDBlocks(): Seq[ReplicateBlock] = master.getReplicateInfoForRDDBlocks(blockManagerId) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 1cc7ef6a25f92..f0a8e47aa3200 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.util.concurrent.ExecutorService +import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConverters._ import scala.collection.mutable @@ -41,6 +42,12 @@ private[storage] class BlockManagerDecommissioner( private val maxReplicationFailuresForDecommission = conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) + // Used for tracking if our migrations are complete. Readable for testing + @volatile private[storage] var lastRDDMigrationTime: Long = 0 + @volatile private[storage] var lastShuffleMigrationTime: Long = 0 + @volatile private[storage] var rddBlocksLeft: Boolean = true + @volatile private[storage] var shuffleBlocksLeft: Boolean = true + /** * This runnable consumes any shuffle blocks in the queue for migration. This part of a * producer/consumer where the main migration loop updates the queue of blocks to be migrated @@ -91,10 +98,11 @@ private[storage] class BlockManagerDecommissioner( null)// class tag, we don't need for shuffle logDebug(s"Migrated sub block ${blockId}") } - logInfo(s"Migrated ${shuffleBlockInfo} to ${peer}") + logDebug(s"Migrated ${shuffleBlockInfo} to ${peer}") } else { logError(s"Skipping block ${shuffleBlockInfo} because it has failed ${retryCount}") } + numMigratedShuffles.incrementAndGet() } } // This catch is intentionally outside of the while running block. @@ -115,12 +123,21 @@ private[storage] class BlockManagerDecommissioner( // Shuffles which are either in queue for migrations or migrated private val migratingShuffles = mutable.HashSet[ShuffleBlockInfo]() + // Shuffles which have migrated. This used to know when we are "done", being done can change + // if a new shuffle file is created by a running task. + private val numMigratedShuffles = new AtomicInteger(0) + // Shuffles which are queued for migration & number of retries so far. + // Visible in storage for testing. private[storage] val shufflesToMigrate = new java.util.concurrent.ConcurrentLinkedQueue[(ShuffleBlockInfo, Int)]() // Set if we encounter an error attempting to migrate and stop. @volatile private var stopped = false + @volatile private var stoppedRDD = + !conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED) + @volatile private var stoppedShuffle = + !conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED) private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]() @@ -133,22 +150,31 @@ private[storage] class BlockManagerDecommissioner( override def run(): Unit = { assert(conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) - while (!stopped && !Thread.interrupted()) { + while (!stopped && !stoppedRDD && !Thread.interrupted()) { logInfo("Iterating on migrating from the block manager.") + // Validate we have peers to migrate to. + val peers = bm.getPeers(false) + // If we have no peers give up. + if (peers.isEmpty) { + stopped = true + stoppedRDD = true + } try { + val startTime = System.nanoTime() logDebug("Attempting to replicate all cached RDD blocks") - decommissionRddCacheBlocks() + rddBlocksLeft = decommissionRddCacheBlocks() + lastRDDMigrationTime = startTime logInfo("Attempt to replicate all cached blocks done") logInfo(s"Waiting for ${sleepInterval} before refreshing migrations.") Thread.sleep(sleepInterval) } catch { case e: InterruptedException => - logInfo("Interrupted during migration, will not refresh migrations.") - stopped = true + logInfo("Interrupted during RDD migration, stopping") + stoppedRDD = true case NonFatal(e) => - logError("Error occurred while trying to replicate for block manager decommissioning.", + logError("Error occurred replicating RDD for block manager decommissioning.", e) - stopped = true + stoppedRDD = true } } } @@ -162,20 +188,22 @@ private[storage] class BlockManagerDecommissioner( override def run() { assert(conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) - while (!stopped && !Thread.interrupted()) { + while (!stopped && !stoppedShuffle && !Thread.interrupted()) { try { logDebug("Attempting to replicate all shuffle blocks") - refreshOffloadingShuffleBlocks() + val startTime = System.nanoTime() + shuffleBlocksLeft = refreshOffloadingShuffleBlocks() + lastShuffleMigrationTime = startTime logInfo("Done starting workers to migrate shuffle blocks") Thread.sleep(sleepInterval) } catch { case e: InterruptedException => logInfo("Interrupted during migration, will not refresh migrations.") - stopped = true + stoppedShuffle = true case NonFatal(e) => logError("Error occurred while trying to replicate for block manager decommissioning.", e) - stopped = true + stoppedShuffle = true } } } @@ -191,8 +219,9 @@ private[storage] class BlockManagerDecommissioner( * but rather shadows them. * Requires an Indexed based shuffle resolver. * Note: if called in testing please call stopOffloadingShuffleBlocks to avoid thread leakage. + * Returns true if we are not done migrating shuffle blocks. */ - private[storage] def refreshOffloadingShuffleBlocks(): Unit = { + private[storage] def refreshOffloadingShuffleBlocks(): Boolean = { // Update the queue of shuffles to be migrated logInfo("Offloading shuffle blocks") val localShuffles = bm.migratableResolver.getStoredShuffles().toSet @@ -215,6 +244,12 @@ private[storage] class BlockManagerDecommissioner( deadPeers.foreach { peer => migrationPeers.get(peer).foreach(_.running = false) } + // If we don't have anyone to migrate to give up + if (migrationPeers.values.find(_.running == true).isEmpty) { + stoppedShuffle = true + } + // If we found any new shuffles to migrate or otherwise have not migrated everything. + newShufflesToMigrate.nonEmpty || migratingShuffles.size < numMigratedShuffles.get() } /** @@ -231,16 +266,18 @@ private[storage] class BlockManagerDecommissioner( /** * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers * Visible for testing + * Returns true if we have not migrated all of our RDD blocks. */ - private[storage] def decommissionRddCacheBlocks(): Unit = { + private[storage] def decommissionRddCacheBlocks(): Boolean = { val replicateBlocksInfo = bm.getMigratableRDDBlocks() + // Refresh peers and validate we have somewhere to move blocks. if (replicateBlocksInfo.nonEmpty) { logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + "for block manager decommissioning") } else { logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") - return + return false } // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) @@ -252,7 +289,9 @@ private[storage] class BlockManagerDecommissioner( if (blocksFailedReplication.nonEmpty) { logWarning("Blocks failed replication in cache decommissioning " + s"process: ${blocksFailedReplication.mkString(",")}") + return true } + return false } private def migrateBlock(blockToReplicate: ReplicateBlock): Boolean = { @@ -327,4 +366,33 @@ private[storage] class BlockManagerDecommissioner( } logInfo("Stopped storage decommissioner") } + + /* + * Returns the last migration time and a boolean for if all blocks have been migrated. + * The last migration time is calculated to be the minimum of the last migration of any + * running migration (and if there are now current running migrations it is set to current). + * This provides a timeStamp which, if there have been no tasks running since that time + * we can know that all potential blocks that can be have been migrated off. + */ + private[storage] def lastMigrationInfo(): (Long, Boolean) = { + if (stopped || (stoppedRDD && stoppedShuffle)) { + // Since we don't have anything left to migrate ever (since we don't restart once + // stopped), return that we're done with a validity timestamp that doesn't expire. + (Long.MaxValue, true) + } else { + // Chose the min of the active times. See the function description for more information. + val lastMigrationTime = if (!stoppedRDD && !stoppedShuffle) { + Math.min(lastRDDMigrationTime, lastShuffleMigrationTime) + } else if (!stoppedShuffle) { + lastShuffleMigrationTime + } else { + lastRDDMigrationTime + } + + // Technically we could have blocks left if we encountered an error, but those blocks will + // never be migrated, so we don't care about them. + val blocksMigrated = (!shuffleBlocksLeft || stoppedShuffle) && (!rddBlocksLeft || stoppedRDD) + (lastMigrationTime, blocksMigrated) + } + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala index 3c34070e8bb97..bb0c33acc0af5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala @@ -47,7 +47,12 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { assert(sleepyRdd.count() === 10) } - test("verify a task with all workers decommissioned succeeds") { + test("verify a running task with all workers decommissioned succeeds") { + // Wait for the executors to come up + TestUtils.waitUntilExecutorsUp(sc = sc, + numExecutors = 2, + timeout = 30000) // 30s + val input = sc.parallelize(1 to 10) // Listen for the job val sem = new Semaphore(0) @@ -56,9 +61,7 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { sem.release() } }) - TestUtils.waitUntilExecutorsUp(sc = sc, - numExecutors = 2, - timeout = 30000) // 30s + val sleepyRdd = input.mapPartitions{ x => Thread.sleep(5000) // 5s x @@ -76,13 +79,5 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { execs.foreach(execId => sched.decommissionExecutor(execId, ExecutorDecommissionInfo("", false))) val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 20.seconds) assert(asyncCountResult === 10) - // Try and launch task after decommissioning, this should fail - val postDecommissioned = input.map(x => x) - val postDecomAsyncCount = postDecommissioned.countAsync() - val thrown = intercept[java.util.concurrent.TimeoutException]{ - val result = ThreadUtils.awaitResult(postDecomAsyncCount, 20.seconds) - } - assert(postDecomAsyncCount.isCompleted === false, - "After exec decommission new task could not launch") } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index 6a52f72938c6c..25145dac52681 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, Semaphore} +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, Semaphore, TimeUnit} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -69,9 +69,9 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS .set(config.STORAGE_DECOMMISSION_ENABLED, true) .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, persist) .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, shuffle) - // Just replicate blocks as fast as we can during testing, there isn't another + // Just replicate blocks quickly during testing, there isn't another // workload we need to worry about. - .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 1L) + .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) if (whenToDecom == TaskStarted) { // We are using accumulators below, make sure those are reported frequently. @@ -266,18 +266,17 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS val execIdToBlocksMapping = storageStatus.map( status => (status.blockManagerId.executorId, status.blocks)).toMap // No cached blocks should be present on executor which was decommissioned - assert(execIdToBlocksMapping(execToDecommission).keys.filter(_.isRDD).toSeq === Seq(), + assert( + !execIdToBlocksMapping.contains(execToDecommission) || + execIdToBlocksMapping(execToDecommission).keys.filter(_.isRDD).toSeq === Seq(), "Cache blocks should be migrated") if (persist) { // There should still be all the RDD blocks cached assert(execIdToBlocksMapping.values.flatMap(_.keys).count(_.isRDD) === numParts) } - // Make the executor we decommissioned exit - sched.client.killExecutors(List(execToDecommission)) - - // Wait for the executor to be removed - executorRemovedSem.acquire(1) + // Wait for the executor to be removed automatically after migration. + assert(executorRemovedSem.tryAcquire(1, 5L, TimeUnit.MINUTES)) // Since the RDD is cached or shuffled so further usage of same RDD should use the // cached data. Original RDD partitions should not be recomputed i.e. accum diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala index 41b68d5978d16..74ad8bd2bcf9d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import scala.concurrent.duration._ import org.mockito.{ArgumentMatchers => mc} -import org.mockito.Mockito.{mock, times, verify, when} +import org.mockito.Mockito.{atLeast => least, mock, times, verify, when} import org.scalatest.concurrent.Eventually._ import org.scalatest.matchers.must.Matchers @@ -38,6 +38,9 @@ class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { private val sparkConf = new SparkConf(false) .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, true) .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, true) + // Just replicate blocks quickly during testing, as there isn't another + // workload we need to worry about. + .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) private def registerShuffleBlocks( mockMigratableShuffleResolver: MigratableResolver, @@ -54,6 +57,113 @@ class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { } } + /** + * Validate a given configuration with the mocks. + * The fail variable controls if we expect migration to fail, in which case we expect + * a constant Long.MaxValue timestamp. + */ + private def validateDecommissionTimestamps(conf: SparkConf, bm: BlockManager, + migratableShuffleBlockResolver: MigratableResolver, fail: Boolean = false) = { + // Verify the decommissioning manager timestamps and status + val bmDecomManager = new BlockManagerDecommissioner(conf, bm) + var previousTime: Option[Long] = None + try { + bmDecomManager.start() + eventually(timeout(100.second), interval(10.milliseconds)) { + val (currentTime, done) = bmDecomManager.lastMigrationInfo() + assert(done) + // Make sure the time stamp starts moving forward. + if (!fail) { + previousTime match { + case None => + previousTime = Some(currentTime) + assert(false) + case Some(t) => + assert(t < currentTime) + } + } else { + // If we expect migration to fail we should get the max value quickly. + assert(currentTime === Long.MaxValue) + } + } + if (!fail) { + // Wait 5 seconds and assert times keep moving forward. + Thread.sleep(5000) + val (currentTime, done) = bmDecomManager.lastMigrationInfo() + assert(done && currentTime > previousTime.get) + } + } finally { + bmDecomManager.stop() + } + } + + test("test that with no blocks we finish migration") { + // Set up the mocks so we return empty + val bm = mock(classOf[BlockManager]) + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + when(migratableShuffleBlockResolver.getStoredShuffles()) + .thenReturn(Seq()) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq()) + when(bm.getPeers(mc.any())) + .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345))) + + // Verify the decom manager handles this correctly + validateDecommissionTimestamps(sparkConf, bm, migratableShuffleBlockResolver) + } + + test("block decom manager with no migrations configured") { + val bm = mock(classOf[BlockManager]) + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1))) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq()) + when(bm.getPeers(mc.any())) + .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345))) + + val badConf = new SparkConf(false) + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, false) + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, false) + .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) + // Verify the decom manager handles this correctly + validateDecommissionTimestamps(badConf, bm, migratableShuffleBlockResolver, + fail = true) + } + + test("block decom manager with no peers") { + // Set up the mocks so we return one shuffle block + val bm = mock(classOf[BlockManager]) + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1))) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq()) + when(bm.getPeers(mc.any())) + .thenReturn(Seq()) + + // Verify the decom manager handles this correctly + validateDecommissionTimestamps(sparkConf, bm, migratableShuffleBlockResolver, + fail = true) + } + + + test("block decom manager with only shuffle files time moves forward") { + // Set up the mocks so we return one shuffle block + val bm = mock(classOf[BlockManager]) + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1))) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq()) + when(bm.getPeers(mc.any())) + .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345))) + + // Verify the decom manager handles this correctly + validateDecommissionTimestamps(sparkConf, bm, migratableShuffleBlockResolver) + } + test("test shuffle and cached rdd migration without any error") { val blockTransferService = mock(classOf[BlockTransferService]) val bm = mock(classOf[BlockManager]) @@ -77,13 +187,36 @@ class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { try { bmDecomManager.start() - eventually(timeout(5.second), interval(10.milliseconds)) { + var previousRDDTime: Option[Long] = None + var previousShuffleTime: Option[Long] = None + + // We don't check that all blocks are migrated because out mock is always returning an RDD. + eventually(timeout(100.second), interval(10.milliseconds)) { assert(bmDecomManager.shufflesToMigrate.isEmpty == true) - verify(bm, times(1)).replicateBlock( + verify(bm, least(1)).replicateBlock( mc.eq(storedBlockId1), mc.any(), mc.any(), mc.eq(Some(3))) verify(blockTransferService, times(2)) .uploadBlockSync(mc.eq("host2"), mc.eq(bmPort), mc.eq("exec2"), mc.any(), mc.any(), mc.eq(StorageLevel.DISK_ONLY), mc.isNull()) + // Since we never "finish" the RDD blocks, make sure the time is always moving forward. + assert(bmDecomManager.rddBlocksLeft) + previousRDDTime match { + case None => + previousRDDTime = Some(bmDecomManager.lastRDDMigrationTime) + assert(false) + case Some(t) => + assert(bmDecomManager.lastRDDMigrationTime > t) + } + // Since we do eventually finish the shuffle blocks make sure the shuffle blocks complete + // and that the time keeps moving forward. + assert(!bmDecomManager.shuffleBlocksLeft) + previousShuffleTime match { + case None => + previousShuffleTime = Some(bmDecomManager.lastShuffleMigrationTime) + assert(false) + case Some(t) => + assert(bmDecomManager.lastShuffleMigrationTime > t) + } } } finally { bmDecomManager.stop() From 7f275ee5978e00ac514e25f5ef1d4e3331f8031b Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 6 Aug 2020 05:39:47 +0000 Subject: [PATCH 376/384] [SPARK-32518][CORE] CoarseGrainedSchedulerBackend.maxNumConcurrentTasks should consider all kinds of resources ### What changes were proposed in this pull request? 1. Make `CoarseGrainedSchedulerBackend.maxNumConcurrentTasks()` considers all kinds of resources when calculating the max concurrent tasks 2. Refactor `calculateAvailableSlots()` to make it be able to be used for both `CoarseGrainedSchedulerBackend` and `TaskSchedulerImpl` ### Why are the changes needed? Currently, `CoarseGrainedSchedulerBackend.maxNumConcurrentTasks()` only considers the CPU for the max concurrent tasks. This can cause the application to hang when a barrier stage requires extra custom resources but the cluster doesn't have enough corresponding resources. Because, without the checking for other custom resources in `maxNumConcurrentTasks`, the barrier stage can be submitted to the `TaskSchedulerImpl`. But the `TaskSchedulerImpl` won't launch tasks for the barrier stage due to the insufficient task slots calculated by `TaskSchedulerImpl.calculateAvailableSlots` (which does check all kinds of resources). The application hang issue can be reproduced by the added unit test. ### Does this PR introduce _any_ user-facing change? Yes. In case of a barrier stage requires more custom resources than the cluster has, the application can get hang before this PR but can fail due to insufficient resources at the end after this PR. ### How was this patch tested? Added a unit test. Closes #29332 from Ngone51/fix-slots. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../BarrierJobAllocationFailed.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 10 +- .../scheduler/ExecutorResourceInfo.scala | 1 + .../spark/scheduler/SchedulerBackend.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 113 ++++++++++-------- .../CoarseGrainedSchedulerBackend.scala | 24 +++- .../spark/BarrierStageOnSubmittedSuite.scala | 36 ++++++ 8 files changed, 133 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 78f509c670839..5e0eaa478547c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1603,7 +1603,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Get the max number of tasks that can be concurrent launched based on the ResourceProfile - * being used. + * could be used, even if some of them are being used at the moment. * Note that please don't cache the value returned by this method, because the number can change * due to add/remove executors. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala b/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala index 2274e6898adf6..043c6b90384b4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala @@ -60,6 +60,6 @@ private[spark] object BarrierJobAllocationFailed { val ERROR_MESSAGE_BARRIER_REQUIRE_MORE_SLOTS_THAN_CURRENT_TOTAL_NUMBER = "[SPARK-24819]: Barrier execution mode does not allow run a barrier stage that requires " + "more slots than the total number of slots in the cluster currently. Please init a new " + - "cluster with more CPU cores or repartition the input RDD(s) to reduce the number of " + - "slots required to run this barrier stage." + "cluster with more resources(e.g. CPU, GPU) or repartition the input RDD(s) to reduce " + + "the number of slots required to run this barrier stage." } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 6b376cdadc66b..7641948ed4b30 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -480,10 +480,12 @@ private[spark] class DAGScheduler( * submission. */ private def checkBarrierStageWithNumSlots(rdd: RDD[_], rp: ResourceProfile): Unit = { - val numPartitions = rdd.getNumPartitions - val maxNumConcurrentTasks = sc.maxNumConcurrentTasks(rp) - if (rdd.isBarrier() && numPartitions > maxNumConcurrentTasks) { - throw new BarrierJobSlotsNumberCheckFailed(numPartitions, maxNumConcurrentTasks) + if (rdd.isBarrier()) { + val numPartitions = rdd.getNumPartitions + val maxNumConcurrentTasks = sc.maxNumConcurrentTasks(rp) + if (numPartitions > maxNumConcurrentTasks) { + throw new BarrierJobSlotsNumberCheckFailed(numPartitions, maxNumConcurrentTasks) + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index fd04db8c09d76..508c6cebd9fe3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -36,4 +36,5 @@ private[spark] class ExecutorResourceInfo( override protected def resourceName = this.name override protected def resourceAddresses = this.addresses override protected def slotsPerAddress: Int = numParts + def totalAddressAmount: Int = resourceAddresses.length * slotsPerAddress } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index a5bba645be14c..a566d0a04387c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -83,7 +83,7 @@ private[spark] trait SchedulerBackend { /** * Get the max number of tasks that can be concurrent launched based on the ResourceProfile - * being used. + * could be used, even if some of them are being used at the moment. * Note that please don't cache the value returned by this method, because the number can change * due to add/remove executors. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2551e497a165a..a0c507e7f893b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -468,51 +468,6 @@ private[spark] class TaskSchedulerImpl( Some(localTaskReqAssign.toMap) } - // Use the resource that the resourceProfile has as the limiting resource to calculate the - // total number of slots available based on the current offers. - private def calculateAvailableSlots( - resourceProfileIds: Array[Int], - availableCpus: Array[Int], - availableResources: Array[Map[String, Buffer[String]]], - taskSet: TaskSetManager): Int = { - val resourceProfile = sc.resourceProfileManager.resourceProfileFromId( - taskSet.taskSet.resourceProfileId) - val offersForResourceProfile = resourceProfileIds.zipWithIndex.filter { case (id, _) => - (id == resourceProfile.id) - } - val coresKnown = resourceProfile.isCoresLimitKnown - var limitingResource = resourceProfile.limitingResource(conf) - val taskCpus = ResourceProfile.getTaskCpusOrDefaultForProfile(resourceProfile, conf) - - offersForResourceProfile.map { case (o, index) => - val numTasksPerExecCores = availableCpus(index) / taskCpus - // if limiting resource is empty then we have no other resources, so it has to be CPU - if (limitingResource == ResourceProfile.CPUS || limitingResource.isEmpty) { - numTasksPerExecCores - } else { - val taskLimit = resourceProfile.taskResources.get(limitingResource).map(_.amount) - .getOrElse { - val errorMsg = "limitingResource returns from ResourceProfile " + - s"$resourceProfile doesn't actually contain that task resource!" - taskSet.abort(errorMsg) - throw new SparkException(errorMsg) - } - // available addresses already takes into account if there are fractional - // task resource requests - val availAddrs = availableResources(index).get(limitingResource).map(_.size).getOrElse(0) - val resourceLimit = (availAddrs / taskLimit).toInt - if (!coresKnown) { - // when executor cores config isn't set, we can't calculate the real limiting resource - // and number of tasks per executor ahead of time, so calculate it now based on what - // is available. - if (numTasksPerExecCores <= resourceLimit) numTasksPerExecCores else resourceLimit - } else { - resourceLimit - } - } - }.sum - } - private def minTaskLocality( l1: Option[TaskLocality], l2: Option[TaskLocality]) : Option[TaskLocality] = { @@ -591,9 +546,14 @@ private[spark] class TaskSchedulerImpl( // we only need to calculate available slots if using barrier scheduling, otherwise the // value is -1 val numBarrierSlotsAvailable = if (taskSet.isBarrier) { - val slots = calculateAvailableSlots(resourceProfileIds, availableCpus, availableResources, - taskSet) - slots + val rpId = taskSet.taskSet.resourceProfileId + val availableResourcesAmount = availableResources.map { resourceMap => + // available addresses already takes into account if there are fractional + // task resource requests + resourceMap.map { case (name, addresses) => (name, addresses.length) } + } + calculateAvailableSlots(this, conf, rpId, resourceProfileIds, availableCpus, + availableResourcesAmount) } else { -1 } @@ -1166,6 +1126,63 @@ private[spark] object TaskSchedulerImpl { val SCHEDULER_MODE_PROPERTY = SCHEDULER_MODE.key + /** + * Calculate the max available task slots given the `availableCpus` and `availableResources` + * from a collection of ResourceProfiles. And only those ResourceProfiles who has the + * same id with the `rpId` can be used to calculate the task slots. + * + * @param scheduler the TaskSchedulerImpl instance + * @param conf SparkConf used to calculate the limiting resource and get the cpu amount per task + * @param rpId the target ResourceProfile id. Only those ResourceProfiles who has the same id + * with it can be used to calculate the task slots. + * @param availableRPIds an Array of ids of the available ResourceProfiles from the executors. + * @param availableCpus an Array of the amount of available cpus from the executors. + * @param availableResources an Array of the resources map from the executors. In the resource + * map, it maps from the resource name to its amount. + * @return the number of max task slots + */ + def calculateAvailableSlots( + scheduler: TaskSchedulerImpl, + conf: SparkConf, + rpId: Int, + availableRPIds: Array[Int], + availableCpus: Array[Int], + availableResources: Array[Map[String, Int]]): Int = { + val resourceProfile = scheduler.sc.resourceProfileManager.resourceProfileFromId(rpId) + val coresKnown = resourceProfile.isCoresLimitKnown + val (limitingResource, limitedByCpu) = { + val limiting = resourceProfile.limitingResource(conf) + // if limiting resource is empty then we have no other resources, so it has to be CPU + if (limiting == ResourceProfile.CPUS || limiting.isEmpty) { + (ResourceProfile.CPUS, true) + } else { + (limiting, false) + } + } + val cpusPerTask = ResourceProfile.getTaskCpusOrDefaultForProfile(resourceProfile, conf) + val taskLimit = resourceProfile.taskResources.get(limitingResource).map(_.amount).get + + availableCpus.zip(availableResources).zip(availableRPIds) + .filter { case (_, id) => id == rpId } + .map { case ((cpu, resources), _) => + val numTasksPerExecCores = cpu / cpusPerTask + if (limitedByCpu) { + numTasksPerExecCores + } else { + val availAddrs = resources.getOrElse(limitingResource, 0) + val resourceLimit = (availAddrs / taskLimit).toInt + // when executor cores config isn't set, we can't calculate the real limiting resource + // and number of tasks per executor ahead of time, so calculate it now based on what + // is available. + if (!coresKnown && numTasksPerExecCores <= resourceLimit) { + numTasksPerExecCores + } else { + resourceLimit + } + } + }.sum + } + /** * Used to balance containers across hosts. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d81a617d0ed7d..200f2d87a8a7a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -642,10 +642,28 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } + /** + * Get the max number of tasks that can be concurrent launched based on the ResourceProfile + * could be used, even if some of them are being used at the moment. + * Note that please don't cache the value returned by this method, because the number can change + * due to add/remove executors. + * + * @param rp ResourceProfile which to use to calculate max concurrent tasks. + * @return The max number of tasks that can be concurrent launched currently. + */ override def maxNumConcurrentTasks(rp: ResourceProfile): Int = synchronized { - val cpusPerTask = ResourceProfile.getTaskCpusOrDefaultForProfile(rp, conf) - val executorsWithResourceProfile = executorDataMap.values.filter(_.resourceProfileId == rp.id) - executorsWithResourceProfile.map(_.totalCores / cpusPerTask).sum + val (rpIds, cpus, resources) = { + executorDataMap + .filter { case (id, _) => isExecutorActive(id) } + .values.toArray.map { executor => + ( + executor.resourceProfileId, + executor.totalCores, + executor.resourcesInfo.map { case (name, rInfo) => (name, rInfo.totalAddressAmount) } + ) + }.unzip3 + } + TaskSchedulerImpl.calculateAvailableSlots(scheduler, conf, rp.id, rpIds, cpus, resources) } // this function is for testing only diff --git a/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala b/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala index 435b927068e60..1ba13c2ef1897 100644 --- a/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala +++ b/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala @@ -19,9 +19,12 @@ package org.apache.spark import scala.concurrent.duration._ +import org.apache.spark.TestUtils.createTempScriptWithExpectedOutput import org.apache.spark.internal.config._ import org.apache.spark.rdd.{PartitionPruningRDD, RDD} +import org.apache.spark.resource.TestResourceIDs.{EXECUTOR_GPU_ID, TASK_GPU_ID, WORKER_GPU_ID} import org.apache.spark.scheduler.BarrierJobAllocationFailed._ +import org.apache.spark.scheduler.BarrierJobSlotsNumberCheckFailed import org.apache.spark.util.ThreadUtils /** @@ -259,4 +262,37 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext testSubmitJob(sc, rdd, message = ERROR_MESSAGE_BARRIER_REQUIRE_MORE_SLOTS_THAN_CURRENT_TOTAL_NUMBER) } + + test("SPARK-32518: CoarseGrainedSchedulerBackend.maxNumConcurrentTasks should " + + "consider all kinds of resources for the barrier stage") { + withTempDir { dir => + val discoveryScript = createTempScriptWithExpectedOutput( + dir, "gpuDiscoveryScript", """{"name": "gpu","addresses":["0"]}""") + + val conf = new SparkConf() + // Setup a local cluster which would only has one executor with 2 CPUs and 1 GPU. + .setMaster("local-cluster[1, 2, 1024]") + .setAppName("test-cluster") + .set(WORKER_GPU_ID.amountConf, "1") + .set(WORKER_GPU_ID.discoveryScriptConf, discoveryScript) + .set(EXECUTOR_GPU_ID.amountConf, "1") + .set(TASK_GPU_ID.amountConf, "1") + // disable barrier stage retry to fail the application as soon as possible + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES, 1) + sc = new SparkContext(conf) + TestUtils.waitUntilExecutorsUp(sc, 1, 60000) + + val exception = intercept[BarrierJobSlotsNumberCheckFailed] { + // Setup a barrier stage which contains 2 tasks and each task requires 1 CPU and 1 GPU. + // Therefore, the total resources requirement (2 CPUs and 2 GPUs) of this barrier stage + // can not be satisfied since the cluster only has 2 CPUs and 1 GPU in total. + sc.parallelize(Range(1, 10), 2) + .barrier() + .mapPartitions { iter => iter } + .collect() + } + assert(exception.getMessage.contains("[SPARK-24819]: Barrier execution " + + "mode does not allow run a barrier stage that requires more slots")) + } + } } From e93b8f02cd706bedc47c9b55a73f632fe9e61ec3 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 6 Aug 2020 05:56:59 +0000 Subject: [PATCH 377/384] [SPARK-32539][INFRA] Disallow `FileSystem.get(Configuration conf)` in style check by default ### What changes were proposed in this pull request? Disallow `FileSystem.get(Configuration conf)` in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem()` instead. ### Why are the changes needed? The method `FileSystem.get(Configuration conf)` will return a default FileSystem instance if the conf `fs.file.impl` is not set. This can cause file not found exception on reading a target path of non-default file system, e.g. S3. It is hard to discover such a mistake via unit tests. If we disallow it in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem(Configuration conf)`, we can reduce potential regression and PR review effort. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually run scala style check and test. Closes #29357 from gengliangwang/newStyleRule. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../security/HadoopFSDelegationTokenProvider.scala | 2 ++ .../spark/internal/io/SparkHadoopWriter.scala | 4 ++++ .../org/apache/spark/deploy/yarn/Client.scala | 2 ++ scalastyle-config.xml | 14 ++++++++++++++ .../parquet/ParquetInteroperabilitySuite.scala | 2 +- 5 files changed, 23 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala index 4e91e72361488..a46864e2d3c9c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -139,7 +139,9 @@ private[deploy] object HadoopFSDelegationTokenProvider { def hadoopFSsToAccess( sparkConf: SparkConf, hadoopConf: Configuration): Set[FileSystem] = { + // scalastyle:off FileSystemGet val defaultFS = FileSystem.get(hadoopConf) + // scalastyle:on FileSystemGet val filesystemsToAccess = sparkConf.get(KERBEROS_FILESYSTEMS_TO_ACCESS) .map(new Path(_).getFileSystem(hadoopConf)) diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala index a619f10bbf064..6d174b5e0f81b 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -222,7 +222,9 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf) if (path != null) { path.getFileSystem(getConf) } else { + // scalastyle:off FileSystemGet FileSystem.get(getConf) + // scalastyle:on FileSystemGet } } @@ -285,7 +287,9 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf) if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) { // FileOutputFormat ignores the filesystem parameter + // scalastyle:off FileSystemGet val ignoredFs = FileSystem.get(getConf) + // scalastyle:on FileSystemGet getOutputFormat().checkOutputSpecs(ignoredFs, getConf) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 1045fb089c017..517a4af2e4b02 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -181,10 +181,12 @@ private[spark] class Client( // The app staging dir based on the STAGING_DIR configuration if configured // otherwise based on the users home directory. + // scalastyle:off FileSystemGet val appStagingBaseDir = sparkConf.get(STAGING_DIR) .map { new Path(_, UserGroupInformation.getCurrentUser.getShortUserName) } .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory()) stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId)) + // scalastyle:on FileSystemGet new CallerContext("CLIENT", sparkConf.get(APP_CALLER_CONTEXT), Option(appId.toString)).setCurrentContext() diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 73ac14fdba1cf..c1dc57be56dff 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -264,6 +264,20 @@ This file is divided into 3 sections: of Commons Lang 2 (package org.apache.commons.lang.*) + + FileSystem.get\([a-zA-Z_$][a-zA-Z_$0-9]*\) + + + extractOpt Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index a14f6416199a1..8c4eedfde76cd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -165,7 +165,7 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS // the assumption on column stats, and also the end-to-end behavior. val hadoopConf = spark.sessionState.newHadoopConf() - val fs = FileSystem.get(hadoopConf) + val fs = new Path(tableDir.getAbsolutePath).getFileSystem(hadoopConf) val parts = fs.listStatus(new Path(tableDir.getAbsolutePath), new PathFilter { override def accept(path: Path): Boolean = !path.getName.startsWith("_") }) From dc96f2f8d6e08c4bc30bc11d6b29109d2aeb604b Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 6 Aug 2020 08:35:59 +0000 Subject: [PATCH 378/384] [SPARK-32546][SQL] Get table names directly from Hive tables ### What changes were proposed in this pull request? Get table names directly from a sequence of Hive tables in `HiveClientImpl.listTablesByType()` by skipping conversions Hive tables to Catalog tables. ### Why are the changes needed? A Hive metastore can be shared across many clients. A client can create tables using a SerDe which is not available on other clients, for instance `ROW FORMAT SERDE "com.ibm.spss.hive.serde2.xml.XmlSerDe"`. In the current implementation, other clients get the following exception while getting views: ``` java.lang.RuntimeException: MetaException(message:java.lang.ClassNotFoundException Class com.ibm.spss.hive.serde2.xml.XmlSerDe not found) ``` when `com.ibm.spss.hive.serde2.xml.XmlSerDe` is not available. ### Does this PR introduce _any_ user-facing change? Yes. For example, `SHOW VIEWS` returns a list of views instead of throwing an exception. ### How was this patch tested? - By existing test suites like: ``` $ build/sbt -Phive-2.3 "test:testOnly org.apache.spark.sql.hive.client.VersionsSuite" ``` - And manually: 1. Build Spark with Hive 1.2: `./build/sbt package -Phive-1.2 -Phive -Dhadoop.version=2.8.5` 2. Run spark-shell with a custom Hive SerDe, for instance download [json-serde-1.3.8-jar-with-dependencies.jar](https://github.com/cdamak/Twitter-Hive/blob/master/json-serde-1.3.8-jar-with-dependencies.jar) from https://github.com/cdamak/Twitter-Hive: ``` $ ./bin/spark-shell --jars ../Downloads/json-serde-1.3.8-jar-with-dependencies.jar ``` 3. Create a Hive table using this SerDe: ```scala scala> :paste // Entering paste mode (ctrl-D to finish) sql(s""" |CREATE TABLE json_table2(page_id INT NOT NULL) |ROW FORMAT SERDE 'org.openx.data.jsonserde.JsonSerDe' |""".stripMargin) // Exiting paste mode, now interpreting. res0: org.apache.spark.sql.DataFrame = [] scala> sql("SHOW TABLES").show +--------+-----------+-----------+ |database| tableName|isTemporary| +--------+-----------+-----------+ | default|json_table2| false| +--------+-----------+-----------+ scala> sql("SHOW VIEWS").show +---------+--------+-----------+ |namespace|viewName|isTemporary| +---------+--------+-----------+ +---------+--------+-----------+ ``` 4. Quit from the current `spark-shell` and run it without jars: ``` $ ./bin/spark-shell ``` 5. Show views. Without the fix, it throws the exception: ```scala scala> sql("SHOW VIEWS").show 20/08/06 10:53:36 ERROR log: error in initSerDe: java.lang.ClassNotFoundException Class org.openx.data.jsonserde.JsonSerDe not found java.lang.ClassNotFoundException: Class org.openx.data.jsonserde.JsonSerDe not found at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2273) at org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:385) at org.apache.hadoop.hive.ql.metadata.Table.getDeserializerFromMetaStore(Table.java:276) at org.apache.hadoop.hive.ql.metadata.Table.getDeserializer(Table.java:258) at org.apache.hadoop.hive.ql.metadata.Table.getCols(Table.java:605) ``` After the fix: ```scala scala> sql("SHOW VIEWS").show +---------+--------+-----------+ |namespace|viewName|isTemporary| +---------+--------+-----------+ +---------+--------+-----------+ ``` Closes #29363 from MaxGekk/fix-listTablesByType-for-views. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 3f70387a3b058..58ad5449b49fd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -759,15 +759,17 @@ private[hive] class HiveClientImpl( dbName: String, pattern: String, tableType: CatalogTableType): Seq[String] = withHiveState { + val hiveTableType = toHiveTableType(tableType) try { // Try with Hive API getTablesByType first, it's supported from Hive 2.3+. - shim.getTablesByType(client, dbName, pattern, toHiveTableType(tableType)) + shim.getTablesByType(client, dbName, pattern, hiveTableType) } catch { case _: UnsupportedOperationException => // Fallback to filter logic if getTablesByType not supported. val tableNames = client.getTablesByPattern(dbName, pattern).asScala - val tables = getTablesByName(dbName, tableNames.toSeq).filter(_.tableType == tableType) - tables.map(_.identifier.table) + getRawTablesByName(dbName, tableNames) + .filter(_.getTableType == hiveTableType) + .map(_.getTableName) } } From 6664e282f6cc37582b9e6e5d6a47380097e58fc9 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 6 Aug 2020 14:23:56 +0000 Subject: [PATCH 379/384] [SPARK-32546][SQL][FOLLOWUP] Add `.toSeq` to `tableNames` in `HiveClientImpl.listTablesByType` ### What changes were proposed in this pull request? Explicitly convert `tableNames` to `Seq` in `HiveClientImpl.listTablesByType` as it was done by https://github.com/apache/spark/commit/c28a6fa5112c9ba3839f52b737266f24fdfcf75b#diff-6fd847124f8eae45ba2de1cf7d6296feR769 ### Why are the changes needed? See this PR https://github.com/apache/spark/pull/29111, to compile by Scala 2.13. The changes were discarded by https://github.com/apache/spark/pull/29363 accidentally. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Compiling by Scala 2.13 Closes #29379 from MaxGekk/fix-listTablesByType-for-views-followup. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 58ad5449b49fd..8aa60365f3c1d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -767,7 +767,7 @@ private[hive] class HiveClientImpl( case _: UnsupportedOperationException => // Fallback to filter logic if getTablesByType not supported. val tableNames = client.getTablesByPattern(dbName, pattern).asScala - getRawTablesByName(dbName, tableNames) + getRawTablesByName(dbName, tableNames.toSeq) .filter(_.getTableType == hiveTableType) .map(_.getTableName) } From 75c2c53e931187912a92e0b52dae0f772fa970e3 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 6 Aug 2020 13:54:15 -0700 Subject: [PATCH 380/384] [SPARK-32506][TESTS] Flaky test: StreamingLinearRegressionWithTests ### What changes were proposed in this pull request? The test creates 10 batches of data to train the model and expects to see error on test data improves as model is trained. If the difference between the 2nd error and the 10th error is smaller than 2, the assertion fails: ``` FAIL: test_train_prediction (pyspark.mllib.tests.test_streaming_algorithms.StreamingLinearRegressionWithTests) Test that error on test data improves as model is trained. ---------------------------------------------------------------------- Traceback (most recent call last): File "/home/runner/work/spark/spark/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 466, in test_train_prediction eventually(condition, timeout=180.0) File "/home/runner/work/spark/spark/python/pyspark/testing/utils.py", line 81, in eventually lastValue = condition() File "/home/runner/work/spark/spark/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 461, in condition self.assertGreater(errors[1] - errors[-1], 2) AssertionError: 1.672640157855923 not greater than 2 ``` I saw this quite a few time on Jenkins but was not able to reproduce this on my local. These are the ten errors I got: ``` 4.517395047937127 4.894265404350079 3.0392090466559876 1.8786361640757654 0.8973106042078115 0.3715780507684368 0.20815690742907672 0.17333033743125845 0.15686783249863873 0.12584413600569616 ``` I am thinking of having 15 batches of data instead of 10, so the model can be trained for a longer time. Hopefully the 15th error - 2nd error will always be larger than 2 on Jenkins. These are the 15 errors I got on my local: ``` 4.517395047937127 4.894265404350079 3.0392090466559876 1.8786361640757658 0.8973106042078115 0.3715780507684368 0.20815690742907672 0.17333033743125845 0.15686783249863873 0.12584413600569616 0.11883853835108477 0.09400261862100823 0.08887491447353497 0.05984929624986607 0.07583948141520978 ``` ### Why are the changes needed? Fix flaky test ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually tested Closes #29380 from huaxingao/flaky_test. Authored-by: Huaxin Gao Signed-off-by: Huaxin Gao --- python/pyspark/mllib/tests/test_streaming_algorithms.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/mllib/tests/test_streaming_algorithms.py b/python/pyspark/mllib/tests/test_streaming_algorithms.py index 2f35e07cdf71c..5818a7c088841 100644 --- a/python/pyspark/mllib/tests/test_streaming_algorithms.py +++ b/python/pyspark/mllib/tests/test_streaming_algorithms.py @@ -434,9 +434,9 @@ def test_train_prediction(self): slr = StreamingLinearRegressionWithSGD(stepSize=0.2, numIterations=25) slr.setInitialWeights([0.0]) - # Create ten batches with 100 sample points in each. + # Create fifteen batches with 100 sample points in each. batches = [] - for i in range(10): + for i in range(15): batch = LinearDataGenerator.generateLinearInput( 0.0, [10.0], [0.0], [1.0 / 3.0], 100, 42 + i, 0.1) batches.append(self.sc.parallelize(batch)) From 4e267f3eb9ca0df18647c859b75b61b1af800120 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 7 Aug 2020 11:29:18 +0900 Subject: [PATCH 381/384] [SPARK-32538][CORE][TEST] Use local time zone for the timestamp logged in unit-tests.log ### What changes were proposed in this pull request? This PR lets the logger log timestamp based on local time zone during test. `SparkFunSuite` fixes the default time zone to America/Los_Angeles so the timestamp logged in unit-tests.log is also based on the fixed time zone. ### Why are the changes needed? It's confusable for developers whose time zone is not America/Los_Angeles. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Run existing tests and confirmed uint-tests.log. If your local time zone is America/Los_Angeles, you can test by setting the environment variable `TZ` like as follows. ``` $ TZ=Asia/Tokyo build/sbt "testOnly org.apache.spark.executor.ExecutorSuite" $ tail core/target/unit-tests.log ``` Closes #29356 from sarutak/fix-unit-test-log-timezone. Authored-by: Kousuke Saruta Signed-off-by: Takeshi Yamamuro --- core/src/test/scala/org/apache/spark/SparkFunSuite.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 581786ce0a70c..8d103002a7496 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -65,6 +65,12 @@ abstract class SparkFunSuite with Logging { // scalastyle:on + // Initialize the logger forcibly to let the logger log timestamp + // based on the local time zone depending on environments. + // The default time zone will be set to America/Los_Angeles later + // so this initialization is necessary here. + log + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting From 2cb48eabdc0b3298de92fe963eeb61181e3d55c5 Mon Sep 17 00:00:00 2001 From: Liang Zhang Date: Fri, 7 Aug 2020 11:50:46 +0900 Subject: [PATCH 382/384] [SPARK-32549][PYSPARK] Add column name in _infer_schema error message ### What changes were proposed in this pull request? The current error message from `_infer_type` in `_infer_schema` only includes the unsupported column type but not the column name. This PR adds the column name in the error message to make it easier for users to identify which column should they drop or convert. ### Why are the changes needed? Improve user experience. ### Does this PR introduce _any_ user-facing change? Yes. The error message from `_infer_schema` is changed. Before: "not supported type: foo" After: "Column bar contains not supported type: foo" ### How was this patch tested? Updated the existing unit test. Closes #29365 from liangz1/types-error-colname. Authored-by: Liang Zhang Signed-off-by: HyukjinKwon --- python/pyspark/sql/tests/test_types.py | 2 +- python/pyspark/sql/types.py | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 051c8bde50ad9..05e2717fda870 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -720,7 +720,7 @@ def assertCollectSuccess(typecode, value): unsupported_types = all_types - set(supported_types) # test unsupported types for t in unsupported_types: - with self.assertRaises(TypeError): + with self.assertRaisesRegexp(TypeError, "infer the type of the field myarray"): a = array.array(t) self.spark.createDataFrame([Row(myarray=a)]).collect() diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index cc08482c735b1..43f3a8531871a 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1032,7 +1032,12 @@ def _infer_schema(row, names=None): else: raise TypeError("Can not infer schema for type: %s" % type(row)) - fields = [StructField(k, _infer_type(v), True) for k, v in items] + fields = [] + for k, v in items: + try: + fields.append(StructField(k, _infer_type(v), True)) + except TypeError as e: + raise TypeError("Unable to infer the type of the field {}.".format(k)) from e return StructType(fields) From 9a35b93c8a02939bf02437f47501d55223b6a283 Mon Sep 17 00:00:00 2001 From: "wangguangxin.cn" Date: Fri, 7 Aug 2020 05:00:33 +0000 Subject: [PATCH 383/384] [SPARK-32559][SQL] Fix the trim logic in UTF8String.toInt/toLong did't handle non-ASCII characters correctly ### What changes were proposed in this pull request? The trim logic in Cast expression introduced in https://github.com/apache/spark/pull/26622 trim non-ASCII characters unexpectly. Before this patch ![image](https://user-images.githubusercontent.com/1312321/89513154-caad9b80-d806-11ea-9ebe-17c9e7d1b5b3.png) After this patch ![image](https://user-images.githubusercontent.com/1312321/89513196-d731f400-d806-11ea-959c-6a7dc29dcd49.png) ### Why are the changes needed? The behavior described above doesn't make sense, and also doesn't consistent with the behavior when cast a string to double/float, as well as doesn't consistent with the behavior of Hive ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Added more UT Closes #29375 from WangGuangxin/cast-bugfix. Authored-by: wangguangxin.cn Signed-off-by: Wenchen Fan --- .../apache/spark/unsafe/types/UTF8String.java | 12 +++--- .../test/resources/sql-tests/inputs/cast.sql | 5 +++ .../resources/sql-tests/inputs/datetime.sql | 2 + .../resources/sql-tests/inputs/interval.sql | 1 + .../sql-tests/results/ansi/datetime.sql.out | 30 ++++++++++++- .../sql-tests/results/ansi/interval.sql.out | 16 ++++++- .../resources/sql-tests/results/cast.sql.out | 42 ++++++++++++++++++- .../sql-tests/results/datetime-legacy.sql.out | 30 ++++++++++++- .../sql-tests/results/datetime.sql.out | 30 ++++++++++++- .../sql-tests/results/interval.sql.out | 16 ++++++- 10 files changed, 172 insertions(+), 12 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 7205293aa48c5..43bd7976c5d33 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -575,14 +575,14 @@ public UTF8String trim() { public UTF8String trimAll() { int s = 0; // skip all of the whitespaces (<=0x20) in the left side - while (s < this.numBytes && getByte(s) <= ' ') s++; + while (s < this.numBytes && Character.isWhitespace(getByte(s))) s++; if (s == this.numBytes) { // Everything trimmed return EMPTY_UTF8; } // skip all of the whitespaces (<=0x20) in the right side int e = this.numBytes - 1; - while (e > s && getByte(e) <= ' ') e--; + while (e > s && Character.isWhitespace(getByte(e))) e--; if (s == 0 && e == numBytes - 1) { // Nothing trimmed return this; @@ -1119,11 +1119,11 @@ public boolean toLong(LongWrapper toLongResult) { private boolean toLong(LongWrapper toLongResult, boolean allowDecimal) { int offset = 0; - while (offset < this.numBytes && getByte(offset) <= ' ') offset++; + while (offset < this.numBytes && Character.isWhitespace(getByte(offset))) offset++; if (offset == this.numBytes) return false; int end = this.numBytes - 1; - while (end > offset && getByte(end) <= ' ') end--; + while (end > offset && Character.isWhitespace(getByte(end))) end--; byte b = getByte(offset); final boolean negative = b == '-'; @@ -1216,11 +1216,11 @@ public boolean toInt(IntWrapper intWrapper) { private boolean toInt(IntWrapper intWrapper, boolean allowDecimal) { int offset = 0; - while (offset < this.numBytes && getByte(offset) <= ' ') offset++; + while (offset < this.numBytes && Character.isWhitespace(getByte(offset))) offset++; if (offset == this.numBytes) return false; int end = this.numBytes - 1; - while (end > offset && getByte(end) <= ' ') end--; + while (end > offset && Character.isWhitespace(getByte(end))) end--; byte b = getByte(offset); final boolean negative = b == '-'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/cast.sql index 972ebdd01f61e..81c741a5ca8e9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cast.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cast.sql @@ -70,6 +70,11 @@ select cast(' 1' as bigint); select cast(' 1' as float); select cast(' 1 ' as DOUBLE); select cast('1.0 ' as DEC); +select cast('1中文' as tinyint); +select cast('1中文' as smallint); +select cast('1中文' as INT); +select cast('中文1' as bigint); +select cast('1中文' as bigint); -- trim string before cast to boolean select cast('\t\t true \n\r ' as boolean); 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 bdf11f51db532..0445c7864946c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -48,6 +48,8 @@ select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01'); select date '2019-01-01\t'; select timestamp '2019-01-01\t'; +select date '2020-01-01中文'; +select timestamp '2019-01-01中文'; -- time add/sub select timestamp'2011-11-11 11:11:11' + interval '2' day; diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index e881250ed7bee..7173863313b2f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -171,6 +171,7 @@ select interval '2-2\t' year to month; select interval '-\t2-2\t' year to month; select interval '\n0 12:34:46.789\t' day to second; select interval '\n-\t10\t 12:34:46.789\t' day to second; +select interval '中文 interval 1 day'; -- interval overflow if (ansi) exception else NULL select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); 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 484b67677a91b..d2d66713780d8 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: 106 +-- Number of queries: 108 -- !query @@ -226,6 +226,34 @@ struct 2019-01-01 00:00:00 +-- !query +select date '2020-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7) + +== SQL == +select date '2020-01-01中文' +-------^^^ + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7) + +== SQL == +select timestamp '2019-01-01中文' +-------^^^ + + -- !query select timestamp'2011-11-11 11:11:11' + interval '2' day -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 8644c668782ae..f80bea1d32734 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 93 +-- Number of queries: 94 -- !query @@ -891,6 +891,20 @@ select interval '\n-\t10\t 12:34:46.789\t' day to second ----------------^^^ +-- !query +select interval '中文 interval 1 day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the INTERVAL value: 中文 interval 1 day(line 1, pos 7) + +== SQL == +select interval '中文 interval 1 day' +-------^^^ + + -- !query select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 35b4c0e79720b..d4872ca03199b 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 46 +-- Number of queries: 51 -- !query @@ -353,6 +353,46 @@ struct 1 +-- !query +select cast('1中文' as tinyint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as smallint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as INT) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('中文1' as bigint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as bigint) +-- !query schema +struct +-- !query output +NULL + + -- !query select cast('\t\t true \n\r ' as boolean) -- !query schema 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 index edb49e575f52e..3806764856f5b 100644 --- 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 106 +-- Number of queries: 108 -- !query @@ -200,6 +200,34 @@ struct 2019-01-01 00:00:00 +-- !query +select date '2020-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7) + +== SQL == +select date '2020-01-01中文' +-------^^^ + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7) + +== SQL == +select timestamp '2019-01-01中文' +-------^^^ + + -- !query select timestamp'2011-11-11 11:11:11' + interval '2' day -- !query schema 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 9f9351a4809af..5feeaa9addef7 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: 106 +-- Number of queries: 108 -- !query @@ -200,6 +200,34 @@ struct 2019-01-01 00:00:00 +-- !query +select date '2020-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7) + +== SQL == +select date '2020-01-01中文' +-------^^^ + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7) + +== SQL == +select timestamp '2019-01-01中文' +-------^^^ + + -- !query select timestamp'2011-11-11 11:11:11' + interval '2' day -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 438a2766061b4..297c4fcd0cb95 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 93 +-- Number of queries: 94 -- !query @@ -868,6 +868,20 @@ select interval '\n-\t10\t 12:34:46.789\t' day to second ----------------^^^ +-- !query +select interval '中文 interval 1 day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the INTERVAL value: 中文 interval 1 day(line 1, pos 7) + +== SQL == +select interval '中文 interval 1 day' +-------^^^ + + -- !query select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) -- !query schema From aa4d3c19fead4ec2f89b4957b4ccc7482e121e4d Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Fri, 7 Aug 2020 14:29:32 +0900 Subject: [PATCH 384/384] [SPARK-32560][SQL] Improve exception message at InsertIntoHiveTable.processInsert ### What changes were proposed in this pull request? improve exception message ### Why are the changes needed? the before message lack of single quotes, we may improve it to keep consisent. ![image](https://user-images.githubusercontent.com/46367746/89595808-15bbc300-d888-11ea-9914-b05ea7b66461.png) ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? No ,it is only improving the message. Closes #29376 from GuoPhilipse/improve-exception-message. Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Co-authored-by: GuoPhilipse Signed-off-by: HyukjinKwon --- .../apache/spark/sql/hive/execution/InsertIntoHiveTable.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 116217ecec0ba..3c3f31ac2994a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -245,7 +245,7 @@ case class InsertIntoHiveTable( if (fs.exists(partitionPath)) { if (!fs.delete(partitionPath, true)) { throw new RuntimeException( - "Cannot remove partition directory '" + partitionPath.toString) + s"Cannot remove partition directory '$partitionPath'") } } } @@ -311,7 +311,7 @@ case class InsertIntoHiveTable( if (fs.exists(path)) { if (!fs.delete(path, true)) { throw new RuntimeException( - "Cannot remove partition directory '" + path.toString) + s"Cannot remove partition directory '$path'") } // Don't let Hive do overwrite operation since it is slower. doHiveOverwrite = false