From ce70a4ef4d6410d0a738a5440dd2b7d91c7e4822 Mon Sep 17 00:00:00 2001 From: sujithjay Date: Tue, 21 Aug 2018 13:50:48 +0530 Subject: [PATCH 0001/1072] [SPARK-24985][SQL] Fix OOM in Full Outer Join in presence of data skew. Change SortMergeJoinExec to use ExternalAppendOnlyUnsafeRowArray for Full Outer Join. This would spill data into disk if the buffered rows exceed beyond a threshold, thus preventing OOM errors. Change corresponding test case in JoinSuite. --- .../execution/joins/SortMergeJoinExec.scala | 62 +++++++++++-------- .../org/apache/spark/sql/JoinSuite.scala | 4 +- 2 files changed, 38 insertions(+), 28 deletions(-) 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 f4b9d132122e..48584712a226 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 @@ -248,7 +248,9 @@ case class SortMergeJoinExec( rightIter = RowIterator.fromScala(rightIter), boundCondition, leftNullRow, - rightNullRow) + rightNullRow, + inMemoryThreshold, + spillThreshold) new FullOuterIterator( smjScanner, @@ -966,7 +968,9 @@ private class SortMergeFullOuterJoinScanner( rightIter: RowIterator, boundCondition: InternalRow => Boolean, leftNullRow: InternalRow, - rightNullRow: InternalRow) { + rightNullRow: InternalRow, + inMemoryThreshold: Int, + spillThreshold: Int) { private[this] val joinedRow: JoinedRow = new JoinedRow() private[this] var leftRow: InternalRow = _ private[this] var leftRowKey: InternalRow = _ @@ -975,8 +979,10 @@ private class SortMergeFullOuterJoinScanner( private[this] var leftIndex: Int = 0 private[this] var rightIndex: Int = 0 - private[this] val leftMatches: ArrayBuffer[InternalRow] = new ArrayBuffer[InternalRow] - private[this] val rightMatches: ArrayBuffer[InternalRow] = new ArrayBuffer[InternalRow] + private[this] val leftMatches: ExternalAppendOnlyUnsafeRowArray = + new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold) + private[this] val rightMatches: ExternalAppendOnlyUnsafeRowArray = + new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold) private[this] var leftMatched: BitSet = new BitSet(1) private[this] var rightMatched: BitSet = new BitSet(1) @@ -1028,23 +1034,23 @@ private class SortMergeFullOuterJoinScanner( rightIndex = 0 while (leftRowKey != null && keyOrdering.compare(leftRowKey, matchingKey) == 0) { - leftMatches += leftRow.copy() + leftMatches.add(leftRow.copy().asInstanceOf[UnsafeRow]) advancedLeft() } while (rightRowKey != null && keyOrdering.compare(rightRowKey, matchingKey) == 0) { - rightMatches += rightRow.copy() + rightMatches.add(rightRow.copy().asInstanceOf[UnsafeRow]) advancedRight() } - if (leftMatches.size <= leftMatched.capacity) { - leftMatched.clearUntil(leftMatches.size) + if (leftMatches.length <= leftMatched.capacity) { + leftMatched.clearUntil(leftMatches.length) } else { - leftMatched = new BitSet(leftMatches.size) + leftMatched = new BitSet(leftMatches.length) } - if (rightMatches.size <= rightMatched.capacity) { - rightMatched.clearUntil(rightMatches.size) + if (rightMatches.length <= rightMatched.capacity) { + rightMatched.clearUntil(rightMatches.length) } else { - rightMatched = new BitSet(rightMatches.size) + rightMatched = new BitSet(rightMatches.length) } } @@ -1058,31 +1064,37 @@ private class SortMergeFullOuterJoinScanner( * @return true if a valid match is found, false otherwise. */ private def scanNextInBuffered(): Boolean = { - while (leftIndex < leftMatches.size) { - while (rightIndex < rightMatches.size) { - joinedRow(leftMatches(leftIndex), rightMatches(rightIndex)) - if (boundCondition(joinedRow)) { - leftMatched.set(leftIndex) - rightMatched.set(rightIndex) + val leftMatchesIterator = leftMatches.generateIterator(leftIndex) + + while (leftMatchesIterator.hasNext) { + val leftCurRow = leftMatchesIterator.next() + val rightMatchesIterator = rightMatches.generateIterator(rightIndex) + while (rightMatchesIterator.hasNext) { + joinedRow(leftCurRow, rightMatchesIterator.next()) + if (boundCondition(joinedRow)) { + leftMatched.set(leftIndex) + rightMatched.set(rightIndex) + rightIndex += 1 + return true + } rightIndex += 1 - return true - } - rightIndex += 1 } rightIndex = 0 if (!leftMatched.get(leftIndex)) { // the left row has never matched any right row, join it with null row - joinedRow(leftMatches(leftIndex), rightNullRow) + joinedRow(leftCurRow, rightNullRow) leftIndex += 1 return true } leftIndex += 1 } - while (rightIndex < rightMatches.size) { + val rightMatchesIterator = rightMatches.generateIterator(rightIndex) + while (rightMatchesIterator.hasNext) { + val rightCurRow = rightMatchesIterator.next() if (!rightMatched.get(rightIndex)) { // the right row has never matched any left row, join it with null row - joinedRow(leftNullRow, rightMatches(rightIndex)) + joinedRow(leftNullRow, rightCurRow) rightIndex += 1 return true } @@ -1099,7 +1111,7 @@ private class SortMergeFullOuterJoinScanner( def advanceNext(): Boolean = { // If we already buffered some matching rows, use them directly - if (leftIndex <= leftMatches.size || rightIndex <= rightMatches.size) { + if (leftIndex <= leftMatches.length || rightIndex <= rightMatches.length) { if (scanNextInBuffered()) { return true } 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 44767dfc9249..7535fb5a4a32 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 @@ -759,9 +759,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { ) } - // FULL OUTER JOIN still does not use [[ExternalAppendOnlyUnsafeRowArray]] - // so should not cause any spill - assertNotSpilled(sparkContext, "full outer join") { + assertSpilled(sparkContext, "full outer join") { checkAnswer( sql( """ From d1ae0a40bd5ffdb65afd684a32370e8ab5217bf0 Mon Sep 17 00:00:00 2001 From: sujithjay Date: Tue, 21 Aug 2018 22:29:33 +0530 Subject: [PATCH 0002/1072] [SPARK-24985][SQL] Clean up unused imports --- .../apache/spark/sql/execution/joins/SortMergeJoinExec.scala | 2 -- 1 file changed, 2 deletions(-) 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 48584712a226..c7c82681d741 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 @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ From 121b3745b0fd22efab4825926389c54d841f08c3 Mon Sep 17 00:00:00 2001 From: sujithjay Date: Tue, 21 Aug 2018 22:35:51 +0530 Subject: [PATCH 0003/1072] [SPARK-24985][SQL] Remove `copy()` on row instance before adding to `ExternalAppendOnlyUnsafeRowArray`. `ExternalAppendOnlyUnsafeRowArray` handles copy internally. --- .../apache/spark/sql/execution/joins/SortMergeJoinExec.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 c7c82681d741..ca77fe98d6ab 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 @@ -1032,11 +1032,11 @@ private class SortMergeFullOuterJoinScanner( rightIndex = 0 while (leftRowKey != null && keyOrdering.compare(leftRowKey, matchingKey) == 0) { - leftMatches.add(leftRow.copy().asInstanceOf[UnsafeRow]) + leftMatches.add(leftRow.asInstanceOf[UnsafeRow]) advancedLeft() } while (rightRowKey != null && keyOrdering.compare(rightRowKey, matchingKey) == 0) { - rightMatches.add(rightRow.copy().asInstanceOf[UnsafeRow]) + rightMatches.add(rightRow.asInstanceOf[UnsafeRow]) advancedRight() } From 09a91d98bdecb86ecad4647b7ef5fb3f69bdc671 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 28 Nov 2018 16:21:42 +0800 Subject: [PATCH 0004/1072] [SPARK-26021][SQL][FOLLOWUP] add test for special floating point values ## What changes were proposed in this pull request? a followup of https://github.com/apache/spark/pull/23043 . Add a test to show the minor behavior change introduced by #23043 , and add migration guide. ## How was this patch tested? a new test Closes #23141 from cloud-fan/follow. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../spark/unsafe/PlatformUtilSuite.java | 12 +++++--- docs/sql-migration-guide-upgrade.md | 6 ++-- .../catalyst/expressions/UnsafeArrayData.java | 6 ---- .../spark/sql/DatasetPrimitiveSuite.scala | 29 +++++++++++++++++++ .../org/apache/spark/sql/QueryTest.scala | 7 +++++ 5 files changed, 48 insertions(+), 12 deletions(-) diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java index ab34324eb54c..2474081dad5c 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -165,10 +165,14 @@ public void writeMinusZeroIsReplacedWithZero() { byte[] floatBytes = new byte[Float.BYTES]; Platform.putDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET, -0.0d); Platform.putFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET, -0.0f); - double doubleFromPlatform = Platform.getDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET); - float floatFromPlatform = Platform.getFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET); - Assert.assertEquals(Double.doubleToLongBits(0.0d), Double.doubleToLongBits(doubleFromPlatform)); - Assert.assertEquals(Float.floatToIntBits(0.0f), Float.floatToIntBits(floatFromPlatform)); + byte[] doubleBytes2 = new byte[Double.BYTES]; + byte[] floatBytes2 = new byte[Float.BYTES]; + Platform.putDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET, 0.0d); + Platform.putFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET, 0.0f); + + // Make sure the bytes we write from 0.0 and -0.0 are same. + Assert.assertArrayEquals(doubleBytes, doubleBytes2); + Assert.assertArrayEquals(floatBytes, floatBytes2); } } diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 68cb8f5a0d18..25cd54119091 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -17,14 +17,16 @@ displayTitle: Spark SQL Upgrading Guide - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. - - In Spark version 2.4 and earlier, the `from_json` function produces `null`s for JSON strings and JSON datasource skips the same independetly of its mode if there is no valid root JSON token in its input (` ` for example). Since Spark 3.0, such input is treated as a bad record and handled according to specified mode. For example, in the `PERMISSIVE` mode the ` ` input is converted to `Row(null, null)` if specified schema is `key STRING, value INT`. + - In Spark version 2.4 and earlier, the `from_json` function produces `null`s for JSON strings and JSON datasource skips the same independetly of its mode if there is no valid root JSON token in its input (` ` for example). Since Spark 3.0, such input is treated as a bad record and handled according to specified mode. For example, in the `PERMISSIVE` mode the ` ` input is converted to `Row(null, null)` if specified schema is `key STRING, value INT`. - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. - In Spark version 2.4 and earlier, users can create map values with map type key via built-in function like `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can still read map values with map type key from data source or Java/Scala collections, though they are not very useful. - + - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a grouped dataset with key attribute wrongly named as "value", if the key is non-struct type, e.g. int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries weird. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behaviour is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`. + - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal to 0.0, but users can still distinguish them via `Dataset.show`, `Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 internally, and users can't distinguish them any more. + ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index 9002abdcfd47..d5f679fe23d4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -334,17 +334,11 @@ public void setLong(int ordinal, long value) { } public void setFloat(int ordinal, float value) { - if (Float.isNaN(value)) { - value = Float.NaN; - } assertIndexIsValid(ordinal); Platform.putFloat(baseObject, getElementOffset(ordinal, 4), value); } public void setDouble(int ordinal, double value) { - if (Double.isNaN(value)) { - value = Double.NaN; - } assertIndexIsValid(ordinal); Platform.putDouble(baseObject, getElementOffset(ordinal, 8), value); } 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 96a6792f52f3..0ded5d8ce1e2 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 @@ -393,4 +393,33 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { val ds = spark.createDataset(data) checkDataset(ds, data: _*) } + + test("special floating point values") { + import org.scalatest.exceptions.TestFailedException + + // Spark treats -0.0 as 0.0 + intercept[TestFailedException] { + checkDataset(Seq(-0.0d).toDS(), -0.0d) + } + intercept[TestFailedException] { + checkDataset(Seq(-0.0f).toDS(), -0.0f) + } + intercept[TestFailedException] { + checkDataset(Seq(Tuple1(-0.0)).toDS(), Tuple1(-0.0)) + } + + val floats = Seq[Float](-0.0f, 0.0f, Float.NaN).toDS() + checkDataset(floats, 0.0f, 0.0f, Float.NaN) + + val doubles = Seq[Double](-0.0d, 0.0d, Double.NaN).toDS() + checkDataset(doubles, 0.0, 0.0, Double.NaN) + + checkDataset(Seq(Tuple1(Float.NaN)).toDS(), Tuple1(Float.NaN)) + checkDataset(Seq(Tuple1(-0.0f)).toDS(), Tuple1(0.0f)) + checkDataset(Seq(Tuple1(Double.NaN)).toDS(), Tuple1(Double.NaN)) + checkDataset(Seq(Tuple1(-0.0)).toDS(), Tuple1(0.0)) + + val complex = Map(Array(Seq(Tuple1(Double.NaN))) -> Map(Tuple2(Float.NaN, null))) + checkDataset(Seq(complex).toDS(), complex) + } } 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 8ba67239fb90..a547676c5ed5 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 @@ -132,6 +132,13 @@ abstract class QueryTest extends PlanTest { a.length == b.length && a.zip(b).forall { case (l, r) => compare(l, r)} case (a: Iterable[_], b: Iterable[_]) => a.size == b.size && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a: Product, b: Product) => + compare(a.productIterator.toSeq, b.productIterator.toSeq) + // 0.0 == -0.0, turn float/double to binary before comparison, to distinguish 0.0 and -0.0. + case (a: Double, b: Double) => + java.lang.Double.doubleToRawLongBits(a) == java.lang.Double.doubleToRawLongBits(b) + case (a: Float, b: Float) => + java.lang.Float.floatToRawIntBits(a) == java.lang.Float.floatToRawIntBits(b) case (a, b) => a == b } From 93112e693082f3fba24cebaf9a98dcf5c1eb84af Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 28 Nov 2018 20:18:13 +0800 Subject: [PATCH 0005/1072] [SPARK-26142][SQL] Implement shuffle read metrics in SQL ## What changes were proposed in this pull request? Implement `SQLShuffleMetricsReporter` on the sql side as the customized ShuffleMetricsReporter, which extended the `TempShuffleReadMetrics` and update SQLMetrics, in this way shuffle metrics can be reported in the SQL UI. ## How was this patch tested? Add UT in SQLMetricsSuite. Manual test locally, before: ![image](https://user-images.githubusercontent.com/4833765/48960517-30f97880-efa8-11e8-982c-92d05938fd1d.png) after: ![image](https://user-images.githubusercontent.com/4833765/48960587-b54bfb80-efa8-11e8-8e95-7a3c8c74cc5c.png) Closes #23128 from xuanyuanking/SPARK-26142. Lead-authored-by: Yuanjian Li Co-authored-by: liyuanjian Signed-off-by: Wenchen Fan --- .../spark/sql/execution/ShuffledRowRDD.scala | 9 ++- .../exchange/ShuffleExchangeExec.scala | 5 +- .../apache/spark/sql/execution/limit.scala | 10 ++- .../sql/execution/metric/SQLMetrics.scala | 20 ++++++ .../metric/SQLShuffleMetricsReporter.scala | 67 +++++++++++++++++++ .../execution/UnsafeRowSerializerSuite.scala | 5 +- .../execution/metric/SQLMetricsSuite.scala | 21 ++++-- 7 files changed, 126 insertions(+), 11 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala 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 542266bc1ae0..9b05faaed045 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 @@ -22,6 +22,7 @@ import java.util.Arrays import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleMetricsReporter} /** * The [[Partition]] used by [[ShuffledRowRDD]]. A post-shuffle partition @@ -112,6 +113,7 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A */ class ShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], + metrics: Map[String, SQLMetric], specifiedPartitionStartIndices: Option[Array[Int]] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { @@ -154,7 +156,10 @@ class ShuffledRowRDD( override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val shuffledRowPartition = split.asInstanceOf[ShuffledRowRDDPartition] - val metrics = context.taskMetrics().createTempShuffleReadMetrics() + val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() + // `SQLShuffleMetricsReporter` will update its own metrics for SQL exchange operator, + // as well as the `tempMetrics` for basic shuffle metrics. + val sqlMetricsReporter = new SQLShuffleMetricsReporter(tempMetrics, metrics) // The range of pre-shuffle partitions that we are fetching at here is // [startPreShufflePartitionIndex, endPreShufflePartitionIndex - 1]. val reader = @@ -163,7 +168,7 @@ class ShuffledRowRDD( shuffledRowPartition.startPreShufflePartitionIndex, shuffledRowPartition.endPreShufflePartitionIndex, context, - metrics) + sqlMetricsReporter) reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) } 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 d6742ab3e0f3..8938d93da90e 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 @@ -48,7 +48,8 @@ case class ShuffleExchangeExec( // e.g. it can be null on the Executor side override lazy val metrics = Map( - "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size")) + "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size") + ) ++ SQLMetrics.getShuffleReadMetrics(sparkContext) override def nodeName: String = { val extraInfo = coordinator match { @@ -108,7 +109,7 @@ case class ShuffleExchangeExec( assert(newPartitioning.isInstanceOf[HashPartitioning]) newPartitioning = UnknownPartitioning(indices.length) } - new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices) + new ShuffledRowRDD(shuffleDependency, metrics, specifiedPartitionStartIndices) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 90dafcf53591..ea845da8438f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGe import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.metric.SQLMetrics /** * Take the first `limit` elements and collect them to a single partition. @@ -37,11 +38,13 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode override def outputPartitioning: Partitioning = SinglePartition override def executeCollect(): Array[InternalRow] = child.executeTake(limit) private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) + override lazy val metrics = SQLMetrics.getShuffleReadMetrics(sparkContext) protected override def doExecute(): RDD[InternalRow] = { val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit)) val shuffled = new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( - locallyLimited, child.output, SinglePartition, serializer)) + locallyLimited, child.output, SinglePartition, serializer), + metrics) shuffled.mapPartitionsInternal(_.take(limit)) } } @@ -151,6 +154,8 @@ case class TakeOrderedAndProjectExec( private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) + override lazy val metrics = SQLMetrics.getShuffleReadMetrics(sparkContext) + protected override def doExecute(): RDD[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) val localTopK: RDD[InternalRow] = { @@ -160,7 +165,8 @@ case class TakeOrderedAndProjectExec( } val shuffled = new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( - localTopK, child.output, SinglePartition, serializer)) + localTopK, child.output, SinglePartition, serializer), + metrics) shuffled.mapPartitions { iter => val topK = org.apache.spark.util.collection.Utils.takeOrdered(iter.map(_.copy()), limit)(ord) if (projectList != child.output) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index cbf707f4a9cf..0b5ee3a5e057 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -82,6 +82,14 @@ object SQLMetrics { private val baseForAvgMetric: Int = 10 + val REMOTE_BLOCKS_FETCHED = "remoteBlocksFetched" + val LOCAL_BLOCKS_FETCHED = "localBlocksFetched" + val REMOTE_BYTES_READ = "remoteBytesRead" + val REMOTE_BYTES_READ_TO_DISK = "remoteBytesReadToDisk" + val LOCAL_BYTES_READ = "localBytesRead" + val FETCH_WAIT_TIME = "fetchWaitTime" + val RECORDS_READ = "recordsRead" + /** * Converts a double value to long value by multiplying a base integer, so we can store it in * `SQLMetrics`. It only works for average metrics. When showing the metrics on UI, we restore @@ -194,4 +202,16 @@ object SQLMetrics { SparkListenerDriverAccumUpdates(executionId.toLong, metrics.map(m => m.id -> m.value))) } } + + /** + * Create all shuffle read relative metrics and return the Map. + */ + def getShuffleReadMetrics(sc: SparkContext): Map[String, SQLMetric] = Map( + REMOTE_BLOCKS_FETCHED -> createMetric(sc, "remote blocks fetched"), + LOCAL_BLOCKS_FETCHED -> createMetric(sc, "local blocks fetched"), + REMOTE_BYTES_READ -> createSizeMetric(sc, "remote bytes read"), + REMOTE_BYTES_READ_TO_DISK -> createSizeMetric(sc, "remote bytes read to disk"), + LOCAL_BYTES_READ -> createSizeMetric(sc, "local bytes read"), + FETCH_WAIT_TIME -> createTimingMetric(sc, "fetch wait time"), + RECORDS_READ -> createMetric(sc, "records read")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala new file mode 100644 index 000000000000..542141ea4b4e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.metric + +import org.apache.spark.executor.TempShuffleReadMetrics + +/** + * A shuffle metrics reporter for SQL exchange operators. + * @param tempMetrics [[TempShuffleReadMetrics]] created in TaskContext. + * @param metrics All metrics in current SparkPlan. This param should not empty and + * contains all shuffle metrics defined in [[SQLMetrics.getShuffleReadMetrics]]. + */ +private[spark] class SQLShuffleMetricsReporter( + tempMetrics: TempShuffleReadMetrics, + metrics: Map[String, SQLMetric]) extends TempShuffleReadMetrics { + private[this] val _remoteBlocksFetched = metrics(SQLMetrics.REMOTE_BLOCKS_FETCHED) + private[this] val _localBlocksFetched = metrics(SQLMetrics.LOCAL_BLOCKS_FETCHED) + private[this] val _remoteBytesRead = metrics(SQLMetrics.REMOTE_BYTES_READ) + private[this] val _remoteBytesReadToDisk = metrics(SQLMetrics.REMOTE_BYTES_READ_TO_DISK) + private[this] val _localBytesRead = metrics(SQLMetrics.LOCAL_BYTES_READ) + private[this] val _fetchWaitTime = metrics(SQLMetrics.FETCH_WAIT_TIME) + private[this] val _recordsRead = metrics(SQLMetrics.RECORDS_READ) + + override def incRemoteBlocksFetched(v: Long): Unit = { + _remoteBlocksFetched.add(v) + tempMetrics.incRemoteBlocksFetched(v) + } + override def incLocalBlocksFetched(v: Long): Unit = { + _localBlocksFetched.add(v) + tempMetrics.incLocalBlocksFetched(v) + } + override def incRemoteBytesRead(v: Long): Unit = { + _remoteBytesRead.add(v) + tempMetrics.incRemoteBytesRead(v) + } + override def incRemoteBytesReadToDisk(v: Long): Unit = { + _remoteBytesReadToDisk.add(v) + tempMetrics.incRemoteBytesReadToDisk(v) + } + override def incLocalBytesRead(v: Long): Unit = { + _localBytesRead.add(v) + tempMetrics.incLocalBytesRead(v) + } + override def incFetchWaitTime(v: Long): Unit = { + _fetchWaitTime.add(v) + tempMetrics.incFetchWaitTime(v) + } + override def incRecordsRead(v: Long): Unit = { + _recordsRead.add(v) + tempMetrics.incRecordsRead(v) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index d305ce3e698a..96b3aa5ee75b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{LocalSparkSession, Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types._ import org.apache.spark.storage.ShuffleBlockId import org.apache.spark.util.collection.ExternalSorter @@ -137,7 +138,9 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { rowsRDD, new PartitionIdPassthrough(2), new UnsafeRowSerializer(2)) - val shuffled = new ShuffledRowRDD(dependency) + val shuffled = new ShuffledRowRDD( + dependency, + SQLMetrics.getShuffleReadMetrics(spark.sparkContext)) shuffled.count() } } 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 b955c157a620..0f1d08b6af5d 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 @@ -94,8 +94,13 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"), Map("number of output rows" -> 1L, "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) + val shuffleExpected1 = Map( + "records read" -> 2L, + "local blocks fetched" -> 2L, + "remote blocks fetched" -> 0L) testSparkPlanMetrics(df, 1, Map( 2L -> (("HashAggregate", expected1(0))), + 1L -> (("Exchange", shuffleExpected1)), 0L -> (("HashAggregate", expected1(1)))) ) @@ -106,8 +111,13 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"), Map("number of output rows" -> 3L, "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) + val shuffleExpected2 = Map( + "records read" -> 4L, + "local blocks fetched" -> 4L, + "remote blocks fetched" -> 0L) testSparkPlanMetrics(df2, 1, Map( 2L -> (("HashAggregate", expected2(0))), + 1L -> (("Exchange", shuffleExpected2)), 0L -> (("HashAggregate", expected2(1)))) ) } @@ -191,7 +201,11 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared testSparkPlanMetrics(df, 1, Map( 0L -> (("SortMergeJoin", Map( // It's 4 because we only read 3 rows in the first partition and 1 row in the second one - "number of output rows" -> 4L)))) + "number of output rows" -> 4L))), + 2L -> (("Exchange", Map( + "records read" -> 4L, + "local blocks fetched" -> 2L, + "remote blocks fetched" -> 0L)))) ) } } @@ -208,7 +222,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "SELECT * FROM testData2 left JOIN testDataForJoin ON testData2.a = testDataForJoin.a") testSparkPlanMetrics(df, 1, Map( 0L -> (("SortMergeJoin", Map( - // It's 4 because we only read 3 rows in the first partition and 1 row in the second one + // It's 8 because we read 6 rows in the left and 2 row in the right one "number of output rows" -> 8L)))) ) @@ -216,7 +230,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "SELECT * FROM testDataForJoin right JOIN testData2 ON testData2.a = testDataForJoin.a") testSparkPlanMetrics(df2, 1, Map( 0L -> (("SortMergeJoin", Map( - // It's 4 because we only read 3 rows in the first partition and 1 row in the second one + // It's 8 because we read 6 rows in the left and 2 row in the right one "number of output rows" -> 8L)))) ) } @@ -287,7 +301,6 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared // Assume the execution plan is // ... -> ShuffledHashJoin(nodeId = 1) -> Project(nodeId = 0) val df = df1.join(df2, "key") - val metrics = getSparkPlanMetrics(df, 1, Set(1L)) testSparkPlanMetrics(df, 1, Map( 1L -> (("ShuffledHashJoin", Map( "number of output rows" -> 2L, From 438f8fd675d8f819373b6643dea3a77d954b6822 Mon Sep 17 00:00:00 2001 From: Sergey Zhemzhitsky Date: Wed, 28 Nov 2018 20:22:24 +0800 Subject: [PATCH 0006/1072] [SPARK-26114][CORE] ExternalSorter's readingIterator field leak ## What changes were proposed in this pull request? This pull request fixes [SPARK-26114](https://issues.apache.org/jira/browse/SPARK-26114) issue that occurs when trying to reduce the number of partitions by means of coalesce without shuffling after shuffle-based transformations. The leak occurs because of not cleaning up `ExternalSorter`'s `readingIterator` field as it's done for its `map` and `buffer` fields. Additionally there are changes to the `CompletionIterator` to prevent capturing its `sub`-iterator and holding it even after the completion iterator completes. It is necessary because in some cases, e.g. in case of standard scala's `flatMap` iterator (which is used is `CoalescedRDD`'s `compute` method) the next value of the main iterator is assigned to `flatMap`'s `cur` field only after it is available. For DAGs where ShuffledRDD is a parent of CoalescedRDD it means that the data should be fetched from the map-side of the shuffle, but the process of fetching this data consumes quite a lot of memory in addition to the memory already consumed by the iterator held by `flatMap`'s `cur` field (until it is reassigned). For the following data ```scala import org.apache.hadoop.io._ import org.apache.hadoop.io.compress._ import org.apache.commons.lang._ import org.apache.spark._ // generate 100M records of sample data sc.makeRDD(1 to 1000, 1000) .flatMap(item => (1 to 100000) .map(i => new Text(RandomStringUtils.randomAlphanumeric(3).toLowerCase) -> new Text(RandomStringUtils.randomAlphanumeric(1024)))) .saveAsSequenceFile("/tmp/random-strings", Some(classOf[GzipCodec])) ``` and the following job ```scala import org.apache.hadoop.io._ import org.apache.spark._ import org.apache.spark.storage._ val rdd = sc.sequenceFile("/tmp/random-strings", classOf[Text], classOf[Text]) rdd .map(item => item._1.toString -> item._2.toString) .repartitionAndSortWithinPartitions(new HashPartitioner(1000)) .coalesce(10,false) .count ``` ... executed like the following ```bash spark-shell \ --num-executors=5 \ --executor-cores=2 \ --master=yarn \ --deploy-mode=client \ --conf spark.executor.memoryOverhead=512 \ --conf spark.executor.memory=1g \ --conf spark.dynamicAllocation.enabled=false \ --conf spark.executor.extraJavaOptions='-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp -Dio.netty.noUnsafe=true' ``` ... executors are always failing with OutOfMemoryErrors. The main issue is multiple leaks of ExternalSorter references. For example, in case of 2 tasks per executor it is expected to be 2 simultaneous instances of ExternalSorter per executor but heap dump generated on OutOfMemoryError shows that there are more ones. ![run1-noparams-dominator-tree-externalsorter](https://user-images.githubusercontent.com/1523889/48703665-782ce580-ec05-11e8-95a9-d6c94e8285ab.png) P.S. This PR does not cover cases with CoGroupedRDDs which use ExternalAppendOnlyMap internally, which itself can lead to OutOfMemoryErrors in many places. ## How was this patch tested? - Existing unit tests - New unit tests - Job executions on the live environment Here is the screenshot before applying this patch ![run3-noparams-failure-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700395-f769eb80-ebfc-11e8-831b-e94c757d416c.png) Here is the screenshot after applying this patch ![run3-noparams-success-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700610-7a8b4180-ebfd-11e8-9761-baaf38a58e66.png) And in case of reducing the number of executors even more the job is still stable ![run3-noparams-success-ui-2x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700619-82e37c80-ebfd-11e8-98ed-a38e1f1f1fd9.png) Closes #23083 from szhem/SPARK-26114-externalsorter-leak. Authored-by: Sergey Zhemzhitsky Signed-off-by: Wenchen Fan --- .../spark/util/CompletionIterator.scala | 7 ++++-- .../util/collection/ExternalSorter.scala | 3 ++- .../spark/util/CompletionIteratorSuite.scala | 22 +++++++++++++++++++ 3 files changed, 29 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala index 21acaa95c564..f4d6c7a28d2e 100644 --- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala @@ -25,11 +25,14 @@ private[spark] abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterator[A] { private[this] var completed = false - def next(): A = sub.next() + private[this] var iter = sub + def next(): A = iter.next() def hasNext: Boolean = { - val r = sub.hasNext + val r = iter.hasNext if (!r && !completed) { completed = true + // reassign to release resources of highly resource consuming iterators early + iter = Iterator.empty.asInstanceOf[I] completion() } r 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 eac3db01158d..46279e79d78d 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 @@ -727,9 +727,10 @@ private[spark] class ExternalSorter[K, V, C]( spills.clear() forceSpillFiles.foreach(s => s.file.delete()) forceSpillFiles.clear() - if (map != null || buffer != null) { + if (map != null || buffer != null || readingIterator != null) { map = null // So that the memory can be garbage-collected buffer = null // So that the memory can be garbage-collected + readingIterator = null // So that the memory can be garbage-collected releaseMemory() } } diff --git a/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala index 688fcd9f9aab..29421f7aa9e3 100644 --- a/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.util +import java.lang.ref.PhantomReference +import java.lang.ref.ReferenceQueue + import org.apache.spark.SparkFunSuite class CompletionIteratorSuite extends SparkFunSuite { @@ -44,4 +47,23 @@ class CompletionIteratorSuite extends SparkFunSuite { assert(!completionIter.hasNext) assert(numTimesCompleted === 1) } + test("reference to sub iterator should not be available after completion") { + var sub = Iterator(1, 2, 3) + + val refQueue = new ReferenceQueue[Iterator[Int]] + val ref = new PhantomReference[Iterator[Int]](sub, refQueue) + + val iter = CompletionIterator[Int, Iterator[Int]](sub, {}) + sub = null + iter.toArray + + for (_ <- 1 to 100 if !ref.isEnqueued) { + System.gc() + if (!ref.isEnqueued) { + Thread.sleep(10) + } + } + assert(ref.isEnqueued) + assert(refQueue.poll() === ref) + } } From affe80958d366f399466a9dba8e03da7f3b7b9bf Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 28 Nov 2018 20:38:42 +0800 Subject: [PATCH 0007/1072] [SPARK-26147][SQL] only pull out unevaluable python udf from join condition ## What changes were proposed in this pull request? https://github.com/apache/spark/pull/22326 made a mistake that, not all python UDFs are unevaluable in join condition. Only python UDFs that refer to attributes from both join side are unevaluable. This PR fixes this mistake. ## How was this patch tested? a new test Closes #23153 from cloud-fan/join. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- python/pyspark/sql/tests/test_udf.py | 12 ++ .../spark/sql/catalyst/optimizer/joins.scala | 22 ++-- ...PullOutPythonUDFInJoinConditionSuite.scala | 120 ++++++++++++------ 3 files changed, 106 insertions(+), 48 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index d2dfb52f5447..ed298f724d55 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -209,6 +209,18 @@ def test_udf_in_join_condition(self): with self.sql_conf({"spark.sql.crossJoin.enabled": True}): self.assertEqual(df.collect(), [Row(a=1, b=1)]) + def test_udf_in_left_outer_join_condition(self): + # regression test for SPARK-26147 + from pyspark.sql.functions import udf, col + left = self.spark.createDataFrame([Row(a=1)]) + right = self.spark.createDataFrame([Row(b=1)]) + f = udf(lambda a: str(a), StringType()) + # The join condition can't be pushed down, as it refers to attributes from both sides. + # The Python UDF only refer to attributes from one side, so it's evaluable. + df = left.join(right, f("a") == col("b").cast("string"), how="left_outer") + with self.sql_conf({"spark.sql.crossJoin.enabled": True}): + self.assertEqual(df.collect(), [Row(a=1, b=1)]) + def test_udf_in_left_semi_join_condition(self): # regression test for SPARK-25314 from pyspark.sql.functions import udf diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 7149edee0173..6ebb194d71c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -155,19 +155,20 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { } /** - * PythonUDF in join condition can not be evaluated, this rule will detect the PythonUDF - * and pull them out from join condition. For python udf accessing attributes from only one side, - * they are pushed down by operation push down rules. If not (e.g. user disables filter push - * down rules), we need to pull them out in this rule too. + * PythonUDF in join condition can't be evaluated if it refers to attributes from both join sides. + * See `ExtractPythonUDFs` for details. This rule will detect un-evaluable PythonUDF and pull them + * out from join condition. */ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateHelper { - def hasPythonUDF(expression: Expression): Boolean = { - expression.collectFirst { case udf: PythonUDF => udf }.isDefined + + private def hasUnevaluablePythonUDF(expr: Expression, j: Join): Boolean = { + expr.find { e => + PythonUDF.isScalarPythonUDF(e) && !canEvaluate(e, j.left) && !canEvaluate(e, j.right) + }.isDefined } override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case j @ Join(_, _, joinType, condition) - if condition.isDefined && hasPythonUDF(condition.get) => + case j @ Join(_, _, joinType, Some(cond)) if hasUnevaluablePythonUDF(cond, j) => if (!joinType.isInstanceOf[InnerLike] && joinType != LeftSemi) { // The current strategy only support InnerLike and LeftSemi join because for other type, // it breaks SQL semantic if we run the join condition as a filter after join. If we pass @@ -179,10 +180,9 @@ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateH } // If condition expression contains python udf, it will be moved out from // the new join conditions. - val (udf, rest) = - splitConjunctivePredicates(condition.get).partition(hasPythonUDF) + val (udf, rest) = splitConjunctivePredicates(cond).partition(hasUnevaluablePythonUDF(_, j)) val newCondition = if (rest.isEmpty) { - logWarning(s"The join condition:$condition of the join plan contains PythonUDF only," + + logWarning(s"The join condition:$cond of the join plan contains PythonUDF only," + s" it will be moved out and the join plan will be turned to cross join.") None } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala index d3867f2b6bd0..3f1c91df7f2e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.optimizer -import org.scalatest.Matchers._ - import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -28,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.internal.SQLConf._ -import org.apache.spark.sql.types.BooleanType +import org.apache.spark.sql.types.{BooleanType, IntegerType} class PullOutPythonUDFInJoinConditionSuite extends PlanTest { @@ -40,13 +38,29 @@ class PullOutPythonUDFInJoinConditionSuite extends PlanTest { CheckCartesianProducts) :: Nil } - val testRelationLeft = LocalRelation('a.int, 'b.int) - val testRelationRight = LocalRelation('c.int, 'd.int) + val attrA = 'a.int + val attrB = 'b.int + val attrC = 'c.int + val attrD = 'd.int + + val testRelationLeft = LocalRelation(attrA, attrB) + val testRelationRight = LocalRelation(attrC, attrD) + + // This join condition refers to attributes from 2 tables, but the PythonUDF inside it only + // refer to attributes from one side. + val evaluableJoinCond = { + val pythonUDF = PythonUDF("evaluable", null, + IntegerType, + Seq(attrA), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + pythonUDF === attrC + } - // Dummy python UDF for testing. Unable to execute. - val pythonUDF = PythonUDF("pythonUDF", null, + // This join condition is a PythonUDF which refers to attributes from 2 tables. + val unevaluableJoinCond = PythonUDF("unevaluable", null, BooleanType, - Seq.empty, + Seq(attrA, attrC), PythonEvalType.SQL_BATCHED_UDF, udfDeterministic = true) @@ -66,62 +80,76 @@ class PullOutPythonUDFInJoinConditionSuite extends PlanTest { } } - test("inner join condition with python udf only") { - val query = testRelationLeft.join( + test("inner join condition with python udf") { + val query1 = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some(pythonUDF)) - val expected = testRelationLeft.join( + condition = Some(unevaluableJoinCond)) + val expected1 = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = None).where(pythonUDF).analyze - comparePlanWithCrossJoinEnable(query, expected) + condition = None).where(unevaluableJoinCond).analyze + comparePlanWithCrossJoinEnable(query1, expected1) + + // evaluable PythonUDF will not be touched + val query2 = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond)) + comparePlans(Optimize.execute(query2), query2) } - test("left semi join condition with python udf only") { - val query = testRelationLeft.join( + test("left semi join condition with python udf") { + val query1 = testRelationLeft.join( testRelationRight, joinType = LeftSemi, - condition = Some(pythonUDF)) - val expected = testRelationLeft.join( + condition = Some(unevaluableJoinCond)) + val expected1 = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = None).where(pythonUDF).select('a, 'b).analyze - comparePlanWithCrossJoinEnable(query, expected) + condition = None).where(unevaluableJoinCond).select('a, 'b).analyze + comparePlanWithCrossJoinEnable(query1, expected1) + + // evaluable PythonUDF will not be touched + val query2 = testRelationLeft.join( + testRelationRight, + joinType = LeftSemi, + condition = Some(evaluableJoinCond)) + comparePlans(Optimize.execute(query2), query2) } - test("python udf and common condition") { + test("unevaluable python udf and common condition") { val query = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some(pythonUDF && 'a.attr === 'c.attr)) + condition = Some(unevaluableJoinCond && 'a.attr === 'c.attr)) val expected = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some('a.attr === 'c.attr)).where(pythonUDF).analyze + condition = Some('a.attr === 'c.attr)).where(unevaluableJoinCond).analyze val optimized = Optimize.execute(query.analyze) comparePlans(optimized, expected) } - test("python udf or common condition") { + test("unevaluable python udf or common condition") { val query = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some(pythonUDF || 'a.attr === 'c.attr)) + condition = Some(unevaluableJoinCond || 'a.attr === 'c.attr)) val expected = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = None).where(pythonUDF || 'a.attr === 'c.attr).analyze + condition = None).where(unevaluableJoinCond || 'a.attr === 'c.attr).analyze comparePlanWithCrossJoinEnable(query, expected) } - test("pull out whole complex condition with multiple python udf") { + test("pull out whole complex condition with multiple unevaluable python udf") { val pythonUDF1 = PythonUDF("pythonUDF1", null, BooleanType, - Seq.empty, + Seq(attrA, attrC), PythonEvalType.SQL_BATCHED_UDF, udfDeterministic = true) - val condition = (pythonUDF || 'a.attr === 'c.attr) && pythonUDF1 + val condition = (unevaluableJoinCond || 'a.attr === 'c.attr) && pythonUDF1 val query = testRelationLeft.join( testRelationRight, @@ -134,13 +162,13 @@ class PullOutPythonUDFInJoinConditionSuite extends PlanTest { comparePlanWithCrossJoinEnable(query, expected) } - test("partial pull out complex condition with multiple python udf") { + test("partial pull out complex condition with multiple unevaluable python udf") { val pythonUDF1 = PythonUDF("pythonUDF1", null, BooleanType, - Seq.empty, + Seq(attrA, attrC), PythonEvalType.SQL_BATCHED_UDF, udfDeterministic = true) - val condition = (pythonUDF || pythonUDF1) && 'a.attr === 'c.attr + val condition = (unevaluableJoinCond || pythonUDF1) && 'a.attr === 'c.attr val query = testRelationLeft.join( testRelationRight, @@ -149,23 +177,41 @@ class PullOutPythonUDFInJoinConditionSuite extends PlanTest { val expected = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some('a.attr === 'c.attr)).where(pythonUDF || pythonUDF1).analyze + condition = Some('a.attr === 'c.attr)).where(unevaluableJoinCond || pythonUDF1).analyze + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + + test("pull out unevaluable python udf when it's mixed with evaluable one") { + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond && unevaluableJoinCond)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond)).where(unevaluableJoinCond).analyze val optimized = Optimize.execute(query.analyze) comparePlans(optimized, expected) } test("throw an exception for not support join type") { for (joinType <- unsupportedJoinTypes) { - val thrownException = the [AnalysisException] thrownBy { + val e = intercept[AnalysisException] { val query = testRelationLeft.join( testRelationRight, joinType, - condition = Some(pythonUDF)) + condition = Some(unevaluableJoinCond)) Optimize.execute(query.analyze) } - assert(thrownException.message.contentEquals( + assert(e.message.contentEquals( s"Using PythonUDF in join condition of join type $joinType is not supported.")) + + val query2 = testRelationLeft.join( + testRelationRight, + joinType, + condition = Some(evaluableJoinCond)) + comparePlans(Optimize.execute(query2), query2) } } } - From ce61bac1d84f8577b180400e44bd9bf22292e0b6 Mon Sep 17 00:00:00 2001 From: Mark Pavey Date: Wed, 28 Nov 2018 07:19:47 -0800 Subject: [PATCH 0008/1072] =?UTF-8?q?[SPARK-26137][CORE]=20Use=20Java=20sy?= =?UTF-8?q?stem=20property=20"file.separator"=20inste=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … of hard coded "/" in DependencyUtils ## What changes were proposed in this pull request? Use Java system property "file.separator" instead of hard coded "/" in DependencyUtils. ## How was this patch tested? Manual test: Submit Spark application via REST API that reads data from Elasticsearch using spark-elasticsearch library. Without fix application fails with error: 18/11/22 10:36:20 ERROR Version: Multiple ES-Hadoop versions detected in the classpath; please use only one jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar jar:file:/C:/<...>/myApp-assembly-1.0.jar 18/11/22 10:36:20 ERROR Main: Application [MyApp] failed: java.lang.Error: Multiple ES-Hadoop versions detected in the classpath; please use only one jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar jar:file:/C:/<...>/myApp-assembly-1.0.jar at org.elasticsearch.hadoop.util.Version.(Version.java:73) at org.elasticsearch.hadoop.rest.RestService.findPartitions(RestService.java:214) at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions$lzycompute(AbstractEsRDD.scala:73) at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions(AbstractEsRDD.scala:72) at org.elasticsearch.spark.rdd.AbstractEsRDD.getPartitions(AbstractEsRDD.scala:44) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.rdd.RDD.partitions(RDD.scala:251) at org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.rdd.RDD.partitions(RDD.scala:251) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2126) at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:945) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) at org.apache.spark.rdd.RDD.withScope(RDD.scala:363) at org.apache.spark.rdd.RDD.collect(RDD.scala:944) ... at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.worker.DriverWrapper$.main(DriverWrapper.scala:65) at org.apache.spark.deploy.worker.DriverWrapper.main(DriverWrapper.scala) With fix application runs successfully. Closes #23102 from markpavey/JIRA_SPARK-26137_DependencyUtilsFileSeparatorFix. Authored-by: Mark Pavey Signed-off-by: Sean Owen --- .../apache/spark/deploy/DependencyUtils.scala | 3 ++- .../spark/deploy/SparkSubmitSuite.scala | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala index 178bdcfccb60..5a17a6b6e169 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala @@ -61,11 +61,12 @@ private[deploy] object DependencyUtils extends Logging { hadoopConf: Configuration, secMgr: SecurityManager): String = { val targetDir = Utils.createTempDir() + val userJarName = userJar.split(File.separatorChar).last Option(jars) .map { resolveGlobPaths(_, hadoopConf) .split(",") - .filterNot(_.contains(userJar.split("/").last)) + .filterNot(_.contains(userJarName)) .mkString(",") } .filterNot(_ == "") 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 652c36ffa6e7..c093789244bf 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -962,6 +962,25 @@ class SparkSubmitSuite } } + test("remove copies of application jar from classpath") { + val fs = File.separator + val sparkConf = new SparkConf(false) + val hadoopConf = new Configuration() + val secMgr = new SecurityManager(sparkConf) + + val appJarName = "myApp.jar" + val jar1Name = "myJar1.jar" + val jar2Name = "myJar2.jar" + val userJar = s"file:/path${fs}to${fs}app${fs}jar$fs$appJarName" + val jars = s"file:/$jar1Name,file:/$appJarName,file:/$jar2Name" + + val resolvedJars = DependencyUtils + .resolveAndDownloadJars(jars, userJar, sparkConf, hadoopConf, secMgr) + + assert(!resolvedJars.contains(appJarName)) + assert(resolvedJars.contains(jar1Name) && resolvedJars.contains(jar2Name)) + } + test("Avoid re-upload remote resources in yarn client mode") { val hadoopConf = new Configuration() updateConfWithFakeS3Fs(hadoopConf) From 87bd9c75df6b67bef903751269a4fd381f9140d9 Mon Sep 17 00:00:00 2001 From: Brandon Krieger Date: Wed, 28 Nov 2018 07:22:48 -0800 Subject: [PATCH 0009/1072] [SPARK-25998][CORE] Change TorrentBroadcast to hold weak reference of broadcast object ## What changes were proposed in this pull request? This PR changes the broadcast object in TorrentBroadcast from a strong reference to a weak reference. This allows it to be garbage collected even if the Dataset is held in memory. This is ok, because the broadcast object can always be re-read. ## How was this patch tested? Tested in Spark shell by taking a heap dump, full repro steps listed in https://issues.apache.org/jira/browse/SPARK-25998. Closes #22995 from bkrieger/bk/torrent-broadcast-weak. Authored-by: Brandon Krieger Signed-off-by: Sean Owen --- .../spark/broadcast/TorrentBroadcast.scala | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index cbd49e070f2e..26ead57316e1 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -18,6 +18,7 @@ package org.apache.spark.broadcast import java.io._ +import java.lang.ref.SoftReference import java.nio.ByteBuffer import java.util.zip.Adler32 @@ -61,9 +62,11 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) * Value of the broadcast object on executors. This is reconstructed by [[readBroadcastBlock]], * which builds this value by reading blocks from the driver and/or other executors. * - * On the driver, if the value is required, it is read lazily from the block manager. + * On the driver, if the value is required, it is read lazily from the block manager. We hold + * a soft reference so that it can be garbage collected if required, as we can always reconstruct + * in the future. */ - @transient private lazy val _value: T = readBroadcastBlock() + @transient private var _value: SoftReference[T] = _ /** The compression codec to use, or None if compression is disabled */ @transient private var compressionCodec: Option[CompressionCodec] = _ @@ -92,8 +95,15 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) /** The checksum for all the blocks. */ private var checksums: Array[Int] = _ - override protected def getValue() = { - _value + override protected def getValue() = synchronized { + val memoized: T = if (_value == null) null.asInstanceOf[T] else _value.get + if (memoized != null) { + memoized + } else { + val newlyRead = readBroadcastBlock() + _value = new SoftReference[T](newlyRead) + newlyRead + } } private def calcChecksum(block: ByteBuffer): Int = { @@ -205,8 +215,8 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) } private def readBroadcastBlock(): T = Utils.tryOrIOException { - TorrentBroadcast.synchronized { - val broadcastCache = SparkEnv.get.broadcastManager.cachedValues + val broadcastCache = SparkEnv.get.broadcastManager.cachedValues + broadcastCache.synchronized { Option(broadcastCache.get(broadcastId)).map(_.asInstanceOf[T]).getOrElse { setConf(SparkEnv.get.conf) From 9fde3deab87c8f9c6d8dd147f5d52d243ff4b7ad Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Wed, 28 Nov 2018 07:33:34 -0800 Subject: [PATCH 0010/1072] [SPARK-25989][ML] OneVsRestModel handle empty outputCols incorrectly ## What changes were proposed in this pull request? ignore empty output columns ## How was this patch tested? added tests Closes #22991 from zhengruifeng/ovrm_empty_outcol. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../spark/ml/classification/OneVsRest.scala | 35 ++++++++++++------- .../ml/classification/OneVsRestSuite.scala | 26 ++++++++++++++ 2 files changed, 48 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 1835a91775e0..2f42a5922054 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -37,7 +37,7 @@ import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} import org.apache.spark.ml.param.shared.{HasParallelism, HasWeightCol} import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel @@ -169,6 +169,12 @@ final class OneVsRestModel private[ml] ( // Check schema transformSchema(dataset.schema, logging = true) + if (getPredictionCol == "" && getRawPredictionCol == "") { + logWarning(s"$uid: OneVsRestModel.transform() was called as NOOP" + + " since no output columns were set.") + return dataset.toDF + } + // determine the input columns: these need to be passed through val origCols = dataset.schema.map(f => col(f.name)) @@ -209,6 +215,9 @@ final class OneVsRestModel private[ml] ( newDataset.unpersist() } + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + if (getRawPredictionCol != "") { val numClass = models.length @@ -219,24 +228,24 @@ final class OneVsRestModel private[ml] ( Vectors.dense(predArray) } - // output the index of the classifier with highest confidence as prediction - val labelUDF = udf { (rawPredictions: Vector) => rawPredictions.argmax.toDouble } + predictionColNames = predictionColNames :+ getRawPredictionCol + predictionColumns = predictionColumns :+ rawPredictionUDF(col(accColName)) + } - // output confidence as raw prediction, label and label metadata as prediction - aggregatedDataset - .withColumn(getRawPredictionCol, rawPredictionUDF(col(accColName))) - .withColumn(getPredictionCol, labelUDF(col(getRawPredictionCol)), labelMetadata) - .drop(accColName) - } else { + if (getPredictionCol != "") { // output the index of the classifier with highest confidence as prediction val labelUDF = udf { (predictions: Map[Int, Double]) => predictions.maxBy(_._2)._1.toDouble } - // output label and label metadata as prediction - aggregatedDataset - .withColumn(getPredictionCol, labelUDF(col(accColName)), labelMetadata) - .drop(accColName) + + predictionColNames = predictionColNames :+ getPredictionCol + predictionColumns = predictionColumns :+ labelUDF(col(accColName)) + .as(getPredictionCol, labelMetadata) } + + aggregatedDataset + .withColumns(predictionColNames, predictionColumns) + .drop(accColName) } @Since("1.4.1") diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 519ec1720eb9..b6e8c927403a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -290,6 +290,32 @@ class OneVsRestSuite extends MLTest with DefaultReadWriteTest { checkModelData(ovaModel, newOvaModel) } + test("should ignore empty output cols") { + val lr = new LogisticRegression().setMaxIter(1) + val ovr = new OneVsRest().setClassifier(lr) + val ovrModel = ovr.fit(dataset) + + val output1 = ovrModel.setPredictionCol("").setRawPredictionCol("") + .transform(dataset) + assert(output1.schema.fieldNames.toSet === + Set("label", "features")) + + val output2 = ovrModel.setPredictionCol("prediction").setRawPredictionCol("") + .transform(dataset) + assert(output2.schema.fieldNames.toSet === + Set("label", "features", "prediction")) + + val output3 = ovrModel.setPredictionCol("").setRawPredictionCol("rawPrediction") + .transform(dataset) + assert(output3.schema.fieldNames.toSet === + Set("label", "features", "rawPrediction")) + + val output4 = ovrModel.setPredictionCol("prediction").setRawPredictionCol("rawPrediction") + .transform(dataset) + assert(output4.schema.fieldNames.toSet === + Set("label", "features", "prediction", "rawPrediction")) + } + test("should support all NumericType labels and not support other types") { val ovr = new OneVsRest().setClassifier(new LogisticRegression().setMaxIter(1)) MLTestingUtils.checkNumericTypes[OneVsRestModel, OneVsRest]( From fa0d4bf69929c5acd676d602e758a969713d19d8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 28 Nov 2018 23:42:13 +0800 Subject: [PATCH 0011/1072] [SPARK-25829][SQL] remove duplicated map keys with last wins policy ## What changes were proposed in this pull request? Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway. updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`. For other places: 1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys. 2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2. 3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1. 4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup. ## How was this patch tested? updated tests and new tests Closes #23124 from cloud-fan/map. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- docs/sql-migration-guide-upgrade.md | 2 + .../spark/sql/avro/AvroDeserializer.scala | 4 +- python/pyspark/sql/functions.py | 10 +- .../catalyst/expressions/UnsafeMapData.java | 3 + .../sql/catalyst/CatalystTypeConverters.scala | 6 - .../spark/sql/catalyst/InternalRow.scala | 48 ++-- .../catalyst/expressions/BoundAttribute.scala | 8 +- .../expressions/collectionOperations.scala | 242 ++++-------------- .../expressions/complexTypeCreator.scala | 106 +++----- .../expressions/higherOrderFunctions.scala | 8 +- .../expressions/objects/objects.scala | 8 +- .../sql/catalyst/json/JacksonParser.scala | 2 + .../catalyst/util/ArrayBasedMapBuilder.scala | 120 +++++++++ .../sql/catalyst/util/ArrayBasedMapData.scala | 15 ++ .../spark/sql/catalyst/util/ArrayData.scala | 18 +- .../CollectionExpressionsSuite.scala | 87 +++---- .../expressions/ComplexTypeSuite.scala | 20 +- .../HigherOrderFunctionsSuite.scala | 37 +-- .../util/ArrayBasedMapBuilderSuite.scala | 105 ++++++++ .../datasources/orc/OrcDeserializer.scala | 2 + .../parquet/ParquetRowConverter.scala | 6 +- .../spark/sql/DataFrameFunctionsSuite.scala | 6 +- 22 files changed, 444 insertions(+), 419 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 25cd54119091..55838e773e4b 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -27,6 +27,8 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal to 0.0, but users can still distinguish them via `Dataset.show`, `Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 internally, and users can't distinguish them any more. + - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be udefined. + ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. 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 272e7d5b388d..4e2224b058a0 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.avro -import java.math.{BigDecimal} +import java.math.BigDecimal import java.nio.ByteBuffer import scala.collection.JavaConverters._ @@ -218,6 +218,8 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { i += 1 } + // The Avro map will never have null or duplicated map keys, it's safe to create a + // ArrayBasedMapData directly here. updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray)) case (UNION, _) => diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 286ef219a69e..f98e550e39da 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2656,11 +2656,11 @@ def map_concat(*cols): >>> from pyspark.sql.functions import map_concat >>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as map1, map(3, 'c', 1, 'd') as map2") >>> df.select(map_concat("map1", "map2").alias("map3")).show(truncate=False) - +--------------------------------+ - |map3 | - +--------------------------------+ - |[1 -> a, 2 -> b, 3 -> c, 1 -> d]| - +--------------------------------+ + +------------------------+ + |map3 | + +------------------------+ + |[1 -> d, 2 -> b, 3 -> c]| + +------------------------+ """ sc = SparkContext._active_spark_context if len(cols) == 1 and isinstance(cols[0], (list, set)): diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java index f17441dfccb6..a0833a6df8bb 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java @@ -28,6 +28,9 @@ * Currently we just use 2 UnsafeArrayData to represent UnsafeMapData, with extra 8 bytes at head * to indicate the number of bytes of the unsafe key array. * [unsafe key array numBytes] [unsafe key array] [unsafe value array] + * + * Note that, user is responsible to guarantee that the key array does not have duplicated + * elements, otherwise the behavior is undefined. */ // TODO: Use a more efficient format which doesn't depend on unsafe array. public final class UnsafeMapData extends MapData { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 6f5fbdd79e66..93df73ab1eaf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -431,12 +431,6 @@ object CatalystTypeConverters { map, (key: Any) => convertToCatalyst(key), (value: Any) => convertToCatalyst(value)) - case (keys: Array[_], values: Array[_]) => - // case for mapdata with duplicate keys - new ArrayBasedMapData( - new GenericArrayData(keys.map(convertToCatalyst)), - new GenericArrayData(values.map(convertToCatalyst)) - ) case other => other } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala index 274d75e680f0..e49c10be6be4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala @@ -125,22 +125,36 @@ object InternalRow { * actually takes a `SpecializedGetters` input because it can be generalized to other classes * that implements `SpecializedGetters` (e.g., `ArrayData`) too. */ - def getAccessor(dataType: DataType): (SpecializedGetters, Int) => Any = dataType match { - case BooleanType => (input, ordinal) => input.getBoolean(ordinal) - case ByteType => (input, ordinal) => input.getByte(ordinal) - case ShortType => (input, ordinal) => input.getShort(ordinal) - case IntegerType | DateType => (input, ordinal) => input.getInt(ordinal) - case LongType | TimestampType => (input, ordinal) => input.getLong(ordinal) - case FloatType => (input, ordinal) => input.getFloat(ordinal) - case DoubleType => (input, ordinal) => input.getDouble(ordinal) - case StringType => (input, ordinal) => input.getUTF8String(ordinal) - case BinaryType => (input, ordinal) => input.getBinary(ordinal) - case CalendarIntervalType => (input, ordinal) => input.getInterval(ordinal) - case t: DecimalType => (input, ordinal) => input.getDecimal(ordinal, t.precision, t.scale) - case t: StructType => (input, ordinal) => input.getStruct(ordinal, t.size) - case _: ArrayType => (input, ordinal) => input.getArray(ordinal) - case _: MapType => (input, ordinal) => input.getMap(ordinal) - case u: UserDefinedType[_] => getAccessor(u.sqlType) - case _ => (input, ordinal) => input.get(ordinal, dataType) + def getAccessor(dt: DataType, nullable: Boolean = true): (SpecializedGetters, Int) => Any = { + val getValueNullSafe: (SpecializedGetters, Int) => Any = dt match { + case BooleanType => (input, ordinal) => input.getBoolean(ordinal) + case ByteType => (input, ordinal) => input.getByte(ordinal) + case ShortType => (input, ordinal) => input.getShort(ordinal) + case IntegerType | DateType => (input, ordinal) => input.getInt(ordinal) + case LongType | TimestampType => (input, ordinal) => input.getLong(ordinal) + case FloatType => (input, ordinal) => input.getFloat(ordinal) + case DoubleType => (input, ordinal) => input.getDouble(ordinal) + case StringType => (input, ordinal) => input.getUTF8String(ordinal) + case BinaryType => (input, ordinal) => input.getBinary(ordinal) + case CalendarIntervalType => (input, ordinal) => input.getInterval(ordinal) + case t: DecimalType => (input, ordinal) => input.getDecimal(ordinal, t.precision, t.scale) + case t: StructType => (input, ordinal) => input.getStruct(ordinal, t.size) + case _: ArrayType => (input, ordinal) => input.getArray(ordinal) + case _: MapType => (input, ordinal) => input.getMap(ordinal) + case u: UserDefinedType[_] => getAccessor(u.sqlType, nullable) + case _ => (input, ordinal) => input.get(ordinal, dt) + } + + if (nullable) { + (getter, index) => { + if (getter.isNullAt(index)) { + null + } else { + getValueNullSafe(getter, index) + } + } + } else { + getValueNullSafe + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 77582e10f9ff..ea8c369ee49e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -34,15 +34,11 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) override def toString: String = s"input[$ordinal, ${dataType.simpleString}, $nullable]" - private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType) + private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType, nullable) // Use special getter for primitive types (for UnsafeRow) override def eval(input: InternalRow): Any = { - if (nullable && input.isNullAt(ordinal)) { - null - } else { - accessor(input, ordinal) - } + accessor(input, ordinal) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { 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 43116743e995..fa8e38acd522 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 @@ -546,33 +546,25 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres override def nullable: Boolean = children.exists(_.nullable) + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) + override def eval(input: InternalRow): Any = { - val maps = children.map(_.eval(input)) + val maps = children.map(_.eval(input).asInstanceOf[MapData]) if (maps.contains(null)) { return null } - val keyArrayDatas = maps.map(_.asInstanceOf[MapData].keyArray()) - val valueArrayDatas = maps.map(_.asInstanceOf[MapData].valueArray()) - val numElements = keyArrayDatas.foldLeft(0L)((sum, ad) => sum + ad.numElements()) + val numElements = maps.foldLeft(0L)((sum, ad) => sum + ad.numElements()) if (numElements > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { throw new RuntimeException(s"Unsuccessful attempt to concat maps with $numElements " + s"elements due to exceeding the map size limit " + s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") } - val finalKeyArray = new Array[AnyRef](numElements.toInt) - val finalValueArray = new Array[AnyRef](numElements.toInt) - var position = 0 - for (i <- keyArrayDatas.indices) { - val keyArray = keyArrayDatas(i).toObjectArray(dataType.keyType) - val valueArray = valueArrayDatas(i).toObjectArray(dataType.valueType) - Array.copy(keyArray, 0, finalKeyArray, position, keyArray.length) - Array.copy(valueArray, 0, finalValueArray, position, valueArray.length) - position += keyArray.length - } - new ArrayBasedMapData(new GenericArrayData(finalKeyArray), - new GenericArrayData(finalValueArray)) + for (map <- maps) { + mapBuilder.putAll(map.keyArray(), map.valueArray()) + } + mapBuilder.build() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -581,16 +573,7 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres val valueType = dataType.valueType val argsName = ctx.freshName("args") val hasNullName = ctx.freshName("hasNull") - val mapDataClass = classOf[MapData].getName - val arrayBasedMapDataClass = classOf[ArrayBasedMapData].getName - val arrayDataClass = classOf[ArrayData].getName - - val init = - s""" - |$mapDataClass[] $argsName = new $mapDataClass[${mapCodes.size}]; - |boolean ${ev.isNull}, $hasNullName = false; - |$mapDataClass ${ev.value} = null; - """.stripMargin + val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) val assignments = mapCodes.zip(children.map(_.nullable)).zipWithIndex.map { case ((m, true), i) => @@ -613,10 +596,10 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres """.stripMargin } - val codes = ctx.splitExpressionsWithCurrentInputs( + val prepareMaps = ctx.splitExpressionsWithCurrentInputs( expressions = assignments, funcName = "getMapConcatInputs", - extraArguments = (s"$mapDataClass[]", argsName) :: ("boolean", hasNullName) :: Nil, + extraArguments = (s"MapData[]", argsName) :: ("boolean", hasNullName) :: Nil, returnType = "boolean", makeSplitFunction = body => s""" @@ -646,34 +629,34 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres val mapMerge = s""" - |${ev.isNull} = $hasNullName; - |if (!${ev.isNull}) { - | $arrayDataClass[] $keyArgsName = new $arrayDataClass[${mapCodes.size}]; - | $arrayDataClass[] $valArgsName = new $arrayDataClass[${mapCodes.size}]; - | long $numElementsName = 0; - | for (int $idxName = 0; $idxName < $argsName.length; $idxName++) { - | $keyArgsName[$idxName] = $argsName[$idxName].keyArray(); - | $valArgsName[$idxName] = $argsName[$idxName].valueArray(); - | $numElementsName += $argsName[$idxName].numElements(); - | } - | if ($numElementsName > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | throw new RuntimeException("Unsuccessful attempt to concat maps with " + - | $numElementsName + " elements due to exceeding the map size limit " + - | "${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}."); - | } - | $arrayDataClass $finKeysName = $keyConcat($keyArgsName, - | (int) $numElementsName); - | $arrayDataClass $finValsName = $valueConcat($valArgsName, - | (int) $numElementsName); - | ${ev.value} = new $arrayBasedMapDataClass($finKeysName, $finValsName); + |ArrayData[] $keyArgsName = new ArrayData[${mapCodes.size}]; + |ArrayData[] $valArgsName = new ArrayData[${mapCodes.size}]; + |long $numElementsName = 0; + |for (int $idxName = 0; $idxName < $argsName.length; $idxName++) { + | $keyArgsName[$idxName] = $argsName[$idxName].keyArray(); + | $valArgsName[$idxName] = $argsName[$idxName].valueArray(); + | $numElementsName += $argsName[$idxName].numElements(); |} + |if ($numElementsName > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { + | throw new RuntimeException("Unsuccessful attempt to concat maps with " + + | $numElementsName + " elements due to exceeding the map size limit " + + | "${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}."); + |} + |ArrayData $finKeysName = $keyConcat($keyArgsName, (int) $numElementsName); + |ArrayData $finValsName = $valueConcat($valArgsName, (int) $numElementsName); + |${ev.value} = $builderTerm.from($finKeysName, $finValsName); """.stripMargin ev.copy( code = code""" - |$init - |$codes - |$mapMerge + |MapData[] $argsName = new MapData[${mapCodes.size}]; + |boolean $hasNullName = false; + |$prepareMaps + |boolean ${ev.isNull} = $hasNullName; + |MapData ${ev.value} = null; + |if (!$hasNullName) { + | $mapMerge + |} """.stripMargin) } @@ -751,171 +734,44 @@ case class MapFromEntries(child: Expression) extends UnaryExpression { s"${child.dataType.catalogString} type. $prettyName accepts only arrays of pair structs.") } + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) + override protected def nullSafeEval(input: Any): Any = { - val arrayData = input.asInstanceOf[ArrayData] - val numEntries = arrayData.numElements() + val entries = input.asInstanceOf[ArrayData] + val numEntries = entries.numElements() var i = 0 - if(nullEntries) { + if (nullEntries) { while (i < numEntries) { - if (arrayData.isNullAt(i)) return null + if (entries.isNullAt(i)) return null i += 1 } } - val keyArray = new Array[AnyRef](numEntries) - val valueArray = new Array[AnyRef](numEntries) + i = 0 while (i < numEntries) { - val entry = arrayData.getStruct(i, 2) - val key = entry.get(0, dataType.keyType) - if (key == null) { - throw new RuntimeException("The first field from a struct (key) can't be null.") - } - keyArray.update(i, key) - val value = entry.get(1, dataType.valueType) - valueArray.update(i, value) + mapBuilder.put(entries.getStruct(i, 2)) i += 1 } - ArrayBasedMapData(keyArray, valueArray) + mapBuilder.build() } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, c => { val numEntries = ctx.freshName("numEntries") - val isKeyPrimitive = CodeGenerator.isPrimitiveType(dataType.keyType) - val isValuePrimitive = CodeGenerator.isPrimitiveType(dataType.valueType) - val code = if (isKeyPrimitive && isValuePrimitive) { - genCodeForPrimitiveElements(ctx, c, ev.value, numEntries) - } else { - genCodeForAnyElements(ctx, c, ev.value, numEntries) - } + val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) + val i = ctx.freshName("idx") ctx.nullArrayElementsSaveExec(nullEntries, ev.isNull, c) { s""" |final int $numEntries = $c.numElements(); - |$code + |for (int $i = 0; $i < $numEntries; $i++) { + | $builderTerm.put($c.getStruct($i, 2)); + |} + |${ev.value} = $builderTerm.build(); """.stripMargin } }) } - private def genCodeForAssignmentLoop( - ctx: CodegenContext, - childVariable: String, - mapData: String, - numEntries: String, - keyAssignment: (String, String) => String, - valueAssignment: (String, String) => String): String = { - val entry = ctx.freshName("entry") - val i = ctx.freshName("idx") - - val nullKeyCheck = if (dataTypeDetails.get._2) { - s""" - |if ($entry.isNullAt(0)) { - | throw new RuntimeException("The first field from a struct (key) can't be null."); - |} - """.stripMargin - } else { - "" - } - - s""" - |for (int $i = 0; $i < $numEntries; $i++) { - | InternalRow $entry = $childVariable.getStruct($i, 2); - | $nullKeyCheck - | ${keyAssignment(CodeGenerator.getValue(entry, dataType.keyType, "0"), i)} - | ${valueAssignment(entry, i)} - |} - """.stripMargin - } - - private def genCodeForPrimitiveElements( - ctx: CodegenContext, - childVariable: String, - mapData: String, - numEntries: String): String = { - val byteArraySize = ctx.freshName("byteArraySize") - val keySectionSize = ctx.freshName("keySectionSize") - val valueSectionSize = ctx.freshName("valueSectionSize") - val data = ctx.freshName("byteArray") - val unsafeMapData = ctx.freshName("unsafeMapData") - val keyArrayData = ctx.freshName("keyArrayData") - val valueArrayData = ctx.freshName("valueArrayData") - - val baseOffset = Platform.BYTE_ARRAY_OFFSET - val keySize = dataType.keyType.defaultSize - val valueSize = dataType.valueType.defaultSize - val kByteSize = s"UnsafeArrayData.calculateSizeOfUnderlyingByteArray($numEntries, $keySize)" - val vByteSize = s"UnsafeArrayData.calculateSizeOfUnderlyingByteArray($numEntries, $valueSize)" - - val keyAssignment = (key: String, idx: String) => - CodeGenerator.setArrayElement(keyArrayData, dataType.keyType, idx, key) - val valueAssignment = (entry: String, idx: String) => - CodeGenerator.createArrayAssignment( - valueArrayData, dataType.valueType, entry, idx, "1", dataType.valueContainsNull) - val assignmentLoop = genCodeForAssignmentLoop( - ctx, - childVariable, - mapData, - numEntries, - keyAssignment, - valueAssignment - ) - - s""" - |final long $keySectionSize = $kByteSize; - |final long $valueSectionSize = $vByteSize; - |final long $byteArraySize = 8 + $keySectionSize + $valueSectionSize; - |if ($byteArraySize > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | ${genCodeForAnyElements(ctx, childVariable, mapData, numEntries)} - |} else { - | final byte[] $data = new byte[(int)$byteArraySize]; - | UnsafeMapData $unsafeMapData = new UnsafeMapData(); - | Platform.putLong($data, $baseOffset, $keySectionSize); - | Platform.putLong($data, ${baseOffset + 8}, $numEntries); - | Platform.putLong($data, ${baseOffset + 8} + $keySectionSize, $numEntries); - | $unsafeMapData.pointTo($data, $baseOffset, (int)$byteArraySize); - | ArrayData $keyArrayData = $unsafeMapData.keyArray(); - | ArrayData $valueArrayData = $unsafeMapData.valueArray(); - | $assignmentLoop - | $mapData = $unsafeMapData; - |} - """.stripMargin - } - - private def genCodeForAnyElements( - ctx: CodegenContext, - childVariable: String, - mapData: String, - numEntries: String): String = { - val keys = ctx.freshName("keys") - val values = ctx.freshName("values") - val mapDataClass = classOf[ArrayBasedMapData].getName() - - val isValuePrimitive = CodeGenerator.isPrimitiveType(dataType.valueType) - val valueAssignment = (entry: String, idx: String) => { - val value = CodeGenerator.getValue(entry, dataType.valueType, "1") - if (dataType.valueContainsNull && isValuePrimitive) { - s"$values[$idx] = $entry.isNullAt(1) ? null : (Object)$value;" - } else { - s"$values[$idx] = $value;" - } - } - val keyAssignment = (key: String, idx: String) => s"$keys[$idx] = $key;" - val assignmentLoop = genCodeForAssignmentLoop( - ctx, - childVariable, - mapData, - numEntries, - keyAssignment, - valueAssignment) - - s""" - |final Object[] $keys = new Object[$numEntries]; - |final Object[] $values = new Object[$numEntries]; - |$assignmentLoop - |$mapData = $mapDataClass.apply($keys, $values); - """.stripMargin - } - override def prettyName: String = "map_from_entries" } 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 6b77996789f1..4e722c9237a9 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 @@ -24,8 +24,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.UTF8String /** @@ -62,7 +60,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val et = dataType.elementType val (allocation, assigns, arrayData) = - GenArrayData.genCodeToCreateArrayData(ctx, et, children, false, "createArray") + GenArrayData.genCodeToCreateArrayData(ctx, et, children, "createArray") ev.copy( code = code"${allocation}${assigns}", value = JavaCode.variable(arrayData, dataType), @@ -79,7 +77,6 @@ private [sql] object GenArrayData { * @param ctx a [[CodegenContext]] * @param elementType data type of underlying array elements * @param elementsExpr concatenated set of [[Expression]] for each element of an underlying array - * @param isMapKey if true, throw an exception when the element is null * @param functionName string to include in the error message * @return (array allocation, concatenated assignments to each array elements, arrayData name) */ @@ -87,7 +84,6 @@ private [sql] object GenArrayData { ctx: CodegenContext, elementType: DataType, elementsExpr: Seq[Expression], - isMapKey: Boolean, functionName: String): (String, String, String) = { val arrayDataName = ctx.freshName("arrayData") val numElements = s"${elementsExpr.length}L" @@ -103,15 +99,9 @@ private [sql] object GenArrayData { val assignment = if (!expr.nullable) { setArrayElement } else { - val isNullAssignment = if (!isMapKey) { - s"$arrayDataName.setNullAt($i);" - } else { - "throw new RuntimeException(\"Cannot use null as map key!\");" - } - s""" |if (${eval.isNull}) { - | $isNullAssignment + | $arrayDataName.setNullAt($i); |} else { | $setArrayElement |} @@ -165,7 +155,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { } } - override def dataType: MapType = { + override lazy val dataType: MapType = { MapType( keyType = TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(keys.map(_.dataType)) .getOrElse(StringType), @@ -176,32 +166,33 @@ case class CreateMap(children: Seq[Expression]) extends Expression { override def nullable: Boolean = false + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) + override def eval(input: InternalRow): Any = { - val keyArray = keys.map(_.eval(input)).toArray - if (keyArray.contains(null)) { - throw new RuntimeException("Cannot use null as map key!") + var i = 0 + while (i < keys.length) { + mapBuilder.put(keys(i).eval(input), values(i).eval(input)) + i += 1 } - val valueArray = values.map(_.eval(input)).toArray - new ArrayBasedMapData(new GenericArrayData(keyArray), new GenericArrayData(valueArray)) + mapBuilder.build() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val mapClass = classOf[ArrayBasedMapData].getName val MapType(keyDt, valueDt, _) = dataType val (allocationKeyData, assignKeys, keyArrayData) = - GenArrayData.genCodeToCreateArrayData(ctx, keyDt, keys, true, "createMap") + GenArrayData.genCodeToCreateArrayData(ctx, keyDt, keys, "createMap") val (allocationValueData, assignValues, valueArrayData) = - GenArrayData.genCodeToCreateArrayData(ctx, valueDt, values, false, "createMap") + GenArrayData.genCodeToCreateArrayData(ctx, valueDt, values, "createMap") + val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) val code = code""" - final boolean ${ev.isNull} = false; $allocationKeyData $assignKeys $allocationValueData $assignValues - final MapData ${ev.value} = new $mapClass($keyArrayData, $valueArrayData); + final MapData ${ev.value} = $builderTerm.from($keyArrayData, $valueArrayData); """ - ev.copy(code = code) + ev.copy(code = code, isNull = FalseLiteral) } override def prettyName: String = "map" @@ -234,53 +225,25 @@ case class MapFromArrays(left: Expression, right: Expression) } } - override def dataType: DataType = { + override def dataType: MapType = { MapType( keyType = left.dataType.asInstanceOf[ArrayType].elementType, valueType = right.dataType.asInstanceOf[ArrayType].elementType, valueContainsNull = right.dataType.asInstanceOf[ArrayType].containsNull) } + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) + override def nullSafeEval(keyArray: Any, valueArray: Any): Any = { val keyArrayData = keyArray.asInstanceOf[ArrayData] val valueArrayData = valueArray.asInstanceOf[ArrayData] - if (keyArrayData.numElements != valueArrayData.numElements) { - throw new RuntimeException("The given two arrays should have the same length") - } - val leftArrayType = left.dataType.asInstanceOf[ArrayType] - if (leftArrayType.containsNull) { - var i = 0 - while (i < keyArrayData.numElements) { - if (keyArrayData.isNullAt(i)) { - throw new RuntimeException("Cannot use null as map key!") - } - i += 1 - } - } - new ArrayBasedMapData(keyArrayData.copy(), valueArrayData.copy()) + mapBuilder.from(keyArrayData.copy(), valueArrayData.copy()) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, (keyArrayData, valueArrayData) => { - val arrayBasedMapData = classOf[ArrayBasedMapData].getName - val leftArrayType = left.dataType.asInstanceOf[ArrayType] - val keyArrayElemNullCheck = if (!leftArrayType.containsNull) "" else { - val i = ctx.freshName("i") - s""" - |for (int $i = 0; $i < $keyArrayData.numElements(); $i++) { - | if ($keyArrayData.isNullAt($i)) { - | throw new RuntimeException("Cannot use null as map key!"); - | } - |} - """.stripMargin - } - s""" - |if ($keyArrayData.numElements() != $valueArrayData.numElements()) { - | throw new RuntimeException("The given two arrays should have the same length"); - |} - |$keyArrayElemNullCheck - |${ev.value} = new $arrayBasedMapData($keyArrayData.copy(), $valueArrayData.copy()); - """.stripMargin + val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) + s"${ev.value} = $builderTerm.from($keyArrayData.copy(), $valueArrayData.copy());" }) } @@ -488,28 +451,25 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E } } + private lazy val mapBuilder = new ArrayBasedMapBuilder(StringType, StringType) + override def nullSafeEval( inputString: Any, stringDelimiter: Any, keyValueDelimiter: Any): Any = { val keyValues = inputString.asInstanceOf[UTF8String].split(stringDelimiter.asInstanceOf[UTF8String], -1) - - val iterator = new Iterator[(UTF8String, UTF8String)] { - var index = 0 - val keyValueDelimiterUTF8String = keyValueDelimiter.asInstanceOf[UTF8String] - - override def hasNext: Boolean = { - keyValues.length > index - } - - override def next(): (UTF8String, UTF8String) = { - val keyValueArray = keyValues(index).split(keyValueDelimiterUTF8String, 2) - index += 1 - (keyValueArray(0), if (keyValueArray.length < 2) null else keyValueArray(1)) - } + val keyValueDelimiterUTF8String = keyValueDelimiter.asInstanceOf[UTF8String] + + var i = 0 + while (i < keyValues.length) { + val keyValueArray = keyValues(i).split(keyValueDelimiterUTF8String, 2) + val key = keyValueArray(0) + val value = if (keyValueArray.length < 2) null else keyValueArray(1) + mapBuilder.put(key, value) + i += 1 } - ArrayBasedMapData(iterator, keyValues.size, identity, identity) + mapBuilder.build() } override def prettyName: String = "str_to_map" 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 8b3102186622..a8639d29f964 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 @@ -512,7 +512,7 @@ case class TransformKeys( @transient lazy val MapType(keyType, valueType, valueContainsNull) = argument.dataType - override def dataType: DataType = MapType(function.dataType, valueType, valueContainsNull) + override def dataType: MapType = MapType(function.dataType, valueType, valueContainsNull) override def checkInputDataTypes(): TypeCheckResult = { TypeUtils.checkForMapKeyType(function.dataType) @@ -525,6 +525,7 @@ case class TransformKeys( @transient lazy val LambdaFunction( _, (keyVar: NamedLambdaVariable) :: (valueVar: NamedLambdaVariable) :: Nil, _) = function + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { val map = argumentValue.asInstanceOf[MapData] @@ -534,13 +535,10 @@ case class TransformKeys( keyVar.value.set(map.keyArray().get(i, keyVar.dataType)) valueVar.value.set(map.valueArray().get(i, valueVar.dataType)) val result = functionForEval.eval(inputRow) - if (result == null) { - throw new RuntimeException("Cannot use null as map key!") - } resultKeys.update(i, result) i += 1 } - new ArrayBasedMapData(resultKeys, map.valueArray()) + mapBuilder.from(resultKeys, map.valueArray()) } override def prettyName: String = "transform_keys" 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 59c897b6a53c..8182730feb4b 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 @@ -587,17 +587,13 @@ case class LambdaVariable( dataType: DataType, nullable: Boolean = true) extends LeafExpression with NonSQLExpression { - private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType) + private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType, nullable) // Interpreted execution of `LambdaVariable` always get the 0-index element from input row. override def eval(input: InternalRow): Any = { assert(input.numFields == 1, "The input row of interpreted LambdaVariable should have only 1 field.") - if (nullable && input.isNullAt(0)) { - null - } else { - accessor(input, 0) - } + accessor(input, 0) } override def genCode(ctx: CodegenContext): ExprCode = { 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 773ff5a7a401..92517aac053b 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 @@ -367,6 +367,8 @@ class JacksonParser( values += fieldConverter.apply(parser) } + // The JSON map will never have null or duplicated map keys, it's safe to create a + // ArrayBasedMapData directly here. ArrayBasedMapData(keys.toArray, values.toArray) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala new file mode 100644 index 000000000000..e7cd61655dc9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types._ + +/** + * A builder of [[ArrayBasedMapData]], which fails if a null map key is detected, and removes + * duplicated map keys w.r.t. the last wins policy. + */ +class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Serializable { + assert(!keyType.existsRecursively(_.isInstanceOf[MapType]), "key of map cannot be/contain map") + assert(keyType != NullType, "map key cannot be null type.") + + private lazy val keyToIndex = keyType match { + // Binary type data is `byte[]`, which can't use `==` to check equality. + case _: AtomicType | _: CalendarIntervalType if !keyType.isInstanceOf[BinaryType] => + new java.util.HashMap[Any, Int]() + case _ => + // for complex types, use interpreted ordering to be able to compare unsafe data with safe + // data, e.g. UnsafeRow vs GenericInternalRow. + new java.util.TreeMap[Any, Int](TypeUtils.getInterpretedOrdering(keyType)) + } + + // TODO: specialize it + private lazy val keys = mutable.ArrayBuffer.empty[Any] + private lazy val values = mutable.ArrayBuffer.empty[Any] + + private lazy val keyGetter = InternalRow.getAccessor(keyType) + private lazy val valueGetter = InternalRow.getAccessor(valueType) + + def put(key: Any, value: Any): Unit = { + if (key == null) { + throw new RuntimeException("Cannot use null as map key.") + } + + val index = keyToIndex.getOrDefault(key, -1) + if (index == -1) { + keyToIndex.put(key, values.length) + keys.append(key) + values.append(value) + } else { + // Overwrite the previous value, as the policy is last wins. + values(index) = value + } + } + + // write a 2-field row, the first field is key and the second field is value. + def put(entry: InternalRow): Unit = { + if (entry.isNullAt(0)) { + throw new RuntimeException("Cannot use null as map key.") + } + put(keyGetter(entry, 0), valueGetter(entry, 1)) + } + + def putAll(keyArray: ArrayData, valueArray: ArrayData): Unit = { + if (keyArray.numElements() != valueArray.numElements()) { + throw new RuntimeException( + "The key array and value array of MapData must have the same length.") + } + + var i = 0 + while (i < keyArray.numElements()) { + put(keyGetter(keyArray, i), valueGetter(valueArray, i)) + i += 1 + } + } + + private def reset(): Unit = { + keyToIndex.clear() + keys.clear() + values.clear() + } + + /** + * Builds the result [[ArrayBasedMapData]] and reset this builder to free up the resources. The + * builder becomes fresh afterward and is ready to take input and build another map. + */ + def build(): ArrayBasedMapData = { + val map = new ArrayBasedMapData( + new GenericArrayData(keys.toArray), new GenericArrayData(values.toArray)) + reset() + map + } + + /** + * Builds a [[ArrayBasedMapData]] from the given key and value array and reset this builder. The + * builder becomes fresh afterward and is ready to take input and build another map. + */ + def from(keyArray: ArrayData, valueArray: ArrayData): ArrayBasedMapData = { + assert(keyToIndex.isEmpty, "'from' can only be called with a fresh ArrayBasedMapBuilder.") + putAll(keyArray, valueArray) + if (keyToIndex.size == keyArray.numElements()) { + // If there is no duplicated map keys, creates the MapData with the input key and value array, + // as they might already in unsafe format and are more efficient. + reset() + new ArrayBasedMapData(keyArray, valueArray) + } else { + build() + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala index 91b313944369..0989af26b8c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala @@ -19,6 +19,12 @@ package org.apache.spark.sql.catalyst.util import java.util.{Map => JavaMap} +/** + * A simple `MapData` implementation which is backed by 2 arrays. + * + * Note that, user is responsible to guarantee that the key array does not have duplicated + * elements, otherwise the behavior is undefined. + */ class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) extends MapData { require(keyArray.numElements() == valueArray.numElements()) @@ -83,6 +89,9 @@ object ArrayBasedMapData { * Creates a [[ArrayBasedMapData]] by applying the given converters over * each (key -> value) pair from the given iterator * + * Note that, user is responsible to guarantee that the key array does not have duplicated + * elements, otherwise the behavior is undefined. + * * @param iterator Input iterator * @param size Number of elements * @param keyConverter This function is applied over all the keys extracted from the @@ -108,6 +117,12 @@ object ArrayBasedMapData { ArrayBasedMapData(keys, values) } + /** + * Creates a [[ArrayBasedMapData]] from a key and value array. + * + * Note that, user is responsible to guarantee that the key array does not have duplicated + * elements, otherwise the behavior is undefined. + */ def apply(keys: Array[_], values: Array[_]): ArrayBasedMapData = { new ArrayBasedMapData(new GenericArrayData(keys), new GenericArrayData(values)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala index 4da8ce05fe8a..ebbf241088f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala @@ -172,11 +172,7 @@ abstract class ArrayData extends SpecializedGetters with Serializable { val values = new Array[T](size) var i = 0 while (i < size) { - if (isNullAt(i)) { - values(i) = null.asInstanceOf[T] - } else { - values(i) = accessor(this, i).asInstanceOf[T] - } + values(i) = accessor(this, i).asInstanceOf[T] i += 1 } values @@ -187,11 +183,7 @@ abstract class ArrayData extends SpecializedGetters with Serializable { val accessor = InternalRow.getAccessor(elementType) var i = 0 while (i < size) { - if (isNullAt(i)) { - f(i, null) - } else { - f(i, accessor(this, i)) - } + f(i, accessor(this, i)) i += 1 } } @@ -208,11 +200,7 @@ class ArrayDataIndexedSeq[T](arrayData: ArrayData, dataType: DataType) extends I override def apply(idx: Int): T = if (0 <= idx && idx < arrayData.numElements()) { - if (arrayData.isNullAt(idx)) { - null.asInstanceOf[T] - } else { - accessor(arrayData, idx).asInstanceOf[T] - } + accessor(arrayData, idx).asInstanceOf[T] } else { throw new IndexOutOfBoundsException( s"Index $idx must be between 0 and the length of the ArrayData.") 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 d2edb2f24688..bed8547dbc83 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 @@ -114,13 +114,13 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper val m1 = Literal.create(create_map("c" -> "3", "a" -> "4"), MapType(StringType, StringType, valueContainsNull = false)) val m2 = Literal.create(create_map("d" -> "4", "e" -> "5"), MapType(StringType, StringType)) - val m3 = Literal.create(create_map("a" -> "1", "b" -> "2"), MapType(StringType, StringType)) + val m3 = Literal.create(create_map("f" -> "1", "g" -> "2"), MapType(StringType, StringType)) val m4 = Literal.create(create_map("a" -> null, "c" -> "3"), MapType(StringType, StringType)) val m5 = Literal.create(create_map("a" -> 1, "b" -> 2), MapType(StringType, IntegerType)) - val m6 = Literal.create(create_map("a" -> null, "c" -> 3), MapType(StringType, IntegerType)) + val m6 = Literal.create(create_map("c" -> null, "d" -> 3), MapType(StringType, IntegerType)) val m7 = Literal.create(create_map(List(1, 2) -> 1, List(3, 4) -> 2), MapType(ArrayType(IntegerType), IntegerType)) - val m8 = Literal.create(create_map(List(5, 6) -> 3, List(1, 2) -> 4), + val m8 = Literal.create(create_map(List(5, 6) -> 3, List(7, 8) -> 4), MapType(ArrayType(IntegerType), IntegerType)) val m9 = Literal.create(create_map(1 -> "1", 2 -> "2"), MapType(IntegerType, StringType, valueContainsNull = false)) @@ -134,57 +134,33 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper MapType(IntegerType, IntegerType, valueContainsNull = true)) val mNull = Literal.create(null, MapType(StringType, StringType)) - // overlapping maps - checkEvaluation(MapConcat(Seq(m0, m1)), - ( - Array("a", "b", "c", "a"), // keys - Array("1", "2", "3", "4") // values - ) - ) + // overlapping maps should remove duplicated map keys w.r.t. last win policy. + checkEvaluation(MapConcat(Seq(m0, m1)), create_map("a" -> "4", "b" -> "2", "c" -> "3")) // maps with no overlap checkEvaluation(MapConcat(Seq(m0, m2)), create_map("a" -> "1", "b" -> "2", "d" -> "4", "e" -> "5")) // 3 maps - checkEvaluation(MapConcat(Seq(m0, m1, m2)), - ( - Array("a", "b", "c", "a", "d", "e"), // keys - Array("1", "2", "3", "4", "4", "5") // values - ) - ) + checkEvaluation(MapConcat(Seq(m0, m2, m3)), + create_map("a" -> "1", "b" -> "2", "d" -> "4", "e" -> "5", "f" -> "1", "g" -> "2")) // null reference values - checkEvaluation(MapConcat(Seq(m3, m4)), - ( - Array("a", "b", "a", "c"), // keys - Array("1", "2", null, "3") // values - ) - ) + checkEvaluation(MapConcat(Seq(m2, m4)), + create_map("d" -> "4", "e" -> "5", "a" -> null, "c" -> "3")) // null primitive values checkEvaluation(MapConcat(Seq(m5, m6)), - ( - Array("a", "b", "a", "c"), // keys - Array(1, 2, null, 3) // values - ) - ) + create_map("a" -> 1, "b" -> 2, "c" -> null, "d" -> 3)) // keys that are primitive checkEvaluation(MapConcat(Seq(m9, m10)), - ( - Array(1, 2, 3, 4), // keys - Array("1", "2", "3", "4") // values - ) - ) + create_map(1 -> "1", 2 -> "2", 3 -> "3", 4 -> "4")) - // keys that are arrays, with overlap + // keys that are arrays checkEvaluation(MapConcat(Seq(m7, m8)), - ( - Array(List(1, 2), List(3, 4), List(5, 6), List(1, 2)), // keys - Array(1, 2, 3, 4) // values - ) - ) + create_map(List(1, 2) -> 1, List(3, 4) -> 2, List(5, 6) -> 3, List(7, 8) -> 4)) + // both keys and value are primitive and valueContainsNull = false checkEvaluation(MapConcat(Seq(m11, m12)), create_map(1 -> 2, 3 -> 4, 5 -> 6)) @@ -205,15 +181,14 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(MapConcat(Seq.empty), Map.empty) // force split expressions for input in generated code - val expectedKeys = Array.fill(65)(Seq("a", "b")).flatten ++ Array("d", "e") - val expectedValues = Array.fill(65)(Seq("1", "2")).flatten ++ Array("4", "5") - checkEvaluation(MapConcat( - Seq( - m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, - m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, - m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m2 - )), - (expectedKeys, expectedValues)) + val expectedKeys = (1 to 65).map(_.toString) + val expectedValues = (1 to 65).map(_.toString) + checkEvaluation( + MapConcat( + expectedKeys.zip(expectedValues).map { + case (k, v) => Literal.create(create_map(k -> v), MapType(StringType, StringType)) + }), + create_map(expectedKeys.zip(expectedValues): _*)) // argument checking assert(MapConcat(Seq(m0, m1)).checkInputDataTypes().isSuccess) @@ -248,7 +223,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper ArrayType(IntegerType, containsNull = true), ArrayType(StringType, containsNull = true), valueContainsNull = true)) - checkEvaluation(mapConcat, Map( + checkEvaluation(mapConcat, create_map( Seq(1, 2) -> Seq("a", "b"), Seq(3, 4, null) -> Seq("c", "d", null), Seq(6) -> null)) @@ -282,7 +257,9 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper val ai1 = Literal.create(Seq(row(1, null), row(2, 20), row(3, null)), aiType) val ai2 = Literal.create(Seq.empty, aiType) val ai3 = Literal.create(null, aiType) + // The map key is duplicated val ai4 = Literal.create(Seq(row(1, 10), row(1, 20)), aiType) + // The map key is null val ai5 = Literal.create(Seq(row(1, 10), row(null, 20)), aiType) val ai6 = Literal.create(Seq(null, row(2, 20), null), aiType) @@ -290,10 +267,12 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(MapFromEntries(ai1), create_map(1 -> null, 2 -> 20, 3 -> null)) checkEvaluation(MapFromEntries(ai2), Map.empty) checkEvaluation(MapFromEntries(ai3), null) - checkEvaluation(MapKeys(MapFromEntries(ai4)), Seq(1, 1)) + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation(MapFromEntries(ai4), create_map(1 -> 20)) + // Map key can't be null checkExceptionInExpression[RuntimeException]( MapFromEntries(ai5), - "The first field from a struct (key) can't be null.") + "Cannot use null as map key") checkEvaluation(MapFromEntries(ai6), null) // Non-primitive-type keys and values @@ -310,13 +289,13 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(MapFromEntries(as1), create_map("a" -> null, "b" -> "bb", "c" -> null)) checkEvaluation(MapFromEntries(as2), Map.empty) checkEvaluation(MapFromEntries(as3), null) - checkEvaluation(MapKeys(MapFromEntries(as4)), Seq("a", "a")) - checkEvaluation(MapFromEntries(as6), null) - + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation(MapFromEntries(as4), create_map("a" -> "bb")) // Map key can't be null checkExceptionInExpression[RuntimeException]( MapFromEntries(as5), - "The first field from a struct (key) can't be null.") + "Cannot use null as map key") + checkEvaluation(MapFromEntries(as6), null) // map key can't be map val structOfMap = row(create_map(1 -> 1), 1) 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 d95f42e04e37..dc6046481504 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 @@ -183,6 +183,11 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { CreateMap(interlace(strWithNull, intSeq.map(Literal(_)))), "Cannot use null as map key") + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation( + CreateMap(Seq(Literal(1), Literal(2), Literal(1), Literal(3))), + create_map(1 -> 3)) + // ArrayType map key and value val map = CreateMap(Seq( Literal.create(intSeq, ArrayType(IntegerType, containsNull = false)), @@ -243,12 +248,18 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { MapFromArrays(intWithNullArray, strArray), "Cannot use null as map key") + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation( + MapFromArrays( + Literal.create(Seq(1, 1), ArrayType(IntegerType)), + Literal.create(Seq(2, 3), ArrayType(IntegerType))), + create_map(1 -> 3)) + // map key can't be map val arrayOfMap = Seq(create_map(1 -> "a", 2 -> "b")) val map = MapFromArrays( Literal.create(arrayOfMap, ArrayType(MapType(IntegerType, StringType))), - Literal.create(Seq(1), ArrayType(IntegerType)) - ) + Literal.create(Seq(1), ArrayType(IntegerType))) map.checkInputDataTypes() match { case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key") case TypeCheckResult.TypeCheckFailure(msg) => @@ -356,6 +367,11 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val m5 = Map("a" -> null) checkEvaluation(new StringToMap(s5), m5) + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation( + new StringToMap(Literal("a:1,b:2,a:3")), + create_map("a" -> "3", "b" -> "2")) + // arguments checking assert(new StringToMap(Literal("a:1,b:2,c:3")).checkInputDataTypes().isSuccess) assert(new StringToMap(Literal(null)).checkInputDataTypes().isFailure) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala index 66bf18af9579..03fb75e330c6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala @@ -330,8 +330,8 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation( transformKeys(transformKeys(ai0, plusOne), plusValue), create_map(3 -> 1, 5 -> 2, 7 -> 3, 9 -> 4)) - checkEvaluation(transformKeys(ai0, modKey), - ArrayBasedMapData(Array(1, 2, 0, 1), Array(1, 2, 3, 4))) + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation(transformKeys(ai0, modKey), create_map(1 -> 4, 2 -> 2, 0 -> 3)) checkEvaluation(transformKeys(ai1, plusOne), Map.empty[Int, Int]) checkEvaluation(transformKeys(ai1, plusOne), Map.empty[Int, Int]) checkEvaluation( @@ -467,16 +467,13 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper .bind(validateBinding) } - val mii0 = Literal.create(Map(1 -> 10, 2 -> 20, 3 -> 30), + val mii0 = Literal.create(create_map(1 -> 10, 2 -> 20, 3 -> 30), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii1 = Literal.create(Map(1 -> -1, 2 -> -2, 4 -> -4), + val mii1 = Literal.create(create_map(1 -> -1, 2 -> -2, 4 -> -4), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii2 = Literal.create(Map(1 -> null, 2 -> -2, 3 -> null), + val mii2 = Literal.create(create_map(1 -> null, 2 -> -2, 3 -> null), MapType(IntegerType, IntegerType, valueContainsNull = true)) val mii3 = Literal.create(Map(), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii4 = MapFromArrays( - Literal.create(Seq(2, 2), ArrayType(IntegerType, false)), - Literal.create(Seq(20, 200), ArrayType(IntegerType, false))) val miin = Literal.create(null, MapType(IntegerType, IntegerType, valueContainsNull = false)) val multiplyKeyWithValues: (Expression, Expression, Expression) => Expression = { @@ -492,12 +489,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation( map_zip_with(mii0, mii3, multiplyKeyWithValues), Map(1 -> null, 2 -> null, 3 -> null)) - checkEvaluation( - map_zip_with(mii0, mii4, multiplyKeyWithValues), - Map(1 -> null, 2 -> 800, 3 -> null)) - checkEvaluation( - map_zip_with(mii4, mii0, multiplyKeyWithValues), - Map(2 -> 800, 1 -> null, 3 -> null)) checkEvaluation( map_zip_with(mii0, miin, multiplyKeyWithValues), null) @@ -511,9 +502,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val mss2 = Literal.create(Map("c" -> null, "b" -> "t", "a" -> null), MapType(StringType, StringType, valueContainsNull = true)) val mss3 = Literal.create(Map(), MapType(StringType, StringType, valueContainsNull = false)) - val mss4 = MapFromArrays( - Literal.create(Seq("a", "a"), ArrayType(StringType, false)), - Literal.create(Seq("a", "n"), ArrayType(StringType, false))) val mssn = Literal.create(null, MapType(StringType, StringType, valueContainsNull = false)) val concat: (Expression, Expression, Expression) => Expression = { @@ -529,12 +517,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation( map_zip_with(mss0, mss3, concat), Map("a" -> null, "b" -> null, "d" -> null)) - checkEvaluation( - map_zip_with(mss0, mss4, concat), - Map("a" -> "axa", "b" -> null, "d" -> null)) - checkEvaluation( - map_zip_with(mss4, mss0, concat), - Map("a" -> "aax", "b" -> null, "d" -> null)) checkEvaluation( map_zip_with(mss0, mssn, concat), null) @@ -550,9 +532,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val mbb2 = Literal.create(Map(b(1, 3) -> null, b(1, 2) -> b(2), b(2, 1) -> null), MapType(BinaryType, BinaryType, valueContainsNull = true)) val mbb3 = Literal.create(Map(), MapType(BinaryType, BinaryType, valueContainsNull = false)) - val mbb4 = MapFromArrays( - Literal.create(Seq(b(2, 1), b(2, 1)), ArrayType(BinaryType, false)), - Literal.create(Seq(b(1), b(9)), ArrayType(BinaryType, false))) val mbbn = Literal.create(null, MapType(BinaryType, BinaryType, valueContainsNull = false)) checkEvaluation( @@ -564,12 +543,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation( map_zip_with(mbb0, mbb3, concat), Map(b(1, 2) -> null, b(2, 1) -> null, b(1, 3) -> null)) - checkEvaluation( - map_zip_with(mbb0, mbb4, concat), - Map(b(1, 2) -> null, b(2, 1) -> b(2, 1, 5, 1), b(1, 3) -> null)) - checkEvaluation( - map_zip_with(mbb4, mbb0, concat), - Map(b(2, 1) -> b(2, 1, 1, 5), b(1, 2) -> null, b(1, 3) -> null)) checkEvaluation( map_zip_with(mbb0, mbbn, concat), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala new file mode 100644 index 000000000000..8509bce17712 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeRow} +import org.apache.spark.sql.types.{ArrayType, BinaryType, IntegerType, StructType} +import org.apache.spark.unsafe.Platform + +class ArrayBasedMapBuilderSuite extends SparkFunSuite { + + test("basic") { + val builder = new ArrayBasedMapBuilder(IntegerType, IntegerType) + builder.put(1, 1) + builder.put(InternalRow(2, 2)) + builder.putAll(new GenericArrayData(Seq(3)), new GenericArrayData(Seq(3))) + val map = builder.build() + assert(map.numElements() == 3) + assert(ArrayBasedMapData.toScalaMap(map) == Map(1 -> 1, 2 -> 2, 3 -> 3)) + } + + test("fail with null key") { + val builder = new ArrayBasedMapBuilder(IntegerType, IntegerType) + builder.put(1, null) // null value is OK + val e = intercept[RuntimeException](builder.put(null, 1)) + assert(e.getMessage.contains("Cannot use null as map key")) + } + + test("remove duplicated keys with last wins policy") { + val builder = new ArrayBasedMapBuilder(IntegerType, IntegerType) + builder.put(1, 1) + builder.put(2, 2) + builder.put(1, 2) + val map = builder.build() + assert(map.numElements() == 2) + assert(ArrayBasedMapData.toScalaMap(map) == Map(1 -> 2, 2 -> 2)) + } + + test("binary type key") { + val builder = new ArrayBasedMapBuilder(BinaryType, IntegerType) + builder.put(Array(1.toByte), 1) + builder.put(Array(2.toByte), 2) + builder.put(Array(1.toByte), 3) + val map = builder.build() + assert(map.numElements() == 2) + val entries = ArrayBasedMapData.toScalaMap(map).iterator.toSeq + assert(entries(0)._1.asInstanceOf[Array[Byte]].toSeq == Seq(1)) + assert(entries(0)._2 == 3) + assert(entries(1)._1.asInstanceOf[Array[Byte]].toSeq == Seq(2)) + assert(entries(1)._2 == 2) + } + + test("struct type key") { + val builder = new ArrayBasedMapBuilder(new StructType().add("i", "int"), IntegerType) + builder.put(InternalRow(1), 1) + builder.put(InternalRow(2), 2) + val unsafeRow = { + val row = new UnsafeRow(1) + val bytes = new Array[Byte](16) + row.pointTo(bytes, 16) + row.setInt(0, 1) + row + } + builder.put(unsafeRow, 3) + val map = builder.build() + assert(map.numElements() == 2) + assert(ArrayBasedMapData.toScalaMap(map) == Map(InternalRow(1) -> 3, InternalRow(2) -> 2)) + } + + test("array type key") { + val builder = new ArrayBasedMapBuilder(ArrayType(IntegerType), IntegerType) + builder.put(new GenericArrayData(Seq(1, 1)), 1) + builder.put(new GenericArrayData(Seq(2, 2)), 2) + val unsafeArray = { + val array = new UnsafeArrayData() + val bytes = new Array[Byte](24) + Platform.putLong(bytes, Platform.BYTE_ARRAY_OFFSET, 2) + array.pointTo(bytes, Platform.BYTE_ARRAY_OFFSET, 24) + array.setInt(0, 1) + array.setInt(1, 1) + array + } + builder.put(unsafeArray, 3) + val map = builder.build() + assert(map.numElements() == 2) + assert(ArrayBasedMapData.toScalaMap(map) == + Map(new GenericArrayData(Seq(1, 1)) -> 3, new GenericArrayData(Seq(2, 2)) -> 2)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala index 4ecc54bd2fd9..ee16b3ab07f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala @@ -179,6 +179,8 @@ class OrcDeserializer( i += 1 } + // The ORC map will never have null or duplicated map keys, it's safe to create a + // ArrayBasedMapData directly here. updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray)) case udt: UserDefinedType[_] => newWriter(udt.sqlType, updater) 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 119972594184..004a96d13413 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 @@ -558,8 +558,12 @@ private[parquet] class ParquetRowConverter( override def getConverter(fieldIndex: Int): Converter = keyValueConverter - override def end(): Unit = + override def end(): Unit = { + // The parquet map may contains null or duplicated map keys. When it happens, the behavior is + // undefined. + // TODO (SPARK-26174): disallow it with a config. updater.set(ArrayBasedMapData(currentKeys.toArray, currentValues.toArray)) + } // NOTE: We can't reuse the mutable Map here and must instantiate a new `Map` for the next // value. `Row.copy()` only copies row cells, it doesn't do deep copy to objects stored in row 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 666ba35d7a8f..e6d1a038a591 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 @@ -89,13 +89,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { val msg1 = intercept[Exception] { df5.select(map_from_arrays($"k", $"v")).collect }.getMessage - assert(msg1.contains("Cannot use null as map key!")) + assert(msg1.contains("Cannot use null as map key")) val df6 = Seq((Seq(1, 2), Seq("a"))).toDF("k", "v") val msg2 = intercept[Exception] { df6.select(map_from_arrays($"k", $"v")).collect }.getMessage - assert(msg2.contains("The given two arrays should have the same length")) + assert(msg2.contains("The key array and value array of MapData must have the same length")) } test("struct with column name") { @@ -2588,7 +2588,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { val ex3 = intercept[Exception] { dfExample1.selectExpr("transform_keys(i, (k, v) -> v)").show() } - assert(ex3.getMessage.contains("Cannot use null as map key!")) + assert(ex3.getMessage.contains("Cannot use null as map key")) val ex4 = intercept[AnalysisException] { dfExample2.selectExpr("transform_keys(j, (k, v) -> k + 1)") From 8bfea86b1c8a65ce73711af02d9e4140659a926d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 29 Nov 2018 01:54:06 +0000 Subject: [PATCH 0012/1072] [SPARK-26133][ML] Remove deprecated OneHotEncoder and rename OneHotEncoderEstimator to OneHotEncoder ## What changes were proposed in this pull request? We have deprecated `OneHotEncoder` at Spark 2.3.0 and introduced `OneHotEncoderEstimator`. At 3.0.0, we remove deprecated `OneHotEncoder` and rename `OneHotEncoderEstimator` to `OneHotEncoder`. TODO: According to ML migration guide, we need to keep `OneHotEncoderEstimator` as an alias after renaming. This is not done at this patch in order to facilitate review. ## How was this patch tested? Existing tests. Closes #23100 from viirya/remove_one_hot_encoder. Authored-by: Liang-Chi Hsieh Signed-off-by: DB Tsai --- docs/ml-features.md | 24 +- docs/ml-guide.md | 6 + ...ple.java => JavaOneHotEncoderExample.java} | 8 +- ...r_example.py => onehot_encoder_example.py} | 8 +- ...ample.scala => OneHotEncoderExample.scala} | 8 +- .../spark/ml/feature/OneHotEncoder.scala | 530 +++++++++++++++--- .../ml/feature/OneHotEncoderEstimator.scala | 528 ----------------- .../apache/spark/ml/feature/RFormula.scala | 4 +- .../feature/OneHotEncoderEstimatorSuite.scala | 422 -------------- .../spark/ml/feature/OneHotEncoderSuite.scala | 411 +++++++++++--- project/MimaExcludes.scala | 12 + python/pyspark/ml/feature.py | 102 +--- 12 files changed, 841 insertions(+), 1222 deletions(-) rename examples/src/main/java/org/apache/spark/examples/ml/{JavaOneHotEncoderEstimatorExample.java => JavaOneHotEncoderExample.java} (91%) rename examples/src/main/python/ml/{onehot_encoder_estimator_example.py => onehot_encoder_example.py} (83%) rename examples/src/main/scala/org/apache/spark/examples/ml/{OneHotEncoderEstimatorExample.scala => OneHotEncoderExample.scala} (89%) delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoderEstimator.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderEstimatorSuite.scala diff --git a/docs/ml-features.md b/docs/ml-features.md index 882b895a9d15..83a211ce02e6 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -779,43 +779,37 @@ for more details on the API. -## OneHotEncoder (Deprecated since 2.3.0) - -Because this existing `OneHotEncoder` is a stateless transformer, it is not usable on new data where the number of categories may differ from the training data. In order to fix this, a new `OneHotEncoderEstimator` was created that produces an `OneHotEncoderModel` when fitting. For more detail, please see [SPARK-13030](https://issues.apache.org/jira/browse/SPARK-13030). - -`OneHotEncoder` has been deprecated in 2.3.0 and will be removed in 3.0.0. Please use [OneHotEncoderEstimator](ml-features.html#onehotencoderestimator) instead. - -## OneHotEncoderEstimator +## OneHotEncoder [One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a categorical feature, represented as a label index, to a binary vector with at most a single one-value indicating the presence of a specific feature value from among the set of all feature values. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features. For string type input data, it is common to encode categorical features using [StringIndexer](ml-features.html#stringindexer) first. -`OneHotEncoderEstimator` can transform multiple columns, returning an one-hot-encoded output vector column for each input column. It is common to merge these vectors into a single feature vector using [VectorAssembler](ml-features.html#vectorassembler). +`OneHotEncoder` can transform multiple columns, returning an one-hot-encoded output vector column for each input column. It is common to merge these vectors into a single feature vector using [VectorAssembler](ml-features.html#vectorassembler). -`OneHotEncoderEstimator` supports the `handleInvalid` parameter to choose how to handle invalid input during transforming data. Available options include 'keep' (any invalid inputs are assigned to an extra categorical index) and 'error' (throw an error). +`OneHotEncoder` supports the `handleInvalid` parameter to choose how to handle invalid input during transforming data. Available options include 'keep' (any invalid inputs are assigned to an extra categorical index) and 'error' (throw an error). **Examples**
-Refer to the [OneHotEncoderEstimator Scala docs](api/scala/index.html#org.apache.spark.ml.feature.OneHotEncoderEstimator) for more details on the API. +Refer to the [OneHotEncoder Scala docs](api/scala/index.html#org.apache.spark.ml.feature.OneHotEncoder) for more details on the API. -{% include_example scala/org/apache/spark/examples/ml/OneHotEncoderEstimatorExample.scala %} +{% include_example scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala %}
-Refer to the [OneHotEncoderEstimator Java docs](api/java/org/apache/spark/ml/feature/OneHotEncoderEstimator.html) +Refer to the [OneHotEncoder Java docs](api/java/org/apache/spark/ml/feature/OneHotEncoder.html) for more details on the API. -{% include_example java/org/apache/spark/examples/ml/JavaOneHotEncoderEstimatorExample.java %} +{% include_example java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java %}
-Refer to the [OneHotEncoderEstimator Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.OneHotEncoderEstimator) for more details on the API. +Refer to the [OneHotEncoder Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.OneHotEncoder) for more details on the API. -{% include_example python/ml/onehot_encoder_estimator_example.py %} +{% include_example python/ml/onehot_encoder_example.py %}
diff --git a/docs/ml-guide.md b/docs/ml-guide.md index aea07be34cb8..57d4e1fe9d33 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -104,6 +104,12 @@ MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, and the migration guide below will explain all changes between releases. +## From 2.4 to 3.0 + +### Breaking changes + +* `OneHotEncoder` which is deprecated in 2.3, is removed in 3.0 and `OneHotEncoderEstimator` is now renamed to `OneHotEncoder`. + ## From 2.2 to 2.3 ### Breaking changes diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderEstimatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java similarity index 91% rename from examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderEstimatorExample.java rename to examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java index 6f93cff94b72..4b49bebf7ccf 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderEstimatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java @@ -23,7 +23,7 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.ml.feature.OneHotEncoderEstimator; +import org.apache.spark.ml.feature.OneHotEncoder; import org.apache.spark.ml.feature.OneHotEncoderModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -34,11 +34,11 @@ import org.apache.spark.sql.types.StructType; // $example off$ -public class JavaOneHotEncoderEstimatorExample { +public class JavaOneHotEncoderExample { public static void main(String[] args) { SparkSession spark = SparkSession .builder() - .appName("JavaOneHotEncoderEstimatorExample") + .appName("JavaOneHotEncoderExample") .getOrCreate(); // Note: categorical features are usually first encoded with StringIndexer @@ -59,7 +59,7 @@ public static void main(String[] args) { Dataset df = spark.createDataFrame(data, schema); - OneHotEncoderEstimator encoder = new OneHotEncoderEstimator() + OneHotEncoder encoder = new OneHotEncoder() .setInputCols(new String[] {"categoryIndex1", "categoryIndex2"}) .setOutputCols(new String[] {"categoryVec1", "categoryVec2"}); diff --git a/examples/src/main/python/ml/onehot_encoder_estimator_example.py b/examples/src/main/python/ml/onehot_encoder_example.py similarity index 83% rename from examples/src/main/python/ml/onehot_encoder_estimator_example.py rename to examples/src/main/python/ml/onehot_encoder_example.py index 2723e681cea7..73775b79e36c 100644 --- a/examples/src/main/python/ml/onehot_encoder_estimator_example.py +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -18,14 +18,14 @@ from __future__ import print_function # $example on$ -from pyspark.ml.feature import OneHotEncoderEstimator +from pyspark.ml.feature import OneHotEncoder # $example off$ from pyspark.sql import SparkSession if __name__ == "__main__": spark = SparkSession\ .builder\ - .appName("OneHotEncoderEstimatorExample")\ + .appName("OneHotEncoderExample")\ .getOrCreate() # Note: categorical features are usually first encoded with StringIndexer @@ -39,8 +39,8 @@ (2.0, 0.0) ], ["categoryIndex1", "categoryIndex2"]) - encoder = OneHotEncoderEstimator(inputCols=["categoryIndex1", "categoryIndex2"], - outputCols=["categoryVec1", "categoryVec2"]) + encoder = OneHotEncoder(inputCols=["categoryIndex1", "categoryIndex2"], + outputCols=["categoryVec1", "categoryVec2"]) model = encoder.fit(df) encoded = model.transform(df) encoded.show() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderEstimatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala similarity index 89% rename from examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderEstimatorExample.scala rename to examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala index 45d816808ed8..742f3cdeea35 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderEstimatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala @@ -19,15 +19,15 @@ package org.apache.spark.examples.ml // $example on$ -import org.apache.spark.ml.feature.OneHotEncoderEstimator +import org.apache.spark.ml.feature.OneHotEncoder // $example off$ import org.apache.spark.sql.SparkSession -object OneHotEncoderEstimatorExample { +object OneHotEncoderExample { def main(args: Array[String]): Unit = { val spark = SparkSession .builder - .appName("OneHotEncoderEstimatorExample") + .appName("OneHotEncoderExample") .getOrCreate() // Note: categorical features are usually first encoded with StringIndexer @@ -41,7 +41,7 @@ object OneHotEncoderEstimatorExample { (2.0, 0.0) )).toDF("categoryIndex1", "categoryIndex2") - val encoder = new OneHotEncoderEstimator() + val encoder = new OneHotEncoder() .setInputCols(Array("categoryIndex1", "categoryIndex2")) .setOutputCols(Array("categoryVec1", "categoryVec2")) val model = encoder.fit(df) 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 27e4869a020b..ec9792cbbda8 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 @@ -17,126 +17,512 @@ package org.apache.spark.ml.feature +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException import org.apache.spark.annotation.Since -import org.apache.spark.ml.Transformer +import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCols, HasOutputCols} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.functions.{col, udf} -import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.functions.{col, lit, udf} +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} + +/** Private trait for params and common methods for OneHotEncoder and OneHotEncoderModel */ +private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid + with HasInputCols with HasOutputCols { + + /** + * Param for how to handle invalid data during transform(). + * Options are 'keep' (invalid data presented as an extra categorical feature) or + * 'error' (throw an error). + * Note that this Param is only used during transform; during fitting, invalid data + * will result in an error. + * Default: "error" + * @group param + */ + @Since("2.3.0") + override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", + "How to handle invalid data during transform(). " + + "Options are 'keep' (invalid data presented as an extra categorical feature) " + + "or error (throw an error). Note that this Param is only used during transform; " + + "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 + */ + @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) + + protected def validateAndTransformSchema( + schema: StructType, + dropLast: Boolean, + keepInvalid: Boolean): StructType = { + val inputColNames = $(inputCols) + val outputColNames = $(outputCols) + + require(inputColNames.length == outputColNames.length, + s"The number of input columns ${inputColNames.length} must be the same as the number of " + + s"output columns ${outputColNames.length}.") + + // Input columns must be NumericType. + inputColNames.foreach(SchemaUtils.checkNumericType(schema, _)) + + // Prepares output columns with proper attributes by examining input columns. + val inputFields = $(inputCols).map(schema(_)) + + val outputFields = inputFields.zip(outputColNames).map { case (inputField, outputColName) => + OneHotEncoderCommon.transformOutputColumnSchema( + inputField, outputColName, dropLast, keepInvalid) + } + outputFields.foldLeft(schema) { case (newSchema, outputField) => + SchemaUtils.appendColumn(newSchema, outputField) + } + } +} /** * A one-hot encoder that maps a column of category indices to a column of binary vectors, with * at most a single one-value per row that indicates the input category index. * For example with 5 categories, an input value of 2.0 would map to an output vector of * `[0.0, 0.0, 1.0, 0.0]`. - * The last category is not included by default (configurable via `OneHotEncoder!.dropLast` + * The last category is not included by default (configurable via `dropLast`), * because it makes the vector entries sum up to one, and hence linearly dependent. * So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. * * @note This is different from scikit-learn's OneHotEncoder, which keeps all categories. * The output vectors are sparse. * + * When `handleInvalid` is configured to 'keep', an extra "category" indicating invalid values is + * added as last category. So when `dropLast` is true, invalid values are encoded as all-zeros + * vector. + * + * @note When encoding multi-column by using `inputCols` and `outputCols` params, input/output cols + * come in pairs, specified by the order in the arrays, and each pair is treated independently. + * * @see `StringIndexer` for converting categorical values into category indices - * @deprecated `OneHotEncoderEstimator` will be renamed `OneHotEncoder` and this `OneHotEncoder` - * will be removed in 3.0.0. */ -@Since("1.4.0") -@deprecated("`OneHotEncoderEstimator` will be renamed `OneHotEncoder` and this `OneHotEncoder`" + - " will be removed in 3.0.0.", "2.3.0") -class OneHotEncoder @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer - with HasInputCol with HasOutputCol with DefaultParamsWritable { +@Since("3.0.0") +class OneHotEncoder @Since("3.0.0") (@Since("3.0.0") override val uid: String) + extends Estimator[OneHotEncoderModel] with OneHotEncoderBase with DefaultParamsWritable { - @Since("1.4.0") - def this() = this(Identifiable.randomUID("oneHot")) + @Since("3.0.0") + def this() = this(Identifiable.randomUID("oneHotEncoder")) - /** - * Whether to drop the last category in the encoded vector (default: true) - * @group param - */ - @Since("1.4.0") - final val dropLast: BooleanParam = - new BooleanParam(this, "dropLast", "whether to drop the last category") - setDefault(dropLast -> true) + /** @group setParam */ + @Since("3.0.0") + def setInputCols(values: Array[String]): this.type = set(inputCols, values) - /** @group getParam */ - @Since("2.0.0") - def getDropLast: Boolean = $(dropLast) + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(values: Array[String]): this.type = set(outputCols, values) /** @group setParam */ - @Since("1.4.0") + @Since("3.0.0") def setDropLast(value: Boolean): this.type = set(dropLast, value) /** @group setParam */ - @Since("1.4.0") - def setInputCol(value: String): this.type = set(inputCol, value) + @Since("3.0.0") + def setHandleInvalid(value: String): this.type = set(handleInvalid, value) + + @Since("3.0.0") + override def transformSchema(schema: StructType): StructType = { + val keepInvalid = $(handleInvalid) == OneHotEncoder.KEEP_INVALID + validateAndTransformSchema(schema, dropLast = $(dropLast), + keepInvalid = keepInvalid) + } + + @Since("3.0.0") + override def fit(dataset: Dataset[_]): OneHotEncoderModel = { + transformSchema(dataset.schema) + + // Compute the plain number of categories without `handleInvalid` and + // `dropLast` taken into account. + val transformedSchema = validateAndTransformSchema(dataset.schema, dropLast = false, + keepInvalid = false) + val categorySizes = new Array[Int]($(outputCols).length) + + val columnToScanIndices = $(outputCols).zipWithIndex.flatMap { case (outputColName, idx) => + val numOfAttrs = AttributeGroup.fromStructField( + transformedSchema(outputColName)).size + if (numOfAttrs < 0) { + Some(idx) + } else { + categorySizes(idx) = numOfAttrs + None + } + } + + // Some input columns don't have attributes or their attributes don't have necessary info. + // We need to scan the data to get the number of values for each column. + if (columnToScanIndices.length > 0) { + val inputColNames = columnToScanIndices.map($(inputCols)(_)) + val outputColNames = columnToScanIndices.map($(outputCols)(_)) + + // When fitting data, we want the plain number of categories without `handleInvalid` and + // `dropLast` taken into account. + val attrGroups = OneHotEncoderCommon.getOutputAttrGroupFromData( + dataset, inputColNames, outputColNames, dropLast = false) + attrGroups.zip(columnToScanIndices).foreach { case (attrGroup, idx) => + categorySizes(idx) = attrGroup.size + } + } + + val model = new OneHotEncoderModel(uid, categorySizes).setParent(this) + copyValues(model) + } + + @Since("3.0.0") + override def copy(extra: ParamMap): OneHotEncoder = defaultCopy(extra) +} + +@Since("3.0.0") +object OneHotEncoder extends DefaultParamsReadable[OneHotEncoder] { + + private[feature] val KEEP_INVALID: String = "keep" + private[feature] val ERROR_INVALID: String = "error" + private[feature] val supportedHandleInvalids: Array[String] = Array(KEEP_INVALID, ERROR_INVALID) + + @Since("3.0.0") + override def load(path: String): OneHotEncoder = super.load(path) +} + +/** + * @param categorySizes Original number of categories for each feature being encoded. + * The array contains one value for each input column, in order. + */ +@Since("3.0.0") +class OneHotEncoderModel private[ml] ( + @Since("3.0.0") override val uid: String, + @Since("3.0.0") val categorySizes: Array[Int]) + extends Model[OneHotEncoderModel] with OneHotEncoderBase with MLWritable { + + import OneHotEncoderModel._ + + // Returns the category size for each index with `dropLast` and `handleInvalid` + // taken into account. + private def getConfigedCategorySizes: Array[Int] = { + val dropLast = getDropLast + val keepInvalid = getHandleInvalid == OneHotEncoder.KEEP_INVALID + + if (!dropLast && keepInvalid) { + // When `handleInvalid` is "keep", an extra category is added as last category + // for invalid data. + categorySizes.map(_ + 1) + } else if (dropLast && !keepInvalid) { + // When `dropLast` is true, the last category is removed. + categorySizes.map(_ - 1) + } else { + // When `dropLast` is true and `handleInvalid` is "keep", the extra category for invalid + // data is removed. Thus, it is the same as the plain number of categories. + categorySizes + } + } + + private def encoder: UserDefinedFunction = { + val keepInvalid = getHandleInvalid == OneHotEncoder.KEEP_INVALID + val configedSizes = getConfigedCategorySizes + val localCategorySizes = categorySizes + + // The udf performed on input data. The first parameter is the input value. The second + // parameter is the index in inputCols of the column being encoded. + udf { (label: Double, colIdx: Int) => + val origCategorySize = localCategorySizes(colIdx) + // idx: index in vector of the single 1-valued element + val idx = if (label >= 0 && label < origCategorySize) { + label + } else { + if (keepInvalid) { + origCategorySize + } else { + if (label < 0) { + throw new SparkException(s"Negative value: $label. Input can't be negative. " + + s"To handle invalid values, set Param handleInvalid to " + + s"${OneHotEncoder.KEEP_INVALID}") + } else { + throw new SparkException(s"Unseen value: $label. To handle unseen values, " + + s"set Param handleInvalid to ${OneHotEncoder.KEEP_INVALID}.") + } + } + } + + val size = configedSizes(colIdx) + if (idx < size) { + Vectors.sparse(size, Array(idx.toInt), Array(1.0)) + } else { + Vectors.sparse(size, Array.empty[Int], Array.empty[Double]) + } + } + } /** @group setParam */ - @Since("1.4.0") - def setOutputCol(value: String): this.type = set(outputCol, value) + @Since("3.0.0") + def setInputCols(values: Array[String]): this.type = set(inputCols, values) - @Since("1.4.0") + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(values: Array[String]): this.type = set(outputCols, values) + + /** @group setParam */ + @Since("3.0.0") + def setDropLast(value: Boolean): this.type = set(dropLast, value) + + /** @group setParam */ + @Since("3.0.0") + def setHandleInvalid(value: String): this.type = set(handleInvalid, value) + + @Since("3.0.0") override def transformSchema(schema: StructType): StructType = { - val inputColName = $(inputCol) - val outputColName = $(outputCol) - val inputFields = schema.fields + val inputColNames = $(inputCols) + + require(inputColNames.length == categorySizes.length, + s"The number of input columns ${inputColNames.length} must be the same as the number of " + + s"features ${categorySizes.length} during fitting.") + + val keepInvalid = $(handleInvalid) == OneHotEncoder.KEEP_INVALID + val transformedSchema = validateAndTransformSchema(schema, dropLast = $(dropLast), + keepInvalid = keepInvalid) + verifyNumOfValues(transformedSchema) + } - require(schema(inputColName).dataType.isInstanceOf[NumericType], - s"Input column must be of type ${NumericType.simpleString} but got " + - schema(inputColName).dataType.catalogString) - require(!inputFields.exists(_.name == outputColName), - s"Output column $outputColName already exists.") + /** + * If the metadata of input columns also specifies the number of categories, we need to + * compare with expected category number with `handleInvalid` and `dropLast` taken into + * account. Mismatched numbers will cause exception. + */ + private def verifyNumOfValues(schema: StructType): StructType = { + val configedSizes = getConfigedCategorySizes + $(outputCols).zipWithIndex.foreach { case (outputColName, idx) => + val inputColName = $(inputCols)(idx) + val attrGroup = AttributeGroup.fromStructField(schema(outputColName)) - val outputField = OneHotEncoderCommon.transformOutputColumnSchema( - schema(inputColName), outputColName, $(dropLast)) - val outputFields = inputFields :+ outputField - StructType(outputFields) + // If the input metadata specifies number of category for output column, + // comparing with expected category number with `handleInvalid` and + // `dropLast` taken into account. + if (attrGroup.attributes.nonEmpty) { + val numCategories = configedSizes(idx) + require(attrGroup.size == numCategories, "OneHotEncoderModel expected " + + s"$numCategories categorical values for input column $inputColName, " + + s"but the input column had metadata specifying ${attrGroup.size} values.") + } + } + schema } - @Since("2.0.0") + @Since("3.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - // schema transformation - val inputColName = $(inputCol) - val outputColName = $(outputCol) + val transformedSchema = transformSchema(dataset.schema, logging = true) + val keepInvalid = $(handleInvalid) == OneHotEncoder.KEEP_INVALID - val outputAttrGroupFromSchema = AttributeGroup.fromStructField( - transformSchema(dataset.schema)(outputColName)) + val encodedColumns = $(inputCols).indices.map { idx => + val inputColName = $(inputCols)(idx) + val outputColName = $(outputCols)(idx) - val outputAttrGroup = if (outputAttrGroupFromSchema.size < 0) { - OneHotEncoderCommon.getOutputAttrGroupFromData( - dataset, Seq(inputColName), Seq(outputColName), $(dropLast))(0) - } else { - outputAttrGroupFromSchema + val outputAttrGroupFromSchema = + AttributeGroup.fromStructField(transformedSchema(outputColName)) + + val metadata = if (outputAttrGroupFromSchema.size < 0) { + OneHotEncoderCommon.createAttrGroupForAttrNames(outputColName, + categorySizes(idx), $(dropLast), keepInvalid).toMetadata() + } else { + outputAttrGroupFromSchema.toMetadata() + } + + encoder(col(inputColName).cast(DoubleType), lit(idx)) + .as(outputColName, metadata) + } + dataset.withColumns($(outputCols), encodedColumns) + } + + @Since("3.0.0") + override def copy(extra: ParamMap): OneHotEncoderModel = { + val copied = new OneHotEncoderModel(uid, categorySizes) + copyValues(copied, extra).setParent(parent) + } + + @Since("3.0.0") + override def write: MLWriter = new OneHotEncoderModelWriter(this) +} + +@Since("3.0.0") +object OneHotEncoderModel extends MLReadable[OneHotEncoderModel] { + + private[OneHotEncoderModel] + class OneHotEncoderModelWriter(instance: OneHotEncoderModel) extends MLWriter { + + private case class Data(categorySizes: Array[Int]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.categorySizes) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class OneHotEncoderModelReader extends MLReader[OneHotEncoderModel] { + + private val className = classOf[OneHotEncoderModel].getName + + override def load(path: String): OneHotEncoderModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("categorySizes") + .head() + val categorySizes = data.getAs[Seq[Int]](0).toArray + val model = new OneHotEncoderModel(metadata.uid, categorySizes) + metadata.getAndSetParams(model) + model + } + } + + @Since("3.0.0") + override def read: MLReader[OneHotEncoderModel] = new OneHotEncoderModelReader + + @Since("3.0.0") + override def load(path: String): OneHotEncoderModel = super.load(path) +} + +/** + * Provides some helper methods used by `OneHotEncoder`. + */ +private[feature] object OneHotEncoderCommon { + + private def genOutputAttrNames(inputCol: StructField): Option[Array[String]] = { + val inputAttr = Attribute.fromStructField(inputCol) + inputAttr match { + case nominal: NominalAttribute => + if (nominal.values.isDefined) { + nominal.values + } else if (nominal.numValues.isDefined) { + nominal.numValues.map(n => Array.tabulate(n)(_.toString)) + } else { + None + } + case binary: BinaryAttribute => + if (binary.values.isDefined) { + binary.values + } else { + Some(Array.tabulate(2)(_.toString)) + } + case _: NumericAttribute => + throw new RuntimeException( + s"The input column ${inputCol.name} cannot be continuous-value.") + case _ => + None // optimistic about unknown attributes } + } - val metadata = outputAttrGroup.toMetadata() + /** Creates an `AttributeGroup` filled by the `BinaryAttribute` named as required. */ + private def genOutputAttrGroup( + outputAttrNames: Option[Array[String]], + outputColName: String): AttributeGroup = { + outputAttrNames.map { attrNames => + val attrs: Array[Attribute] = attrNames.map { name => + BinaryAttribute.defaultAttr.withName(name) + } + new AttributeGroup(outputColName, attrs) + }.getOrElse{ + new AttributeGroup(outputColName) + } + } - // data transformation - val size = outputAttrGroup.size - val oneValue = Array(1.0) - val emptyValues = Array.empty[Double] - val emptyIndices = Array.empty[Int] - val encode = udf { label: Double => - if (label < size) { - Vectors.sparse(size, Array(label.toInt), oneValue) + /** + * Prepares the `StructField` with proper metadata for `OneHotEncoder`'s output column. + */ + def transformOutputColumnSchema( + inputCol: StructField, + outputColName: String, + dropLast: Boolean, + keepInvalid: Boolean = false): StructField = { + val outputAttrNames = genOutputAttrNames(inputCol) + val filteredOutputAttrNames = outputAttrNames.map { names => + if (dropLast && !keepInvalid) { + require(names.length > 1, + s"The input column ${inputCol.name} should have at least two distinct values.") + names.dropRight(1) + } else if (!dropLast && keepInvalid) { + names ++ Seq("invalidValues") } else { - Vectors.sparse(size, emptyIndices, emptyValues) + names } } - dataset.select(col("*"), encode(col(inputColName).cast(DoubleType)).as(outputColName, metadata)) + genOutputAttrGroup(filteredOutputAttrNames, outputColName).toStructField() } - @Since("1.4.1") - override def copy(extra: ParamMap): OneHotEncoder = defaultCopy(extra) -} + /** + * This method is called when we want to generate `AttributeGroup` from actual data for + * one-hot encoder. + */ + def getOutputAttrGroupFromData( + dataset: Dataset[_], + inputColNames: Seq[String], + outputColNames: Seq[String], + dropLast: Boolean): Seq[AttributeGroup] = { + // The RDD approach has advantage of early-stop if any values are invalid. It seems that + // DataFrame ops don't have equivalent functions. + val columns = inputColNames.map { inputColName => + col(inputColName).cast(DoubleType) + } + val numOfColumns = columns.length -@Since("1.6.0") -object OneHotEncoder extends DefaultParamsReadable[OneHotEncoder] { + val numAttrsArray = dataset.select(columns: _*).rdd.map { row => + (0 until numOfColumns).map(idx => row.getDouble(idx)).toArray + }.treeAggregate(new Array[Double](numOfColumns))( + (maxValues, curValues) => { + (0 until numOfColumns).foreach { idx => + val x = curValues(idx) + assert(x <= Int.MaxValue, + s"OneHotEncoder only supports up to ${Int.MaxValue} indices, but got $x.") + assert(x >= 0.0 && x == x.toInt, + s"Values from column ${inputColNames(idx)} must be indices, but got $x.") + maxValues(idx) = math.max(maxValues(idx), x) + } + maxValues + }, + (m0, m1) => { + (0 until numOfColumns).foreach { idx => + m0(idx) = math.max(m0(idx), m1(idx)) + } + m0 + } + ).map(_.toInt + 1) - @Since("1.6.0") - override def load(path: String): OneHotEncoder = super.load(path) + outputColNames.zip(numAttrsArray).map { case (outputColName, numAttrs) => + createAttrGroupForAttrNames(outputColName, numAttrs, dropLast, keepInvalid = false) + } + } + + /** Creates an `AttributeGroup` with the required number of `BinaryAttribute`. */ + def createAttrGroupForAttrNames( + outputColName: String, + numAttrs: Int, + dropLast: Boolean, + keepInvalid: Boolean): AttributeGroup = { + val outputAttrNames = Array.tabulate(numAttrs)(_.toString) + val filtered = if (dropLast && !keepInvalid) { + outputAttrNames.dropRight(1) + } else if (!dropLast && keepInvalid) { + outputAttrNames ++ Seq("invalidValues") + } else { + outputAttrNames + } + genOutputAttrGroup(Some(filtered), outputColName) + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoderEstimator.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoderEstimator.scala deleted file mode 100644 index 4a44f3186538..000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoderEstimator.scala +++ /dev/null @@ -1,528 +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.ml.feature - -import org.apache.hadoop.fs.Path - -import org.apache.spark.SparkException -import org.apache.spark.annotation.Since -import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.attribute._ -import org.apache.spark.ml.linalg.Vectors -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCols, HasOutputCols} -import org.apache.spark.ml.util._ -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.expressions.UserDefinedFunction -import org.apache.spark.sql.functions.{col, lit, udf} -import org.apache.spark.sql.types.{DoubleType, StructField, StructType} - -/** Private trait for params and common methods for OneHotEncoderEstimator and OneHotEncoderModel */ -private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid - with HasInputCols with HasOutputCols { - - /** - * Param for how to handle invalid data during transform(). - * Options are 'keep' (invalid data presented as an extra categorical feature) or - * 'error' (throw an error). - * Note that this Param is only used during transform; during fitting, invalid data - * will result in an error. - * Default: "error" - * @group param - */ - @Since("2.3.0") - override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", - "How to handle invalid data during transform(). " + - "Options are 'keep' (invalid data presented as an extra categorical feature) " + - "or error (throw an error). Note that this Param is only used during transform; " + - "during fitting, invalid data will result in an error.", - ParamValidators.inArray(OneHotEncoderEstimator.supportedHandleInvalids)) - - setDefault(handleInvalid, OneHotEncoderEstimator.ERROR_INVALID) - - /** - * Whether to drop the last category in the encoded vector (default: true) - * @group param - */ - @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) - - protected def validateAndTransformSchema( - schema: StructType, - dropLast: Boolean, - keepInvalid: Boolean): StructType = { - val inputColNames = $(inputCols) - val outputColNames = $(outputCols) - - require(inputColNames.length == outputColNames.length, - s"The number of input columns ${inputColNames.length} must be the same as the number of " + - s"output columns ${outputColNames.length}.") - - // Input columns must be NumericType. - inputColNames.foreach(SchemaUtils.checkNumericType(schema, _)) - - // Prepares output columns with proper attributes by examining input columns. - val inputFields = $(inputCols).map(schema(_)) - - val outputFields = inputFields.zip(outputColNames).map { case (inputField, outputColName) => - OneHotEncoderCommon.transformOutputColumnSchema( - inputField, outputColName, dropLast, keepInvalid) - } - outputFields.foldLeft(schema) { case (newSchema, outputField) => - SchemaUtils.appendColumn(newSchema, outputField) - } - } -} - -/** - * A one-hot encoder that maps a column of category indices to a column of binary vectors, with - * at most a single one-value per row that indicates the input category index. - * For example with 5 categories, an input value of 2.0 would map to an output vector of - * `[0.0, 0.0, 1.0, 0.0]`. - * The last category is not included by default (configurable via `dropLast`), - * because it makes the vector entries sum up to one, and hence linearly dependent. - * So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. - * - * @note This is different from scikit-learn's OneHotEncoder, which keeps all categories. - * The output vectors are sparse. - * - * When `handleInvalid` is configured to 'keep', an extra "category" indicating invalid values is - * added as last category. So when `dropLast` is true, invalid values are encoded as all-zeros - * vector. - * - * @note When encoding multi-column by using `inputCols` and `outputCols` params, input/output cols - * come in pairs, specified by the order in the arrays, and each pair is treated independently. - * - * @see `StringIndexer` for converting categorical values into category indices - */ -@Since("2.3.0") -class OneHotEncoderEstimator @Since("2.3.0") (@Since("2.3.0") override val uid: String) - extends Estimator[OneHotEncoderModel] with OneHotEncoderBase with DefaultParamsWritable { - - @Since("2.3.0") - def this() = this(Identifiable.randomUID("oneHotEncoder")) - - /** @group setParam */ - @Since("2.3.0") - def setInputCols(values: Array[String]): this.type = set(inputCols, values) - - /** @group setParam */ - @Since("2.3.0") - def setOutputCols(values: Array[String]): this.type = set(outputCols, values) - - /** @group setParam */ - @Since("2.3.0") - def setDropLast(value: Boolean): this.type = set(dropLast, value) - - /** @group setParam */ - @Since("2.3.0") - def setHandleInvalid(value: String): this.type = set(handleInvalid, value) - - @Since("2.3.0") - override def transformSchema(schema: StructType): StructType = { - val keepInvalid = $(handleInvalid) == OneHotEncoderEstimator.KEEP_INVALID - validateAndTransformSchema(schema, dropLast = $(dropLast), - keepInvalid = keepInvalid) - } - - @Since("2.3.0") - override def fit(dataset: Dataset[_]): OneHotEncoderModel = { - transformSchema(dataset.schema) - - // Compute the plain number of categories without `handleInvalid` and - // `dropLast` taken into account. - val transformedSchema = validateAndTransformSchema(dataset.schema, dropLast = false, - keepInvalid = false) - val categorySizes = new Array[Int]($(outputCols).length) - - val columnToScanIndices = $(outputCols).zipWithIndex.flatMap { case (outputColName, idx) => - val numOfAttrs = AttributeGroup.fromStructField( - transformedSchema(outputColName)).size - if (numOfAttrs < 0) { - Some(idx) - } else { - categorySizes(idx) = numOfAttrs - None - } - } - - // Some input columns don't have attributes or their attributes don't have necessary info. - // We need to scan the data to get the number of values for each column. - if (columnToScanIndices.length > 0) { - val inputColNames = columnToScanIndices.map($(inputCols)(_)) - val outputColNames = columnToScanIndices.map($(outputCols)(_)) - - // When fitting data, we want the plain number of categories without `handleInvalid` and - // `dropLast` taken into account. - val attrGroups = OneHotEncoderCommon.getOutputAttrGroupFromData( - dataset, inputColNames, outputColNames, dropLast = false) - attrGroups.zip(columnToScanIndices).foreach { case (attrGroup, idx) => - categorySizes(idx) = attrGroup.size - } - } - - val model = new OneHotEncoderModel(uid, categorySizes).setParent(this) - copyValues(model) - } - - @Since("2.3.0") - override def copy(extra: ParamMap): OneHotEncoderEstimator = defaultCopy(extra) -} - -@Since("2.3.0") -object OneHotEncoderEstimator extends DefaultParamsReadable[OneHotEncoderEstimator] { - - private[feature] val KEEP_INVALID: String = "keep" - private[feature] val ERROR_INVALID: String = "error" - private[feature] val supportedHandleInvalids: Array[String] = Array(KEEP_INVALID, ERROR_INVALID) - - @Since("2.3.0") - override def load(path: String): OneHotEncoderEstimator = super.load(path) -} - -/** - * @param categorySizes Original number of categories for each feature being encoded. - * The array contains one value for each input column, in order. - */ -@Since("2.3.0") -class OneHotEncoderModel private[ml] ( - @Since("2.3.0") override val uid: String, - @Since("2.3.0") val categorySizes: Array[Int]) - extends Model[OneHotEncoderModel] with OneHotEncoderBase with MLWritable { - - import OneHotEncoderModel._ - - // Returns the category size for each index with `dropLast` and `handleInvalid` - // taken into account. - private def getConfigedCategorySizes: Array[Int] = { - val dropLast = getDropLast - val keepInvalid = getHandleInvalid == OneHotEncoderEstimator.KEEP_INVALID - - if (!dropLast && keepInvalid) { - // When `handleInvalid` is "keep", an extra category is added as last category - // for invalid data. - categorySizes.map(_ + 1) - } else if (dropLast && !keepInvalid) { - // When `dropLast` is true, the last category is removed. - categorySizes.map(_ - 1) - } else { - // When `dropLast` is true and `handleInvalid` is "keep", the extra category for invalid - // data is removed. Thus, it is the same as the plain number of categories. - categorySizes - } - } - - private def encoder: UserDefinedFunction = { - val keepInvalid = getHandleInvalid == OneHotEncoderEstimator.KEEP_INVALID - val configedSizes = getConfigedCategorySizes - val localCategorySizes = categorySizes - - // The udf performed on input data. The first parameter is the input value. The second - // parameter is the index in inputCols of the column being encoded. - udf { (label: Double, colIdx: Int) => - val origCategorySize = localCategorySizes(colIdx) - // idx: index in vector of the single 1-valued element - val idx = if (label >= 0 && label < origCategorySize) { - label - } else { - if (keepInvalid) { - origCategorySize - } else { - if (label < 0) { - throw new SparkException(s"Negative value: $label. Input can't be negative. " + - s"To handle invalid values, set Param handleInvalid to " + - s"${OneHotEncoderEstimator.KEEP_INVALID}") - } else { - throw new SparkException(s"Unseen value: $label. To handle unseen values, " + - s"set Param handleInvalid to ${OneHotEncoderEstimator.KEEP_INVALID}.") - } - } - } - - val size = configedSizes(colIdx) - if (idx < size) { - Vectors.sparse(size, Array(idx.toInt), Array(1.0)) - } else { - Vectors.sparse(size, Array.empty[Int], Array.empty[Double]) - } - } - } - - /** @group setParam */ - @Since("2.3.0") - def setInputCols(values: Array[String]): this.type = set(inputCols, values) - - /** @group setParam */ - @Since("2.3.0") - def setOutputCols(values: Array[String]): this.type = set(outputCols, values) - - /** @group setParam */ - @Since("2.3.0") - def setDropLast(value: Boolean): this.type = set(dropLast, value) - - /** @group setParam */ - @Since("2.3.0") - def setHandleInvalid(value: String): this.type = set(handleInvalid, value) - - @Since("2.3.0") - override def transformSchema(schema: StructType): StructType = { - val inputColNames = $(inputCols) - - require(inputColNames.length == categorySizes.length, - s"The number of input columns ${inputColNames.length} must be the same as the number of " + - s"features ${categorySizes.length} during fitting.") - - val keepInvalid = $(handleInvalid) == OneHotEncoderEstimator.KEEP_INVALID - val transformedSchema = validateAndTransformSchema(schema, dropLast = $(dropLast), - keepInvalid = keepInvalid) - verifyNumOfValues(transformedSchema) - } - - /** - * If the metadata of input columns also specifies the number of categories, we need to - * compare with expected category number with `handleInvalid` and `dropLast` taken into - * account. Mismatched numbers will cause exception. - */ - private def verifyNumOfValues(schema: StructType): StructType = { - val configedSizes = getConfigedCategorySizes - $(outputCols).zipWithIndex.foreach { case (outputColName, idx) => - val inputColName = $(inputCols)(idx) - val attrGroup = AttributeGroup.fromStructField(schema(outputColName)) - - // If the input metadata specifies number of category for output column, - // comparing with expected category number with `handleInvalid` and - // `dropLast` taken into account. - if (attrGroup.attributes.nonEmpty) { - val numCategories = configedSizes(idx) - require(attrGroup.size == numCategories, "OneHotEncoderModel expected " + - s"$numCategories categorical values for input column $inputColName, " + - s"but the input column had metadata specifying ${attrGroup.size} values.") - } - } - schema - } - - @Since("2.3.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val transformedSchema = transformSchema(dataset.schema, logging = true) - val keepInvalid = $(handleInvalid) == OneHotEncoderEstimator.KEEP_INVALID - - val encodedColumns = $(inputCols).indices.map { idx => - val inputColName = $(inputCols)(idx) - val outputColName = $(outputCols)(idx) - - val outputAttrGroupFromSchema = - AttributeGroup.fromStructField(transformedSchema(outputColName)) - - val metadata = if (outputAttrGroupFromSchema.size < 0) { - OneHotEncoderCommon.createAttrGroupForAttrNames(outputColName, - categorySizes(idx), $(dropLast), keepInvalid).toMetadata() - } else { - outputAttrGroupFromSchema.toMetadata() - } - - encoder(col(inputColName).cast(DoubleType), lit(idx)) - .as(outputColName, metadata) - } - dataset.withColumns($(outputCols), encodedColumns) - } - - @Since("2.3.0") - override def copy(extra: ParamMap): OneHotEncoderModel = { - val copied = new OneHotEncoderModel(uid, categorySizes) - copyValues(copied, extra).setParent(parent) - } - - @Since("2.3.0") - override def write: MLWriter = new OneHotEncoderModelWriter(this) -} - -@Since("2.3.0") -object OneHotEncoderModel extends MLReadable[OneHotEncoderModel] { - - private[OneHotEncoderModel] - class OneHotEncoderModelWriter(instance: OneHotEncoderModel) extends MLWriter { - - private case class Data(categorySizes: Array[Int]) - - override protected def saveImpl(path: String): Unit = { - DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.categorySizes) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } - } - - private class OneHotEncoderModelReader extends MLReader[OneHotEncoderModel] { - - private val className = classOf[OneHotEncoderModel].getName - - override def load(path: String): OneHotEncoderModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - val data = sparkSession.read.parquet(dataPath) - .select("categorySizes") - .head() - val categorySizes = data.getAs[Seq[Int]](0).toArray - val model = new OneHotEncoderModel(metadata.uid, categorySizes) - metadata.getAndSetParams(model) - model - } - } - - @Since("2.3.0") - override def read: MLReader[OneHotEncoderModel] = new OneHotEncoderModelReader - - @Since("2.3.0") - override def load(path: String): OneHotEncoderModel = super.load(path) -} - -/** - * Provides some helper methods used by both `OneHotEncoder` and `OneHotEncoderEstimator`. - */ -private[feature] object OneHotEncoderCommon { - - private def genOutputAttrNames(inputCol: StructField): Option[Array[String]] = { - val inputAttr = Attribute.fromStructField(inputCol) - inputAttr match { - case nominal: NominalAttribute => - if (nominal.values.isDefined) { - nominal.values - } else if (nominal.numValues.isDefined) { - nominal.numValues.map(n => Array.tabulate(n)(_.toString)) - } else { - None - } - case binary: BinaryAttribute => - if (binary.values.isDefined) { - binary.values - } else { - Some(Array.tabulate(2)(_.toString)) - } - case _: NumericAttribute => - throw new RuntimeException( - s"The input column ${inputCol.name} cannot be continuous-value.") - case _ => - None // optimistic about unknown attributes - } - } - - /** Creates an `AttributeGroup` filled by the `BinaryAttribute` named as required. */ - private def genOutputAttrGroup( - outputAttrNames: Option[Array[String]], - outputColName: String): AttributeGroup = { - outputAttrNames.map { attrNames => - val attrs: Array[Attribute] = attrNames.map { name => - BinaryAttribute.defaultAttr.withName(name) - } - new AttributeGroup(outputColName, attrs) - }.getOrElse{ - new AttributeGroup(outputColName) - } - } - - /** - * Prepares the `StructField` with proper metadata for `OneHotEncoder`'s output column. - */ - def transformOutputColumnSchema( - inputCol: StructField, - outputColName: String, - dropLast: Boolean, - keepInvalid: Boolean = false): StructField = { - val outputAttrNames = genOutputAttrNames(inputCol) - val filteredOutputAttrNames = outputAttrNames.map { names => - if (dropLast && !keepInvalid) { - require(names.length > 1, - s"The input column ${inputCol.name} should have at least two distinct values.") - names.dropRight(1) - } else if (!dropLast && keepInvalid) { - names ++ Seq("invalidValues") - } else { - names - } - } - - genOutputAttrGroup(filteredOutputAttrNames, outputColName).toStructField() - } - - /** - * This method is called when we want to generate `AttributeGroup` from actual data for - * one-hot encoder. - */ - def getOutputAttrGroupFromData( - dataset: Dataset[_], - inputColNames: Seq[String], - outputColNames: Seq[String], - dropLast: Boolean): Seq[AttributeGroup] = { - // The RDD approach has advantage of early-stop if any values are invalid. It seems that - // DataFrame ops don't have equivalent functions. - val columns = inputColNames.map { inputColName => - col(inputColName).cast(DoubleType) - } - val numOfColumns = columns.length - - val numAttrsArray = dataset.select(columns: _*).rdd.map { row => - (0 until numOfColumns).map(idx => row.getDouble(idx)).toArray - }.treeAggregate(new Array[Double](numOfColumns))( - (maxValues, curValues) => { - (0 until numOfColumns).foreach { idx => - val x = curValues(idx) - assert(x <= Int.MaxValue, - s"OneHotEncoder only supports up to ${Int.MaxValue} indices, but got $x.") - assert(x >= 0.0 && x == x.toInt, - s"Values from column ${inputColNames(idx)} must be indices, but got $x.") - maxValues(idx) = math.max(maxValues(idx), x) - } - maxValues - }, - (m0, m1) => { - (0 until numOfColumns).foreach { idx => - m0(idx) = math.max(m0(idx), m1(idx)) - } - m0 - } - ).map(_.toInt + 1) - - outputColNames.zip(numAttrsArray).map { case (outputColName, numAttrs) => - createAttrGroupForAttrNames(outputColName, numAttrs, dropLast, keepInvalid = false) - } - } - - /** Creates an `AttributeGroup` with the required number of `BinaryAttribute`. */ - def createAttrGroupForAttrNames( - outputColName: String, - numAttrs: Int, - dropLast: Boolean, - keepInvalid: Boolean): AttributeGroup = { - val outputAttrNames = Array.tabulate(numAttrs)(_.toString) - val filtered = if (dropLast && !keepInvalid) { - outputAttrNames.dropRight(1) - } else if (!dropLast && keepInvalid) { - outputAttrNames ++ Seq("invalidValues") - } else { - outputAttrNames - } - genOutputAttrGroup(Some(filtered), outputColName) - } -} 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 346e1823f00b..d7eb13772aa6 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 @@ -246,7 +246,7 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) // Formula w/o intercept, one of the categories in the first category feature is // being used as reference category, we will not drop any category for that feature. if (!hasIntercept && !keepReferenceCategory) { - encoderStages += new OneHotEncoderEstimator(uid) + encoderStages += new OneHotEncoder(uid) .setInputCols(Array(indexed(term))) .setOutputCols(Array(encodedCol)) .setDropLast(false) @@ -269,7 +269,7 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) if (oneHotEncodeColumns.nonEmpty) { val (inputCols, outputCols) = oneHotEncodeColumns.toArray.unzip - encoderStages += new OneHotEncoderEstimator(uid) + encoderStages += new OneHotEncoder(uid) .setInputCols(inputCols) .setOutputCols(outputCols) .setDropLast(true) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderEstimatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderEstimatorSuite.scala deleted file mode 100644 index d549e1326227..000000000000 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderEstimatorSuite.scala +++ /dev/null @@ -1,422 +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.ml.feature - -import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} -import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} -import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} -import org.apache.spark.sql.{Encoder, Row} -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types._ - -class OneHotEncoderEstimatorSuite extends MLTest with DefaultReadWriteTest { - - import testImplicits._ - - test("params") { - ParamsSuite.checkParams(new OneHotEncoderEstimator) - } - - test("OneHotEncoderEstimator dropLast = false") { - val data = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - assert(encoder.getDropLast === true) - encoder.setDropLast(false) - assert(encoder.getDropLast === false) - val model = encoder.fit(df) - testTransformer[(Double, Vector)](df, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - } - } - - test("OneHotEncoderEstimator dropLast = true") { - val data = Seq( - Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(2, Seq((1, 1.0)))), - Row(2.0, Vectors.sparse(2, Seq())), - Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(2, Seq()))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - - val model = encoder.fit(df) - testTransformer[(Double, Vector)](df, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - } - } - - test("input column with ML attribute") { - val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") - val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") - .select(col("size").as("size", attr.toMetadata())) - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("size")) - .setOutputCols(Array("encoded")) - val model = encoder.fit(df) - testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => - val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) - assert(group.size === 2) - assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("small").withIndex(0)) - assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("medium").withIndex(1)) - } - } - - test("input column without ML attribute") { - val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("index") - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("index")) - .setOutputCols(Array("encoded")) - val model = encoder.fit(df) - testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => - val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) - assert(group.size === 2) - assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("0").withIndex(0)) - assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("1").withIndex(1)) - } - } - - test("read/write") { - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("index")) - .setOutputCols(Array("encoded")) - testDefaultReadWrite(encoder) - } - - test("OneHotEncoderModel read/write") { - val instance = new OneHotEncoderModel("myOneHotEncoderModel", Array(1, 2, 3)) - val newInstance = testDefaultReadWrite(instance) - assert(newInstance.categorySizes === instance.categorySizes) - } - - test("OneHotEncoderEstimator with varying types") { - val data = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - class NumericTypeWithEncoder[A](val numericType: NumericType) - (implicit val encoder: Encoder[(A, Vector)]) - - val types = Seq( - new NumericTypeWithEncoder[Short](ShortType), - new NumericTypeWithEncoder[Long](LongType), - new NumericTypeWithEncoder[Int](IntegerType), - new NumericTypeWithEncoder[Float](FloatType), - new NumericTypeWithEncoder[Byte](ByteType), - new NumericTypeWithEncoder[Double](DoubleType), - new NumericTypeWithEncoder[Decimal](DecimalType(10, 0))(ExpressionEncoder())) - - for (t <- types) { - val dfWithTypes = df.select(col("input").cast(t.numericType), col("expected")) - val estimator = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - .setDropLast(false) - - val model = estimator.fit(dfWithTypes) - testTransformer(dfWithTypes, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - }(t.encoder) - } - } - - test("OneHotEncoderEstimator: encoding multiple columns and dropLast = false") { - val data = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), 3.0, Vectors.sparse(4, Seq((3, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 1.0, Vectors.sparse(4, Seq((1, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0))))) - - val schema = StructType(Array( - StructField("input1", DoubleType), - StructField("expected1", new VectorUDT), - StructField("input2", DoubleType), - StructField("expected2", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input1", "input2")) - .setOutputCols(Array("output1", "output2")) - assert(encoder.getDropLast === true) - encoder.setDropLast(false) - assert(encoder.getDropLast === false) - - val model = encoder.fit(df) - testTransformer[(Double, Vector, Double, Vector)]( - df, - model, - "output1", - "output2", - "expected1", - "expected2") { - case Row(output1: Vector, output2: Vector, expected1: Vector, expected2: Vector) => - assert(output1 === expected1) - assert(output2 === expected2) - } - } - - test("OneHotEncoderEstimator: encoding multiple columns and dropLast = true") { - val data = Seq( - Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 2.0, Vectors.sparse(3, Seq((2, 1.0)))), - Row(1.0, Vectors.sparse(2, Seq((1, 1.0))), 3.0, Vectors.sparse(3, Seq())), - Row(2.0, Vectors.sparse(2, Seq()), 0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 1.0, Vectors.sparse(3, Seq((1, 1.0)))), - Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(2, Seq()), 2.0, Vectors.sparse(3, Seq((2, 1.0))))) - - val schema = StructType(Array( - StructField("input1", DoubleType), - StructField("expected1", new VectorUDT), - StructField("input2", DoubleType), - StructField("expected2", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input1", "input2")) - .setOutputCols(Array("output1", "output2")) - - val model = encoder.fit(df) - testTransformer[(Double, Vector, Double, Vector)]( - df, - model, - "output1", - "output2", - "expected1", - "expected2") { - case Row(output1: Vector, output2: Vector, expected1: Vector, expected2: Vector) => - assert(output1 === expected1) - assert(output2 === expected2) - } - } - - test("Throw error on invalid values") { - val trainingData = Seq((0, 0), (1, 1), (2, 2)) - val trainingDF = trainingData.toDF("id", "a") - val testData = Seq((0, 0), (1, 2), (1, 3)) - val testDF = testData.toDF("id", "a") - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("a")) - .setOutputCols(Array("encoded")) - - val model = encoder.fit(trainingDF) - testTransformerByInterceptingException[(Int, Int)]( - testDF, - model, - expectedMessagePart = "Unseen value: 3.0. To handle unseen values", - firstResultCol = "encoded") - - } - - test("Can't transform on negative input") { - val trainingDF = Seq((0, 0), (1, 1), (2, 2)).toDF("a", "b") - val testDF = Seq((0, 0), (-1, 2), (1, 3)).toDF("a", "b") - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("a")) - .setOutputCols(Array("encoded")) - - val model = encoder.fit(trainingDF) - testTransformerByInterceptingException[(Int, Int)]( - testDF, - model, - expectedMessagePart = "Negative value: -1.0. Input can't be negative", - firstResultCol = "encoded") - } - - test("Keep on invalid values: dropLast = false") { - val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") - - val testData = Seq( - Row(0.0, Vectors.sparse(4, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(4, Seq((1, 1.0)))), - Row(3.0, Vectors.sparse(4, Seq((3, 1.0))))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val testDF = spark.createDataFrame(sc.parallelize(testData), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - .setHandleInvalid("keep") - .setDropLast(false) - - val model = encoder.fit(trainingDF) - testTransformer[(Double, Vector)](testDF, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - } - } - - test("Keep on invalid values: dropLast = true") { - val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") - - val testData = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), - Row(3.0, Vectors.sparse(3, Seq()))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val testDF = spark.createDataFrame(sc.parallelize(testData), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - .setHandleInvalid("keep") - .setDropLast(true) - - val model = encoder.fit(trainingDF) - testTransformer[(Double, Vector)](testDF, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - } - } - - test("OneHotEncoderModel changes dropLast") { - val data = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), Vectors.sparse(2, Seq((1, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), Vectors.sparse(2, Seq())), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), Vectors.sparse(2, Seq()))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected1", new VectorUDT), - StructField("expected2", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - - val model = encoder.fit(df) - - model.setDropLast(false) - testTransformer[(Double, Vector, Vector)](df, model, "output", "expected1") { - case Row(output: Vector, expected1: Vector) => - assert(output === expected1) - } - - model.setDropLast(true) - testTransformer[(Double, Vector, Vector)](df, model, "output", "expected2") { - case Row(output: Vector, expected2: Vector) => - assert(output === expected2) - } - } - - test("OneHotEncoderModel changes handleInvalid") { - val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") - - val testData = Seq( - Row(0.0, Vectors.sparse(4, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(4, Seq((1, 1.0)))), - Row(3.0, Vectors.sparse(4, Seq((3, 1.0))))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val testDF = spark.createDataFrame(sc.parallelize(testData), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - - val model = encoder.fit(trainingDF) - model.setHandleInvalid("error") - - testTransformerByInterceptingException[(Double, Vector)]( - testDF, - model, - expectedMessagePart = "Unseen value: 3.0. To handle unseen values", - firstResultCol = "output") - - model.setHandleInvalid("keep") - testTransformerByGlobalCheckFunc[(Double, Vector)](testDF, model, "output") { _ => } - } - - test("Transforming on mismatched attributes") { - val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") - val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") - .select(col("size").as("size", attr.toMetadata())) - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("size")) - .setOutputCols(Array("encoded")) - val model = encoder.fit(df) - - val testAttr = NominalAttribute.defaultAttr.withValues("tiny", "small", "medium", "large") - val testDF = Seq(0.0, 1.0, 2.0, 3.0).map(Tuple1.apply).toDF("size") - .select(col("size").as("size", testAttr.toMetadata())) - testTransformerByInterceptingException[(Double)]( - testDF, - model, - expectedMessagePart = "OneHotEncoderModel expected 2 categorical values", - firstResultCol = "encoded") - } -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala index 41b32b2ffa09..d92313f4ce03 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala @@ -18,72 +18,71 @@ package org.apache.spark.ml.feature import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} -import org.apache.spark.sql.{DataFrame, Encoder, Row} +import org.apache.spark.sql.{Encoder, Row} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.functions.col import org.apache.spark.sql.types._ -class OneHotEncoderSuite - extends MLTest with DefaultReadWriteTest { +class OneHotEncoderSuite extends MLTest with DefaultReadWriteTest { import testImplicits._ - def stringIndexed(): DataFrame = { - val data = Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) - val df = data.toDF("id", "label") - val indexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("labelIndex") - .fit(df) - indexer.transform(df) - } - test("params") { ParamsSuite.checkParams(new OneHotEncoder) } test("OneHotEncoder dropLast = false") { - val transformed = stringIndexed() + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + val encoder = new OneHotEncoder() - .setInputCol("labelIndex") - .setOutputCol("labelVec") + .setInputCols(Array("input")) + .setOutputCols(Array("output")) assert(encoder.getDropLast === true) encoder.setDropLast(false) assert(encoder.getDropLast === false) - val expected = Seq( - (0, Vectors.sparse(3, Seq((0, 1.0)))), - (1, Vectors.sparse(3, Seq((2, 1.0)))), - (2, Vectors.sparse(3, Seq((1, 1.0)))), - (3, Vectors.sparse(3, Seq((0, 1.0)))), - (4, Vectors.sparse(3, Seq((0, 1.0)))), - (5, Vectors.sparse(3, Seq((1, 1.0))))).toDF("id", "expected") - - val withExpected = transformed.join(expected, "id") - testTransformer[(Int, String, Double, Vector)](withExpected, encoder, "labelVec", "expected") { + val model = encoder.fit(df) + testTransformer[(Double, Vector)](df, model, "output", "expected") { case Row(output: Vector, expected: Vector) => assert(output === expected) } } test("OneHotEncoder dropLast = true") { - val transformed = stringIndexed() + val data = Seq( + Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(2, Seq())), + Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(2, Seq()))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + val encoder = new OneHotEncoder() - .setInputCol("labelIndex") - .setOutputCol("labelVec") - val expected = Seq( - (0, Vectors.sparse(2, Seq((0, 1.0)))), - (1, Vectors.sparse(2, Seq())), - (2, Vectors.sparse(2, Seq((1, 1.0)))), - (3, Vectors.sparse(2, Seq((0, 1.0)))), - (4, Vectors.sparse(2, Seq((0, 1.0)))), - (5, Vectors.sparse(2, Seq((1, 1.0))))).toDF("id", "expected") - - val withExpected = transformed.join(expected, "id") - testTransformer[(Int, String, Double, Vector)](withExpected, encoder, "labelVec", "expected") { + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + + val model = encoder.fit(df) + testTransformer[(Double, Vector)](df, model, "output", "expected") { case Row(output: Vector, expected: Vector) => assert(output === expected) } @@ -94,52 +93,61 @@ class OneHotEncoderSuite val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") .select(col("size").as("size", attr.toMetadata())) val encoder = new OneHotEncoder() - .setInputCol("size") - .setOutputCol("encoded") - testTransformerByGlobalCheckFunc[(Double)](df, encoder, "encoded") { rows => - val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) - assert(group.size === 2) - assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("small").withIndex(0)) - assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("medium").withIndex(1)) + .setInputCols(Array("size")) + .setOutputCols(Array("encoded")) + val model = encoder.fit(df) + testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => + val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) + assert(group.size === 2) + assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("small").withIndex(0)) + assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("medium").withIndex(1)) } } - test("input column without ML attribute") { val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("index") val encoder = new OneHotEncoder() - .setInputCol("index") - .setOutputCol("encoded") - val rows = encoder.transform(df).select("encoded").collect() - val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) - assert(group.size === 2) - assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("0").withIndex(0)) - assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("1").withIndex(1)) + .setInputCols(Array("index")) + .setOutputCols(Array("encoded")) + val model = encoder.fit(df) + testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => + val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) + assert(group.size === 2) + assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("0").withIndex(0)) + assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("1").withIndex(1)) + } } test("read/write") { - val t = new OneHotEncoder() - .setInputCol("myInputCol") - .setOutputCol("myOutputCol") - .setDropLast(false) - testDefaultReadWrite(t) + val encoder = new OneHotEncoder() + .setInputCols(Array("index")) + .setOutputCols(Array("encoded")) + testDefaultReadWrite(encoder) + } + + test("OneHotEncoderModel read/write") { + val instance = new OneHotEncoderModel("myOneHotEncoderModel", Array(1, 2, 3)) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.categorySizes === instance.categorySizes) } test("OneHotEncoder with varying types") { - val df = stringIndexed() - val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") - val expected = Seq( - (0, Vectors.sparse(3, Seq((0, 1.0)))), - (1, Vectors.sparse(3, Seq((2, 1.0)))), - (2, Vectors.sparse(3, Seq((1, 1.0)))), - (3, Vectors.sparse(3, Seq((0, 1.0)))), - (4, Vectors.sparse(3, Seq((0, 1.0)))), - (5, Vectors.sparse(3, Seq((1, 1.0))))).toDF("id", "expected") + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) - val withExpected = df.join(expected, "id") + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) class NumericTypeWithEncoder[A](val numericType: NumericType) - (implicit val encoder: Encoder[(A, Vector)]) + (implicit val encoder: Encoder[(A, Vector)]) val types = Seq( new NumericTypeWithEncoder[Short](ShortType), @@ -151,17 +159,264 @@ class OneHotEncoderSuite new NumericTypeWithEncoder[Decimal](DecimalType(10, 0))(ExpressionEncoder())) for (t <- types) { - val dfWithTypes = withExpected.select(col("labelIndex") - .cast(t.numericType).as("labelIndex", attr.toMetadata()), col("expected")) - val encoder = new OneHotEncoder() - .setInputCol("labelIndex") - .setOutputCol("labelVec") + val dfWithTypes = df.select(col("input").cast(t.numericType), col("expected")) + val estimator = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) .setDropLast(false) - testTransformer(dfWithTypes, encoder, "labelVec", "expected") { + val model = estimator.fit(dfWithTypes) + testTransformer(dfWithTypes, model, "output", "expected") { case Row(output: Vector, expected: Vector) => assert(output === expected) }(t.encoder) } } + + test("OneHotEncoder: encoding multiple columns and dropLast = false") { + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), 3.0, Vectors.sparse(4, Seq((3, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 1.0, Vectors.sparse(4, Seq((1, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input1", DoubleType), + StructField("expected1", new VectorUDT), + StructField("input2", DoubleType), + StructField("expected2", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("output1", "output2")) + assert(encoder.getDropLast === true) + encoder.setDropLast(false) + assert(encoder.getDropLast === false) + + val model = encoder.fit(df) + testTransformer[(Double, Vector, Double, Vector)]( + df, + model, + "output1", + "output2", + "expected1", + "expected2") { + case Row(output1: Vector, output2: Vector, expected1: Vector, expected2: Vector) => + assert(output1 === expected1) + assert(output2 === expected2) + } + } + + test("OneHotEncoder: encoding multiple columns and dropLast = true") { + val data = Seq( + Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 2.0, Vectors.sparse(3, Seq((2, 1.0)))), + Row(1.0, Vectors.sparse(2, Seq((1, 1.0))), 3.0, Vectors.sparse(3, Seq())), + Row(2.0, Vectors.sparse(2, Seq()), 0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(2, Seq()), 2.0, Vectors.sparse(3, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input1", DoubleType), + StructField("expected1", new VectorUDT), + StructField("input2", DoubleType), + StructField("expected2", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("output1", "output2")) + + val model = encoder.fit(df) + testTransformer[(Double, Vector, Double, Vector)]( + df, + model, + "output1", + "output2", + "expected1", + "expected2") { + case Row(output1: Vector, output2: Vector, expected1: Vector, expected2: Vector) => + assert(output1 === expected1) + assert(output2 === expected2) + } + } + + test("Throw error on invalid values") { + val trainingData = Seq((0, 0), (1, 1), (2, 2)) + val trainingDF = trainingData.toDF("id", "a") + val testData = Seq((0, 0), (1, 2), (1, 3)) + val testDF = testData.toDF("id", "a") + + val encoder = new OneHotEncoder() + .setInputCols(Array("a")) + .setOutputCols(Array("encoded")) + + val model = encoder.fit(trainingDF) + testTransformerByInterceptingException[(Int, Int)]( + testDF, + model, + expectedMessagePart = "Unseen value: 3.0. To handle unseen values", + firstResultCol = "encoded") + + } + + test("Can't transform on negative input") { + val trainingDF = Seq((0, 0), (1, 1), (2, 2)).toDF("a", "b") + val testDF = Seq((0, 0), (-1, 2), (1, 3)).toDF("a", "b") + + val encoder = new OneHotEncoder() + .setInputCols(Array("a")) + .setOutputCols(Array("encoded")) + + val model = encoder.fit(trainingDF) + testTransformerByInterceptingException[(Int, Int)]( + testDF, + model, + expectedMessagePart = "Negative value: -1.0. Input can't be negative", + firstResultCol = "encoded") + } + + test("Keep on invalid values: dropLast = false") { + val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") + + val testData = Seq( + Row(0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(4, Seq((1, 1.0)))), + Row(3.0, Vectors.sparse(4, Seq((3, 1.0))))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val testDF = spark.createDataFrame(sc.parallelize(testData), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + .setHandleInvalid("keep") + .setDropLast(false) + + val model = encoder.fit(trainingDF) + testTransformer[(Double, Vector)](testDF, model, "output", "expected") { + case Row(output: Vector, expected: Vector) => + assert(output === expected) + } + } + + test("Keep on invalid values: dropLast = true") { + val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") + + val testData = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(3.0, Vectors.sparse(3, Seq()))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val testDF = spark.createDataFrame(sc.parallelize(testData), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + .setHandleInvalid("keep") + .setDropLast(true) + + val model = encoder.fit(trainingDF) + testTransformer[(Double, Vector)](testDF, model, "output", "expected") { + case Row(output: Vector, expected: Vector) => + assert(output === expected) + } + } + + test("OneHotEncoderModel changes dropLast") { + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), Vectors.sparse(2, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), Vectors.sparse(2, Seq())), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), Vectors.sparse(2, Seq()))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected1", new VectorUDT), + StructField("expected2", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + + val model = encoder.fit(df) + + model.setDropLast(false) + testTransformer[(Double, Vector, Vector)](df, model, "output", "expected1") { + case Row(output: Vector, expected1: Vector) => + assert(output === expected1) + } + + model.setDropLast(true) + testTransformer[(Double, Vector, Vector)](df, model, "output", "expected2") { + case Row(output: Vector, expected2: Vector) => + assert(output === expected2) + } + } + + test("OneHotEncoderModel changes handleInvalid") { + val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") + + val testData = Seq( + Row(0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(4, Seq((1, 1.0)))), + Row(3.0, Vectors.sparse(4, Seq((3, 1.0))))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val testDF = spark.createDataFrame(sc.parallelize(testData), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + + val model = encoder.fit(trainingDF) + model.setHandleInvalid("error") + + testTransformerByInterceptingException[(Double, Vector)]( + testDF, + model, + expectedMessagePart = "Unseen value: 3.0. To handle unseen values", + firstResultCol = "output") + + model.setHandleInvalid("keep") + testTransformerByGlobalCheckFunc[(Double, Vector)](testDF, model, "output") { _ => } + } + + test("Transforming on mismatched attributes") { + val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") + val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") + .select(col("size").as("size", attr.toMetadata())) + val encoder = new OneHotEncoder() + .setInputCols(Array("size")) + .setOutputCols(Array("encoded")) + val model = encoder.fit(df) + + val testAttr = NominalAttribute.defaultAttr.withValues("tiny", "small", "medium", "large") + val testDF = Seq(0.0, 1.0, 2.0, 3.0).map(Tuple1.apply).toDF("size") + .select(col("size").as("size", testAttr.toMetadata())) + testTransformerByInterceptingException[(Double)]( + testDF, + model, + expectedMessagePart = "OneHotEncoderModel expected 2 categorical values", + firstResultCol = "encoded") + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 3fabec0f6012..5e97d826370f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -228,6 +228,18 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createWriter"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter"), + // [SPARK-26133][ML] Remove deprecated OneHotEncoder and rename OneHotEncoderEstimator to OneHotEncoder + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.OneHotEncoderEstimator"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.OneHotEncoder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.transform"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.getInputCol"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.getOutputCol"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.inputCol"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.setInputCol"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.setOutputCol"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.outputCol"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.OneHotEncoderEstimator$"), + // [SPARK-26141] Enable custom metrics implementation in shuffle write // Following are Java private classes ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.sort.UnsafeShuffleWriter.this"), diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 3d2370024259..6cc80e181e5e 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -44,8 +44,7 @@ 'MinMaxScaler', 'MinMaxScalerModel', 'NGram', 'Normalizer', - 'OneHotEncoder', - 'OneHotEncoderEstimator', 'OneHotEncoderModel', + 'OneHotEncoder', 'OneHotEncoderModel', 'PCA', 'PCAModel', 'PolynomialExpansion', 'QuantileDiscretizer', @@ -1642,91 +1641,8 @@ def getP(self): @inherit_doc -class OneHotEncoder(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): - """ - A one-hot encoder that maps a column of category indices to a - column of binary vectors, with at most a single one-value per row - that indicates the input category index. - For example with 5 categories, an input value of 2.0 would map to - an output vector of `[0.0, 0.0, 1.0, 0.0]`. - The last category is not included by default (configurable via - :py:attr:`dropLast`) because it makes the vector entries sum up to - one, and hence linearly dependent. - So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. - - .. note:: This is different from scikit-learn's OneHotEncoder, - which keeps all categories. The output vectors are sparse. - - .. note:: Deprecated in 2.3.0. :py:class:`OneHotEncoderEstimator` will be renamed to - :py:class:`OneHotEncoder` and this :py:class:`OneHotEncoder` will be removed in 3.0.0. - - .. seealso:: - - :py:class:`StringIndexer` for converting categorical values into - category indices - - >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") - >>> model = stringIndexer.fit(stringIndDf) - >>> td = model.transform(stringIndDf) - >>> encoder = OneHotEncoder(inputCol="indexed", outputCol="features") - >>> encoder.transform(td).head().features - SparseVector(2, {0: 1.0}) - >>> encoder.setParams(outputCol="freqs").transform(td).head().freqs - SparseVector(2, {0: 1.0}) - >>> params = {encoder.dropLast: False, encoder.outputCol: "test"} - >>> encoder.transform(td, params).head().test - SparseVector(3, {0: 1.0}) - >>> onehotEncoderPath = temp_path + "/onehot-encoder" - >>> encoder.save(onehotEncoderPath) - >>> loadedEncoder = OneHotEncoder.load(onehotEncoderPath) - >>> loadedEncoder.getDropLast() == encoder.getDropLast() - True - - .. versionadded:: 1.4.0 - """ - - dropLast = Param(Params._dummy(), "dropLast", "whether to drop the last category", - typeConverter=TypeConverters.toBoolean) - - @keyword_only - def __init__(self, dropLast=True, inputCol=None, outputCol=None): - """ - __init__(self, dropLast=True, inputCol=None, outputCol=None) - """ - super(OneHotEncoder, self).__init__() - self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.OneHotEncoder", self.uid) - self._setDefault(dropLast=True) - kwargs = self._input_kwargs - self.setParams(**kwargs) - - @keyword_only - @since("1.4.0") - def setParams(self, dropLast=True, inputCol=None, outputCol=None): - """ - setParams(self, dropLast=True, inputCol=None, outputCol=None) - Sets params for this OneHotEncoder. - """ - kwargs = self._input_kwargs - return self._set(**kwargs) - - @since("1.4.0") - def setDropLast(self, value): - """ - Sets the value of :py:attr:`dropLast`. - """ - return self._set(dropLast=value) - - @since("1.4.0") - def getDropLast(self): - """ - Gets the value of dropLast or its default value. - """ - return self.getOrDefault(self.dropLast) - - -@inherit_doc -class OneHotEncoderEstimator(JavaEstimator, HasInputCols, HasOutputCols, HasHandleInvalid, - JavaMLReadable, JavaMLWritable): +class OneHotEncoder(JavaEstimator, HasInputCols, HasOutputCols, HasHandleInvalid, + JavaMLReadable, JavaMLWritable): """ A one-hot encoder that maps a column of category indices to a column of binary vectors, with at most a single one-value per row that indicates the input category index. @@ -1751,13 +1667,13 @@ class OneHotEncoderEstimator(JavaEstimator, HasInputCols, HasOutputCols, HasHand >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(0.0,), (1.0,), (2.0,)], ["input"]) - >>> ohe = OneHotEncoderEstimator(inputCols=["input"], outputCols=["output"]) + >>> ohe = OneHotEncoder(inputCols=["input"], outputCols=["output"]) >>> model = ohe.fit(df) >>> model.transform(df).head().output SparseVector(2, {0: 1.0}) >>> ohePath = temp_path + "/oheEstimator" >>> ohe.save(ohePath) - >>> loadedOHE = OneHotEncoderEstimator.load(ohePath) + >>> loadedOHE = OneHotEncoder.load(ohePath) >>> loadedOHE.getInputCols() == ohe.getInputCols() True >>> modelPath = temp_path + "/ohe-model" @@ -1784,9 +1700,9 @@ def __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropL """ __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True) """ - super(OneHotEncoderEstimator, self).__init__() + super(OneHotEncoder, self).__init__() self._java_obj = self._new_java_obj( - "org.apache.spark.ml.feature.OneHotEncoderEstimator", self.uid) + "org.apache.spark.ml.feature.OneHotEncoder", self.uid) self._setDefault(handleInvalid="error", dropLast=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1796,7 +1712,7 @@ def __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropL def setParams(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True): """ setParams(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True) - Sets params for this OneHotEncoderEstimator. + Sets params for this OneHotEncoder. """ kwargs = self._input_kwargs return self._set(**kwargs) @@ -1821,7 +1737,7 @@ def _create_model(self, java_model): class OneHotEncoderModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - Model fitted by :py:class:`OneHotEncoderEstimator`. + Model fitted by :py:class:`OneHotEncoder`. .. versionadded:: 2.3.0 """ From 7a83d71403edf7d24fa5efc0ef913f3ce76d88b8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 29 Nov 2018 22:15:12 +0800 Subject: [PATCH 0013/1072] [SPARK-26163][SQL] Parsing decimals from JSON using locale ## What changes were proposed in this pull request? In the PR, I propose using of the locale option to parse (and infer) decimals from JSON input. After the changes, `JacksonParser` converts input string to `BigDecimal` and to Spark's Decimal by using `java.text.DecimalFormat`. New behaviour can be switched off via SQL config `spark.sql.legacy.decimalParsing.enabled`. ## How was this patch tested? Added 2 tests to `JsonExpressionsSuite` for the `en-US`, `ko-KR`, `ru-RU`, `de-DE` locales: - Inferring decimal type using locale from JSON field values - Converting JSON field values to specified decimal type using the locales. Closes #23132 from MaxGekk/json-decimal-parsing-locale. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/ExprUtils.scala | 21 +++++ .../expressions/jsonExpressions.scala | 7 +- .../sql/catalyst/json/JacksonParser.scala | 6 ++ .../sql/catalyst/json/JsonInferSchema.scala | 89 +++++++++++-------- .../expressions/JsonExpressionsSuite.scala | 42 ++++++++- .../datasources/json/JsonDataSource.scala | 4 +- .../datasources/json/JsonSuite.scala | 15 ++-- 7 files changed, 132 insertions(+), 52 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala index 89e9071324ef..3f3d6b2b63a0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.catalyst.expressions +import java.text.{DecimalFormat, DecimalFormatSymbols, ParsePosition} +import java.util.Locale + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.util.ArrayBasedMapData import org.apache.spark.sql.types.{DataType, MapType, StringType, StructType} @@ -83,4 +86,22 @@ object ExprUtils { } } } + + def getDecimalParser(locale: Locale): String => java.math.BigDecimal = { + if (locale == Locale.US) { // Special handling the default locale for backward compatibility + (s: String) => new java.math.BigDecimal(s.replaceAll(",", "")) + } else { + val decimalFormat = new DecimalFormat("", new DecimalFormatSymbols(locale)) + decimalFormat.setParseBigDecimal(true) + (s: String) => { + val pos = new ParsePosition(0) + val result = decimalFormat.parse(s, pos).asInstanceOf[java.math.BigDecimal] + if (pos.getIndex() != s.length() || pos.getErrorIndex() != -1) { + throw new IllegalArgumentException("Cannot parse any decimal"); + } else { + result + } + } + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 47304d835fdf..e0cab537ce1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -23,12 +23,10 @@ import scala.util.parsing.combinator.RegexParsers import com.fasterxml.jackson.core._ -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.json._ -import org.apache.spark.sql.catalyst.json.JsonInferSchema.inferField import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -775,6 +773,9 @@ case class SchemaOfJson( factory } + @transient + private lazy val jsonInferSchema = new JsonInferSchema(jsonOptions) + @transient private lazy val json = child.eval().asInstanceOf[UTF8String] @@ -787,7 +788,7 @@ case class SchemaOfJson( override def eval(v: InternalRow): Any = { val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => parser.nextToken() - inferField(parser, jsonOptions) + jsonInferSchema.inferField(parser) } UTF8String.fromString(dt.catalogString) 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 92517aac053b..2357595906b1 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 @@ -29,6 +29,7 @@ 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._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -135,6 +136,8 @@ class JacksonParser( } } + private val decimalParser = ExprUtils.getDecimalParser(options.locale) + /** * Create a converter which converts the JSON documents held by the `JsonParser` * to a value according to a desired schema. @@ -261,6 +264,9 @@ class JacksonParser( (parser: JsonParser) => parseJsonToken[Decimal](parser, dataType) { case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) => Decimal(parser.getDecimalValue, dt.precision, dt.scale) + case VALUE_STRING if parser.getTextLength >= 1 => + val bigDecimal = decimalParser(parser.getText) + Decimal(bigDecimal, dt.precision, dt.scale) } case st: StructType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index 9999a005106f..263e05de3207 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -19,18 +19,23 @@ package org.apache.spark.sql.catalyst.json import java.util.Comparator +import scala.util.control.Exception.allCatch + import com.fasterxml.jackson.core._ import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion +import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.JacksonUtils.nextUntil import org.apache.spark.sql.catalyst.util.{DropMalformedMode, FailFastMode, ParseMode, PermissiveMode} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -private[sql] object JsonInferSchema { +private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { + + private val decimalParser = ExprUtils.getDecimalParser(options.locale) /** * Infer the type of a collection of json records in three stages: @@ -40,21 +45,20 @@ private[sql] object JsonInferSchema { */ def infer[T]( json: RDD[T], - configOptions: JSONOptions, createParser: (JsonFactory, T) => JsonParser): StructType = { - val parseMode = configOptions.parseMode - val columnNameOfCorruptRecord = configOptions.columnNameOfCorruptRecord + val parseMode = options.parseMode + val columnNameOfCorruptRecord = options.columnNameOfCorruptRecord // In each RDD partition, perform schema inference on each row and merge afterwards. - val typeMerger = compatibleRootType(columnNameOfCorruptRecord, parseMode) + val typeMerger = JsonInferSchema.compatibleRootType(columnNameOfCorruptRecord, parseMode) val mergedTypesFromPartitions = json.mapPartitions { iter => val factory = new JsonFactory() - configOptions.setJacksonOptions(factory) + options.setJacksonOptions(factory) iter.flatMap { row => try { Utils.tryWithResource(createParser(factory, row)) { parser => parser.nextToken() - Some(inferField(parser, configOptions)) + Some(inferField(parser)) } } catch { case e @ (_: RuntimeException | _: JsonProcessingException) => parseMode match { @@ -82,7 +86,7 @@ private[sql] object JsonInferSchema { } json.sparkContext.runJob(mergedTypesFromPartitions, foldPartition, mergeResult) - canonicalizeType(rootType, configOptions) match { + canonicalizeType(rootType, options) match { case Some(st: StructType) => st case _ => // canonicalizeType erases all empty structs, including the only one we want to keep @@ -90,34 +94,17 @@ private[sql] object JsonInferSchema { } } - private[this] val structFieldComparator = new Comparator[StructField] { - override def compare(o1: StructField, o2: StructField): Int = { - o1.name.compareTo(o2.name) - } - } - - private def isSorted(arr: Array[StructField]): Boolean = { - var i: Int = 0 - while (i < arr.length - 1) { - if (structFieldComparator.compare(arr(i), arr(i + 1)) > 0) { - return false - } - i += 1 - } - true - } - /** * Infer the type of a json document from the parser's token stream */ - def inferField(parser: JsonParser, configOptions: JSONOptions): DataType = { + def inferField(parser: JsonParser): DataType = { import com.fasterxml.jackson.core.JsonToken._ parser.getCurrentToken match { case null | VALUE_NULL => NullType case FIELD_NAME => parser.nextToken() - inferField(parser, configOptions) + inferField(parser) case VALUE_STRING if parser.getTextLength < 1 => // Zero length strings and nulls have special handling to deal @@ -128,18 +115,25 @@ private[sql] object JsonInferSchema { // record fields' types have been combined. NullType + case VALUE_STRING if options.prefersDecimal => + val decimalTry = allCatch opt { + val bigDecimal = decimalParser(parser.getText) + DecimalType(bigDecimal.precision, bigDecimal.scale) + } + decimalTry.getOrElse(StringType) case VALUE_STRING => StringType + case START_OBJECT => val builder = Array.newBuilder[StructField] while (nextUntil(parser, END_OBJECT)) { builder += StructField( parser.getCurrentName, - inferField(parser, configOptions), + inferField(parser), nullable = true) } val fields: Array[StructField] = builder.result() // Note: other code relies on this sorting for correctness, so don't remove it! - java.util.Arrays.sort(fields, structFieldComparator) + java.util.Arrays.sort(fields, JsonInferSchema.structFieldComparator) StructType(fields) case START_ARRAY => @@ -148,15 +142,15 @@ private[sql] object JsonInferSchema { // the type as we pass through all JSON objects. var elementType: DataType = NullType while (nextUntil(parser, END_ARRAY)) { - elementType = compatibleType( - elementType, inferField(parser, configOptions)) + elementType = JsonInferSchema.compatibleType( + elementType, inferField(parser)) } ArrayType(elementType) - case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) if configOptions.primitivesAsString => StringType + case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) if options.primitivesAsString => StringType - case (VALUE_TRUE | VALUE_FALSE) if configOptions.primitivesAsString => StringType + case (VALUE_TRUE | VALUE_FALSE) if options.primitivesAsString => StringType case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT => import JsonParser.NumberType._ @@ -172,7 +166,7 @@ private[sql] object JsonInferSchema { } else { DoubleType } - case FLOAT | DOUBLE if configOptions.prefersDecimal => + case FLOAT | DOUBLE if options.prefersDecimal => val v = parser.getDecimalValue if (Math.max(v.precision(), v.scale()) <= DecimalType.MAX_PRECISION) { DecimalType(Math.max(v.precision(), v.scale()), v.scale()) @@ -217,12 +211,31 @@ private[sql] object JsonInferSchema { case other => Some(other) } +} + +object JsonInferSchema { + val structFieldComparator = new Comparator[StructField] { + override def compare(o1: StructField, o2: StructField): Int = { + o1.name.compareTo(o2.name) + } + } + + def isSorted(arr: Array[StructField]): Boolean = { + var i: Int = 0 + while (i < arr.length - 1) { + if (structFieldComparator.compare(arr(i), arr(i + 1)) > 0) { + return false + } + i += 1 + } + true + } - private def withCorruptField( + def withCorruptField( struct: StructType, other: DataType, columnNameOfCorruptRecords: String, - parseMode: ParseMode) = parseMode match { + parseMode: ParseMode): StructType = parseMode match { case PermissiveMode => // If we see any other data type at the root level, we get records that cannot be // parsed. So, we use the struct as the data type and add the corrupt field to the schema. @@ -230,7 +243,7 @@ private[sql] object JsonInferSchema { // If this given struct does not have a column used for corrupt records, // add this field. val newFields: Array[StructField] = - StructField(columnNameOfCorruptRecords, StringType, nullable = true) +: struct.fields + StructField(columnNameOfCorruptRecords, StringType, nullable = true) +: struct.fields // Note: other code relies on this sorting for correctness, so don't remove it! java.util.Arrays.sort(newFields, structFieldComparator) StructType(newFields) @@ -253,7 +266,7 @@ private[sql] object JsonInferSchema { /** * Remove top-level ArrayType wrappers and merge the remaining schemas */ - private def compatibleRootType( + def compatibleRootType( columnNameOfCorruptRecords: String, parseMode: ParseMode): (DataType, DataType) => DataType = { // Since we support array of json objects at the top level, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 9b89a27c2377..5d60cefc1389 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import java.text.SimpleDateFormat +import java.text.{DecimalFormat, DecimalFormatSymbols, SimpleDateFormat} import java.util.{Calendar, Locale} import org.scalatest.exceptions.TestFailedException @@ -765,4 +765,44 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with timeZoneId = gmtId), expectedErrMsg = "The field for corrupt records must be string type and nullable") } + + def decimalInput(langTag: String): (Decimal, String) = { + val decimalVal = new java.math.BigDecimal("1000.001") + val decimalType = new DecimalType(10, 5) + val expected = Decimal(decimalVal, decimalType.precision, decimalType.scale) + val decimalFormat = new DecimalFormat("", + new DecimalFormatSymbols(Locale.forLanguageTag(langTag))) + val input = s"""{"d": "${decimalFormat.format(expected.toBigDecimal)}"}""" + + (expected, input) + } + + test("parse decimals using locale") { + def checkDecimalParsing(langTag: String): Unit = { + val schema = new StructType().add("d", DecimalType(10, 5)) + val options = Map("locale" -> langTag) + val (expected, input) = decimalInput(langTag) + + checkEvaluation( + JsonToStructs(schema, options, Literal.create(input), gmtId), + InternalRow(expected)) + } + + Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach(checkDecimalParsing) + } + + test("inferring the decimal type using locale") { + def checkDecimalInfer(langTag: String, expectedType: String): Unit = { + val options = Map("locale" -> langTag, "prefersDecimal" -> "true") + val (_, input) = decimalInput(langTag) + + checkEvaluation( + SchemaOfJson(Literal.create(input), options), + expectedType) + } + + Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach { + checkDecimalInfer(_, """struct""") + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index c7608e2e881f..456f08a2a2ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -107,7 +107,7 @@ object TextInputJsonDataSource extends JsonDataSource { }.getOrElse(CreateJacksonParser.internalRow(_: JsonFactory, _: InternalRow)) SQLExecution.withSQLConfPropagated(json.sparkSession) { - JsonInferSchema.infer(rdd, parsedOptions, rowParser) + new JsonInferSchema(parsedOptions).infer(rdd, rowParser) } } @@ -166,7 +166,7 @@ object MultiLineJsonDataSource extends JsonDataSource { .getOrElse(createParser(_: JsonFactory, _: PortableDataStream)) SQLExecution.withSQLConfPropagated(sparkSession) { - JsonInferSchema.infer[PortableDataStream](sampled, parsedOptions, parser) + new JsonInferSchema(parsedOptions).infer[PortableDataStream](sampled, parser) } } 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 9ea9189cdf7f..ee31077e12ef 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 @@ -31,8 +31,7 @@ import org.apache.hadoop.io.compress.GzipCodec import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{functions => F, _} -import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JsonInferSchema, JSONOptions} -import org.apache.spark.sql.catalyst.json.JsonInferSchema.compatibleType +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 @@ -118,10 +117,10 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("Get compatible type") { def checkDataType(t1: DataType, t2: DataType, expected: DataType) { - var actual = compatibleType(t1, t2) + var actual = JsonInferSchema.compatibleType(t1, t2) assert(actual == expected, s"Expected $expected as the most general data type for $t1 and $t2, found $actual") - actual = compatibleType(t2, t1) + actual = JsonInferSchema.compatibleType(t2, t1) assert(actual == expected, s"Expected $expected as the most general data type for $t1 and $t2, found $actual") } @@ -1373,9 +1372,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-6245 JsonInferSchema.infer on empty RDD") { // This is really a test that it doesn't throw an exception - val emptySchema = JsonInferSchema.infer( + val options = new JSONOptions(Map.empty[String, String], "GMT") + val emptySchema = new JsonInferSchema(options).infer( empty.rdd, - new JSONOptions(Map.empty[String, String], "GMT"), CreateJacksonParser.string) assert(StructType(Seq()) === emptySchema) } @@ -1400,9 +1399,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-8093 Erase empty structs") { - val emptySchema = JsonInferSchema.infer( + val options = new JSONOptions(Map.empty[String, String], "GMT") + val emptySchema = new JsonInferSchema(options).infer( emptyRecords.rdd, - new JSONOptions(Map.empty[String, String], "GMT"), CreateJacksonParser.string) assert(StructType(Seq()) === emptySchema) } From b9b68a6dc7d0f735163e980392ea957f2d589923 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 29 Nov 2018 22:37:02 +0800 Subject: [PATCH 0014/1072] [SPARK-26211][SQL] Fix InSet for binary, and struct and array with null. ## What changes were proposed in this pull request? Currently `InSet` doesn't work properly for binary type, or struct and array type with null value in the set. Because, as for binary type, the `HashSet` doesn't work properly for `Array[Byte]`, and as for struct and array type with null value in the set, the `ordering` will throw a `NPE`. ## How was this patch tested? Added a few tests. Closes #23176 from ueshin/issues/SPARK-26211/inset. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/predicates.scala | 33 ++++++------ .../catalyst/expressions/PredicateSuite.scala | 50 ++++++++++++++++++- 2 files changed, 63 insertions(+), 20 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 16e0bc3aaf35..01ecb99025ea 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 @@ -367,31 +367,26 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with } @transient lazy val set: Set[Any] = child.dataType match { - case _: AtomicType => hset + case t: AtomicType if !t.isInstanceOf[BinaryType] => hset case _: NullType => hset case _ => // for structs use interpreted ordering to be able to compare UnsafeRows with non-UnsafeRows - TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ hset + TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ (hset - null) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val setTerm = ctx.addReferenceObj("set", set) - val childGen = child.genCode(ctx) - val setIsNull = if (hasNull) { - s"${ev.isNull} = !${ev.value};" - } else { - "" - } - ev.copy(code = - code""" - |${childGen.code} - |${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = ${childGen.isNull}; - |${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = false; - |if (!${ev.isNull}) { - | ${ev.value} = $setTerm.contains(${childGen.value}); - | $setIsNull - |} - """.stripMargin) + nullSafeCodeGen(ctx, ev, c => { + val setTerm = ctx.addReferenceObj("set", set) + val setIsNull = if (hasNull) { + s"${ev.isNull} = !${ev.value};" + } else { + "" + } + s""" + |${ev.value} = $setTerm.contains($c); + |$setIsNull + """.stripMargin + }) } override def sql: String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index ac76b17ef476..3b60d1d88b3c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -268,7 +268,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(InSet(nl, nS), null) val primitiveTypes = Seq(IntegerType, FloatType, DoubleType, StringType, ByteType, ShortType, - LongType, BinaryType, BooleanType, DecimalType.USER_DEFAULT, TimestampType) + LongType, BooleanType, DecimalType.USER_DEFAULT, TimestampType) primitiveTypes.foreach { t => val dataGen = RandomDataGenerator.forType(t, nullable = true).get val inputData = Seq.fill(10) { @@ -293,6 +293,54 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { } } + test("INSET: binary") { + val hS = HashSet[Any]() + Array(1.toByte, 2.toByte) + Array(3.toByte) + val nS = HashSet[Any]() + Array(1.toByte, 2.toByte) + Array(3.toByte) + null + val onetwo = Literal(Array(1.toByte, 2.toByte)) + val three = Literal(Array(3.toByte)) + val threefour = Literal(Array(3.toByte, 4.toByte)) + val nl = Literal(null, onetwo.dataType) + checkEvaluation(InSet(onetwo, hS), true) + checkEvaluation(InSet(three, hS), true) + checkEvaluation(InSet(three, nS), true) + checkEvaluation(InSet(threefour, hS), false) + checkEvaluation(InSet(threefour, nS), null) + checkEvaluation(InSet(nl, hS), null) + checkEvaluation(InSet(nl, nS), null) + } + + test("INSET: struct") { + val hS = HashSet[Any]() + Literal.create((1, "a")).value + Literal.create((2, "b")).value + val nS = HashSet[Any]() + Literal.create((1, "a")).value + Literal.create((2, "b")).value + null + val oneA = Literal.create((1, "a")) + val twoB = Literal.create((2, "b")) + val twoC = Literal.create((2, "c")) + val nl = Literal(null, oneA.dataType) + checkEvaluation(InSet(oneA, hS), true) + checkEvaluation(InSet(twoB, hS), true) + checkEvaluation(InSet(twoB, nS), true) + checkEvaluation(InSet(twoC, hS), false) + checkEvaluation(InSet(twoC, nS), null) + checkEvaluation(InSet(nl, hS), null) + checkEvaluation(InSet(nl, nS), null) + } + + test("INSET: array") { + val hS = HashSet[Any]() + Literal.create(Seq(1, 2)).value + Literal.create(Seq(3)).value + val nS = HashSet[Any]() + Literal.create(Seq(1, 2)).value + Literal.create(Seq(3)).value + null + val onetwo = Literal.create(Seq(1, 2)) + val three = Literal.create(Seq(3)) + val threefour = Literal.create(Seq(3, 4)) + val nl = Literal(null, onetwo.dataType) + checkEvaluation(InSet(onetwo, hS), true) + checkEvaluation(InSet(three, hS), true) + checkEvaluation(InSet(three, nS), true) + checkEvaluation(InSet(threefour, hS), false) + checkEvaluation(InSet(threefour, nS), null) + checkEvaluation(InSet(nl, hS), null) + checkEvaluation(InSet(nl, nS), null) + } + private case class MyStruct(a: Long, b: String) private case class MyStruct2(a: MyStruct, b: Array[Int]) private val udt = new ExamplePointUDT From 06a87711b8a3a71c32897003cd9c6203e1c0c42e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 29 Nov 2018 08:48:12 -0600 Subject: [PATCH 0015/1072] [SPARK-26024][FOLLOWUP][MINOR] Follow-up to remove extra blank lines in R function descriptions ## What changes were proposed in this pull request? Follow-up to remove extra blank lines in R function descriptions ## How was this patch tested? N/A Closes #23167 from srowen/SPARK-26024.2. Authored-by: Sean Owen Signed-off-by: Sean Owen --- R/pkg/R/DataFrame.R | 2 -- 1 file changed, 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ad9cd845f696..745bb3e15932 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -766,7 +766,6 @@ setMethod("repartition", #' \item{2.} {Return a new SparkDataFrame range partitioned by the given column(s), #' using \code{spark.sql.shuffle.partitions} as number of partitions.} #'} -#' #' At least one partition-by expression must be specified. #' When no explicit sort order is specified, "ascending nulls first" is assumed. #' @@ -828,7 +827,6 @@ setMethod("repartitionByRange", #' toJSON #' #' Converts a SparkDataFrame into a SparkDataFrame of JSON string. -#' #' Each row is turned into a JSON document with columns as different fields. #' The returned SparkDataFrame has a single character column with the name \code{value} #' From e3ea93ab6c8e434faa360af831947e682206d50d Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 29 Nov 2018 08:53:12 -0600 Subject: [PATCH 0016/1072] [MINOR][ML] add missing params to Instr ## What changes were proposed in this pull request? add following param to instr: GBTC: validationTol GBTR: validationTol, validationIndicatorCol colnames in LiR, LinearSVC, etc ## How was this patch tested? existing tests Closes #23122 from zhengruifeng/instr_append_missing_params. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../spark/ml/classification/DecisionTreeClassifier.scala | 3 ++- .../org/apache/spark/ml/classification/GBTClassifier.scala | 2 +- .../scala/org/apache/spark/ml/classification/LinearSVC.scala | 4 ++-- .../apache/spark/ml/classification/LogisticRegression.scala | 5 +++-- .../ml/classification/MultilayerPerceptronClassifier.scala | 4 ++-- .../scala/org/apache/spark/ml/classification/OneVsRest.scala | 3 ++- .../scala/org/apache/spark/ml/regression/GBTRegressor.scala | 3 ++- 7 files changed, 14 insertions(+), 10 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 bcf89766b087..d9292a547676 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 @@ -115,7 +115,8 @@ class DecisionTreeClassifier @Since("1.4.0") ( val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, numClasses) val strategy = getOldStrategy(categoricalFeatures, numClasses) - instr.logParams(this, maxDepth, maxBins, minInstancesPerNode, minInfoGain, maxMemoryInMB, + instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, + probabilityCol, maxDepth, maxBins, minInstancesPerNode, minInfoGain, maxMemoryInMB, cacheNodeIds, checkpointInterval, impurity, seed) val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, featureSubsetStrategy = "all", 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 09a9df6d15ec..abe2d1febfdf 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 @@ -194,7 +194,7 @@ class GBTClassifier @Since("1.4.0") ( instr.logParams(this, labelCol, featuresCol, predictionCol, impurity, lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, featureSubsetStrategy, - validationIndicatorCol) + validationIndicatorCol, validationTol) instr.logNumClasses(numClasses) val (baseLearners, learnerWeights) = if (withValidation) { 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 1b5c02fc9a57..ff801abef9a9 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 @@ -173,8 +173,8 @@ class LinearSVC @Since("2.2.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, regParam, maxIter, fitIntercept, tol, standardization, threshold, - aggregationDepth) + instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, + regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth) val (summarizer, labelSummarizer) = { val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer), 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 6f0804f0c8e4..27a7db0b2f5d 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 @@ -503,8 +503,9 @@ class LogisticRegression @Since("1.2.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, regParam, elasticNetParam, standardization, threshold, - maxIter, tol, fitIntercept) + instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, + probabilityCol, regParam, elasticNetParam, standardization, threshold, maxIter, tol, + fitIntercept) val (summarizer, labelSummarizer) = { val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer), 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 4feddce1d9f2..47b8a8df637b 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 @@ -205,8 +205,8 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( dataset: Dataset[_]): MultilayerPerceptronClassificationModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, predictionCol, layers, maxIter, tol, - blockSize, solver, stepSize, seed) + instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, layers, maxIter, + tol, blockSize, solver, stepSize, seed) val myLayers = $(layers) val labels = myLayers.last diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 2f42a5922054..e1fceb1fc96a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -377,7 +377,8 @@ final class OneVsRest @Since("1.4.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, predictionCol, parallelism, rawPredictionCol) + instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, + rawPredictionCol, parallelism) instr.logNamedValue("classifier", $(classifier).getClass.getCanonicalName) // determine number of classes either from metadata if provided, or via computation. diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 9b386ef5eed8..9a5b7d59e9ae 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -171,7 +171,8 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) instr.logDataset(dataset) instr.logParams(this, labelCol, featuresCol, predictionCol, impurity, lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, - seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, featureSubsetStrategy) + seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, featureSubsetStrategy, + validationIndicatorCol, validationTol) val (baseLearners, learnerWeights) = if (withValidation) { GradientBoostedTrees.runWithValidation(trainDataset, validationDataset, boostingStrategy, From 9a09e91a3e880b7a07b11a957fb6766578f5a1af Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Thu, 29 Nov 2018 08:54:31 -0600 Subject: [PATCH 0017/1072] [SPARK-26177] Automated formatting for Scala code ## What changes were proposed in this pull request? Add a maven plugin and wrapper script to use scalafmt to format files that differ from git master. Intention is for contributors to be able to use this to automate fixing code style, not to include it in build pipeline yet. If this PR is accepted, I'd make a different PR to update the code style section of https://spark.apache.org/contributing.html to mention the script ## How was this patch tested? Manually tested by modifying a few files and running ./dev/scalafmt then checking that ./dev/scalastyle still passed. Closes #23148 from koeninger/scalafmt. Authored-by: cody koeninger Signed-off-by: Sean Owen --- dev/.scalafmt.conf | 24 ++++++++++++++++++++++++ dev/scalafmt | 23 +++++++++++++++++++++++ pom.xml | 21 +++++++++++++++++++++ 3 files changed, 68 insertions(+) create mode 100644 dev/.scalafmt.conf create mode 100755 dev/scalafmt diff --git a/dev/.scalafmt.conf b/dev/.scalafmt.conf new file mode 100644 index 000000000000..def67e026982 --- /dev/null +++ b/dev/.scalafmt.conf @@ -0,0 +1,24 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +align = none +align.openParenDefnSite = false +align.openParenCallSite = false +align.tokens = [] +docstrings = JavaDoc +maxColumn = 98 + diff --git a/dev/scalafmt b/dev/scalafmt new file mode 100755 index 000000000000..76f688a2f5b8 --- /dev/null +++ b/dev/scalafmt @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# by default, format only files that differ from git master +params="${@:---diff}" + +./build/mvn mvn-scalafmt_2.12:format -Dscalafmt.skip=false -Dscalafmt.parameters="$params" diff --git a/pom.xml b/pom.xml index 93075e9b06a6..3ca2f739ce0e 100644 --- a/pom.xml +++ b/pom.xml @@ -156,6 +156,9 @@ 3.2.2 2.12.7 2.12 + --diff --test + + true 1.9.13 2.9.6 1.1.7.1 @@ -2600,6 +2603,24 @@ + + org.antipathy + mvn-scalafmt_2.12 + 0.9_1.5.1 + + ${scalafmt.parameters} + ${scalafmt.skip} + dev/.scalafmt.conf + + + + validate + + format + + + + From 31c4fab3fb0343edf971de9070a319c6b3094647 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 29 Nov 2018 10:31:31 -0600 Subject: [PATCH 0018/1072] [SPARK-26081][SQL] Prevent empty files for empty partitions in Text datasources ## What changes were proposed in this pull request? In the PR, I propose to postpone creation of `OutputStream`/`Univocity`/`JacksonGenerator` till the first row should be written. This prevents creation of empty files for empty partitions. So, no need to open and to read such files back while loading data from the location. ## How was this patch tested? Added tests for Text, JSON and CSV datasource where empty dataset is written but should not produce any files. Closes #23052 from MaxGekk/text-empty-files. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Sean Owen --- .../datasources/csv/CSVFileFormat.scala | 20 ++++++++++++------- .../datasources/json/JsonFileFormat.scala | 19 +++++++++--------- .../datasources/text/TextFileFormat.scala | 16 +++++++++++---- .../execution/datasources/csv/CSVSuite.scala | 9 +++++++++ .../datasources/json/JsonSuite.scala | 13 ++++++++++-- .../datasources/text/TextSuite.scala | 9 +++++++++ 6 files changed, 64 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index ff1911d69a6b..4c5a1d327023 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -169,13 +169,19 @@ private[csv] class CsvOutputWriter( context: TaskAttemptContext, params: CSVOptions) extends OutputWriter with Logging { - private val charset = Charset.forName(params.charset) - - private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path), charset) - - private val gen = new UnivocityGenerator(dataSchema, writer, params) + private var univocityGenerator: Option[UnivocityGenerator] = None + + override def write(row: InternalRow): Unit = { + val gen = univocityGenerator.getOrElse { + val charset = Charset.forName(params.charset) + val os = CodecStreams.createOutputStreamWriter(context, new Path(path), charset) + val newGen = new UnivocityGenerator(dataSchema, os, params) + univocityGenerator = Some(newGen) + newGen + } - override def write(row: InternalRow): Unit = gen.write(row) + gen.write(row) + } - override def close(): Unit = gen.close() + override def close(): Unit = univocityGenerator.map(_.close()) } 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 610f0d1619fc..3042133ee43a 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 @@ -175,19 +175,20 @@ private[json] class JsonOutputWriter( " which can be read back by Spark only if multiLine is enabled.") } - private val writer = CodecStreams.createOutputStreamWriter( - context, new Path(path), encoding) - - // create the Generator without separator inserted between 2 records - private[this] val gen = new JacksonGenerator(dataSchema, writer, options) + private var jacksonGenerator: Option[JacksonGenerator] = None override def write(row: InternalRow): Unit = { + val gen = jacksonGenerator.getOrElse { + val os = CodecStreams.createOutputStreamWriter(context, new Path(path), encoding) + // create the Generator without separator inserted between 2 records + val newGen = new JacksonGenerator(dataSchema, os, options) + jacksonGenerator = Some(newGen) + newGen + } + gen.write(row) gen.writeLineEnding() } - override def close(): Unit = { - gen.close() - writer.close() - } + override def close(): Unit = jacksonGenerator.map(_.close()) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index 268297148b52..01948ab25d63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.text +import java.io.OutputStream + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} @@ -148,17 +150,23 @@ class TextOutputWriter( context: TaskAttemptContext) extends OutputWriter { - private val writer = CodecStreams.createOutputStream(context, new Path(path)) + private var outputStream: Option[OutputStream] = None override def write(row: InternalRow): Unit = { + val os = outputStream.getOrElse{ + val newStream = CodecStreams.createOutputStream(context, new Path(path)) + outputStream = Some(newStream) + newStream + } + if (!row.isNullAt(0)) { val utf8string = row.getUTF8String(0) - utf8string.writeTo(writer) + utf8string.writeTo(os) } - writer.write(lineSeparator) + os.write(lineSeparator) } override def close(): Unit = { - writer.close() + outputStream.map(_.close()) } } 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 c275d63d32cc..e14e8d49db5c 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 @@ -1986,4 +1986,13 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te }.getMessage assert(errMsg2.contains("'lineSep' can contain only 1 character")) } + + test("do not produce empty files for empty partitions") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.emptyDataset[String].write.csv(path) + val files = new File(path).listFiles() + assert(!files.exists(_.getName.endsWith("csv"))) + } + } } 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 ee31077e12ef..ee5176e23e34 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 @@ -1897,7 +1897,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .text(path) val jsonDF = spark.read.option("multiLine", true).option("mode", "PERMISSIVE").json(path) - assert(jsonDF.count() === corruptRecordCount + 1) // null row for empty file + assert(jsonDF.count() === corruptRecordCount) assert(jsonDF.schema === new StructType() .add("_corrupt_record", StringType) .add("dummy", StringType)) @@ -1910,7 +1910,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { F.count($"dummy").as("valid"), F.count($"_corrupt_record").as("corrupt"), F.count("*").as("count")) - checkAnswer(counts, Row(1, 5, 7)) // null row for empty file + checkAnswer(counts, Row(1, 4, 6)) // null row for empty file } } @@ -2555,4 +2555,13 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { emptyString(StringType, "") emptyString(BinaryType, "".getBytes(StandardCharsets.UTF_8)) } + + test("do not produce empty files for empty partitions") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.emptyDataset[String].write.json(path) + val files = new File(path).listFiles() + assert(!files.exists(_.getName.endsWith("json"))) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 0e7f3afa9c3a..a86d5ee37f3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -233,4 +233,13 @@ class TextSuite extends QueryTest with SharedSQLContext { assert(data(3) == Row("\"doh\"")) assert(data.length == 4) } + + test("do not produce empty files for empty partitions") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.emptyDataset[String].write.text(path) + val files = new File(path).listFiles() + assert(!files.exists(_.getName.endsWith("txt"))) + } + } } From de4228152771390b0c5ba15254e9c5b832095366 Mon Sep 17 00:00:00 2001 From: Keiji Yoshida Date: Thu, 29 Nov 2018 10:39:00 -0600 Subject: [PATCH 0019/1072] [MINOR][DOCS][WIP] Fix Typos ## What changes were proposed in this pull request? Fix Typos. ## How was this patch tested? NA Closes #23145 from kjmrknsn/docUpdate. Authored-by: Keiji Yoshida Signed-off-by: Sean Owen --- docs/index.md | 4 +-- docs/rdd-programming-guide.md | 8 +++--- docs/running-on-mesos.md | 2 +- docs/sql-data-sources-avro.md | 6 ++-- docs/sql-data-sources-hive-tables.md | 2 +- docs/sql-data-sources-jdbc.md | 2 +- docs/sql-data-sources-load-save-functions.md | 2 +- docs/sql-getting-started.md | 2 +- docs/sql-programming-guide.md | 2 +- docs/sql-pyspark-pandas-with-arrow.md | 2 +- docs/sql-reference.md | 6 ++-- docs/streaming-programming-guide.md | 2 +- .../structured-streaming-programming-guide.md | 28 +++++++++---------- 13 files changed, 34 insertions(+), 34 deletions(-) diff --git a/docs/index.md b/docs/index.md index bd287e3f8d83..8864239eb164 100644 --- a/docs/index.md +++ b/docs/index.md @@ -66,8 +66,8 @@ Example applications are also provided in Python. For example, ./bin/spark-submit examples/src/main/python/pi.py 10 -Spark also provides an experimental [R API](sparkr.html) since 1.4 (only DataFrames APIs included). -To run Spark interactively in a R interpreter, use `bin/sparkR`: +Spark also provides an [R API](sparkr.html) since 1.4 (only DataFrames APIs included). +To run Spark interactively in an R interpreter, use `bin/sparkR`: ./bin/sparkR --master local[2] diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index 9a07d6ca24b6..2d1ddae5780d 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -332,7 +332,7 @@ One important parameter for parallel collections is the number of *partitions* t Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes a URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight scala %} scala> val distFile = sc.textFile("data.txt") @@ -365,7 +365,7 @@ Apart from text files, Spark's Scala API also supports several other data format Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes a URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight java %} JavaRDD distFile = sc.textFile("data.txt"); @@ -397,7 +397,7 @@ Apart from text files, Spark's Java API also supports several other data formats PySpark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes a URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight python %} >>> distFile = sc.textFile("data.txt") @@ -1122,7 +1122,7 @@ costly operation. #### Background -To understand what happens during the shuffle we can consider the example of the +To understand what happens during the shuffle, we can consider the example of the [`reduceByKey`](#ReduceByLink) operation. The `reduceByKey` operation generates a new RDD where all values for a single key are combined into a tuple - the key and the result of executing a reduce function against all values associated with that key. The challenge is that not all values for a diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 2502cd4ca86f..b3ba4b255b71 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -687,7 +687,7 @@ See the [configuration page](configuration.html) for information on Spark config 0 Set the maximum number GPU resources to acquire for this job. Note that executors will still launch when no GPU resources are found - since this configuration is just a upper limit and not a guaranteed amount. + since this configuration is just an upper limit and not a guaranteed amount. diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md index bfe641d1c6d1..b403a66fad79 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/sql-data-sources-avro.md @@ -66,9 +66,9 @@ write.df(select(df, "name", "favorite_color"), "namesAndFavColors.avro", "avro") ## to_avro() and from_avro() The Avro package provides function `to_avro` to encode a column as binary in Avro format, and `from_avro()` to decode Avro binary data into a column. Both functions transform one column to -another column, and the input/output SQL data type can be complex type or primitive type. +another column, and the input/output SQL data type can be a complex type or a primitive type. -Using Avro record as columns are useful when reading from or writing to a streaming source like Kafka. Each +Using Avro record as columns is useful when reading from or writing to a streaming source like Kafka. Each Kafka key-value record will be augmented with some metadata, such as the ingestion timestamp into Kafka, the offset in Kafka, etc. * If the "value" field that contains your data is in Avro, you could use `from_avro()` to extract your data, enrich it, clean it, and then push it downstream to Kafka again or write it out to a file. * `to_avro()` can be used to turn structs into Avro records. This method is particularly useful when you would like to re-encode multiple columns into a single one when writing data out to Kafka. @@ -151,7 +151,7 @@ Data source options of Avro can be set via: avroSchema None - Optional Avro schema provided by an user in JSON format. The date type and naming of record fields + Optional Avro schema provided by a user in JSON format. The date type and naming of record fields should match the input Avro data or Catalyst data, otherwise the read/write action will fail. read and write diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md index 28e1a3962666..3b39a32d4324 100644 --- a/docs/sql-data-sources-hive-tables.md +++ b/docs/sql-data-sources-hive-tables.md @@ -74,7 +74,7 @@ creating table, you can create a table using storage handler at Hive side, and u inputFormat, outputFormat These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, - e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in pair, and you can not + e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in a pair, and you can not specify them if you already specified the `fileFormat` option. diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index 057e8217241a..9a5d0fc7d424 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -55,7 +55,7 @@ the following case-insensitive options: as a subquery in the FROM clause. Spark will also assign an alias to the subquery clause. As an example, spark will issue a query of the following form to the JDBC Source.

SELECT <columns> FROM (<user_specified_query>) spark_gen_alias

- Below are couple of restrictions while using this option.
+ Below are a couple of restrictions while using this option.
  1. It is not allowed to specify `dbtable` and `query` options at the same time.
  2. It is not allowed to specify `query` and `partitionColumn` options at the same time. When specifying diff --git a/docs/sql-data-sources-load-save-functions.md b/docs/sql-data-sources-load-save-functions.md index e4c7b1766f91..4386caedb38b 100644 --- a/docs/sql-data-sources-load-save-functions.md +++ b/docs/sql-data-sources-load-save-functions.md @@ -324,4 +324,4 @@ CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS; `partitionBy` creates a directory structure as described in the [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) section. Thus, it has limited applicability to columns with high cardinality. In contrast `bucketBy` distributes -data across a fixed number of buckets and can be used when a number of unique values is unbounded. +data across a fixed number of buckets and can be used when the number of unique values is unbounded. diff --git a/docs/sql-getting-started.md b/docs/sql-getting-started.md index 88512205894a..0c3f0fb20610 100644 --- a/docs/sql-getting-started.md +++ b/docs/sql-getting-started.md @@ -99,7 +99,7 @@ Here we include some basic examples of structured data processing using Datasets
    {% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). +For a complete list of the types of operations that can be performed on a Dataset, refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$).
    diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index eca8915dfa97..9c85a15827bb 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -7,7 +7,7 @@ title: Spark SQL and DataFrames Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, Spark SQL uses this extra information to perform extra optimizations. There are several ways to -interact with Spark SQL including SQL and the Dataset API. When computing a result +interact with Spark SQL including SQL and the Dataset API. When computing a result, the same execution engine is used, independent of which API/language you are using to express the computation. This unification means that developers can easily switch back and forth between different APIs based on which provides the most natural way to express a given transformation. diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md index d04b955f9bf8..d18ca0beb0fc 100644 --- a/docs/sql-pyspark-pandas-with-arrow.md +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -129,7 +129,7 @@ For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/p Currently, all Spark SQL data types are supported by Arrow-based conversion except `MapType`, `ArrayType` of `TimestampType`, and nested `StructType`. `BinaryType` is supported only when -installed PyArrow is equal to or higher then 0.10.0. +installed PyArrow is equal to or higher than 0.10.0. ### Setting Arrow Batch Size diff --git a/docs/sql-reference.md b/docs/sql-reference.md index 9e4239b6bad2..88d0596f3876 100644 --- a/docs/sql-reference.md +++ b/docs/sql-reference.md @@ -38,15 +38,15 @@ Spark SQL and DataFrames support the following data types: elements with the type of `elementType`. `containsNull` is used to indicate if elements in a `ArrayType` value can have `null` values. - `MapType(keyType, valueType, valueContainsNull)`: - Represents values comprising a set of key-value pairs. The data type of keys are - described by `keyType` and the data type of values are described by `valueType`. + Represents values comprising a set of key-value pairs. The data type of keys is + described by `keyType` and the data type of values is described by `valueType`. For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` is used to indicate if values of a `MapType` value can have `null` values. - `StructType(fields)`: Represents values with the structure described by a sequence of `StructField`s (`fields`). * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. The name of a field is indicated by `name`. The data type of a field is indicated - by `dataType`. `nullable` is used to indicate if values of this fields can have + by `dataType`. `nullable` is used to indicate if values of these fields can have `null` values.
    diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 70bee5032a24..94c61205bd53 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -733,7 +733,7 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea Python API As of Spark {{site.SPARK_VERSION_SHORT}}, out of these sources, Kafka and Kinesis are available in the Python API. -This category of sources require interfacing with external non-Spark libraries, some of them with +This category of sources requires interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka). Hence, to minimize issues related to version conflicts of dependencies, the functionality to create DStreams from these sources has been moved to separate libraries that can be [linked](#linking) to explicitly when necessary. diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 8cea98c2cc52..32d61dcdb459 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1493,7 +1493,7 @@ Additional details on supported joins: ### Streaming Deduplication You can deduplicate records in data streams using a unique identifier in the events. This is exactly same as deduplication on static using a unique identifier column. The query will store the necessary amount of data from previous records such that it can filter duplicate records. Similar to aggregations, you can use deduplication with or without watermarking. -- *With watermark* - If there is a upper bound on how late a duplicate record may arrive, then you can define a watermark on a event time column and deduplicate using both the guid and the event time columns. The query will use the watermark to remove old state data from past records that are not expected to get any duplicates any more. This bounds the amount of the state the query has to maintain. +- *With watermark* - If there is an upper bound on how late a duplicate record may arrive, then you can define a watermark on an event time column and deduplicate using both the guid and the event time columns. The query will use the watermark to remove old state data from past records that are not expected to get any duplicates any more. This bounds the amount of the state the query has to maintain. - *Without watermark* - Since there are no bounds on when a duplicate record may arrive, the query stores the data from all the past records as state. @@ -1577,7 +1577,7 @@ event time seen in each input stream, calculates watermarks based on the corresp and chooses a single global watermark with them to be used for stateful operations. By default, the minimum is chosen as the global watermark because it ensures that no data is accidentally dropped as too late if one of the streams falls behind the others -(for example, one of the streams stop receiving data due to upstream failures). In other words, +(for example, one of the streams stops receiving data due to upstream failures). In other words, the global watermark will safely move at the pace of the slowest stream and the query output will be delayed accordingly. @@ -1598,7 +1598,7 @@ Some of them are as follows. - Multiple streaming aggregations (i.e. a chain of aggregations on a streaming DF) are not yet supported on streaming Datasets. -- Limit and take first N rows are not supported on streaming Datasets. +- Limit and take the first N rows are not supported on streaming Datasets. - Distinct operations on streaming Datasets are not supported. @@ -1634,7 +1634,7 @@ returned through `Dataset.writeStream()`. You will have to specify one or more o - *Query name:* Optionally, specify a unique name of the query for identification. -- *Trigger interval:* Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the previous processing has completed. If a trigger time is missed because the previous processing has not completed, then the system will trigger processing immediately. +- *Trigger interval:* Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the previous processing has been completed. If a trigger time is missed because the previous processing has not been completed, then the system will trigger processing immediately. - *Checkpoint location:* For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system will write all the checkpoint information. This should be a directory in an HDFS-compatible fault-tolerant file system. The semantics of checkpointing is discussed in more detail in the next section. @@ -2106,7 +2106,7 @@ With `foreachBatch`, you can do the following. ###### Foreach If `foreachBatch` is not an option (for example, corresponding batch data writer does not exist, or -continuous processing mode), then you can express you custom writer logic using `foreach`. +continuous processing mode), then you can express your custom writer logic using `foreach`. Specifically, you can express the data writing logic by dividing it into three methods: `open`, `process`, and `close`. Since Spark 2.4, `foreach` is available in Scala, Java and Python. @@ -2236,8 +2236,8 @@ When the streaming query is started, Spark calls the function or the object’s in the continuous mode, then this guarantee does not hold and therefore should not be used for deduplication. #### Triggers -The trigger settings of a streaming query defines the timing of streaming data processing, whether -the query is going to executed as micro-batch query with a fixed batch interval or as a continuous processing query. +The trigger settings of a streaming query define the timing of streaming data processing, whether +the query is going to be executed as micro-batch query with a fixed batch interval or as a continuous processing query. Here are the different kinds of triggers that are supported. @@ -2960,7 +2960,7 @@ the effect of the change is not well-defined. For all of them: - Addition/deletion/modification of rate limits is allowed: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "topic").option("maxOffsetsPerTrigger", ...)` - - Changes to subscribed topics/files is generally not allowed as the results are unpredictable: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "newTopic")` + - Changes to subscribed topics/files are generally not allowed as the results are unpredictable: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "newTopic")` - *Changes in the type of output sink*: Changes between a few specific combinations of sinks are allowed. This needs to be verified on a case-by-case basis. Here are a few examples. @@ -2974,17 +2974,17 @@ the effect of the change is not well-defined. For all of them: - *Changes in the parameters of output sink*: Whether this is allowed and whether the semantics of the change are well-defined depends on the sink and the query. Here are a few examples. - - Changes to output directory of a file sink is not allowed: `sdf.writeStream.format("parquet").option("path", "/somePath")` to `sdf.writeStream.format("parquet").option("path", "/anotherPath")` + - Changes to output directory of a file sink are not allowed: `sdf.writeStream.format("parquet").option("path", "/somePath")` to `sdf.writeStream.format("parquet").option("path", "/anotherPath")` - - Changes to output topic is allowed: `sdf.writeStream.format("kafka").option("topic", "someTopic")` to `sdf.writeStream.format("kafka").option("topic", "anotherTopic")` + - Changes to output topic are allowed: `sdf.writeStream.format("kafka").option("topic", "someTopic")` to `sdf.writeStream.format("kafka").option("topic", "anotherTopic")` - - Changes to the user-defined foreach sink (that is, the `ForeachWriter` code) is allowed, but the semantics of the change depends on the code. + - Changes to the user-defined foreach sink (that is, the `ForeachWriter` code) are allowed, but the semantics of the change depends on the code. - *Changes in projection / filter / map-like operations**: Some cases are allowed. For example: - Addition / deletion of filters is allowed: `sdf.selectExpr("a")` to `sdf.where(...).selectExpr("a").filter(...)`. - - Changes in projections with same output schema is allowed: `sdf.selectExpr("stringColumn AS json").writeStream` to `sdf.selectExpr("anotherStringColumn AS json").writeStream` + - Changes in projections with same output schema are allowed: `sdf.selectExpr("stringColumn AS json").writeStream` to `sdf.selectExpr("anotherStringColumn AS json").writeStream` - Changes in projections with different output schema are conditionally allowed: `sdf.selectExpr("a").writeStream` to `sdf.selectExpr("b").writeStream` is allowed only if the output sink allows the schema change from `"a"` to `"b"`. @@ -3000,7 +3000,7 @@ the effect of the change is not well-defined. For all of them: - *Streaming deduplication*: For example, `sdf.dropDuplicates("a")`. Any change in number or type of grouping keys or aggregates is not allowed. - *Stream-stream join*: For example, `sdf1.join(sdf2, ...)` (i.e. both inputs are generated with `sparkSession.readStream`). Changes - in the schema or equi-joining columns are not allowed. Changes in join type (outer or inner) not allowed. Other changes in the join condition are ill-defined. + in the schema or equi-joining columns are not allowed. Changes in join type (outer or inner) are not allowed. Other changes in the join condition are ill-defined. - *Arbitrary stateful operation*: For example, `sdf.groupByKey(...).mapGroupsWithState(...)` or `sdf.groupByKey(...).flatMapGroupsWithState(...)`. Any change to the schema of the user-defined state and the type of timeout is not allowed. @@ -3083,7 +3083,7 @@ spark \ -A checkpoint interval of 1 second means that the continuous processing engine will records the progress of the query every second. The resulting checkpoints are in a format compatible with the micro-batch engine, hence any query can be restarted with any trigger. For example, a supported query started with the micro-batch mode can be restarted in continuous mode, and vice versa. Note that any time you switch to continuous mode, you will get at-least-once fault-tolerance guarantees. +A checkpoint interval of 1 second means that the continuous processing engine will record the progress of the query every second. The resulting checkpoints are in a format compatible with the micro-batch engine, hence any query can be restarted with any trigger. For example, a supported query started with the micro-batch mode can be restarted in continuous mode, and vice versa. Note that any time you switch to continuous mode, you will get at-least-once fault-tolerance guarantees. ## Supported Queries {:.no_toc} From 24e78b7f163acf6129d934633ae6d3e6d568656a Mon Sep 17 00:00:00 2001 From: Shahid Date: Thu, 29 Nov 2018 09:48:18 -0800 Subject: [PATCH 0020/1072] [SPARK-26186][SPARK-26184][CORE] Last updated time is not getting updated for the Inprogress application ## What changes were proposed in this pull request? When the 'spark.history.fs.inProgressOptimization.enabled' is true, inProgress application's last updated time is not getting updated in the History UI. Also, during the cleaning time, InProgress application is getting removed from the listing, even if the last updated time is within the cleaning threshold time. In this PR, if the fastInprogressOptimization enabled, we update the `lastUpdateTime` of the application as last scan time. This will update the `lastUpdateTime` in the historyUI and also while cleaning, it won't remove if the updateTime is within the cleaning interval ## How was this patch tested? Added UT, attached screen shot. Before patch: ![screenshot from 2018-11-27 23-22-38](https://user-images.githubusercontent.com/23054875/49101600-9b5a3380-f29c-11e8-8efc-3fb594e4279a.png) ![screenshot from 2018-11-27 23-20-11](https://user-images.githubusercontent.com/23054875/49101601-9c8b6080-f29c-11e8-928e-643a8c8f4477.png) After Patch: ![screenshot from 2018-11-27 23-37-10](https://user-images.githubusercontent.com/23054875/49101911-669aac00-f29d-11e8-8181-663e4a08ab0e.png) ![screenshot from 2018-11-27 23-39-04](https://user-images.githubusercontent.com/23054875/49102010-a5306680-f29d-11e8-947a-e8a2a09a785a.png) Closes #23158 from shahidki31/HistoryLastUpdateTime. Authored-by: Shahid Signed-off-by: Marcelo Vanzin --- .../deploy/history/FsHistoryProvider.scala | 22 +++++++++++ .../history/FsHistoryProviderSuite.scala | 39 +++++++++++++++++++ 2 files changed, 61 insertions(+) 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 4f4a00b7d831..da6e5f03aabb 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 @@ -461,6 +461,28 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) if (info.appId.isDefined && fastInProgressParsing) { // When fast in-progress parsing is on, we don't need to re-parse when the // size changes, but we do need to invalidate any existing UIs. + // Also, we need to update the `lastUpdated time` to display the updated time in + // the HistoryUI and to avoid cleaning the inprogress app while running. + val appInfo = listing.read(classOf[ApplicationInfoWrapper], info.appId.get) + + val attemptList = appInfo.attempts.map { attempt => + if (attempt.info.attemptId == info.attemptId) { + new AttemptInfoWrapper( + attempt.info.copy(lastUpdated = new Date(newLastScanTime)), + attempt.logPath, + attempt.fileSize, + attempt.adminAcls, + attempt.viewAcls, + attempt.adminAclsGroups, + attempt.viewAclsGroups) + } else { + attempt + } + } + + val updatedAppInfo = new ApplicationInfoWrapper(appInfo.info, attemptList) + listing.write(updatedAppInfo) + invalidateUI(info.appId.get, info.attemptId) false } else { 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 b0ced46c9c7b..527c654a7cd6 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 @@ -334,6 +334,45 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc assert(!log2.exists()) } + test("should not clean inprogress application with lastUpdated time less than maxTime") { + val firstFileModifiedTime = TimeUnit.DAYS.toMillis(1) + val secondFileModifiedTime = TimeUnit.DAYS.toMillis(6) + val maxAge = TimeUnit.DAYS.toMillis(7) + val clock = new ManualClock(0) + val provider = new FsHistoryProvider( + createTestConf().set(MAX_LOG_AGE_S, maxAge / 1000), clock) + val log = newLogFile("inProgressApp1", None, inProgress = true) + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")) + ) + clock.setTime(firstFileModifiedTime) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")), + SparkListenerJobStart(0, 1L, Nil, null) + ) + + clock.setTime(secondFileModifiedTime) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + clock.setTime(TimeUnit.DAYS.toMillis(10)) + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")), + SparkListenerJobStart(0, 1L, Nil, null), + SparkListenerJobEnd(0, 1L, JobSucceeded) + ) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + // This should not trigger any cleanup + updateAndCheck(provider) { list => + list.size should be(1) + } + } + test("log cleaner for inProgress files") { val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10) val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20) From 1144df3b5dc8280ae4a07678cb439f0b44cb17b0 Mon Sep 17 00:00:00 2001 From: Rob Vesse Date: Thu, 29 Nov 2018 09:59:38 -0800 Subject: [PATCH 0021/1072] [SPARK-26015][K8S] Set a default UID for Spark on K8S Images Adds USER directives to the Dockerfiles which is configurable via build argument (`spark_uid`) for easy customisation. A `-u` flag is added to `bin/docker-image-tool.sh` to make it easy to customise this e.g. ``` > bin/docker-image-tool.sh -r rvesse -t uid -u 185 build > bin/docker-image-tool.sh -r rvesse -t uid push ``` If no UID is explicitly specified it defaults to `185` - this is per skonto's suggestion to align with the OpenShift standard reserved UID for Java apps ( https://lists.openshift.redhat.com/openshift-archives/users/2016-March/msg00283.html) Notes: - We have to make the `WORKDIR` writable by the root group or otherwise jobs will fail with `AccessDeniedException` To Do: - [x] Debug and resolve issue with client mode test - [x] Consider whether to always propagate `SPARK_USER_NAME` to environment of driver and executor pods so `entrypoint.sh` can insert that into `/etc/passwd` entry - [x] Rebase once PR #23013 is merged and update documentation accordingly Built the Docker images with the new Dockerfiles that include the `USER` directives. Ran the Spark on K8S integration tests against the new images. All pass except client mode which I am currently debugging further. Also manually dropped myself into the resulting container images via `docker run` and checked `id -u` output to see that UID is as expected. Tried customising the UID from the default via the new `-u` argument to `docker-image-tool.sh` and again checked the resulting image for the correct runtime UID. cc felixcheung skonto vanzin Closes #23017 from rvesse/SPARK-26015. Authored-by: Rob Vesse Signed-off-by: Marcelo Vanzin --- bin/docker-image-tool.sh | 16 +++++++++++++--- docs/running-on-kubernetes.md | 5 +++-- .../docker/src/main/dockerfiles/spark/Dockerfile | 6 ++++++ .../main/dockerfiles/spark/bindings/R/Dockerfile | 9 +++++++++ .../dockerfiles/spark/bindings/python/Dockerfile | 9 +++++++++ .../src/main/dockerfiles/spark/entrypoint.sh | 2 +- .../integrationtest/ClientModeTestsSuite.scala | 3 ++- 7 files changed, 43 insertions(+), 7 deletions(-) diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index 9f735f1148da..fbf9c9e448fd 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -146,6 +146,12 @@ function build { fi local BUILD_ARGS=(${BUILD_PARAMS}) + + # If a custom SPARK_UID was set add it to build arguments + if [ -n "$SPARK_UID" ]; then + BUILD_ARGS+=(--build-arg spark_uid=$SPARK_UID) + fi + local BINDING_BUILD_ARGS=( ${BUILD_PARAMS} --build-arg @@ -207,8 +213,10 @@ Options: -t tag Tag to apply to the built image, or to identify the image to be pushed. -m Use minikube's Docker daemon. -n Build docker image with --no-cache - -b arg Build arg to build or push the image. For multiple build args, this option needs to - be used separately for each build arg. + -u uid UID to use in the USER directive to set the user the main Spark process runs as inside the + resulting container + -b arg Build arg to build or push the image. For multiple build args, this option needs to + be used separately for each build arg. Using minikube when building images will do so directly into minikube's Docker daemon. There is no need to push the images into minikube in that case, they'll be automatically @@ -243,7 +251,8 @@ PYDOCKERFILE= RDOCKERFILE= NOCACHEARG= BUILD_PARAMS= -while getopts f:p:R:mr:t:nb: option +SPARK_UID= +while getopts f:p:R:mr:t:nb:u: option do case "${option}" in @@ -263,6 +272,7 @@ do fi eval $(minikube docker-env) ;; + u) SPARK_UID=${OPTARG};; esac done diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 2c01e1e7155e..5639253d52f5 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -19,9 +19,9 @@ Please see [Spark Security](security.html) and the specific advice below before ## User Identity -Images built from the project provided Dockerfiles do not contain any [`USER`](https://docs.docker.com/engine/reference/builder/#user) directives. This means that the resulting images will be running the Spark processes as `root` inside the container. On unsecured clusters this may provide an attack vector for privilege escalation and container breakout. Therefore security conscious deployments should consider providing custom images with `USER` directives specifying an unprivileged UID and GID. +Images built from the project provided Dockerfiles contain a default [`USER`](https://docs.docker.com/engine/reference/builder/#user) directive with a default UID of `185`. This means that the resulting images will be running the Spark processes as this UID inside the container. Security conscious deployments should consider providing custom images with `USER` directives specifying their desired unprivileged UID and GID. The resulting UID should include the root group in its supplementary groups in order to be able to run the Spark executables. Users building their own images with the provided `docker-image-tool.sh` script can use the `-u ` option to specify the desired UID. -Alternatively the [Pod Template](#pod-template) feature can be used to add a [Security Context](https://kubernetes.io/docs/tasks/configure-pod-container/security-context/#volumes-and-file-systems) with a `runAsUser` to the pods that Spark submits. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/#users-and-groups) if they wish to limit the users that pods may run as. +Alternatively the [Pod Template](#pod-template) feature can be used to add a [Security Context](https://kubernetes.io/docs/tasks/configure-pod-container/security-context/#volumes-and-file-systems) with a `runAsUser` to the pods that Spark submits. This can be used to override the `USER` directives in the images themselves. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/#users-and-groups) if they wish to limit the users that pods may run as. ## Volume Mounts @@ -87,6 +87,7 @@ Example usage is: $ ./bin/docker-image-tool.sh -r -t my-tag build $ ./bin/docker-image-tool.sh -r -t my-tag push ``` +This will build using the projects provided default `Dockerfiles`. To see more options available for customising the behaviour of this tool, including providing custom `Dockerfiles`, please run with the `-h` flag. By default `bin/docker-image-tool.sh` builds docker image for running JVM jobs. You need to opt-in to build additional language binding docker images. diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 89b20e144622..084304032470 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -17,6 +17,8 @@ FROM openjdk:8-alpine +ARG spark_uid=185 + # Before building the docker image, first build and make a Spark distribution following # the instructions in http://spark.apache.org/docs/latest/building-spark.html. # If this docker file is being used in the context of building your images from a Spark @@ -47,5 +49,9 @@ COPY data /opt/spark/data ENV SPARK_HOME /opt/spark WORKDIR /opt/spark/work-dir +RUN chmod g+w /opt/spark/work-dir ENTRYPOINT [ "/opt/entrypoint.sh" ] + +# Specify the User that the actual main process will run as +USER ${spark_uid} diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile index 9f67422efeb3..9ded57c65510 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile @@ -16,8 +16,14 @@ # ARG base_img +ARG spark_uid=185 + FROM $base_img WORKDIR / + +# Reset to root to run installation tasks +USER 0 + RUN mkdir ${SPARK_HOME}/R RUN apk add --no-cache R R-dev @@ -27,3 +33,6 @@ ENV R_HOME /usr/lib/R WORKDIR /opt/spark/work-dir ENTRYPOINT [ "/opt/entrypoint.sh" ] + +# Specify the User that the actual main process will run as +USER ${spark_uid} diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile index 69b6efa6149a..de1a0617b1cc 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -16,8 +16,14 @@ # ARG base_img +ARG spark_uid=185 + FROM $base_img WORKDIR / + +# Reset to root to run installation tasks +USER 0 + RUN mkdir ${SPARK_HOME}/python # TODO: Investigate running both pip and pip3 via virtualenvs RUN apk add --no-cache python && \ @@ -37,3 +43,6 @@ ENV PYTHONPATH ${SPARK_HOME}/python/lib/pyspark.zip:${SPARK_HOME}/python/lib/py4 WORKDIR /opt/spark/work-dir ENTRYPOINT [ "/opt/entrypoint.sh" ] + +# Specify the User that the actual main process will run as +USER ${spark_uid} diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 2b2a4e4cf6bc..2d770075a074 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -30,7 +30,7 @@ set -e # If there is no passwd entry for the container UID, attempt to create one if [ -z "$uidentry" ] ; then if [ -w /etc/passwd ] ; then - echo "$myuid:x:$myuid:$mygid:anonymous uid:$SPARK_HOME:/bin/false" >> /etc/passwd + echo "$myuid:x:$myuid:$mygid:${SPARK_USER_NAME:-anonymous uid}:$SPARK_HOME:/bin/false" >> /etc/passwd else echo "Container ENTRYPOINT failed to add passwd entry for anonymous UID" fi diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala index c8bd584516ea..2720cdf74ca8 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala @@ -62,11 +62,12 @@ private[spark] trait ClientModeTestsSuite { k8sSuite: KubernetesSuite => .endMetadata() .withNewSpec() .withServiceAccountName(kubernetesTestComponents.serviceAccountName) + .withRestartPolicy("Never") .addNewContainer() .withName("spark-example") .withImage(image) .withImagePullPolicy("IfNotPresent") - .withCommand("/opt/spark/bin/run-example") + .addToArgs("/opt/spark/bin/run-example") .addToArgs("--master", s"k8s://https://kubernetes.default.svc") .addToArgs("--deploy-mode", "client") .addToArgs("--conf", s"spark.kubernetes.container.image=$image") From 9fdc7a840daa64d1302d12027fd84ea9894110a1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E4=BA=AE?= Date: Thu, 29 Nov 2018 13:08:53 -0600 Subject: [PATCH 0022/1072] [SPARK-26158][MLLIB] fix covariance accuracy problem for DenseVector MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Enhance accuracy of the covariance logic in RowMatrix for function computeCovariance ## How was this patch tested? Unit test Accuracy test Closes #23126 from KyleLi1985/master. Authored-by: 李亮 Signed-off-by: Sean Owen --- .../mllib/linalg/distributed/RowMatrix.scala | 97 ++++++++++++++----- .../apache/spark/ml/feature/JavaPCASuite.java | 3 +- .../linalg/distributed/RowMatrixSuite.scala | 14 +++ 3 files changed, 90 insertions(+), 24 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index c12b751bfb8e..ff02e5dd3c25 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -128,6 +128,77 @@ class RowMatrix @Since("1.0.0") ( RowMatrix.triuToFull(n, GU.data) } + private def computeDenseVectorCovariance(mean: Vector, n: Int, m: Long): Matrix = { + + val bc = rows.context.broadcast(mean) + + // Computes n*(n+1)/2, avoiding overflow in the multiplication. + // This succeeds when n <= 65535, which is checked above + val nt = if (n % 2 == 0) ((n / 2) * (n + 1)) else (n * ((n + 1) / 2)) + + val MU = rows.treeAggregate(new BDV[Double](nt))( + seqOp = (U, v) => { + + val n = v.size + val na = Array.ofDim[Double](n) + val means = bc.value + + val ta = v.toArray + for (index <- 0 until n) { + na(index) = ta(index) - means(index) + } + + BLAS.spr(1.0, new DenseVector(na), U.data) + U + }, combOp = (U1, U2) => U1 += U2) + + bc.destroy() + + val M = RowMatrix.triuToFull(n, MU.data).asBreeze + + var i = 0 + var j = 0 + val m1 = m - 1.0 + while (i < n) { + j = i + while (j < n) { + val Mij = M(i, j) / m1 + M(i, j) = Mij + M(j, i) = Mij + j += 1 + } + i += 1 + } + + Matrices.fromBreeze(M) + } + + private def computeSparseVectorCovariance(mean: Vector, n: Int, m: Long): Matrix = { + + // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is + // large but Cov(X, Y) is small, but it is good for sparse computation. + // TODO: find a fast and stable way for sparse data. + val G = computeGramianMatrix().asBreeze + + var i = 0 + var j = 0 + val m1 = m - 1.0 + var alpha = 0.0 + while (i < n) { + alpha = m / m1 * mean(i) + j = i + while (j < n) { + val Gij = G(i, j) / m1 - alpha * mean(j) + G(i, j) = Gij + G(j, i) = Gij + j += 1 + } + i += 1 + } + + Matrices.fromBreeze(G) + } + private def checkNumColumns(cols: Int): Unit = { if (cols > 65535) { throw new IllegalArgumentException(s"Argument with more than 65535 cols: $cols") @@ -337,29 +408,11 @@ class RowMatrix @Since("1.0.0") ( " Cannot compute the covariance of a RowMatrix with <= 1 row.") val mean = summary.mean - // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is - // large but Cov(X, Y) is small, but it is good for sparse computation. - // TODO: find a fast and stable way for sparse data. - - val G = computeGramianMatrix().asBreeze - - var i = 0 - var j = 0 - val m1 = m - 1.0 - var alpha = 0.0 - while (i < n) { - alpha = m / m1 * mean(i) - j = i - while (j < n) { - val Gij = G(i, j) / m1 - alpha * mean(j) - G(i, j) = Gij - G(j, i) = Gij - j += 1 - } - i += 1 + if (rows.first().isInstanceOf[DenseVector]) { + computeDenseVectorCovariance(mean, n, m) + } else { + computeSparseVectorCovariance(mean, n, m) } - - Matrices.fromBreeze(G) } /** diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index 683ceffeaed0..2e177edf2a5c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -28,7 +28,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.Vectors; -import org.apache.spark.mllib.linalg.DenseVector; import org.apache.spark.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.distributed.RowMatrix; import org.apache.spark.sql.Dataset; @@ -67,7 +66,7 @@ public void testPCA() { JavaRDD dataRDD = jsc.parallelize(points, 2); RowMatrix mat = new RowMatrix(dataRDD.map( - (Vector vector) -> (org.apache.spark.mllib.linalg.Vector) new DenseVector(vector.toArray()) + (Vector vector) -> org.apache.spark.mllib.linalg.Vectors.fromML(vector) ).rdd()); Matrix pc = mat.computePrincipalComponents(3); diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 7c9e14f8cee7..a4ca4f0a80fa 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -266,6 +266,20 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { } } + test("dense vector covariance accuracy (SPARK-26158)") { + val denseData = Seq( + Vectors.dense(100000.000004, 199999.999999), + Vectors.dense(100000.000012, 200000.000002), + Vectors.dense(99999.9999931, 200000.000003), + Vectors.dense(99999.9999977, 200000.000001) + ) + val denseMat = new RowMatrix(sc.parallelize(denseData, 2)) + + val result = denseMat.computeCovariance() + val expected = breeze.linalg.cov(denseMat.toBreeze()) + assert(closeToZero(abs(expected) - abs(result.asBreeze.asInstanceOf[BDM[Double]]))) + } + test("compute covariance") { for (mat <- Seq(denseMat, sparseMat)) { val result = mat.computeCovariance() From cb368f2c2964797d7313d3a4151e2352ff7847a9 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 29 Nov 2018 12:09:30 -0800 Subject: [PATCH 0023/1072] [SPARK-26142] followup: Move sql shuffle read metrics relatives to SQLShuffleMetricsReporter ## What changes were proposed in this pull request? Follow up for https://github.com/apache/spark/pull/23128, move sql read metrics relatives to `SQLShuffleMetricsReporter`, in order to put sql shuffle read metrics relatives closer and avoid possible problem about forgetting update SQLShuffleMetricsReporter while new metrics added by others. ## How was this patch tested? Existing tests. Closes #23175 from xuanyuanking/SPARK-26142-follow. Authored-by: Yuanjian Li Signed-off-by: Reynold Xin --- .../exchange/ShuffleExchangeExec.scala | 4 +- .../apache/spark/sql/execution/limit.scala | 6 +-- .../sql/execution/metric/SQLMetrics.scala | 20 -------- .../metric/SQLShuffleMetricsReporter.scala | 50 +++++++++++++++---- .../execution/UnsafeRowSerializerSuite.scala | 4 +- 5 files changed, 47 insertions(+), 37 deletions(-) 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 8938d93da90e..c9ca395bceaa 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,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Uns import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.{SQLMetrics, SQLShuffleMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.MutablePair @@ -49,7 +49,7 @@ case class ShuffleExchangeExec( override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size") - ) ++ SQLMetrics.getShuffleReadMetrics(sparkContext) + ) ++ SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) override def nodeName: String = { val extraInfo = coordinator match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index ea845da8438f..e9ab7cd138d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGe import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.SQLShuffleMetricsReporter /** * Take the first `limit` elements and collect them to a single partition. @@ -38,7 +38,7 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode override def outputPartitioning: Partitioning = SinglePartition override def executeCollect(): Array[InternalRow] = child.executeTake(limit) private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) - override lazy val metrics = SQLMetrics.getShuffleReadMetrics(sparkContext) + override lazy val metrics = SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) protected override def doExecute(): RDD[InternalRow] = { val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit)) val shuffled = new ShuffledRowRDD( @@ -154,7 +154,7 @@ case class TakeOrderedAndProjectExec( private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) - override lazy val metrics = SQLMetrics.getShuffleReadMetrics(sparkContext) + override lazy val metrics = SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) protected override def doExecute(): RDD[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 0b5ee3a5e057..cbf707f4a9cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -82,14 +82,6 @@ object SQLMetrics { private val baseForAvgMetric: Int = 10 - val REMOTE_BLOCKS_FETCHED = "remoteBlocksFetched" - val LOCAL_BLOCKS_FETCHED = "localBlocksFetched" - val REMOTE_BYTES_READ = "remoteBytesRead" - val REMOTE_BYTES_READ_TO_DISK = "remoteBytesReadToDisk" - val LOCAL_BYTES_READ = "localBytesRead" - val FETCH_WAIT_TIME = "fetchWaitTime" - val RECORDS_READ = "recordsRead" - /** * Converts a double value to long value by multiplying a base integer, so we can store it in * `SQLMetrics`. It only works for average metrics. When showing the metrics on UI, we restore @@ -202,16 +194,4 @@ object SQLMetrics { SparkListenerDriverAccumUpdates(executionId.toLong, metrics.map(m => m.id -> m.value))) } } - - /** - * Create all shuffle read relative metrics and return the Map. - */ - def getShuffleReadMetrics(sc: SparkContext): Map[String, SQLMetric] = Map( - REMOTE_BLOCKS_FETCHED -> createMetric(sc, "remote blocks fetched"), - LOCAL_BLOCKS_FETCHED -> createMetric(sc, "local blocks fetched"), - REMOTE_BYTES_READ -> createSizeMetric(sc, "remote bytes read"), - REMOTE_BYTES_READ_TO_DISK -> createSizeMetric(sc, "remote bytes read to disk"), - LOCAL_BYTES_READ -> createSizeMetric(sc, "local bytes read"), - FETCH_WAIT_TIME -> createTimingMetric(sc, "fetch wait time"), - RECORDS_READ -> createMetric(sc, "records read")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala index 542141ea4b4e..780f0d762229 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala @@ -17,24 +17,32 @@ package org.apache.spark.sql.execution.metric +import org.apache.spark.SparkContext import org.apache.spark.executor.TempShuffleReadMetrics /** * A shuffle metrics reporter for SQL exchange operators. * @param tempMetrics [[TempShuffleReadMetrics]] created in TaskContext. * @param metrics All metrics in current SparkPlan. This param should not empty and - * contains all shuffle metrics defined in [[SQLMetrics.getShuffleReadMetrics]]. + * contains all shuffle metrics defined in createShuffleReadMetrics. */ private[spark] class SQLShuffleMetricsReporter( - tempMetrics: TempShuffleReadMetrics, - metrics: Map[String, SQLMetric]) extends TempShuffleReadMetrics { - private[this] val _remoteBlocksFetched = metrics(SQLMetrics.REMOTE_BLOCKS_FETCHED) - private[this] val _localBlocksFetched = metrics(SQLMetrics.LOCAL_BLOCKS_FETCHED) - private[this] val _remoteBytesRead = metrics(SQLMetrics.REMOTE_BYTES_READ) - private[this] val _remoteBytesReadToDisk = metrics(SQLMetrics.REMOTE_BYTES_READ_TO_DISK) - private[this] val _localBytesRead = metrics(SQLMetrics.LOCAL_BYTES_READ) - private[this] val _fetchWaitTime = metrics(SQLMetrics.FETCH_WAIT_TIME) - private[this] val _recordsRead = metrics(SQLMetrics.RECORDS_READ) + tempMetrics: TempShuffleReadMetrics, + metrics: Map[String, SQLMetric]) extends TempShuffleReadMetrics { + private[this] val _remoteBlocksFetched = + metrics(SQLShuffleMetricsReporter.REMOTE_BLOCKS_FETCHED) + private[this] val _localBlocksFetched = + metrics(SQLShuffleMetricsReporter.LOCAL_BLOCKS_FETCHED) + private[this] val _remoteBytesRead = + metrics(SQLShuffleMetricsReporter.REMOTE_BYTES_READ) + private[this] val _remoteBytesReadToDisk = + metrics(SQLShuffleMetricsReporter.REMOTE_BYTES_READ_TO_DISK) + private[this] val _localBytesRead = + metrics(SQLShuffleMetricsReporter.LOCAL_BYTES_READ) + private[this] val _fetchWaitTime = + metrics(SQLShuffleMetricsReporter.FETCH_WAIT_TIME) + private[this] val _recordsRead = + metrics(SQLShuffleMetricsReporter.RECORDS_READ) override def incRemoteBlocksFetched(v: Long): Unit = { _remoteBlocksFetched.add(v) @@ -65,3 +73,25 @@ private[spark] class SQLShuffleMetricsReporter( tempMetrics.incRecordsRead(v) } } + +private[spark] object SQLShuffleMetricsReporter { + val REMOTE_BLOCKS_FETCHED = "remoteBlocksFetched" + val LOCAL_BLOCKS_FETCHED = "localBlocksFetched" + val REMOTE_BYTES_READ = "remoteBytesRead" + val REMOTE_BYTES_READ_TO_DISK = "remoteBytesReadToDisk" + val LOCAL_BYTES_READ = "localBytesRead" + val FETCH_WAIT_TIME = "fetchWaitTime" + val RECORDS_READ = "recordsRead" + + /** + * Create all shuffle read relative metrics and return the Map. + */ + def createShuffleReadMetrics(sc: SparkContext): Map[String, SQLMetric] = Map( + REMOTE_BLOCKS_FETCHED -> SQLMetrics.createMetric(sc, "remote blocks fetched"), + LOCAL_BLOCKS_FETCHED -> SQLMetrics.createMetric(sc, "local blocks fetched"), + REMOTE_BYTES_READ -> SQLMetrics.createSizeMetric(sc, "remote bytes read"), + REMOTE_BYTES_READ_TO_DISK -> SQLMetrics.createSizeMetric(sc, "remote bytes read to disk"), + LOCAL_BYTES_READ -> SQLMetrics.createSizeMetric(sc, "local bytes read"), + FETCH_WAIT_TIME -> SQLMetrics.createTimingMetric(sc, "fetch wait time"), + RECORDS_READ -> SQLMetrics.createMetric(sc, "records read")) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index 96b3aa5ee75b..1ad5713ab8ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{LocalSparkSession, Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.SQLShuffleMetricsReporter import org.apache.spark.sql.types._ import org.apache.spark.storage.ShuffleBlockId import org.apache.spark.util.collection.ExternalSorter @@ -140,7 +140,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { new UnsafeRowSerializer(2)) val shuffled = new ShuffledRowRDD( dependency, - SQLMetrics.getShuffleReadMetrics(spark.sparkContext)) + SQLShuffleMetricsReporter.createShuffleReadMetrics(spark.sparkContext)) shuffled.count() } } From 59741887e272be92ebd6e61783f99f7d8fc05456 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Thu, 29 Nov 2018 14:56:34 -0600 Subject: [PATCH 0024/1072] [SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to a ChunkedByteBuffer ## What changes were proposed in this pull request? In `BlockManager`, `getRemoteValues` gets a `ChunkedByteBuffer` (by calling `getRemoteBytes`) and creates an `InputStream` from it. `getRemoteBytes`, in turn, gets a `ManagedBuffer` and converts it to a `ChunkedByteBuffer`. Instead, expose a `getRemoteManagedBuffer` method so `getRemoteValues` can just get this `ManagedBuffer` and use its `InputStream`. When reading a remote cache block from disk, this reduces heap memory usage significantly. Retain `getRemoteBytes` for other callers. ## How was this patch tested? Imran Rashid wrote an application (https://github.com/squito/spark_2gb_test/blob/master/src/main/scala/com/cloudera/sparktest/LargeBlocks.scala), that among other things, tests reading remote cache blocks. I ran this application, using 2500MB blocks, to test reading a cache block on disk. Without this change, with `--executor-memory 5g`, the test fails with `java.lang.OutOfMemoryError: Java heap space`. With the change, the test passes with `--executor-memory 2g`. I also ran the unit tests in core. In particular, `DistributedSuite` has a set of tests that exercise the `getRemoteValues` code path. `BlockManagerSuite` has several tests that call `getRemoteBytes`; I left these unchanged, so `getRemoteBytes` still gets exercised. Closes #23058 from wypoon/SPARK-25905. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid --- .../apache/spark/storage/BlockManager.scala | 43 ++++++++++++------- .../spark/util/io/ChunkedByteBuffer.scala | 2 - .../org/apache/spark/DistributedSuite.scala | 2 +- 3 files changed, 28 insertions(+), 19 deletions(-) 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 1b617297e0a3..1dfbc6effb34 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -692,9 +692,9 @@ private[spark] class BlockManager( */ private def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { val ct = implicitly[ClassTag[T]] - getRemoteBytes(blockId).map { data => + getRemoteManagedBuffer(blockId).map { data => val values = - serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true))(ct) + serializerManager.dataDeserializeStream(blockId, data.createInputStream())(ct) new BlockResult(values, DataReadMethod.Network, data.size) } } @@ -717,13 +717,9 @@ private[spark] class BlockManager( } /** - * Get block from remote block managers as serialized bytes. + * Get block from remote block managers as a ManagedBuffer. */ - def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { - // TODO SPARK-25905 if we change this method to return the ManagedBuffer, then getRemoteValues - // could just use the inputStream on the temp file, rather than reading the file into memory. - // Until then, replication can cause the process to use too much memory and get killed - // even though we've read the data to disk. + private def getRemoteManagedBuffer(blockId: BlockId): Option[ManagedBuffer] = { logDebug(s"Getting remote block $blockId") require(blockId != null, "BlockId is null") var runningFailureCount = 0 @@ -788,14 +784,13 @@ private[spark] class BlockManager( } if (data != null) { - // SPARK-24307 undocumented "escape-hatch" in case there are any issues in converting to - // ChunkedByteBuffer, to go back to old code-path. Can be removed post Spark 2.4 if - // new path is stable. - if (remoteReadNioBufferConversion) { - return Some(new ChunkedByteBuffer(data.nioByteBuffer())) - } else { - return Some(ChunkedByteBuffer.fromManagedBuffer(data)) - } + // If the ManagedBuffer is a BlockManagerManagedBuffer, the disposal of the + // byte buffers backing it may need to be handled after reading the bytes. + // In this case, since we just fetched the bytes remotely, we do not have + // a BlockManagerManagedBuffer. The assert here is to ensure that this holds + // true (or the disposal is handled). + assert(!data.isInstanceOf[BlockManagerManagedBuffer]) + return Some(data) } logDebug(s"The value of block $blockId is null") } @@ -803,6 +798,22 @@ private[spark] class BlockManager( None } + /** + * Get block from remote block managers as serialized bytes. + */ + def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { + getRemoteManagedBuffer(blockId).map { data => + // SPARK-24307 undocumented "escape-hatch" in case there are any issues in converting to + // ChunkedByteBuffer, to go back to old code-path. Can be removed post Spark 2.4 if + // new path is stable. + if (remoteReadNioBufferConversion) { + new ChunkedByteBuffer(data.nioByteBuffer()) + } else { + ChunkedByteBuffer.fromManagedBuffer(data) + } + } + } + /** * Get a block from the block manager (either local or remote). * diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index 128d6ff8cd74..2c3730de08b5 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -172,8 +172,6 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { private[spark] object ChunkedByteBuffer { - - // TODO SPARK-25905 eliminate this method if we switch BlockManager to getting InputStreams def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = { data match { case f: FileSegmentManagedBuffer => diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 629a323042ff..4083b20c2359 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -195,7 +195,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream())(data.elementClassTag).toList assert(deserialized === (1 to 100).toList) } - // This will exercise the getRemoteBytes / getRemoteValues code paths: + // This will exercise the getRemoteValues code path: assert(blockIds.flatMap(id => blockManager.get[Int](id).get.data).toSet === (1 to 1000).toSet) } From f97326bcdba532eabf25d4899b13709e9af2bfea Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 30 Nov 2018 08:27:55 +0800 Subject: [PATCH 0025/1072] [SPARK-25977][SQL] Parsing decimals from CSV using locale ## What changes were proposed in this pull request? In the PR, I propose using of the locale option to parse decimals from CSV input. After the changes, `UnivocityParser` converts input string to `BigDecimal` and to Spark's Decimal by using `java.text.DecimalFormat`. ## How was this patch tested? Added a test for the `en-US`, `ko-KR`, `ru-RU`, `de-DE` locales. Closes #22979 from MaxGekk/decimal-parsing-locale. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: hyukjinkwon --- .../spark/sql/catalyst/csv/CSVExprUtils.scala | 4 + .../sql/catalyst/csv/CSVInferSchema.scala | 72 ++++----- .../sql/catalyst/csv/UnivocityParser.scala | 8 +- .../catalyst/expressions/csvExpressions.scala | 5 +- .../catalyst/csv/CSVInferSchemaSuite.scala | 147 ++++++++++++------ .../catalyst/csv/UnivocityParserSuite.scala | 22 ++- .../datasources/csv/CSVDataSource.scala | 4 +- 7 files changed, 168 insertions(+), 94 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala index bbe27831f01d..6c982a1de9a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala @@ -17,6 +17,10 @@ package org.apache.spark.sql.catalyst.csv +import java.math.BigDecimal +import java.text.{DecimalFormat, DecimalFormatSymbols, ParsePosition} +import java.util.Locale + object CSVExprUtils { /** * Filter ignorable rows for CSV iterator (lines empty and starting with `comment`). 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 799e9994451b..94cb4b114e6b 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 @@ -17,16 +17,19 @@ package org.apache.spark.sql.catalyst.csv -import java.math.BigDecimal - import scala.util.control.Exception.allCatch import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion +import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ -object CSVInferSchema { +class CSVInferSchema(options: CSVOptions) extends Serializable { + + private val decimalParser = { + ExprUtils.getDecimalParser(options.locale) + } /** * Similar to the JSON schema inference @@ -36,14 +39,13 @@ object CSVInferSchema { */ def infer( tokenRDD: RDD[Array[String]], - header: Array[String], - options: CSVOptions): StructType = { + header: Array[String]): StructType = { val fields = if (options.inferSchemaFlag) { val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) val rootTypes: Array[DataType] = - tokenRDD.aggregate(startType)(inferRowType(options), mergeRowTypes) + tokenRDD.aggregate(startType)(inferRowType, mergeRowTypes) - toStructFields(rootTypes, header, options) + toStructFields(rootTypes, header) } else { // By default fields are assumed to be StringType header.map(fieldName => StructField(fieldName, StringType, nullable = true)) @@ -54,8 +56,7 @@ object CSVInferSchema { def toStructFields( fieldTypes: Array[DataType], - header: Array[String], - options: CSVOptions): Array[StructField] = { + header: Array[String]): Array[StructField] = { header.zip(fieldTypes).map { case (thisHeader, rootType) => val dType = rootType match { case _: NullType => StringType @@ -65,11 +66,10 @@ object CSVInferSchema { } } - def inferRowType(options: CSVOptions) - (rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { + def inferRowType(rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { var i = 0 while (i < math.min(rowSoFar.length, next.length)) { // May have columns on right missing. - rowSoFar(i) = inferField(rowSoFar(i), next(i), options) + rowSoFar(i) = inferField(rowSoFar(i), next(i)) i+=1 } rowSoFar @@ -85,20 +85,20 @@ object CSVInferSchema { * Infer type of string field. Given known type Double, and a string "1", there is no * point checking if it is an Int, as the final type must be Double or higher. */ - def inferField(typeSoFar: DataType, field: String, options: CSVOptions): DataType = { + def inferField(typeSoFar: DataType, field: String): DataType = { if (field == null || field.isEmpty || field == options.nullValue) { typeSoFar } else { typeSoFar match { - case NullType => tryParseInteger(field, options) - case IntegerType => tryParseInteger(field, options) - case LongType => tryParseLong(field, options) + 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, options)).getOrElse(StringType) - case DoubleType => tryParseDouble(field, options) - case TimestampType => tryParseTimestamp(field, options) - case BooleanType => tryParseBoolean(field, options) + compatibleType(typeSoFar, tryParseDecimal(field)).getOrElse(StringType) + case DoubleType => tryParseDouble(field) + case TimestampType => tryParseTimestamp(field) + case BooleanType => tryParseBoolean(field) case StringType => StringType case other: DataType => throw new UnsupportedOperationException(s"Unexpected data type $other") @@ -106,30 +106,30 @@ object CSVInferSchema { } } - private def isInfOrNan(field: String, options: CSVOptions): Boolean = { + private def isInfOrNan(field: String): Boolean = { field == options.nanValue || field == options.negativeInf || field == options.positiveInf } - private def tryParseInteger(field: String, options: CSVOptions): DataType = { + private def tryParseInteger(field: String): DataType = { if ((allCatch opt field.toInt).isDefined) { IntegerType } else { - tryParseLong(field, options) + tryParseLong(field) } } - private def tryParseLong(field: String, options: CSVOptions): DataType = { + private def tryParseLong(field: String): DataType = { if ((allCatch opt field.toLong).isDefined) { LongType } else { - tryParseDecimal(field, options) + tryParseDecimal(field) } } - private def tryParseDecimal(field: String, options: CSVOptions): DataType = { + private def tryParseDecimal(field: String): DataType = { val decimalTry = allCatch opt { - // `BigDecimal` conversion can fail when the `field` is not a form of number. - val bigDecimal = new BigDecimal(field) + // The conversion can fail when the `field` is not a form of number. + val bigDecimal = decimalParser(field) // Because many other formats do not support decimal, it reduces the cases for // decimals by disallowing values having scale (eg. `1.1`). if (bigDecimal.scale <= 0) { @@ -138,21 +138,21 @@ object CSVInferSchema { // 2. scale is bigger than precision. DecimalType(bigDecimal.precision, bigDecimal.scale) } else { - tryParseDouble(field, options) + tryParseDouble(field) } } - decimalTry.getOrElse(tryParseDouble(field, options)) + decimalTry.getOrElse(tryParseDouble(field)) } - private def tryParseDouble(field: String, options: CSVOptions): DataType = { - if ((allCatch opt field.toDouble).isDefined || isInfOrNan(field, options)) { + private def tryParseDouble(field: String): DataType = { + if ((allCatch opt field.toDouble).isDefined || isInfOrNan(field)) { DoubleType } else { - tryParseTimestamp(field, options) + tryParseTimestamp(field) } } - private def tryParseTimestamp(field: String, options: CSVOptions): DataType = { + private def tryParseTimestamp(field: String): DataType = { // This case infers a custom `dataFormat` is set. if ((allCatch opt options.timestampFormat.parse(field)).isDefined) { TimestampType @@ -160,11 +160,11 @@ object CSVInferSchema { // We keep this for backwards compatibility. TimestampType } else { - tryParseBoolean(field, options) + tryParseBoolean(field) } } - private def tryParseBoolean(field: String, options: CSVOptions): DataType = { + private def tryParseBoolean(field: String): DataType = { if ((allCatch opt field.toBoolean).isDefined) { BooleanType } else { 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 ed196935e357..85e129224c91 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 @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.csv import java.io.InputStream -import java.math.BigDecimal import scala.util.Try import scala.util.control.NonFatal @@ -27,7 +26,7 @@ import com.univocity.parsers.csv.CsvParser import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} import org.apache.spark.sql.catalyst.util.{BadRecordException, DateTimeUtils, FailureSafeParser} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -104,6 +103,8 @@ class UnivocityParser( requiredSchema.map(f => makeConverter(f.name, f.dataType, f.nullable, options)).toArray } + private val decimalParser = ExprUtils.getDecimalParser(options.locale) + /** * Create a converter which converts the string value to a value according to a desired type. * Currently, we do not support complex types (`ArrayType`, `MapType`, `StructType`). @@ -149,8 +150,7 @@ class UnivocityParser( case dt: DecimalType => (d: String) => nullSafeDatum(d, name, nullable, options) { datum => - val value = new BigDecimal(datum.replaceAll(",", "")) - Decimal(value, dt.precision, dt.scale) + Decimal(decimalParser(datum), dt.precision, dt.scale) } case _: TimestampType => (d: String) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala index 1e4e1c663c90..83b0299bac44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -180,8 +180,9 @@ case class SchemaOfCsv( val header = row.zipWithIndex.map { case (_, index) => s"_c$index" } val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) - val fieldTypes = CSVInferSchema.inferRowType(parsedOptions)(startType, row) - val st = StructType(CSVInferSchema.toStructFields(fieldTypes, header, parsedOptions)) + val inferSchema = new CSVInferSchema(parsedOptions) + val fieldTypes = inferSchema.inferRowType(startType, row) + val st = StructType(inferSchema.toStructFields(fieldTypes, header)) UTF8String.fromString(st.catalogString) } 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 651846d2ebcb..1a020e67a75b 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 @@ -17,126 +17,175 @@ package org.apache.spark.sql.catalyst.csv +import java.text.{DecimalFormat, DecimalFormatSymbols} +import java.util.Locale + import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -class CSVInferSchemaSuite extends SparkFunSuite { +class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { test("String fields types are inferred correctly from null types") { val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(CSVInferSchema.inferField(NullType, "", options) == NullType) - assert(CSVInferSchema.inferField(NullType, null, options) == NullType) - assert(CSVInferSchema.inferField(NullType, "100000000000", options) == LongType) - assert(CSVInferSchema.inferField(NullType, "60", options) == IntegerType) - assert(CSVInferSchema.inferField(NullType, "3.5", options) == DoubleType) - assert(CSVInferSchema.inferField(NullType, "test", options) == StringType) - assert(CSVInferSchema.inferField(NullType, "2015-08-20 15:57:00", options) == TimestampType) - assert(CSVInferSchema.inferField(NullType, "True", options) == BooleanType) - assert(CSVInferSchema.inferField(NullType, "FAlSE", options) == BooleanType) + val inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(NullType, "") == NullType) + assert(inferSchema.inferField(NullType, null) == NullType) + assert(inferSchema.inferField(NullType, "100000000000") == LongType) + assert(inferSchema.inferField(NullType, "60") == IntegerType) + assert(inferSchema.inferField(NullType, "3.5") == DoubleType) + assert(inferSchema.inferField(NullType, "test") == StringType) + assert(inferSchema.inferField(NullType, "2015-08-20 15:57:00") == TimestampType) + assert(inferSchema.inferField(NullType, "True") == BooleanType) + assert(inferSchema.inferField(NullType, "FAlSE") == BooleanType) val textValueOne = Long.MaxValue.toString + "0" val decimalValueOne = new java.math.BigDecimal(textValueOne) val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) - assert(CSVInferSchema.inferField(NullType, textValueOne, options) == expectedTypeOne) + assert(inferSchema.inferField(NullType, textValueOne) == expectedTypeOne) } test("String fields types are inferred correctly from other types") { val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(CSVInferSchema.inferField(LongType, "1.0", options) == DoubleType) - assert(CSVInferSchema.inferField(LongType, "test", options) == StringType) - assert(CSVInferSchema.inferField(IntegerType, "1.0", options) == DoubleType) - assert(CSVInferSchema.inferField(DoubleType, null, options) == DoubleType) - assert(CSVInferSchema.inferField(DoubleType, "test", options) == StringType) - assert(CSVInferSchema.inferField(LongType, "2015-08-20 14:57:00", options) == TimestampType) - assert(CSVInferSchema.inferField(DoubleType, "2015-08-20 15:57:00", options) == TimestampType) - assert(CSVInferSchema.inferField(LongType, "True", options) == BooleanType) - assert(CSVInferSchema.inferField(IntegerType, "FALSE", options) == BooleanType) - assert(CSVInferSchema.inferField(TimestampType, "FALSE", options) == BooleanType) + val inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(LongType, "1.0") == DoubleType) + assert(inferSchema.inferField(LongType, "test") == StringType) + 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) val textValueOne = Long.MaxValue.toString + "0" val decimalValueOne = new java.math.BigDecimal(textValueOne) val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) - assert(CSVInferSchema.inferField(IntegerType, textValueOne, options) == expectedTypeOne) + assert(inferSchema.inferField(IntegerType, textValueOne) == expectedTypeOne) } test("Timestamp field types are inferred correctly via custom data format") { var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm"), false, "GMT") - assert(CSVInferSchema.inferField(TimestampType, "2015-08", options) == TimestampType) + var inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(TimestampType, "2015-08") == TimestampType) + options = new CSVOptions(Map("timestampFormat" -> "yyyy"), false, "GMT") - assert(CSVInferSchema.inferField(TimestampType, "2015", options) == TimestampType) + inferSchema = new CSVInferSchema(options) + assert(inferSchema.inferField(TimestampType, "2015") == TimestampType) } test("Timestamp field types are inferred correctly from other types") { val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(CSVInferSchema.inferField(IntegerType, "2015-08-20 14", options) == StringType) - assert(CSVInferSchema.inferField(DoubleType, "2015-08-20 14:10", options) == StringType) - assert(CSVInferSchema.inferField(LongType, "2015-08 14:49:00", options) == StringType) + val inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(IntegerType, "2015-08-20 14") == StringType) + assert(inferSchema.inferField(DoubleType, "2015-08-20 14:10") == StringType) + assert(inferSchema.inferField(LongType, "2015-08 14:49:00") == StringType) } test("Boolean fields types are inferred correctly from other types") { val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(CSVInferSchema.inferField(LongType, "Fale", options) == StringType) - assert(CSVInferSchema.inferField(DoubleType, "TRUEe", options) == StringType) + val inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(LongType, "Fale") == StringType) + assert(inferSchema.inferField(DoubleType, "TRUEe") == StringType) } test("Type arrays are merged to highest common type") { + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val inferSchema = new CSVInferSchema(options) + assert( - CSVInferSchema.mergeRowTypes(Array(StringType), + inferSchema.mergeRowTypes(Array(StringType), Array(DoubleType)).deep == Array(StringType).deep) assert( - CSVInferSchema.mergeRowTypes(Array(IntegerType), + inferSchema.mergeRowTypes(Array(IntegerType), Array(LongType)).deep == Array(LongType).deep) assert( - CSVInferSchema.mergeRowTypes(Array(DoubleType), + inferSchema.mergeRowTypes(Array(DoubleType), Array(LongType)).deep == Array(DoubleType).deep) } test("Null fields are handled properly when a nullValue is specified") { var options = new CSVOptions(Map("nullValue" -> "null"), false, "GMT") - assert(CSVInferSchema.inferField(NullType, "null", options) == NullType) - assert(CSVInferSchema.inferField(StringType, "null", options) == StringType) - assert(CSVInferSchema.inferField(LongType, "null", options) == LongType) + var inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(NullType, "null") == NullType) + assert(inferSchema.inferField(StringType, "null") == StringType) + assert(inferSchema.inferField(LongType, "null") == LongType) options = new CSVOptions(Map("nullValue" -> "\\N"), false, "GMT") - assert(CSVInferSchema.inferField(IntegerType, "\\N", options) == IntegerType) - assert(CSVInferSchema.inferField(DoubleType, "\\N", options) == DoubleType) - assert(CSVInferSchema.inferField(TimestampType, "\\N", options) == TimestampType) - assert(CSVInferSchema.inferField(BooleanType, "\\N", options) == BooleanType) - assert(CSVInferSchema.inferField(DecimalType(1, 1), "\\N", options) == DecimalType(1, 1)) + inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(IntegerType, "\\N") == IntegerType) + assert(inferSchema.inferField(DoubleType, "\\N") == DoubleType) + assert(inferSchema.inferField(TimestampType, "\\N") == TimestampType) + assert(inferSchema.inferField(BooleanType, "\\N") == BooleanType) + assert(inferSchema.inferField(DecimalType(1, 1), "\\N") == DecimalType(1, 1)) } test("Merging Nulltypes should yield Nulltype.") { - val mergedNullTypes = CSVInferSchema.mergeRowTypes(Array(NullType), Array(NullType)) + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val inferSchema = new CSVInferSchema(options) + + val mergedNullTypes = inferSchema.mergeRowTypes(Array(NullType), Array(NullType)) assert(mergedNullTypes.deep == Array(NullType).deep) } test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { val options = new CSVOptions(Map("TiMeStampFormat" -> "yyyy-mm"), false, "GMT") - assert(CSVInferSchema.inferField(TimestampType, "2015-08", options) == TimestampType) + val inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(TimestampType, "2015-08") == TimestampType) } test("SPARK-18877: `inferField` on DecimalType should find a common type with `typeSoFar`") { val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val inferSchema = new CSVInferSchema(options) // 9.03E+12 is Decimal(3, -10) and 1.19E+11 is Decimal(3, -9). - assert(CSVInferSchema.inferField(DecimalType(3, -10), "1.19E+11", options) == + assert(inferSchema.inferField(DecimalType(3, -10), "1.19E11") == DecimalType(4, -9)) // BigDecimal("12345678901234567890.01234567890123456789") is precision 40 and scale 20. val value = "12345678901234567890.01234567890123456789" - assert(CSVInferSchema.inferField(DecimalType(3, -10), value, options) == DoubleType) + assert(inferSchema.inferField(DecimalType(3, -10), value) == DoubleType) // Seq(s"${Long.MaxValue}1", "2015-12-01 00:00:00") should be StringType - assert(CSVInferSchema.inferField(NullType, s"${Long.MaxValue}1", options) == DecimalType(20, 0)) - assert(CSVInferSchema.inferField(DecimalType(20, 0), "2015-12-01 00:00:00", options) + assert(inferSchema.inferField(NullType, s"${Long.MaxValue}1") == DecimalType(20, 0)) + assert(inferSchema.inferField(DecimalType(20, 0), "2015-12-01 00:00:00") == StringType) } test("DoubleType should be inferred when user defined nan/inf are provided") { val options = new CSVOptions(Map("nanValue" -> "nan", "negativeInf" -> "-inf", "positiveInf" -> "inf"), false, "GMT") - assert(CSVInferSchema.inferField(NullType, "nan", options) == DoubleType) - assert(CSVInferSchema.inferField(NullType, "inf", options) == DoubleType) - assert(CSVInferSchema.inferField(NullType, "-inf", options) == DoubleType) + val inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(NullType, "nan") == DoubleType) + assert(inferSchema.inferField(NullType, "inf") == DoubleType) + assert(inferSchema.inferField(NullType, "-inf") == DoubleType) + } + + test("inferring the decimal type using locale") { + def checkDecimalInfer(langTag: String, expectedType: DataType): Unit = { + val options = new CSVOptions( + parameters = Map("locale" -> langTag, "inferSchema" -> "true", "sep" -> "|"), + columnPruning = false, + defaultTimeZoneId = "GMT") + val inferSchema = new CSVInferSchema(options) + + val df = new DecimalFormat("", new DecimalFormatSymbols(Locale.forLanguageTag(langTag))) + val input = df.format(Decimal(1000001).toBigDecimal) + + assert(inferSchema.inferField(NullType, input) == expectedType) + } + + Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach(checkDecimalInfer(_, DecimalType(7, 0))) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index e4e7dc2e8c0e..7212402ef5cf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -18,13 +18,17 @@ package org.apache.spark.sql.catalyst.csv import java.math.BigDecimal +import java.text.{DecimalFormat, DecimalFormatSymbols} +import java.util.Locale import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -class UnivocityParserSuite extends SparkFunSuite { +class UnivocityParserSuite extends SparkFunSuite with SQLHelper { private val parser = new UnivocityParser( StructType(Seq.empty), new CSVOptions(Map.empty[String, String], false, "GMT")) @@ -196,4 +200,20 @@ class UnivocityParserSuite extends SparkFunSuite { assert(doubleVal2 == Double.PositiveInfinity) } + test("parse decimals using locale") { + def checkDecimalParsing(langTag: String): Unit = { + val decimalVal = new BigDecimal("1000.001") + val decimalType = new DecimalType(10, 5) + val expected = Decimal(decimalVal, decimalType.precision, decimalType.scale) + val df = new DecimalFormat("", new DecimalFormatSymbols(Locale.forLanguageTag(langTag))) + val input = df.format(expected.toBigDecimal) + + val options = new CSVOptions(Map("locale" -> langTag), false, "GMT") + val parser = new UnivocityParser(new StructType().add("d", decimalType), options) + + assert(parser.makeConverter("_1", decimalType, options = options).apply(input) === expected) + } + + Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach(checkDecimalParsing) + } } 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 b35b8851918b..b46dfb94c133 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 @@ -135,7 +135,7 @@ object TextInputCSVDataSource extends CSVDataSource { val parser = new CsvParser(parsedOptions.asParserSettings) linesWithoutHeader.map(parser.parseLine) } - CSVInferSchema.infer(tokenRDD, header, parsedOptions) + new CSVInferSchema(parsedOptions).infer(tokenRDD, header) case _ => // If the first line could not be read, just return the empty schema. StructType(Nil) @@ -208,7 +208,7 @@ object MultiLineCSVDataSource extends CSVDataSource { encoding = parsedOptions.charset) } val sampled = CSVUtils.sample(tokenRDD, parsedOptions) - CSVInferSchema.infer(sampled, header, parsedOptions) + new CSVInferSchema(parsedOptions).infer(sampled, header) case None => // If the first row could not be read, just return the empty schema. StructType(Nil) From 0166c7373eee2654c49c210927e4e290d103f24f Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Thu, 29 Nov 2018 18:00:47 -0800 Subject: [PATCH 0026/1072] [SPARK-25501][SS] Add kafka delegation token support. ## What changes were proposed in this pull request? It adds kafka delegation token support for structured streaming. Please see the relevant [SPIP](https://docs.google.com/document/d/1ouRayzaJf_N5VQtGhVq9FURXVmRpXzEEWYHob0ne3NY/edit?usp=sharing) What this PR contains: * Configuration parameters for the feature * Delegation token fetching from broker * Usage of token through dynamic JAAS configuration * Minor refactoring in the existing code What this PR doesn't contain: * Documentation changes because design can change ## How was this patch tested? Existing tests + added small amount of additional unit tests. Because it's an external service integration mainly tested on cluster. * 4 node cluster * Kafka broker version 1.1.0 * Topic with 4 partitions * security.protocol = SASL_SSL * sasl.mechanism = SCRAM-SHA-256 An example of obtaining a token: ``` 18/10/01 01:07:49 INFO kafka010.TokenUtil: TOKENID HMAC OWNER RENEWERS ISSUEDATE EXPIRYDATE MAXDATE 18/10/01 01:07:49 INFO kafka010.TokenUtil: D1-v__Q5T_uHx55rW16Jwg [hidden] User:user [] 2018-10-01T01:07 2018-10-02T01:07 2018-10-08T01:07 18/10/01 01:07:49 INFO security.KafkaDelegationTokenProvider: Get token from Kafka: Kind: KAFKA_DELEGATION_TOKEN, Service: kafka.server.delegation.token, Ident: 44 31 2d 76 5f 5f 51 35 54 5f 75 48 78 35 35 72 57 31 36 4a 77 67 ``` An example token usage: ``` 18/10/01 01:08:07 INFO kafka010.KafkaSecurityHelper: Scram JAAS params: org.apache.kafka.common.security.scram.ScramLoginModule required tokenauth=true serviceName="kafka" username="D1-v__Q5T_uHx55rW16Jwg" password="[hidden]"; 18/10/01 01:08:07 INFO kafka010.KafkaSourceProvider: Delegation token detected, using it for login. ``` Closes #22598 from gaborgsomogyi/SPARK-25501. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- core/pom.xml | 13 + .../HadoopDelegationTokenManager.scala | 3 +- .../KafkaDelegationTokenProvider.scala | 61 +++++ .../deploy/security/KafkaTokenUtil.scala | 202 +++++++++++++++ .../apache/spark/internal/config/Kafka.scala | 82 ++++++ .../HadoopDelegationTokenManagerSuite.scala | 5 +- .../deploy/security/KafkaTokenUtilSuite.scala | 239 ++++++++++++++++++ external/kafka-0-10-sql/pom.xml | 2 - .../sql/kafka010/KafkaSecurityHelper.scala | 56 ++++ .../sql/kafka010/KafkaSourceProvider.scala | 82 +++--- .../kafka010/KafkaStreamingWriteSupport.scala | 22 +- .../kafka010/KafkaContinuousSinkSuite.scala | 4 +- .../kafka010/KafkaSecurityHelperSuite.scala | 100 ++++++++ external/kafka-0-10/pom.xml | 2 - pom.xml | 2 + 15 files changed, 825 insertions(+), 50 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala create mode 100644 core/src/main/scala/org/apache/spark/internal/config/Kafka.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala diff --git a/core/pom.xml b/core/pom.xml index 36d93212ba9f..49b1a54e3259 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -408,6 +408,19 @@ provided + + + org.apache.kafka + kafka-clients + ${kafka.version} + provided + + target/scala-${scala.binary.version}/classes diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index 1169b2878e99..126a6ab80136 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -274,7 +274,8 @@ private[spark] class HadoopDelegationTokenManager( new HadoopFSDelegationTokenProvider( () => HadoopDelegationTokenManager.this.fileSystemsToAccess())) ++ safeCreateProvider(new HiveDelegationTokenProvider) ++ - safeCreateProvider(new HBaseDelegationTokenProvider) + safeCreateProvider(new HBaseDelegationTokenProvider) ++ + safeCreateProvider(new KafkaDelegationTokenProvider) // Filter out providers for which spark.security.credentials.{service}.enabled is false. providers diff --git a/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala new file mode 100644 index 000000000000..45995be630cc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import scala.language.existentials +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.Credentials +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ + +private[security] class KafkaDelegationTokenProvider + extends HadoopDelegationTokenProvider with Logging { + + override def serviceName: String = "kafka" + + override def obtainDelegationTokens( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = { + try { + logDebug("Attempting to fetch Kafka security token.") + val (token, nextRenewalDate) = KafkaTokenUtil.obtainToken(sparkConf) + creds.addToken(token.getService, token) + return Some(nextRenewalDate) + } catch { + case NonFatal(e) => + logInfo(s"Failed to get token from service $serviceName", e) + } + None + } + + override def delegationTokensRequired( + sparkConf: SparkConf, + hadoopConf: Configuration): Boolean = { + val protocol = sparkConf.get(Kafka.SECURITY_PROTOCOL) + sparkConf.contains(Kafka.BOOTSTRAP_SERVERS) && + (protocol == SASL_SSL.name || + protocol == SSL.name || + protocol == SASL_PLAINTEXT.name) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala b/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala new file mode 100644 index 000000000000..c890cee59ffe --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.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.deploy.security + +import java.{ util => ju } +import java.text.SimpleDateFormat + +import scala.util.control.NonFatal + +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.clients.admin.{AdminClient, CreateDelegationTokenOptions} +import org.apache.kafka.common.config.SaslConfigs +import org.apache.kafka.common.security.JaasContext +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} +import org.apache.kafka.common.security.token.delegation.DelegationToken + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ + +private[spark] object KafkaTokenUtil extends Logging { + val TOKEN_KIND = new Text("KAFKA_DELEGATION_TOKEN") + val TOKEN_SERVICE = new Text("kafka.server.delegation.token") + + private[spark] class KafkaDelegationTokenIdentifier extends AbstractDelegationTokenIdentifier { + override def getKind: Text = TOKEN_KIND + } + + private[security] def obtainToken(sparkConf: SparkConf): (Token[_ <: TokenIdentifier], Long) = { + val adminClient = AdminClient.create(createAdminClientProperties(sparkConf)) + val createDelegationTokenOptions = new CreateDelegationTokenOptions() + val createResult = adminClient.createDelegationToken(createDelegationTokenOptions) + val token = createResult.delegationToken().get() + printToken(token) + + (new Token[KafkaDelegationTokenIdentifier]( + token.tokenInfo.tokenId.getBytes, + token.hmacAsBase64String.getBytes, + TOKEN_KIND, + TOKEN_SERVICE + ), token.tokenInfo.expiryTimestamp) + } + + private[security] def createAdminClientProperties(sparkConf: SparkConf): ju.Properties = { + val adminClientProperties = new ju.Properties + + val bootstrapServers = sparkConf.get(Kafka.BOOTSTRAP_SERVERS) + require(bootstrapServers.nonEmpty, s"Tried to obtain kafka delegation token but bootstrap " + + "servers not configured.") + adminClientProperties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers.get) + + val protocol = sparkConf.get(Kafka.SECURITY_PROTOCOL) + adminClientProperties.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, protocol) + protocol match { + case SASL_SSL.name => + setTrustStoreProperties(sparkConf, adminClientProperties) + + case SSL.name => + setTrustStoreProperties(sparkConf, adminClientProperties) + setKeyStoreProperties(sparkConf, adminClientProperties) + logWarning("Obtaining kafka delegation token with SSL protocol. Please " + + "configure 2-way authentication on the broker side.") + + case SASL_PLAINTEXT.name => + logWarning("Obtaining kafka delegation token through plain communication channel. Please " + + "consider the security impact.") + } + + // There are multiple possibilities to log in and applied in the following order: + // - JVM global security provided -> try to log in with JVM global security configuration + // which can be configured for example with 'java.security.auth.login.config'. + // For this no additional parameter needed. + // - Keytab is provided -> try to log in with kerberos module and keytab using kafka's dynamic + // JAAS configuration. + // - Keytab not provided -> try to log in with kerberos module and ticket cache using kafka's + // dynamic JAAS configuration. + // Kafka client is unable to use subject from JVM which already logged in + // to kdc (see KAFKA-7677) + if (isGlobalJaasConfigurationProvided) { + logDebug("JVM global security configuration detected, using it for login.") + } else { + adminClientProperties.put(SaslConfigs.SASL_MECHANISM, SaslConfigs.GSSAPI_MECHANISM) + if (sparkConf.contains(KEYTAB)) { + logDebug("Keytab detected, using it for login.") + val jaasParams = getKeytabJaasParams(sparkConf) + adminClientProperties.put(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + } else { + logDebug("Using ticket cache for login.") + val jaasParams = getTicketCacheJaasParams(sparkConf) + adminClientProperties.put(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + } + } + + adminClientProperties + } + + def isGlobalJaasConfigurationProvided: Boolean = { + try { + JaasContext.loadClientContext(ju.Collections.emptyMap[String, Object]()) + true + } catch { + case NonFatal(_) => false + } + } + + private def setTrustStoreProperties(sparkConf: SparkConf, properties: ju.Properties): Unit = { + sparkConf.get(Kafka.TRUSTSTORE_LOCATION).foreach { truststoreLocation => + properties.put("ssl.truststore.location", truststoreLocation) + } + sparkConf.get(Kafka.TRUSTSTORE_PASSWORD).foreach { truststorePassword => + properties.put("ssl.truststore.password", truststorePassword) + } + } + + private def setKeyStoreProperties(sparkConf: SparkConf, properties: ju.Properties): Unit = { + sparkConf.get(Kafka.KEYSTORE_LOCATION).foreach { keystoreLocation => + properties.put("ssl.keystore.location", keystoreLocation) + } + sparkConf.get(Kafka.KEYSTORE_PASSWORD).foreach { keystorePassword => + properties.put("ssl.keystore.password", keystorePassword) + } + sparkConf.get(Kafka.KEY_PASSWORD).foreach { keyPassword => + properties.put("ssl.key.password", keyPassword) + } + } + + private[security] def getKeytabJaasParams(sparkConf: SparkConf): String = { + val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) + require(serviceName.nonEmpty, "Kerberos service name must be defined") + + val params = + s""" + |${getKrb5LoginModuleName} required + | useKeyTab=true + | serviceName="${serviceName.get}" + | keyTab="${sparkConf.get(KEYTAB).get}" + | principal="${sparkConf.get(PRINCIPAL).get}"; + """.stripMargin.replace("\n", "") + logDebug(s"Krb keytab JAAS params: $params") + params + } + + def getTicketCacheJaasParams(sparkConf: SparkConf): String = { + val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) + require(serviceName.nonEmpty, "Kerberos service name must be defined") + + val params = + s""" + |${getKrb5LoginModuleName} required + | useTicketCache=true + | serviceName="${serviceName.get}"; + """.stripMargin.replace("\n", "") + logDebug(s"Krb ticket cache JAAS params: $params") + params + } + + /** + * Krb5LoginModule package vary in different JVMs. + * Please see Hadoop UserGroupInformation for further details. + */ + private def getKrb5LoginModuleName(): String = { + if (System.getProperty("java.vendor").contains("IBM")) { + "com.ibm.security.auth.module.Krb5LoginModule" + } else { + "com.sun.security.auth.module.Krb5LoginModule" + } + } + + private def printToken(token: DelegationToken): Unit = { + if (log.isDebugEnabled) { + val dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm") + logDebug("%-15s %-30s %-15s %-25s %-15s %-15s %-15s".format( + "TOKENID", "HMAC", "OWNER", "RENEWERS", "ISSUEDATE", "EXPIRYDATE", "MAXDATE")) + val tokenInfo = token.tokenInfo + logDebug("%-15s [hidden] %-15s %-25s %-15s %-15s %-15s".format( + tokenInfo.tokenId, + tokenInfo.owner, + tokenInfo.renewersAsString, + dateFormat.format(tokenInfo.issueTimestamp), + dateFormat.format(tokenInfo.expiryTimestamp), + dateFormat.format(tokenInfo.maxTimestamp))) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala b/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala new file mode 100644 index 000000000000..85d74c27142a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Kafka.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.internal.config + +private[spark] object Kafka { + + val BOOTSTRAP_SERVERS = + ConfigBuilder("spark.kafka.bootstrap.servers") + .doc("A list of coma separated host/port pairs to use for establishing the initial " + + "connection to the Kafka cluster. For further details please see kafka documentation. " + + "Only used to obtain delegation token.") + .stringConf + .createOptional + + val SECURITY_PROTOCOL = + ConfigBuilder("spark.kafka.security.protocol") + .doc("Protocol used to communicate with brokers. For further details please see kafka " + + "documentation. Only used to obtain delegation token.") + .stringConf + .createWithDefault("SASL_SSL") + + val KERBEROS_SERVICE_NAME = + ConfigBuilder("spark.kafka.sasl.kerberos.service.name") + .doc("The Kerberos principal name that Kafka runs as. This can be defined either in " + + "Kafka's JAAS config or in Kafka's config. For further details please see kafka " + + "documentation. Only used to obtain delegation token.") + .stringConf + .createOptional + + val TRUSTSTORE_LOCATION = + ConfigBuilder("spark.kafka.ssl.truststore.location") + .doc("The location of the trust store file. For further details please see kafka " + + "documentation. Only used to obtain delegation token.") + .stringConf + .createOptional + + val TRUSTSTORE_PASSWORD = + ConfigBuilder("spark.kafka.ssl.truststore.password") + .doc("The store password for the trust store file. This is optional for client and only " + + "needed if ssl.truststore.location is configured. For further details please see kafka " + + "documentation. Only used to obtain delegation token.") + .stringConf + .createOptional + + val KEYSTORE_LOCATION = + ConfigBuilder("spark.kafka.ssl.keystore.location") + .doc("The location of the key store file. This is optional for client and can be used for " + + "two-way authentication for client. For further details please see kafka documentation. " + + "Only used to obtain delegation token.") + .stringConf + .createOptional + + val KEYSTORE_PASSWORD = + ConfigBuilder("spark.kafka.ssl.keystore.password") + .doc("The store password for the key store file. This is optional for client and only " + + "needed if ssl.keystore.location is configured. For further details please see kafka " + + "documentation. Only used to obtain delegation token.") + .stringConf + .createOptional + + val KEY_PASSWORD = + ConfigBuilder("spark.kafka.ssl.key.password") + .doc("The password of the private key in the key store file. This is optional for client. " + + "For further details please see kafka documentation. Only used to obtain delegation token.") + .stringConf + .createOptional +} diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index e0e630e3be63..def9e626a2df 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.deploy.security import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.security.Credentials import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.util.Utils @@ -33,6 +31,7 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { assert(manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) assert(manager.isProviderLoaded("hive")) + assert(manager.isProviderLoaded("kafka")) } test("disable hive credential provider") { @@ -41,6 +40,7 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { assert(manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) assert(!manager.isProviderLoaded("hive")) + assert(manager.isProviderLoaded("kafka")) } test("using deprecated configurations") { @@ -51,6 +51,7 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { assert(!manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) assert(!manager.isProviderLoaded("hive")) + assert(manager.isProviderLoaded("kafka")) } test("SPARK-23209: obtain tokens when Hive classes are not available") { diff --git a/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala new file mode 100644 index 000000000000..682bebde916f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.security + +import java.{ util => ju } +import javax.security.auth.login.{AppConfigurationEntry, Configuration} + +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.SaslConfigs +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ + +class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { + private val bootStrapServers = "127.0.0.1:0" + private val trustStoreLocation = "/path/to/trustStore" + private val trustStorePassword = "trustStoreSecret" + private val keyStoreLocation = "/path/to/keyStore" + private val keyStorePassword = "keyStoreSecret" + private val keyPassword = "keySecret" + private val keytab = "/path/to/keytab" + private val kerberosServiceName = "kafka" + private val principal = "user@domain.com" + + private var sparkConf: SparkConf = null + + private class KafkaJaasConfiguration extends Configuration { + val entry = + new AppConfigurationEntry( + "DummyModule", + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + ju.Collections.emptyMap[String, Object]() + ) + + override def getAppConfigurationEntry(name: String): Array[AppConfigurationEntry] = { + if (name.equals("KafkaClient")) { + Array(entry) + } else { + null + } + } + } + + override def beforeEach(): Unit = { + super.beforeEach() + sparkConf = new SparkConf() + } + + override def afterEach(): Unit = { + try { + resetGlobalConfig() + } finally { + super.afterEach() + } + } + + private def setGlobalKafkaClientConfig(): Unit = { + Configuration.setConfiguration(new KafkaJaasConfiguration) + } + + private def resetGlobalConfig(): Unit = { + Configuration.setConfiguration(null) + } + + test("createAdminClientProperties without bootstrap servers should throw exception") { + val thrown = intercept[IllegalArgumentException] { + KafkaTokenUtil.createAdminClientProperties(sparkConf) + } + assert(thrown.getMessage contains + "Tried to obtain kafka delegation token but bootstrap servers not configured.") + } + + test("createAdminClientProperties with SASL_PLAINTEXT protocol should not include " + + "keystore and truststore config") { + sparkConf.set(Kafka.BOOTSTRAP_SERVERS, bootStrapServers) + sparkConf.set(Kafka.SECURITY_PROTOCOL, SASL_PLAINTEXT.name) + sparkConf.set(Kafka.TRUSTSTORE_LOCATION, trustStoreLocation) + sparkConf.set(Kafka.TRUSTSTORE_PASSWORD, trustStoreLocation) + sparkConf.set(Kafka.KEYSTORE_LOCATION, keyStoreLocation) + sparkConf.set(Kafka.KEYSTORE_PASSWORD, keyStorePassword) + sparkConf.set(Kafka.KEY_PASSWORD, keyPassword) + sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_PLAINTEXT.name) + assert(!adminClientProperties.containsKey("ssl.truststore.location")) + assert(!adminClientProperties.containsKey("ssl.truststore.password")) + assert(!adminClientProperties.containsKey("ssl.keystore.location")) + assert(!adminClientProperties.containsKey("ssl.keystore.password")) + assert(!adminClientProperties.containsKey("ssl.key.password")) + } + + test("createAdminClientProperties with SASL_SSL protocol should include truststore config") { + sparkConf.set(Kafka.BOOTSTRAP_SERVERS, bootStrapServers) + sparkConf.set(Kafka.SECURITY_PROTOCOL, SASL_SSL.name) + sparkConf.set(Kafka.TRUSTSTORE_LOCATION, trustStoreLocation) + sparkConf.set(Kafka.TRUSTSTORE_PASSWORD, trustStorePassword) + sparkConf.set(Kafka.KEYSTORE_LOCATION, keyStoreLocation) + sparkConf.set(Kafka.KEYSTORE_PASSWORD, keyStorePassword) + sparkConf.set(Kafka.KEY_PASSWORD, keyPassword) + sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_SSL.name) + assert(adminClientProperties.get("ssl.truststore.location") === trustStoreLocation) + assert(adminClientProperties.get("ssl.truststore.password") === trustStorePassword) + assert(!adminClientProperties.containsKey("ssl.keystore.location")) + assert(!adminClientProperties.containsKey("ssl.keystore.password")) + assert(!adminClientProperties.containsKey("ssl.key.password")) + } + + test("createAdminClientProperties with SSL protocol should include keystore and truststore " + + "config") { + sparkConf.set(Kafka.BOOTSTRAP_SERVERS, bootStrapServers) + sparkConf.set(Kafka.SECURITY_PROTOCOL, SSL.name) + sparkConf.set(Kafka.TRUSTSTORE_LOCATION, trustStoreLocation) + sparkConf.set(Kafka.TRUSTSTORE_PASSWORD, trustStorePassword) + sparkConf.set(Kafka.KEYSTORE_LOCATION, keyStoreLocation) + sparkConf.set(Kafka.KEYSTORE_PASSWORD, keyStorePassword) + sparkConf.set(Kafka.KEY_PASSWORD, keyPassword) + sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SSL.name) + assert(adminClientProperties.get("ssl.truststore.location") === trustStoreLocation) + assert(adminClientProperties.get("ssl.truststore.password") === trustStorePassword) + assert(adminClientProperties.get("ssl.keystore.location") === keyStoreLocation) + assert(adminClientProperties.get("ssl.keystore.password") === keyStorePassword) + assert(adminClientProperties.get("ssl.key.password") === keyPassword) + } + + test("createAdminClientProperties with global config should not set dynamic jaas config") { + sparkConf.set(Kafka.BOOTSTRAP_SERVERS, bootStrapServers) + sparkConf.set(Kafka.SECURITY_PROTOCOL, SASL_SSL.name) + setGlobalKafkaClientConfig() + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_SSL.name) + assert(!adminClientProperties.containsKey(SaslConfigs.SASL_MECHANISM)) + assert(!adminClientProperties.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) + } + + test("createAdminClientProperties with keytab should set keytab dynamic jaas config") { + sparkConf.set(Kafka.BOOTSTRAP_SERVERS, bootStrapServers) + sparkConf.set(Kafka.SECURITY_PROTOCOL, SASL_SSL.name) + sparkConf.set(KEYTAB, keytab) + sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) + sparkConf.set(PRINCIPAL, principal) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_SSL.name) + assert(adminClientProperties.containsKey(SaslConfigs.SASL_MECHANISM)) + val saslJaasConfig = adminClientProperties.getProperty(SaslConfigs.SASL_JAAS_CONFIG) + assert(saslJaasConfig.contains("Krb5LoginModule required")) + assert(saslJaasConfig.contains("useKeyTab=true")) + } + + test("createAdminClientProperties without keytab should set ticket cache dynamic jaas config") { + sparkConf.set(Kafka.BOOTSTRAP_SERVERS, bootStrapServers) + sparkConf.set(Kafka.SECURITY_PROTOCOL, SASL_SSL.name) + sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_SSL.name) + assert(adminClientProperties.containsKey(SaslConfigs.SASL_MECHANISM)) + val saslJaasConfig = adminClientProperties.getProperty(SaslConfigs.SASL_JAAS_CONFIG) + assert(saslJaasConfig.contains("Krb5LoginModule required")) + assert(saslJaasConfig.contains("useTicketCache=true")) + } + + test("isGlobalJaasConfigurationProvided without global config should return false") { + assert(!KafkaTokenUtil.isGlobalJaasConfigurationProvided) + } + + test("isGlobalJaasConfigurationProvided with global config should return false") { + setGlobalKafkaClientConfig() + + assert(KafkaTokenUtil.isGlobalJaasConfigurationProvided) + } + + test("getKeytabJaasParams with keytab no service should throw exception") { + sparkConf.set(KEYTAB, keytab) + + val thrown = intercept[IllegalArgumentException] { + KafkaTokenUtil.getKeytabJaasParams(sparkConf) + } + + assert(thrown.getMessage contains "Kerberos service name must be defined") + } + + test("getTicketCacheJaasParams without service should throw exception") { + val thrown = intercept[IllegalArgumentException] { + KafkaTokenUtil.getTicketCacheJaasParams(sparkConf) + } + + assert(thrown.getMessage contains "Kerberos service name must be defined") + } +} diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 1af407167597..de8731c4b774 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -29,8 +29,6 @@ spark-sql-kafka-0-10_2.12 sql-kafka-0-10 - - 2.1.0 jar Kafka 0.10+ Source for Structured Streaming diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala new file mode 100644 index 000000000000..74d5ef9c05f1 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.kafka.common.security.scram.ScramLoginModule + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.security.KafkaTokenUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ + +private[kafka010] object KafkaSecurityHelper extends Logging { + def isTokenAvailable(): Boolean = { + UserGroupInformation.getCurrentUser().getCredentials.getToken( + KafkaTokenUtil.TOKEN_SERVICE) != null + } + + def getTokenJaasParams(sparkConf: SparkConf): String = { + val token = UserGroupInformation.getCurrentUser().getCredentials.getToken( + KafkaTokenUtil.TOKEN_SERVICE) + val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) + require(serviceName.isDefined, "Kerberos service name must be defined") + val username = new String(token.getIdentifier) + val password = new String(token.getPassword) + + val loginModuleName = classOf[ScramLoginModule].getName + val params = + s""" + |$loginModuleName required + | tokenauth=true + | serviceName="${serviceName.get}" + | username="$username" + | password="$password"; + """.stripMargin.replace("\n", "") + logDebug(s"Scram JAAS params: ${params.replaceAll("password=\".*\"", "password=\"[hidden]\"")}") + + params + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index f770f0c2a04c..0ac330435e5c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -18,16 +18,19 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.util.{Locale, Optional, UUID} +import java.util.{Locale, UUID} import scala.collection.JavaConverters._ import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} +import org.apache.spark.SparkEnv +import org.apache.spark.deploy.security.KafkaTokenUtil import org.apache.spark.internal.Logging -import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SparkSession, SQLContext} +import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ import org.apache.spark.sql.sources.v2._ @@ -80,12 +83,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val uniqueGroupId = streamingUniqueGroupId(parameters, metadataPath) val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + val specifiedKafkaParams = convertToSpecifiedParams(parameters) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) @@ -122,12 +120,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val uniqueGroupId = streamingUniqueGroupId(parameters, metadataPath) val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + val specifiedKafkaParams = convertToSpecifiedParams(parameters) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) @@ -198,12 +191,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters: Map[String, String]): BaseRelation = { validateBatchOptions(parameters) val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + val specifiedKafkaParams = convertToSpecifiedParams(parameters) val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) @@ -230,8 +218,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister outputMode: OutputMode): Sink = { val defaultTopic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) val specifiedKafkaParams = kafkaParamsForProducer(parameters) - new KafkaSink(sqlContext, - new ju.HashMap[String, Object](specifiedKafkaParams.asJava), defaultTopic) + new KafkaSink(sqlContext, specifiedKafkaParams, defaultTopic) } override def createRelation( @@ -248,8 +235,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } val topic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) val specifiedKafkaParams = kafkaParamsForProducer(parameters) - KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, - new ju.HashMap[String, Object](specifiedKafkaParams.asJava), topic) + KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, specifiedKafkaParams, + topic) /* This method is suppose to return a relation that reads the data that was written. * We cannot support this for Kafka. Therefore, in order to make things consistent, @@ -274,13 +261,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister options: DataSourceOptions): StreamingWriteSupport = { import scala.collection.JavaConverters._ - val spark = SparkSession.getActiveSession.get val topic = Option(options.get(TOPIC_OPTION_KEY).orElse(null)).map(_.trim) // We convert the options argument from V2 -> Java map -> scala mutable -> scala immutable. val producerParams = kafkaParamsForProducer(options.asMap.asScala.toMap) - KafkaWriter.validateQuery( - schema.toAttributes, new java.util.HashMap[String, Object](producerParams.asJava), topic) + KafkaWriter.validateQuery(schema.toAttributes, producerParams, topic) new KafkaStreamingWriteSupport(topic, producerParams, schema) } @@ -481,6 +466,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { + private val serClassName = classOf[ByteArraySerializer].getName private val deserClassName = classOf[ByteArrayDeserializer].getName def getKafkaOffsetRangeLimit( @@ -515,6 +501,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { // If buffer config is not set, set it to reasonable value to work around // buffer issues (see KAFKA-3135) .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .setTokenJaasConfigIfNeeded() .build() def kafkaParamsForExecutors( @@ -536,6 +523,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { // If buffer config is not set, set it to reasonable value to work around // buffer issues (see KAFKA-3135) .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .setTokenJaasConfigIfNeeded() .build() /** @@ -568,11 +556,32 @@ private[kafka010] object KafkaSourceProvider extends Logging { this } + def setTokenJaasConfigIfNeeded(): ConfigUpdater = { + // There are multiple possibilities to log in and applied in the following order: + // - JVM global security provided -> try to log in with JVM global security configuration + // which can be configured for example with 'java.security.auth.login.config'. + // For this no additional parameter needed. + // - Token is provided -> try to log in with scram module using kafka's dynamic JAAS + // configuration. + if (KafkaTokenUtil.isGlobalJaasConfigurationProvided) { + logDebug("JVM global security configuration detected, using it for login.") + } else if (KafkaSecurityHelper.isTokenAvailable()) { + logDebug("Delegation token detected, using it for login.") + val jaasParams = KafkaSecurityHelper.getTokenJaasParams(SparkEnv.get.conf) + val mechanism = kafkaParams + .getOrElse(SaslConfigs.SASL_MECHANISM, SaslConfigs.DEFAULT_SASL_MECHANISM) + require(mechanism.startsWith("SCRAM"), + "Delegation token works only with SCRAM mechanism.") + set(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + } + this + } + def build(): ju.Map[String, Object] = map } private[kafka010] def kafkaParamsForProducer( - parameters: Map[String, String]): Map[String, String] = { + parameters: Map[String, String]): ju.Map[String, Object] = { val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( @@ -580,17 +589,26 @@ private[kafka010] object KafkaSourceProvider extends Logging { + "are serialized with ByteArraySerializer.") } - if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) - { + if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}' is not supported as " + "value are serialized with ByteArraySerializer.") } + + val specifiedKafkaParams = convertToSpecifiedParams(parameters) + + ConfigUpdater("executor", specifiedKafkaParams) + .set(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, serClassName) + .set(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, serClassName) + .setTokenJaasConfigIfNeeded() + .build() + } + + private def convertToSpecifiedParams(parameters: Map[String, String]): Map[String, String] = { parameters .keySet .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) .map { k => k.drop(6).toString -> parameters(k) } - .toMap + (ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName, - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName) + .toMap } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala index 927c56d9ce82..0d831c388460 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.kafka010 -import scala.collection.JavaConverters._ +import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute @@ -41,10 +41,12 @@ case object KafkaWriterCommitMessage extends WriterCommitMessage * @param schema The schema of the input data. */ class KafkaStreamingWriteSupport( - topic: Option[String], producerParams: Map[String, String], schema: StructType) + topic: Option[String], + producerParams: ju.Map[String, Object], + schema: StructType) extends StreamingWriteSupport { - validateQuery(schema.toAttributes, producerParams.toMap[String, Object].asJava, topic) + validateQuery(schema.toAttributes, producerParams, topic) override def createStreamingWriterFactory(): KafkaStreamWriterFactory = KafkaStreamWriterFactory(topic, producerParams, schema) @@ -62,7 +64,9 @@ class KafkaStreamingWriteSupport( * @param schema The schema of the input data. */ case class KafkaStreamWriterFactory( - topic: Option[String], producerParams: Map[String, String], schema: StructType) + topic: Option[String], + producerParams: ju.Map[String, Object], + schema: StructType) extends StreamingDataWriterFactory { override def createWriter( @@ -83,12 +87,12 @@ case class KafkaStreamWriterFactory( * @param inputSchema The attributes in the input data. */ class KafkaStreamDataWriter( - targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) + targetTopic: Option[String], + producerParams: ju.Map[String, Object], + inputSchema: Seq[Attribute]) extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] { - import scala.collection.JavaConverters._ - private lazy val producer = CachedKafkaProducer.getOrCreate( - new java.util.HashMap[String, Object](producerParams.asJava)) + private lazy val producer = CachedKafkaProducer.getOrCreate(producerParams) def write(row: InternalRow): Unit = { checkForErrors() @@ -112,7 +116,7 @@ class KafkaStreamDataWriter( if (producer != null) { producer.flush() checkForErrors() - CachedKafkaProducer.close(new java.util.HashMap[String, Object](producerParams.asJava)) + CachedKafkaProducer.close(producerParams) } } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala index 3f6fcf6b2e52..b21037b1340c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala @@ -409,7 +409,7 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { */ val topic = newTopic() testUtils.createTopic(topic, 1) - val options = new java.util.HashMap[String, String] + val options = new java.util.HashMap[String, Object] options.put("bootstrap.servers", testUtils.brokerAddress) options.put("buffer.memory", "16384") // min buffer size options.put("block.on.buffer.full", "true") @@ -417,7 +417,7 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { options.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer].getName) val inputSchema = Seq(AttributeReference("value", BinaryType)()) val data = new Array[Byte](15000) // large value - val writeTask = new KafkaStreamDataWriter(Some(topic), options.asScala.toMap, inputSchema) + val writeTask = new KafkaStreamDataWriter(Some(topic), options, inputSchema) try { val fieldTypes: Array[DataType] = Array(BinaryType) val converter = UnsafeProjection.create(fieldTypes) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala new file mode 100644 index 000000000000..772fe4614bad --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.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.kafka010 + +import java.util.UUID + +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.Token +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.security.KafkaTokenUtil +import org.apache.spark.deploy.security.KafkaTokenUtil.KafkaDelegationTokenIdentifier +import org.apache.spark.internal.config._ + +class KafkaSecurityHelperSuite extends SparkFunSuite with BeforeAndAfterEach { + private val keytab = "/path/to/keytab" + private val kerberosServiceName = "kafka" + private val principal = "user@domain.com" + private val tokenId = "tokenId" + UUID.randomUUID().toString + private val tokenPassword = "tokenPassword" + UUID.randomUUID().toString + + private var sparkConf: SparkConf = null + + override def beforeEach(): Unit = { + super.beforeEach() + sparkConf = new SparkConf() + } + + override def afterEach(): Unit = { + try { + resetUGI + } finally { + super.afterEach() + } + } + + private def addTokenToUGI(): Unit = { + val token = new Token[KafkaDelegationTokenIdentifier]( + tokenId.getBytes, + tokenPassword.getBytes, + KafkaTokenUtil.TOKEN_KIND, + KafkaTokenUtil.TOKEN_SERVICE + ) + val creds = new Credentials() + creds.addToken(KafkaTokenUtil.TOKEN_SERVICE, token) + UserGroupInformation.getCurrentUser.addCredentials(creds) + } + + private def resetUGI: Unit = { + UserGroupInformation.setLoginUser(null) + } + + test("isTokenAvailable without token should return false") { + assert(!KafkaSecurityHelper.isTokenAvailable()) + } + + test("isTokenAvailable with token should return true") { + addTokenToUGI() + + assert(KafkaSecurityHelper.isTokenAvailable()) + } + + test("getTokenJaasParams with token no service should throw exception") { + addTokenToUGI() + + val thrown = intercept[IllegalArgumentException] { + KafkaSecurityHelper.getTokenJaasParams(sparkConf) + } + + assert(thrown.getMessage contains "Kerberos service name must be defined") + } + + test("getTokenJaasParams with token should return scram module") { + addTokenToUGI() + sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) + + val jaasParams = KafkaSecurityHelper.getTokenJaasParams(sparkConf) + + assert(jaasParams.contains("ScramLoginModule required")) + assert(jaasParams.contains("tokenauth=true")) + assert(jaasParams.contains(tokenId)) + assert(jaasParams.contains(tokenPassword)) + } +} diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index ea18b7e03591..333572e99b1c 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -28,8 +28,6 @@ spark-streaming-kafka-0-10_2.12 streaming-kafka-0-10 - - 2.1.0 jar Spark Integration for Kafka 0.10 diff --git a/pom.xml b/pom.xml index 3ca2f739ce0e..dfc3c540dc18 100644 --- a/pom.xml +++ b/pom.xml @@ -128,6 +128,8 @@ 1.2.1.spark2 1.2.1 + + 2.1.0 10.12.1.1 1.10.0 1.5.3 From 66b2046462c0e93b2ca167728eba9f4d13a5a67c Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 30 Nov 2018 10:29:30 +0800 Subject: [PATCH 0027/1072] [SPARK-25446][R] Add schema_of_json() and schema_of_csv() to R ## What changes were proposed in this pull request? This PR proposes to expose `schema_of_json` and `schema_of_csv` at R side. **`schema_of_json`**: ```r json <- '{"name":"Bob"}' df <- sql("SELECT * FROM range(1)") head(select(df, schema_of_json(json))) ``` ``` schema_of_json({"name":"Bob"}) 1 struct ``` **`schema_of_csv`**: ```r csv <- "Amsterdam,2018" df <- sql("SELECT * FROM range(1)") head(select(df, schema_of_csv(csv))) ``` ``` schema_of_csv(Amsterdam,2018) 1 struct<_c0:string,_c1:int> ``` ## How was this patch tested? Manually tested, unit tests added, documentation manually built and verified. Closes #22939 from HyukjinKwon/SPARK-25446. Authored-by: hyukjinkwon Signed-off-by: hyukjinkwon --- R/pkg/NAMESPACE | 2 + R/pkg/R/functions.R | 77 ++++++++++++++++++++++++--- R/pkg/R/generics.R | 8 +++ R/pkg/tests/fulltests/test_sparkSQL.R | 16 +++++- 4 files changed, 94 insertions(+), 9 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index cdeafdd90ce4..1f8ba0bcf1cf 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -351,6 +351,8 @@ exportMethods("%<=>%", "row_number", "rpad", "rtrim", + "schema_of_csv", + "schema_of_json", "second", "sha1", "sha2", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index f72645a25779..f568a931ae1f 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -205,11 +205,18 @@ NULL #' also supported for the schema. #' \item \code{from_csv}: a DDL-formatted string #' } -#' @param ... additional argument(s). In \code{to_json}, \code{to_csv} and \code{from_json}, -#' this contains additional named properties to control how it is converted, accepts -#' the same options as the JSON/CSV data source. Additionally \code{to_json} supports -#' the "pretty" option which enables pretty JSON generation. In \code{arrays_zip}, -#' this contains additional Columns of arrays to be merged. +#' @param ... additional argument(s). +#' \itemize{ +#' \item \code{to_json}, \code{from_json} and \code{schema_of_json}: this contains +#' additional named properties to control how it is converted and accepts the +#' same options as the JSON data source. +#' \item \code{to_json}: it supports the "pretty" option which enables pretty +#' JSON generation. +#' \item \code{to_csv}, \code{from_csv} and \code{schema_of_csv}: this contains +#' additional named properties to control how it is converted and accepts the +#' same options as the CSV data source. +#' \item \code{arrays_zip}, this contains additional Columns of arrays to be merged. +#' } #' @name column_collection_functions #' @rdname column_collection_functions #' @family collection functions @@ -1771,12 +1778,16 @@ setMethod("to_date", #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' #' # Converts a map into a JSON object -#' df2 <- sql("SELECT map('name', 'Bob')) as people") +#' df2 <- sql("SELECT map('name', 'Bob') as people") #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' #' # Converts an array of maps into a JSON array #' df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people") -#' df2 <- mutate(df2, people_json = to_json(df2$people))} +#' df2 <- mutate(df2, people_json = to_json(df2$people)) +#' +#' # Converts a map into a pretty JSON object +#' df2 <- sql("SELECT map('name', 'Bob') as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people, pretty = TRUE))} #' @note to_json since 2.2.0 setMethod("to_json", signature(x = "Column"), function(x, ...) { @@ -2285,6 +2296,32 @@ setMethod("from_json", signature(x = "Column", schema = "characterOrstructType") column(jc) }) +#' @details +#' \code{schema_of_json}: Parses a JSON string and infers its schema in DDL format. +#' +#' @rdname column_collection_functions +#' @aliases schema_of_json schema_of_json,characterOrColumn-method +#' @examples +#' +#' \dontrun{ +#' json <- "{\"name\":\"Bob\"}" +#' df <- sql("SELECT * FROM range(1)") +#' head(select(df, schema_of_json(json)))} +#' @note schema_of_json since 3.0.0 +setMethod("schema_of_json", signature(x = "characterOrColumn"), + function(x, ...) { + if (class(x) == "character") { + col <- callJStatic("org.apache.spark.sql.functions", "lit", x) + } else { + col <- x@jc + } + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", + "schema_of_json", + col, options) + column(jc) + }) + #' @details #' \code{from_csv}: Parses a column containing a CSV string into a Column of \code{structType} #' with the specified \code{schema}. @@ -2315,6 +2352,32 @@ setMethod("from_csv", signature(x = "Column", schema = "characterOrColumn"), column(jc) }) +#' @details +#' \code{schema_of_csv}: Parses a CSV string and infers its schema in DDL format. +#' +#' @rdname column_collection_functions +#' @aliases schema_of_csv schema_of_csv,characterOrColumn-method +#' @examples +#' +#' \dontrun{ +#' csv <- "Amsterdam,2018" +#' df <- sql("SELECT * FROM range(1)") +#' head(select(df, schema_of_csv(csv)))} +#' @note schema_of_csv since 3.0.0 +setMethod("schema_of_csv", signature(x = "characterOrColumn"), + function(x, ...) { + if (class(x) == "character") { + col <- callJStatic("org.apache.spark.sql.functions", "lit", x) + } else { + col <- x@jc + } + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", + "schema_of_csv", + col, options) + column(jc) + }) + #' @details #' \code{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 diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index b2ca6e62175e..9d8c24c686c7 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1206,6 +1206,14 @@ setGeneric("rpad", function(x, len, pad) { standardGeneric("rpad") }) #' @name NULL setGeneric("rtrim", function(x, trimString) { standardGeneric("rtrim") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("schema_of_csv", function(x, ...) { standardGeneric("schema_of_csv") }) + +#' @rdname column_collection_functions +#' @name NULL +setGeneric("schema_of_json", function(x, ...) { standardGeneric("schema_of_json") }) + #' @rdname column_aggregate_functions #' @name NULL setGeneric("sd", function(x, na.rm = FALSE) { standardGeneric("sd") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 77a29c9ecad8..0d5118c127f2 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1620,14 +1620,20 @@ test_that("column functions", { expect_equal(collect(select(df, bround(df$x, 0)))[[1]][1], 2) expect_equal(collect(select(df, bround(df$x, 0)))[[1]][2], 4) - # Test from_csv() + # Test from_csv(), schema_of_csv() df <- as.DataFrame(list(list("col" = "1"))) c <- collect(select(df, alias(from_csv(df$col, "a INT"), "csv"))) expect_equal(c[[1]][[1]]$a, 1) c <- collect(select(df, alias(from_csv(df$col, lit("a INT")), "csv"))) expect_equal(c[[1]][[1]]$a, 1) - # Test to_json(), from_json() + df <- as.DataFrame(list(list("col" = "1"))) + c <- collect(select(df, schema_of_csv("Amsterdam,2018"))) + expect_equal(c[[1]], "struct<_c0:string,_c1:int>") + c <- collect(select(df, schema_of_csv(lit("Amsterdam,2018")))) + expect_equal(c[[1]], "struct<_c0:string,_c1:int>") + + # Test to_json(), from_json(), schema_of_json() df <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people") j <- collect(select(df, alias(to_json(df$people), "json"))) expect_equal(j[order(j$json), ][1], "[{\"name\":\"Bob\"},{\"name\":\"Alice\"}]") @@ -1654,6 +1660,12 @@ test_that("column functions", { expect_true(any(apply(s, 1, function(x) { x[[1]]$age == 16 }))) } + df <- as.DataFrame(list(list("col" = "1"))) + c <- collect(select(df, schema_of_json('{"name":"Bob"}'))) + expect_equal(c[[1]], "struct") + c <- collect(select(df, schema_of_json(lit('{"name":"Bob"}')))) + expect_equal(c[[1]], "struct") + # Test to_json() supports arrays of primitive types and arrays df <- sql("SELECT array(19, 42, 70) as age") j <- collect(select(df, alias(to_json(df$age), "json"))) From 8edb64c1b9ee49d836e171a459dd93f524df92bf Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 30 Nov 2018 11:56:25 +0800 Subject: [PATCH 0028/1072] [SPARK-26060][SQL] Track SparkConf entries and make SET command reject such entries. ## What changes were proposed in this pull request? Currently the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. We should track `SparkConf` entries and make the command reject for such entries. ## How was this patch tested? Added a test and existing tests. Closes #23031 from ueshin/issues/SPARK-26060/set_command. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan --- docs/sql-migration-guide-upgrade.md | 2 ++ .../org/apache/spark/sql/internal/SQLConf.scala | 12 +++++++++++- .../scala/org/apache/spark/sql/RuntimeConfig.scala | 4 ++++ .../org/apache/spark/sql/RuntimeConfigSuite.scala | 10 ++++++++++ .../spark/sql/execution/command/DDLSuite.scala | 8 ++++++++ 5 files changed, 35 insertions(+), 1 deletion(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 55838e773e4b..e48125a0972b 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -29,6 +29,8 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be udefined. + - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.execution.setCommandRejectsSparkConfs` to `false`. + ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. 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 7bcf21595ce5..f1c845bc9450 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 @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils object SQLConf { - private val sqlConfEntries = java.util.Collections.synchronizedMap( + private[sql] val sqlConfEntries = java.util.Collections.synchronizedMap( new java.util.HashMap[String, ConfigEntry[_]]()) val staticConfKeys: java.util.Set[String] = @@ -1610,6 +1610,14 @@ object SQLConf { """ "... N more fields" placeholder.""") .intConf .createWithDefault(25) + + val SET_COMMAND_REJECTS_SPARK_CONFS = + buildConf("spark.sql.legacy.execution.setCommandRejectsSparkConfs") + .internal() + .doc("If it is set to true, SET command will fail when the key is registered as " + + "a SparkConf entry.") + .booleanConf + .createWithDefault(true) } /** @@ -2030,6 +2038,8 @@ class SQLConf extends Serializable with Logging { def maxToStringFields: Int = getConf(SQLConf.MAX_TO_STRING_FIELDS) + def setCommandRejectsSparkConfs: Boolean = getConf(SQLConf.SET_COMMAND_REJECTS_SPARK_CONFS) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 5a554eff02e3..d83a01ff9ea6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -153,5 +153,9 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { if (SQLConf.staticConfKeys.contains(key)) { throw new AnalysisException(s"Cannot modify the value of a static config: $key") } + if (sqlConf.setCommandRejectsSparkConfs && + ConfigEntry.findEntry(key) != null && !SQLConf.sqlConfEntries.containsKey(key)) { + throw new AnalysisException(s"Cannot modify the value of a Spark config: $key") + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala index cdcea09ad975..6196757eb701 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config class RuntimeConfigSuite extends SparkFunSuite { @@ -68,4 +69,13 @@ class RuntimeConfigSuite extends SparkFunSuite { assert(!conf.isModifiable("")) assert(!conf.isModifiable("invalid config parameter")) } + + test("reject SparkConf entries") { + val conf = newConf() + + val ex = intercept[AnalysisException] { + conf.set(config.CPUS_PER_TASK.key, 4) + } + assert(ex.getMessage.contains("Spark config")) + } } 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 f8d98dead2d4..9d32fb6d4696 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 @@ -24,6 +24,7 @@ import java.util.Locale import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach +import org.apache.spark.internal.config import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} @@ -2715,4 +2716,11 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } } + + test("set command rejects SparkConf entries") { + val ex = intercept[AnalysisException] { + sql(s"SET ${config.CPUS_PER_TASK.key} = 4") + } + assert(ex.getMessage.contains("Spark config")) + } } From 9cfc3ee6253bed21924424ccaadea0287a6f15f4 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 30 Nov 2018 12:00:55 +0800 Subject: [PATCH 0029/1072] [SPARK-26188][SQL] FileIndex: don't infer data types of partition columns if user specifies schema ## What changes were proposed in this pull request? This PR is to fix a regression introduced in: https://github.com/apache/spark/pull/21004/files#r236998030 If user specifies schema, Spark don't need to infer data type for of partition columns, otherwise the data type might not match with the one user provided. E.g. for partition directory `p=4d`, after data type inference the column value will be `4.0`. See https://issues.apache.org/jira/browse/SPARK-26188 for more details. Note that user specified schema **might not cover all the data columns**: ``` val schema = new StructType() .add("id", StringType) .add("ex", ArrayType(StringType)) val df = spark.read .schema(schema) .format("parquet") .load(src.toString) assert(df.schema.toList === List( StructField("ex", ArrayType(StringType)), StructField("part", IntegerType), // inferred partitionColumn dataType StructField("id", StringType))) // used user provided partitionColumn dataType ``` For the missing columns in user specified schema, Spark still need to infer their data types if `partitionColumnTypeInferenceEnabled` is enabled. To implement the partially inference, refactor `PartitioningUtils.parsePartitions` and pass the user specified schema as parameter to cast partition values. ## How was this patch tested? Add unit test. Closes #23165 from gengliangwang/fixFileIndex. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../PartitioningAwareFileIndex.scala | 47 ++----------------- .../datasources/PartitioningUtils.scala | 39 ++++++++++++--- .../datasources/FileIndexSuite.scala | 16 +++++++ .../ParquetPartitionDiscoverySuite.scala | 22 +++++++-- 4 files changed, 72 insertions(+), 52 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 cc8af7b92c45..7b0e4dbcc25f 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 @@ -126,33 +126,15 @@ abstract class PartitioningAwareFileIndex( val caseInsensitiveOptions = CaseInsensitiveMap(parameters) val timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone) - val inferredPartitionSpec = PartitioningUtils.parsePartitions( + + val caseSensitive = sparkSession.sqlContext.conf.caseSensitiveAnalysis + PartitioningUtils.parsePartitions( leafDirs, typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, basePaths = basePaths, + userSpecifiedSchema = userSpecifiedSchema, + caseSensitive = caseSensitive, timeZoneId = timeZoneId) - userSpecifiedSchema match { - case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => - val userPartitionSchema = - combineInferredAndUserSpecifiedPartitionSchema(inferredPartitionSpec) - - // we need to cast into the data type that user specified. - def castPartitionValuesToUserSchema(row: InternalRow) = { - InternalRow((0 until row.numFields).map { i => - val dt = inferredPartitionSpec.partitionColumns.fields(i).dataType - Cast( - Literal.create(row.get(i, dt), dt), - userPartitionSchema.fields(i).dataType, - Option(timeZoneId)).eval() - }: _*) - } - - PartitionSpec(userPartitionSchema, inferredPartitionSpec.partitions.map { part => - part.copy(values = castPartitionValuesToUserSchema(part.values)) - }) - case _ => - inferredPartitionSpec - } } private def prunePartitions( @@ -233,25 +215,6 @@ abstract class PartitioningAwareFileIndex( val name = path.getName !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) } - - /** - * In the read path, only managed tables by Hive provide the partition columns properly when - * initializing this class. All other file based data sources will try to infer the partitioning, - * and then cast the inferred types to user specified dataTypes if the partition columns exist - * inside `userSpecifiedSchema`, otherwise we can hit data corruption bugs like SPARK-18510, or - * inconsistent data types as reported in SPARK-21463. - * @param spec A partition inference result - * @return The PartitionSchema resolved from inference and cast according to `userSpecifiedSchema` - */ - private def combineInferredAndUserSpecifiedPartitionSchema(spec: PartitionSpec): StructType = { - val equality = sparkSession.sessionState.conf.resolver - val resolved = spec.partitionColumns.map { partitionField => - // SPARK-18510: try to get schema from userSpecifiedSchema, otherwise fallback to inferred - userSpecifiedSchema.flatMap(_.find(f => equality(f.name, partitionField.name))).getOrElse( - partitionField) - } - StructType(resolved) - } } object PartitioningAwareFileIndex { 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 3183fd30e5e0..9d2c9ba0c1a5 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -94,18 +94,34 @@ object PartitioningUtils { paths: Seq[Path], typeInference: Boolean, basePaths: Set[Path], + userSpecifiedSchema: Option[StructType], + caseSensitive: Boolean, timeZoneId: String): PartitionSpec = { - parsePartitions(paths, typeInference, basePaths, DateTimeUtils.getTimeZone(timeZoneId)) + parsePartitions(paths, typeInference, basePaths, userSpecifiedSchema, + caseSensitive, DateTimeUtils.getTimeZone(timeZoneId)) } private[datasources] def parsePartitions( paths: Seq[Path], typeInference: Boolean, basePaths: Set[Path], + userSpecifiedSchema: Option[StructType], + caseSensitive: Boolean, timeZone: TimeZone): PartitionSpec = { + val userSpecifiedDataTypes = if (userSpecifiedSchema.isDefined) { + val nameToDataType = userSpecifiedSchema.get.fields.map(f => f.name -> f.dataType).toMap + if (!caseSensitive) { + CaseInsensitiveMap(nameToDataType) + } else { + nameToDataType + } + } else { + Map.empty[String, DataType] + } + // First, we need to parse every partition's path and see if we can find partition values. val (partitionValues, optDiscoveredBasePaths) = paths.map { path => - parsePartition(path, typeInference, basePaths, timeZone) + parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, timeZone) }.unzip // We create pairs of (path -> path's partition value) here @@ -147,7 +163,7 @@ object PartitioningUtils { columnNames.zip(literals).map { case (name, Literal(_, dataType)) => // We always assume partition columns are nullable since we've no idea whether null values // will be appended in the future. - StructField(name, dataType, nullable = true) + StructField(name, userSpecifiedDataTypes.getOrElse(name, dataType), nullable = true) } } @@ -185,6 +201,7 @@ object PartitioningUtils { path: Path, typeInference: Boolean, basePaths: Set[Path], + userSpecifiedDataTypes: Map[String, DataType], timeZone: TimeZone): (Option[PartitionValues], Option[Path]) = { val columns = ArrayBuffer.empty[(String, Literal)] // Old Hadoop versions don't have `Path.isRoot` @@ -206,7 +223,7 @@ object PartitioningUtils { // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1. // Once we get the string, we try to parse it and find the partition column and value. val maybeColumn = - parsePartitionColumn(currentPath.getName, typeInference, timeZone) + parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes, timeZone) maybeColumn.foreach(columns += _) // Now, we determine if we should stop. @@ -239,6 +256,7 @@ object PartitioningUtils { private def parsePartitionColumn( columnSpec: String, typeInference: Boolean, + userSpecifiedDataTypes: Map[String, DataType], timeZone: TimeZone): Option[(String, Literal)] = { val equalSignIndex = columnSpec.indexOf('=') if (equalSignIndex == -1) { @@ -250,7 +268,16 @@ object PartitioningUtils { val rawColumnValue = columnSpec.drop(equalSignIndex + 1) assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - val literal = inferPartitionColumnValue(rawColumnValue, typeInference, timeZone) + val literal = if (userSpecifiedDataTypes.contains(columnName)) { + // SPARK-26188: if user provides corresponding column schema, get the column value without + // inference, and then cast it as user specified data type. + val columnValue = inferPartitionColumnValue(rawColumnValue, false, timeZone) + val castedValue = + Cast(columnValue, userSpecifiedDataTypes(columnName), Option(timeZone.getID)).eval() + Literal.create(castedValue, userSpecifiedDataTypes(columnName)) + } else { + inferPartitionColumnValue(rawColumnValue, typeInference, timeZone) + } Some(columnName -> literal) } } 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 49e7af4a9896..fdb0511f01a2 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 @@ -30,6 +30,7 @@ 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.test.SharedSQLContext +import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.util.{KnownSizeEstimation, SizeEstimator} class FileIndexSuite extends SharedSQLContext { @@ -49,6 +50,21 @@ class FileIndexSuite extends SharedSQLContext { } } + test("SPARK-26188: don't infer data types of partition columns if user specifies schema") { + withTempDir { dir => + val partitionDirectory = new File(dir, s"a=4d") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val path = new Path(dir.getCanonicalPath) + val schema = StructType(Seq(StructField("a", StringType, false))) + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema)) + val partitionValues = fileIndex.partitionSpec().partitions.map(_.values) + assert(partitionValues.length == 1 && partitionValues(0).numFields == 1 && + partitionValues(0).getString(0) == "4d") + } + } + test("InMemoryFileIndex: input paths are converted to qualified paths") { withTempDir { dir => val file = new File(dir, "text.txt") 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 9966ed94a839..f808ca458aaa 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 @@ -101,7 +101,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/path/a=10.5/b=hello") var exception = intercept[AssertionError] { - parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], timeZoneId) + parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], None, true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -115,6 +115,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/")), + None, + true, timeZoneId) // Valid @@ -128,6 +130,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/something=true/table")), + None, + true, timeZoneId) // Valid @@ -141,6 +145,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/table=true")), + None, + true, timeZoneId) // Invalid @@ -154,6 +160,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/")), + None, + true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -174,6 +182,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/tmp/tables/")), + None, + true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -181,13 +191,13 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partition") { def check(path: String, expected: Option[PartitionValues]): Unit = { - val actual = parsePartition(new Path(path), true, Set.empty[Path], timeZone)._1 + val actual = parsePartition(new Path(path), true, Set.empty[Path], Map.empty, timeZone)._1 assert(expected === actual) } def checkThrows[T <: Throwable: Manifest](path: String, expected: String): Unit = { val message = intercept[T] { - parsePartition(new Path(path), true, Set.empty[Path], timeZone) + parsePartition(new Path(path), true, Set.empty[Path], Map.empty, timeZone) }.getMessage assert(message.contains(expected)) @@ -231,6 +241,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha path = new Path("file://path/a=10"), typeInference = true, basePaths = Set(new Path("file://path/a=10")), + Map.empty, timeZone = timeZone)._1 assert(partitionSpec1.isEmpty) @@ -240,6 +251,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha path = new Path("file://path/a=10"), typeInference = true, basePaths = Set(new Path("file://path")), + Map.empty, timeZone = timeZone)._1 assert(partitionSpec2 == @@ -258,6 +270,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, rootPaths, + None, + true, timeZoneId) assert(actualSpec.partitionColumns === spec.partitionColumns) assert(actualSpec.partitions.length === spec.partitions.length) @@ -370,7 +384,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partitions with type inference disabled") { def check(paths: Seq[String], spec: PartitionSpec): Unit = { val actualSpec = - parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], timeZoneId) + parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], None, true, timeZoneId) assert(actualSpec === spec) } From 2b2c94a3ee89630047bcdd416a977e0d1cdb1926 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 30 Nov 2018 00:02:43 -0800 Subject: [PATCH 0030/1072] [SPARK-25528][SQL] data source v2 API refactor (batch read) ## What changes were proposed in this pull request? This is the first step of the data source v2 API refactor [proposal](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing) It adds the new API for batch read, without removing the old APIs, as they are still needed for streaming sources. More concretely, it adds 1. `TableProvider`, works like an anonymous catalog 2. `Table`, represents a structured data set. 3. `ScanBuilder` and `Scan`, a logical represents of data source scan 4. `Batch`, a physical representation of data source batch scan. ## How was this patch tested? existing tests Closes #23086 from cloud-fan/refactor-batch. Authored-by: Wenchen Fan Signed-off-by: gatorsmile --- .../kafka010/KafkaContinuousSourceSuite.scala | 4 +- .../sql/kafka010/KafkaContinuousTest.scala | 4 +- project/MimaExcludes.scala | 48 ++-- .../sql/sources/v2/SupportsBatchRead.java | 33 +++ .../apache/spark/sql/sources/v2/Table.java | 59 ++++ .../spark/sql/sources/v2/TableProvider.java | 64 +++++ .../spark/sql/sources/v2/reader/Batch.java | 48 ++++ .../reader/OldSupportsReportPartitioning.java | 38 +++ .../reader/OldSupportsReportStatistics.java | 38 +++ .../spark/sql/sources/v2/reader/Scan.java | 68 +++++ .../sql/sources/v2/reader/ScanBuilder.java | 30 ++ .../sql/sources/v2/reader/ScanConfig.java | 4 +- .../sql/sources/v2/reader/Statistics.java | 2 +- .../v2/reader/SupportsPushDownFilters.java | 4 +- .../SupportsPushDownRequiredColumns.java | 4 +- .../v2/reader/SupportsReportPartitioning.java | 8 +- .../v2/reader/SupportsReportStatistics.java | 6 +- .../v2/reader/partitioning/Partitioning.java | 3 +- .../apache/spark/sql/DataFrameReader.scala | 36 ++- .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 90 +++--- .../datasources/v2/DataSourceV2ScanExec.scala | 68 ++--- .../datasources/v2/DataSourceV2Strategy.scala | 34 +-- .../v2/DataSourceV2StreamingScanExec.scala | 120 ++++++++ .../streaming/ProgressReporter.scala | 4 +- .../continuous/ContinuousExecution.scala | 5 +- .../sources/v2/JavaAdvancedDataSourceV2.java | 116 ++++---- .../sources/v2/JavaColumnarDataSourceV2.java | 27 +- .../v2/JavaPartitionAwareDataSource.java | 29 +- .../v2/JavaSchemaRequiredDataSource.java | 36 ++- ...Support.java => JavaSimpleBatchTable.java} | 33 +-- .../sources/v2/JavaSimpleDataSourceV2.java | 19 +- .../sql/sources/v2/DataSourceV2Suite.scala | 260 ++++++++++-------- .../sources/v2/SimpleWritableDataSource.scala | 35 ++- .../continuous/ContinuousSuite.scala | 4 +- 35 files changed, 942 insertions(+), 441 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchRead.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/OldSupportsReportPartitioning.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/OldSupportsReportStatistics.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala rename sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/{JavaSimpleReadSupport.java => JavaSimpleBatchTable.java} (78%) 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 af510219a6f6..9ba066a4cdc3 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.clients.producer.ProducerRecord import org.apache.spark.sql.Dataset -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2StreamingScanExec import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.streaming.Trigger @@ -208,7 +208,7 @@ class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { eventually(timeout(streamingTimeout)) { assert( query.lastExecution.executedPlan.collectFirst { - case scan: DataSourceV2ScanExec + case scan: DataSourceV2StreamingScanExec if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] => scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig] }.exists { config => diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala index aa21f1271b81..5549e821be75 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.SparkContext import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2StreamingScanExec import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.Trigger @@ -47,7 +47,7 @@ trait KafkaContinuousTest extends KafkaSourceTest { eventually(timeout(streamingTimeout)) { assert( query.lastExecution.executedPlan.collectFirst { - case scan: DataSourceV2ScanExec + case scan: DataSourceV2StreamingScanExec if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] => scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig] }.exists(_.knownPartitions.size == newCount), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 5e97d826370f..fcef424c330f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -197,37 +197,6 @@ object MimaExcludes { // [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.nextCredentialRenewalTime"), - // Data Source V2 API changes - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.ContinuousReadSupport"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.ReadSupport"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.WriteSupport"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.StreamWriteSupport"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.MicroBatchReadSupport"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.DataSourceReader"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.sources.v2.reader.SupportsPushDownRequiredColumns"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.sql.sources.v2.reader.ScanConfigBuilder.build"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.InputPartition.createPartitionReader"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics.estimateStatistics"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.sql.sources.v2.reader.ReadSupport.fullSchema"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.sql.sources.v2.reader.ReadSupport.planInputPartitions"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning.outputPartitioning"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning.outputPartitioning"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.sql.sources.v2.reader.ReadSupport.fullSchema"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.sql.sources.v2.reader.ReadSupport.planInputPartitions"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.sources.v2.reader.SupportsPushDownFilters"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.sql.sources.v2.reader.ScanConfigBuilder.build"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.ContinuousInputPartition"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.InputPartitionReader"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.streaming.ContinuousInputPartitionReader"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.DataSourceWriter"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createWriter"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter"), - // [SPARK-26133][ML] Remove deprecated OneHotEncoder and rename OneHotEncoderEstimator to OneHotEncoder ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.OneHotEncoderEstimator"), ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.OneHotEncoder"), @@ -243,7 +212,22 @@ object MimaExcludes { // [SPARK-26141] Enable custom metrics implementation in shuffle write // Following are Java private classes ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.sort.UnsafeShuffleWriter.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.TimeTrackingOutputStream.this") + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.TimeTrackingOutputStream.this"), + + // Data Source V2 API changes + (problem: Problem) => problem match { + case MissingClassProblem(cls) => + !cls.fullName.startsWith("org.apache.spark.sql.sources.v2") + case MissingTypesProblem(newCls, _) => + !newCls.fullName.startsWith("org.apache.spark.sql.sources.v2") + case InheritedNewAbstractMethodProblem(cls, _) => + !cls.fullName.startsWith("org.apache.spark.sql.sources.v2") + case DirectMissingMethodProblem(meth) => + !meth.owner.fullName.startsWith("org.apache.spark.sql.sources.v2") + case ReversedMissingMethodProblem(meth) => + !meth.owner.fullName.startsWith("org.apache.spark.sql.sources.v2") + case _ => true + } ) // Exclude rules for 2.4.x diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchRead.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchRead.java new file mode 100644 index 000000000000..0df89dbb608a --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchRead.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.sources.v2.reader.ScanBuilder; + +/** + * An empty mix-in interface for {@link Table}, to indicate this table supports batch scan. + *

    + * If a {@link Table} implements this interface, its {@link Table#newScanBuilder(DataSourceOptions)} + * must return a {@link ScanBuilder} that builds {@link Scan} with {@link Scan#toBatch()} + * implemented. + *

    + */ +@Evolving +public interface SupportsBatchRead extends Table { } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java new file mode 100644 index 000000000000..0c65fe0f9e76 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.sources.v2.reader.ScanBuilder; +import org.apache.spark.sql.types.StructType; + +/** + * An interface representing a logical structured data set of a data source. For example, the + * implementation can be a directory on the file system, a topic of Kafka, or a table in the + * catalog, etc. + *

    + * This interface can mixin the following interfaces to support different operations: + *

    + *
      + *
    • {@link SupportsBatchRead}: this table can be read in batch queries.
    • + *
    + */ +@Evolving +public interface Table { + + /** + * A name to identify this table. Implementations should provide a meaningful name, like the + * database and table name from catalog, or the location of files for this table. + */ + String name(); + + /** + * Returns the schema of this table. + */ + StructType schema(); + + /** + * Returns a {@link ScanBuilder} which can be used to build a {@link Scan} later. Spark will call + * this method for each data scanning query. + *

    + * The builder can take some query specific information to do operators pushdown, and keep these + * information in the created {@link Scan}. + *

    + */ + ScanBuilder newScanBuilder(DataSourceOptions options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java new file mode 100644 index 000000000000..855d5efe0c69 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.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.sql.sources.v2; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.DataSourceRegister; +import org.apache.spark.sql.types.StructType; + +/** + * The base interface for v2 data sources which don't have a real catalog. Implementations must + * have a public, 0-arg constructor. + *

    + * The major responsibility of this interface is to return a {@link Table} for read/write. + *

    + */ +@Evolving +// TODO: do not extend `DataSourceV2`, after we finish the API refactor completely. +public interface TableProvider extends DataSourceV2 { + + /** + * Return a {@link Table} instance to do read/write with user-specified options. + * + * @param options the user-specified options that can identify a table, e.g. file path, Kafka + * topic name, etc. It's an immutable case-insensitive string-to-string map. + */ + Table getTable(DataSourceOptions options); + + /** + * Return a {@link Table} instance to do read/write with user-specified schema and options. + *

    + * By default this method throws {@link UnsupportedOperationException}, implementations should + * override this method to handle user-specified schema. + *

    + * @param options the user-specified options that can identify a table, e.g. file path, Kafka + * topic name, etc. It's an immutable case-insensitive string-to-string map. + * @param schema the user-specified schema. + * @throws UnsupportedOperationException + */ + default Table getTable(DataSourceOptions options, StructType schema) { + String name; + if (this instanceof DataSourceRegister) { + name = ((DataSourceRegister) this).shortName(); + } else { + name = this.getClass().getName(); + } + throw new UnsupportedOperationException( + name + " source does not support user-specified schema"); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java new file mode 100644 index 000000000000..bcfa1983abb8 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.reader; + +import org.apache.spark.annotation.Evolving; + +/** + * A physical representation of a data source scan for batch queries. This interface is used to + * provide physical information, like how many partitions the scanned data has, and how to read + * records from the partitions. + */ +@Evolving +public interface Batch { + + /** + * Returns a list of {@link InputPartition input partitions}. Each {@link InputPartition} + * represents a data split that can be processed by one Spark task. The number of input + * partitions returned here is the same as the number of RDD partitions this scan outputs. + *

    + * If the {@link Scan} supports filter pushdown, this Batch is likely configured with a filter + * and is responsible for creating splits for that filter, which is not a full scan. + *

    + *

    + * This method will be called only once during a data source scan, to launch one Spark job. + *

    + */ + InputPartition[] planInputPartitions(); + + /** + * Returns a factory, which produces one {@link PartitionReader} for one {@link InputPartition}. + */ + PartitionReaderFactory createReaderFactory(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/OldSupportsReportPartitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/OldSupportsReportPartitioning.java new file mode 100644 index 000000000000..347a465905ac --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/OldSupportsReportPartitioning.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.reader; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; + +/** + * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to + * report data partitioning and try to avoid shuffle at Spark side. + * + * Note that, when a {@link ReadSupport} implementation creates exactly one {@link InputPartition}, + * Spark may avoid adding a shuffle even if the reader does not implement this interface. + */ +@Evolving +// TODO: remove it, after we finish the API refactor completely. +public interface OldSupportsReportPartitioning extends ReadSupport { + + /** + * Returns the output data partitioning that this reader guarantees. + */ + Partitioning outputPartitioning(ScanConfig config); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/OldSupportsReportStatistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/OldSupportsReportStatistics.java new file mode 100644 index 000000000000..0d3ec17107c1 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/OldSupportsReportStatistics.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.reader; + +import org.apache.spark.annotation.Evolving; + +/** + * A mix in interface for {@link BatchReadSupport}. 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. + */ +@Evolving +// TODO: remove it, after we finish the API refactor completely. +public interface OldSupportsReportStatistics extends ReadSupport { + + /** + * Returns the estimated statistics of this data source scan. + */ + Statistics estimateStatistics(ScanConfig config); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java new file mode 100644 index 000000000000..4d84fb19aa02 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.reader; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.sources.v2.SupportsBatchRead; +import org.apache.spark.sql.sources.v2.Table; + +/** + * A logical representation of a data source scan. This interface is used to provide logical + * information, like what the actual read schema is. + *

    + * This logical representation is shared between batch scan, micro-batch streaming scan and + * continuous streaming scan. Data sources must implement the corresponding methods in this + * interface, to match what the table promises to support. For example, {@link #toBatch()} must be + * implemented, if the {@link Table} that creates this {@link Scan} implements + * {@link SupportsBatchRead}. + *

    + */ +@Evolving +public interface Scan { + + /** + * Returns the actual schema of this data source scan, which may be different from the physical + * schema of the underlying storage, as column pruning or other optimizations may happen. + */ + StructType readSchema(); + + /** + * A description string of this scan, which may includes information like: what filters are + * configured for this scan, what's the value of some important options like path, etc. The + * description doesn't need to include {@link #readSchema()}, as Spark already knows it. + *

    + * By default this returns the class name of the implementation. Please override it to provide a + * meaningful description. + *

    + */ + default String description() { + return this.getClass().toString(); + } + + /** + * Returns the physical representation of this scan for batch query. By default this method throws + * exception, data sources must overwrite this method to provide an implementation, if the + * {@link Table} that creates this scan implements {@link SupportsBatchRead}. + * + * @throws UnsupportedOperationException + */ + default Batch toBatch() { + throw new UnsupportedOperationException("Batch scans are not supported"); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java new file mode 100644 index 000000000000..d4bc1ff97713 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.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.sql.sources.v2.reader; + +import org.apache.spark.annotation.Evolving; + +/** + * An interface for building the {@link Scan}. Implementations can mixin SupportsPushDownXYZ + * interfaces to do operator pushdown, and keep the operator pushdown result in the returned + * {@link Scan}. + */ +@Evolving +public interface ScanBuilder { + Scan build(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java index a69872a52774..c8cff68c2ef7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java @@ -28,8 +28,8 @@ * For APIs that take a {@link ScanConfig} as input, like * {@link ReadSupport#planInputPartitions(ScanConfig)}, * {@link BatchReadSupport#createReaderFactory(ScanConfig)} and - * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}, implementations mostly need to - * cast the input {@link ScanConfig} to the concrete {@link ScanConfig} class of the data source. + * {@link OldSupportsReportStatistics#estimateStatistics(ScanConfig)}, implementations mostly need + * to cast the input {@link ScanConfig} to the concrete {@link ScanConfig} class of the data source. */ @Evolving public interface ScanConfig { diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java index 14776f37fed4..a0b194a41f58 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java @@ -23,7 +23,7 @@ /** * An interface to represent statistics for a data source, which is returned by - * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}. + * {@link SupportsReportStatistics#estimateStatistics()}. */ @Evolving public interface Statistics { diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java index 3a89baa1b44c..296d3e47e732 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java @@ -21,11 +21,11 @@ import org.apache.spark.sql.sources.Filter; /** - * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this interface to + * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to * push down filters to the data source and reduce the size of the data to be read. */ @Evolving -public interface SupportsPushDownFilters extends ScanConfigBuilder { +public interface SupportsPushDownFilters extends ScanBuilder { /** * Pushes down filters, and returns filters that need to be evaluated after scanning. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java index 193476322488..60e71c5dd008 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java @@ -21,12 +21,12 @@ import org.apache.spark.sql.types.StructType; /** - * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this + * A mix-in interface for {@link ScanBuilder}. Data sources can implement this * interface to push down required columns to the data source and only read these columns during * scan to reduce the size of the data to be read. */ @Evolving -public interface SupportsPushDownRequiredColumns extends ScanConfigBuilder { +public interface SupportsPushDownRequiredColumns extends ScanBuilder { /** * Applies column pruning w.r.t. the given requiredSchema. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java index 0335c7775c2a..ba175812a88d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java @@ -21,17 +21,17 @@ import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; /** - * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to + * A mix in interface for {@link Batch}. Data sources can implement this interface to * report data partitioning and try to avoid shuffle at Spark side. * - * Note that, when a {@link ReadSupport} implementation creates exactly one {@link InputPartition}, + * Note that, when a {@link Batch} implementation creates exactly one {@link InputPartition}, * Spark may avoid adding a shuffle even if the reader does not implement this interface. */ @Evolving -public interface SupportsReportPartitioning extends ReadSupport { +public interface SupportsReportPartitioning extends Batch { /** * Returns the output data partitioning that this reader guarantees. */ - Partitioning outputPartitioning(ScanConfig config); + Partitioning outputPartitioning(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java index 917372cdd25b..d9f5fb64083a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java @@ -20,7 +20,7 @@ import org.apache.spark.annotation.Evolving; /** - * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to + * A mix in interface for {@link Batch}. 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 @@ -28,10 +28,10 @@ * not improve query performance until the planner can push operators before getting stats. */ @Evolving -public interface SupportsReportStatistics extends ReadSupport { +public interface SupportsReportStatistics extends Batch { /** * Returns the estimated statistics of this data source scan. */ - Statistics estimateStatistics(ScanConfig config); + Statistics estimateStatistics(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java index c9a00262c128..c7370eb3d38a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java @@ -19,12 +19,11 @@ import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.ScanConfig; import org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning; /** * An interface to represent the output data partitioning for a data source, which is returned by - * {@link SupportsReportPartitioning#outputPartitioning(ScanConfig)}. Note that this should work + * {@link SupportsReportPartitioning#outputPartitioning()}. Note that this should work * like a snapshot. Once created, it should be deterministic and always report the same number of * partitions and the same "satisfy" result for a certain distribution. */ 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 da88598eed06..661fe98d8c90 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 @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils -import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, DataSourceOptions, DataSourceV2} +import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.unsafe.types.UTF8String @@ -194,20 +194,26 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) - if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val ds = cls.getConstructor().newInstance().asInstanceOf[DataSourceV2] - if (ds.isInstanceOf[BatchReadSupportProvider]) { - val sessionOptions = DataSourceV2Utils.extractSessionConfigs( - ds = ds, conf = sparkSession.sessionState.conf) - val pathsOption = { - val objectMapper = new ObjectMapper() - DataSourceOptions.PATHS_KEY -> objectMapper.writeValueAsString(paths.toArray) - } - Dataset.ofRows(sparkSession, DataSourceV2Relation.create( - ds, sessionOptions ++ extraOptions.toMap + pathsOption, - userSpecifiedSchema = userSpecifiedSchema)) - } else { - loadV1Source(paths: _*) + if (classOf[TableProvider].isAssignableFrom(cls)) { + val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + ds = provider, conf = sparkSession.sessionState.conf) + val pathsOption = { + val objectMapper = new ObjectMapper() + DataSourceOptions.PATHS_KEY -> objectMapper.writeValueAsString(paths.toArray) + } + val finalOptions = sessionOptions ++ extraOptions.toMap + pathsOption + val dsOptions = new DataSourceOptions(finalOptions.asJava) + val table = userSpecifiedSchema match { + case Some(schema) => provider.getTable(dsOptions, schema) + case _ => provider.getTable(dsOptions) + } + table match { + case s: SupportsBatchRead => + Dataset.ofRows(sparkSession, DataSourceV2Relation.create( + provider, s, finalOptions, userSpecifiedSchema = userSpecifiedSchema)) + + case _ => loadV1Source(paths: _*) } } else { loadV1Source(paths: _*) 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 5a807d3d4b93..b9c4076994e9 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 @@ -252,7 +252,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val options = sessionOptions ++ extraOptions if (mode == SaveMode.Append) { - val relation = DataSourceV2Relation.create(source, options) + val relation = DataSourceV2Relation.createRelationForWrite(source, options) runCommand(df.sparkSession, "save") { AppendData.byName(relation, df.logicalPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index f7e29593a635..0a6b0afe6cfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -22,13 +22,13 @@ import java.util.UUID import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, BatchWriteSupportProvider, DataSourceOptions, DataSourceV2} -import org.apache.spark.sql.sources.v2.reader.{BatchReadSupport, ReadSupport, ScanConfigBuilder, SupportsReportStatistics} +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport import org.apache.spark.sql.types.StructType @@ -40,32 +40,38 @@ import org.apache.spark.sql.types.StructType * @param userSpecifiedSchema The user-specified schema for this scan. */ case class DataSourceV2Relation( - source: DataSourceV2, - readSupport: BatchReadSupport, + // TODO: remove `source` when we finish API refactor for write. + source: TableProvider, + table: SupportsBatchRead, output: Seq[AttributeReference], options: Map[String, String], - tableIdent: Option[TableIdentifier] = None, userSpecifiedSchema: Option[StructType] = None) - extends LeafNode with MultiInstanceRelation with NamedRelation with DataSourceV2StringFormat { + extends LeafNode with MultiInstanceRelation with NamedRelation { import DataSourceV2Relation._ - override def name: String = { - tableIdent.map(_.unquotedString).getOrElse(s"${source.name}:unknown") - } - - override def pushedFilters: Seq[Expression] = Seq.empty + override def name: String = table.name() - override def simpleString: String = "RelationV2 " + metadataString + override def simpleString: String = { + s"RelationV2${truncatedString(output, "[", ", ", "]")} $name" + } def newWriteSupport(): BatchWriteSupport = source.createWriteSupport(options, schema) - override def computeStats(): Statistics = readSupport match { - case r: SupportsReportStatistics => - val statistics = r.estimateStatistics(readSupport.newScanConfigBuilder().build()) - Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) - case _ => - Statistics(sizeInBytes = conf.defaultSizeInBytes) + def newScanBuilder(): ScanBuilder = { + val dsOptions = new DataSourceOptions(options.asJava) + table.newScanBuilder(dsOptions) + } + + override def computeStats(): Statistics = { + val scan = newScanBuilder().build() + scan match { + case r: SupportsReportStatistics => + val statistics = r.estimateStatistics() + Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) + case _ => + Statistics(sizeInBytes = conf.defaultSizeInBytes) + } } override def newInstance(): DataSourceV2Relation = { @@ -109,7 +115,7 @@ case class StreamingDataSourceV2Relation( } override def computeStats(): Statistics = readSupport match { - case r: SupportsReportStatistics => + case r: OldSupportsReportStatistics => val statistics = r.estimateStatistics(scanConfigBuilder.build()) Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) case _ => @@ -119,15 +125,6 @@ case class StreamingDataSourceV2Relation( object DataSourceV2Relation { private implicit class SourceHelpers(source: DataSourceV2) { - def asReadSupportProvider: BatchReadSupportProvider = { - source match { - case provider: BatchReadSupportProvider => - provider - case _ => - throw new AnalysisException(s"Data source is not readable: $name") - } - } - def asWriteSupportProvider: BatchWriteSupportProvider = { source match { case provider: BatchWriteSupportProvider => @@ -146,18 +143,6 @@ object DataSourceV2Relation { } } - def createReadSupport( - options: Map[String, String], - userSpecifiedSchema: Option[StructType]): BatchReadSupport = { - val v2Options = new DataSourceOptions(options.asJava) - userSpecifiedSchema match { - case Some(s) => - asReadSupportProvider.createBatchReadSupport(s, v2Options) - case _ => - asReadSupportProvider.createBatchReadSupport(v2Options) - } - } - def createWriteSupport( options: Map[String, String], schema: StructType): BatchWriteSupport = { @@ -170,20 +155,21 @@ object DataSourceV2Relation { } def create( - source: DataSourceV2, + provider: TableProvider, + table: SupportsBatchRead, options: Map[String, String], - tableIdent: Option[TableIdentifier] = None, userSpecifiedSchema: Option[StructType] = None): DataSourceV2Relation = { - val readSupport = source.createReadSupport(options, userSpecifiedSchema) - val output = readSupport.fullSchema().toAttributes - val ident = tableIdent.orElse(tableFromOptions(options)) - DataSourceV2Relation( - source, readSupport, output, options, ident, userSpecifiedSchema) + val output = table.schema().toAttributes + DataSourceV2Relation(provider, table, output, options, userSpecifiedSchema) } - private def tableFromOptions(options: Map[String, String]): Option[TableIdentifier] = { - options - .get(DataSourceOptions.TABLE_KEY) - .map(TableIdentifier(_, options.get(DataSourceOptions.DATABASE_KEY))) + // TODO: remove this when we finish API refactor for write. + def createRelationForWrite( + source: DataSourceV2, + options: Map[String, String]): DataSourceV2Relation = { + val provider = source.asInstanceOf[TableProvider] + val dsOptions = new DataSourceOptions(options.asJava) + val table = provider.getTable(dsOptions) + create(provider, table.asInstanceOf[SupportsBatchRead], options) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala index 25f86a66a826..725bcc3af3ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala @@ -22,60 +22,47 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ 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.execution.{ColumnarBatchScan, LeafExecNode, WholeStageCodegenExec} -import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.DataSourceV2 import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory, ContinuousReadSupport, MicroBatchReadSupport} /** - * Physical plan node for scanning data from a data source. + * Physical plan node for scanning a batch of data from a data source. */ case class DataSourceV2ScanExec( output: Seq[AttributeReference], - @transient source: DataSourceV2, - @transient options: Map[String, String], - @transient pushedFilters: Seq[Expression], - @transient readSupport: ReadSupport, - @transient scanConfig: ScanConfig) - extends LeafExecNode with DataSourceV2StringFormat with ColumnarBatchScan { + scanDesc: String, + @transient batch: Batch) + extends LeafExecNode with ColumnarBatchScan { - override def simpleString: String = "ScanV2 " + metadataString + override def simpleString: String = { + s"ScanV2${truncatedString(output, "[", ", ", "]")} $scanDesc" + } // TODO: unify the equal/hashCode implementation for all data source v2 query plans. override def equals(other: Any): Boolean = other match { - case other: DataSourceV2ScanExec => - output == other.output && readSupport.getClass == other.readSupport.getClass && - options == other.options + case other: DataSourceV2ScanExec => this.batch == other.batch case _ => false } - override def hashCode(): Int = { - Seq(output, source, options).hashCode() - } + override def hashCode(): Int = batch.hashCode() + + private lazy val partitions = batch.planInputPartitions() + + private lazy val readerFactory = batch.createReaderFactory() - override def outputPartitioning: physical.Partitioning = readSupport match { + override def outputPartitioning: physical.Partitioning = batch match { case _ if partitions.length == 1 => SinglePartition case s: SupportsReportPartitioning => new DataSourcePartitioning( - s.outputPartitioning(scanConfig), AttributeMap(output.map(a => a -> a.name))) + s.outputPartitioning(), AttributeMap(output.map(a => a -> a.name))) case _ => super.outputPartitioning } - private lazy val partitions: Seq[InputPartition] = readSupport.planInputPartitions(scanConfig) - - private lazy val readerFactory = readSupport match { - case r: BatchReadSupport => r.createReaderFactory(scanConfig) - case r: MicroBatchReadSupport => r.createReaderFactory(scanConfig) - case r: ContinuousReadSupport => r.createContinuousReaderFactory(scanConfig) - case _ => throw new IllegalStateException("unknown read support: " + readSupport) - } - - // TODO: clean this up when we have dedicated scan plan for continuous streaming. - override val supportsBatch: Boolean = { + override def supportsBatch: Boolean = { require(partitions.forall(readerFactory.supportColumnarReads) || !partitions.exists(readerFactory.supportColumnarReads), "Cannot mix row-based and columnar input partitions.") @@ -83,25 +70,8 @@ case class DataSourceV2ScanExec( partitions.exists(readerFactory.supportColumnarReads) } - private lazy val inputRDD: RDD[InternalRow] = readSupport match { - case _: ContinuousReadSupport => - assert(!supportsBatch, - "continuous stream reader does not support columnar read yet.") - EpochCoordinatorRef.get( - sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY), - sparkContext.env) - .askSync[Unit](SetReaderPartitions(partitions.size)) - new ContinuousDataSourceRDD( - sparkContext, - sqlContext.conf.continuousStreamingExecutorQueueSize, - sqlContext.conf.continuousStreamingExecutorPollIntervalMs, - partitions, - schema, - readerFactory.asInstanceOf[ContinuousPartitionReaderFactory]) - - case _ => - new DataSourceRDD( - sparkContext, partitions, readerFactory.asInstanceOf[PartitionReaderFactory], supportsBatch) + private lazy val inputRDD: RDD[InternalRow] = { + new DataSourceRDD(sparkContext, partitions, readerFactory, supportsBatch) } override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) 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 9a3109e7c199..2e26fce880b6 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 @@ -37,9 +37,9 @@ object DataSourceV2Strategy extends Strategy { * @return pushed filter and post-scan filters. */ private def pushFilters( - configBuilder: ScanConfigBuilder, + scanBuilder: ScanBuilder, filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - configBuilder match { + scanBuilder match { case r: SupportsPushDownFilters => // A map from translated data source filters to original catalyst filter expressions. val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] @@ -76,18 +76,18 @@ object DataSourceV2Strategy extends Strategy { */ // TODO: nested column pruning. private def pruneColumns( - configBuilder: ScanConfigBuilder, + scanBuilder: ScanBuilder, relation: DataSourceV2Relation, - exprs: Seq[Expression]): (ScanConfig, Seq[AttributeReference]) = { - configBuilder match { + exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { + scanBuilder match { case r: SupportsPushDownRequiredColumns => val requiredColumns = AttributeSet(exprs.flatMap(_.references)) val neededOutput = relation.output.filter(requiredColumns.contains) if (neededOutput != relation.output) { r.pruneColumns(neededOutput.toStructType) - val config = r.build() + val scan = r.build() val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap - config -> config.readSchema().toAttributes.map { + scan -> scan.readSchema().toAttributes.map { // We have to keep the attribute id during transformation. a => a.withExprId(nameToAttr(a.name).exprId) } @@ -95,19 +95,19 @@ object DataSourceV2Strategy extends Strategy { r.build() -> relation.output } - case _ => configBuilder.build() -> relation.output + case _ => scanBuilder.build() -> relation.output } } override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => - val configBuilder = relation.readSupport.newScanConfigBuilder() + val scanBuilder = relation.newScanBuilder() // `pushedFilters` will be pushed down and evaluated in the underlying data sources. // `postScanFilters` need to be evaluated after the scan. // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. - val (pushedFilters, postScanFilters) = pushFilters(configBuilder, filters) - val (config, output) = pruneColumns(configBuilder, relation, project ++ postScanFilters) + val (pushedFilters, postScanFilters) = pushFilters(scanBuilder, filters) + val (scan, output) = pruneColumns(scanBuilder, relation, project ++ postScanFilters) logInfo( s""" |Pushing operators to ${relation.source.getClass} @@ -116,16 +116,10 @@ object DataSourceV2Strategy extends Strategy { |Output: ${output.mkString(", ")} """.stripMargin) - val scan = DataSourceV2ScanExec( - output, - relation.source, - relation.options, - pushedFilters, - relation.readSupport, - config) + val plan = DataSourceV2ScanExec(output, scan.description(), scan.toBatch) val filterCondition = postScanFilters.reduceLeftOption(And) - val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan) + val withFilter = filterCondition.map(FilterExec(_, plan)).getOrElse(plan) // always add the projection, which will produce unsafe rows required by some operators ProjectExec(project, withFilter) :: Nil @@ -135,7 +129,7 @@ object DataSourceV2Strategy extends Strategy { val scanConfig = r.scanConfigBuilder.build() // ensure there is a projection, which will produce unsafe rows required by some operators ProjectExec(r.output, - DataSourceV2ScanExec( + DataSourceV2StreamingScanExec( r.output, r.source, r.options, r.pushedFilters, r.readSupport, scanConfig)) :: Nil case WriteToDataSourceV2(writer, query) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala new file mode 100644 index 000000000000..c87294090996 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +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.plans.physical +import org.apache.spark.sql.catalyst.plans.physical.SinglePartition +import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, WholeStageCodegenExec} +import org.apache.spark.sql.execution.streaming.continuous._ +import org.apache.spark.sql.sources.v2.DataSourceV2 +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory, ContinuousReadSupport, MicroBatchReadSupport} + +/** + * Physical plan node for scanning data from a data source. + */ +// TODO: micro-batch should be handled by `DataSourceV2ScanExec`, after we finish the API refactor +// completely. +case class DataSourceV2StreamingScanExec( + output: Seq[AttributeReference], + @transient source: DataSourceV2, + @transient options: Map[String, String], + @transient pushedFilters: Seq[Expression], + @transient readSupport: ReadSupport, + @transient scanConfig: ScanConfig) + extends LeafExecNode with DataSourceV2StringFormat with ColumnarBatchScan { + + override def simpleString: String = "ScanV2 " + metadataString + + // TODO: unify the equal/hashCode implementation for all data source v2 query plans. + override def equals(other: Any): Boolean = other match { + case other: DataSourceV2StreamingScanExec => + output == other.output && readSupport.getClass == other.readSupport.getClass && + options == other.options + case _ => false + } + + override def hashCode(): Int = { + Seq(output, source, options).hashCode() + } + + override def outputPartitioning: physical.Partitioning = readSupport match { + case _ if partitions.length == 1 => + SinglePartition + + case s: OldSupportsReportPartitioning => + new DataSourcePartitioning( + s.outputPartitioning(scanConfig), AttributeMap(output.map(a => a -> a.name))) + + case _ => super.outputPartitioning + } + + private lazy val partitions: Seq[InputPartition] = readSupport.planInputPartitions(scanConfig) + + private lazy val readerFactory = readSupport match { + case r: MicroBatchReadSupport => r.createReaderFactory(scanConfig) + case r: ContinuousReadSupport => r.createContinuousReaderFactory(scanConfig) + case _ => throw new IllegalStateException("unknown read support: " + readSupport) + } + + override val supportsBatch: Boolean = { + require(partitions.forall(readerFactory.supportColumnarReads) || + !partitions.exists(readerFactory.supportColumnarReads), + "Cannot mix row-based and columnar input partitions.") + + partitions.exists(readerFactory.supportColumnarReads) + } + + private lazy val inputRDD: RDD[InternalRow] = readSupport match { + case _: ContinuousReadSupport => + assert(!supportsBatch, + "continuous stream reader does not support columnar read yet.") + EpochCoordinatorRef.get( + sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY), + sparkContext.env) + .askSync[Unit](SetReaderPartitions(partitions.size)) + new ContinuousDataSourceRDD( + sparkContext, + sqlContext.conf.continuousStreamingExecutorQueueSize, + sqlContext.conf.continuousStreamingExecutorPollIntervalMs, + partitions, + schema, + readerFactory.asInstanceOf[ContinuousPartitionReaderFactory]) + + case _ => + new DataSourceRDD( + sparkContext, partitions, readerFactory.asInstanceOf[PartitionReaderFactory], supportsBatch) + } + + override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) + + override protected def doExecute(): RDD[InternalRow] = { + if (supportsBatch) { + WholeStageCodegenExec(this)(codegenStageId = 0).execute() + } else { + val numOutputRows = longMetric("numOutputRows") + inputRDD.map { r => + numOutputRows += 1 + r + } + } + } +} 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 392229bcb5f5..6a22f0cc8431 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2StreamingScanExec import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport import org.apache.spark.sql.streaming._ import org.apache.spark.sql.streaming.StreamingQueryListener.QueryProgressEvent @@ -256,7 +256,7 @@ trait ProgressReporter extends Logging { // (can happen with self-unions or self-joins). This means the source is scanned multiple // times in the query, we should count the numRows for each scan. val sourceToInputRowsTuples = lastExecution.executedPlan.collect { - case s: DataSourceV2ScanExec if s.readSupport.isInstanceOf[BaseStreamingSource] => + case s: DataSourceV2StreamingScanExec if s.readSupport.isInstanceOf[BaseStreamingSource] => val numRows = s.metrics.get("numOutputRows").map(_.value).getOrElse(0L) val source = s.readSupport.asInstanceOf[BaseStreamingSource] source -> numRows 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 1eab55122e84..af23c5cd3d80 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Curre import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExec, StreamingDataSourceV2Relation} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2StreamingScanExec, StreamingDataSourceV2Relation} import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingRelationV2, _} import org.apache.spark.sql.sources.v2 import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, StreamingWriteSupportProvider} @@ -206,7 +206,8 @@ class ContinuousExecution( } val (readSupport, scanConfig) = lastExecution.executedPlan.collect { - case scan: DataSourceV2ScanExec if scan.readSupport.isInstanceOf[ContinuousReadSupport] => + case scan: DataSourceV2StreamingScanExec + if scan.readSupport.isInstanceOf[ContinuousReadSupport] => scan.readSupport.asInstanceOf[ContinuousReadSupport] -> scan.scanConfig }.head diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java index 5602310219a7..2612b6185fd4 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java @@ -24,62 +24,29 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.sources.Filter; import org.apache.spark.sql.sources.GreaterThan; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.StructType; -public class JavaAdvancedDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { +public class JavaAdvancedDataSourceV2 implements TableProvider { - public class ReadSupport extends JavaSimpleReadSupport { - @Override - public ScanConfigBuilder newScanConfigBuilder() { - return new AdvancedScanConfigBuilder(); - } - - @Override - public InputPartition[] planInputPartitions(ScanConfig config) { - Filter[] filters = ((AdvancedScanConfigBuilder) config).filters; - List res = new ArrayList<>(); - - Integer lowerBound = null; - for (Filter filter : filters) { - if (filter instanceof GreaterThan) { - GreaterThan f = (GreaterThan) filter; - if ("i".equals(f.attribute()) && f.value() instanceof Integer) { - lowerBound = (Integer) f.value(); - break; - } - } - } - - if (lowerBound == null) { - res.add(new JavaRangeInputPartition(0, 5)); - res.add(new JavaRangeInputPartition(5, 10)); - } else if (lowerBound < 4) { - res.add(new JavaRangeInputPartition(lowerBound + 1, 5)); - res.add(new JavaRangeInputPartition(5, 10)); - } else if (lowerBound < 9) { - res.add(new JavaRangeInputPartition(lowerBound + 1, 10)); + @Override + public Table getTable(DataSourceOptions options) { + return new JavaSimpleBatchTable() { + @Override + public ScanBuilder newScanBuilder(DataSourceOptions options) { + return new AdvancedScanBuilder(); } - - return res.stream().toArray(InputPartition[]::new); - } - - @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { - StructType requiredSchema = ((AdvancedScanConfigBuilder) config).requiredSchema; - return new AdvancedReaderFactory(requiredSchema); - } + }; } - public static class AdvancedScanConfigBuilder implements ScanConfigBuilder, ScanConfig, + static class AdvancedScanBuilder implements ScanBuilder, Scan, SupportsPushDownFilters, SupportsPushDownRequiredColumns { - // Exposed for testing. - public StructType requiredSchema = new StructType().add("i", "int").add("j", "int"); - public Filter[] filters = new Filter[0]; + private StructType requiredSchema = new StructType().add("i", "int").add("j", "int"); + private Filter[] filters = new Filter[0]; @Override public void pruneColumns(StructType requiredSchema) { @@ -121,9 +88,58 @@ public Filter[] pushedFilters() { } @Override - public ScanConfig build() { + public Scan build() { return this; } + + @Override + public Batch toBatch() { + return new AdvancedBatch(requiredSchema, filters); + } + } + + public static class AdvancedBatch implements Batch { + // Exposed for testing. + public StructType requiredSchema; + public Filter[] filters; + + AdvancedBatch(StructType requiredSchema, Filter[] filters) { + this.requiredSchema = requiredSchema; + this.filters = filters; + } + + @Override + public InputPartition[] planInputPartitions() { + List res = new ArrayList<>(); + + Integer lowerBound = null; + for (Filter filter : filters) { + if (filter instanceof GreaterThan) { + GreaterThan f = (GreaterThan) filter; + if ("i".equals(f.attribute()) && f.value() instanceof Integer) { + lowerBound = (Integer) f.value(); + break; + } + } + } + + if (lowerBound == null) { + res.add(new JavaRangeInputPartition(0, 5)); + res.add(new JavaRangeInputPartition(5, 10)); + } else if (lowerBound < 4) { + res.add(new JavaRangeInputPartition(lowerBound + 1, 5)); + res.add(new JavaRangeInputPartition(5, 10)); + } else if (lowerBound < 9) { + res.add(new JavaRangeInputPartition(lowerBound + 1, 10)); + } + + return res.stream().toArray(InputPartition[]::new); + } + + @Override + public PartitionReaderFactory createReaderFactory() { + return new AdvancedReaderFactory(requiredSchema); + } } static class AdvancedReaderFactory implements PartitionReaderFactory { @@ -165,10 +181,4 @@ public void close() throws IOException { }; } } - - - @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); - } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java index 28a933039831..d72ab5338aa8 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java @@ -21,21 +21,21 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; -public class JavaColumnarDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { +public class JavaColumnarDataSourceV2 implements TableProvider { - class ReadSupport extends JavaSimpleReadSupport { + class MyScanBuilder extends JavaSimpleScanBuilder { @Override - public InputPartition[] planInputPartitions(ScanConfig config) { + public InputPartition[] planInputPartitions() { InputPartition[] partitions = new InputPartition[2]; partitions[0] = new JavaRangeInputPartition(0, 50); partitions[1] = new JavaRangeInputPartition(50, 90); @@ -43,11 +43,21 @@ public InputPartition[] planInputPartitions(ScanConfig config) { } @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { + public PartitionReaderFactory createReaderFactory() { return new ColumnarReaderFactory(); } } + @Override + public Table getTable(DataSourceOptions options) { + return new JavaSimpleBatchTable() { + @Override + public ScanBuilder newScanBuilder(DataSourceOptions options) { + return new MyScanBuilder(); + } + }; + } + static class ColumnarReaderFactory implements PartitionReaderFactory { private static final int BATCH_SIZE = 20; @@ -106,9 +116,4 @@ public void close() throws IOException { }; } } - - @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); - } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java index 18a11dde8219..a513bfb26ef1 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java @@ -22,18 +22,20 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.*; +import org.apache.spark.sql.sources.v2.DataSourceOptions; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.sources.v2.reader.partitioning.ClusteredDistribution; import org.apache.spark.sql.sources.v2.reader.partitioning.Distribution; import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; -public class JavaPartitionAwareDataSource implements DataSourceV2, BatchReadSupportProvider { +public class JavaPartitionAwareDataSource implements TableProvider { - class ReadSupport extends JavaSimpleReadSupport implements SupportsReportPartitioning { + class MyScanBuilder extends JavaSimpleScanBuilder implements SupportsReportPartitioning { @Override - public InputPartition[] planInputPartitions(ScanConfig config) { + public InputPartition[] planInputPartitions() { InputPartition[] partitions = new InputPartition[2]; partitions[0] = new SpecificInputPartition(new int[]{1, 1, 3}, new int[]{4, 4, 6}); partitions[1] = new SpecificInputPartition(new int[]{2, 4, 4}, new int[]{6, 2, 2}); @@ -41,16 +43,26 @@ public InputPartition[] planInputPartitions(ScanConfig config) { } @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { + public PartitionReaderFactory createReaderFactory() { return new SpecificReaderFactory(); } @Override - public Partitioning outputPartitioning(ScanConfig config) { + public Partitioning outputPartitioning() { return new MyPartitioning(); } } + @Override + public Table getTable(DataSourceOptions options) { + return new JavaSimpleBatchTable() { + @Override + public ScanBuilder newScanBuilder(DataSourceOptions options) { + return new MyScanBuilder(); + } + }; + } + static class MyPartitioning implements Partitioning { @Override @@ -106,9 +118,4 @@ public void close() throws IOException { }; } } - - @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); - } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java index cc9ac04a0dad..815d57ba9413 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java @@ -17,39 +17,51 @@ package test.org.apache.spark.sql.sources.v2; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.StructType; -public class JavaSchemaRequiredDataSource implements DataSourceV2, BatchReadSupportProvider { +public class JavaSchemaRequiredDataSource implements TableProvider { - class ReadSupport extends JavaSimpleReadSupport { - private final StructType schema; + class MyScanBuilder extends JavaSimpleScanBuilder { - ReadSupport(StructType schema) { + private StructType schema; + + MyScanBuilder(StructType schema) { this.schema = schema; } @Override - public StructType fullSchema() { + public StructType readSchema() { return schema; } @Override - public InputPartition[] planInputPartitions(ScanConfig config) { + public InputPartition[] planInputPartitions() { return new InputPartition[0]; } } @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - throw new IllegalArgumentException("requires a user-supplied schema"); + public Table getTable(DataSourceOptions options, StructType schema) { + return new JavaSimpleBatchTable() { + + @Override + public StructType schema() { + return schema; + } + + @Override + public ScanBuilder newScanBuilder(DataSourceOptions options) { + return new MyScanBuilder(schema); + } + }; } @Override - public BatchReadSupport createBatchReadSupport(StructType schema, DataSourceOptions options) { - return new ReadSupport(schema); + public Table getTable(DataSourceOptions options) { + throw new IllegalArgumentException("requires a user-supplied schema"); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java similarity index 78% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java rename to sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java index ced51dde6997..cb5954d5a621 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java @@ -21,43 +21,44 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.sources.v2.SupportsBatchRead; +import org.apache.spark.sql.sources.v2.Table; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.StructType; -abstract class JavaSimpleReadSupport implements BatchReadSupport { +abstract class JavaSimpleBatchTable implements Table, SupportsBatchRead { @Override - public StructType fullSchema() { + public StructType schema() { return new StructType().add("i", "int").add("j", "int"); } @Override - public ScanConfigBuilder newScanConfigBuilder() { - return new JavaNoopScanConfigBuilder(fullSchema()); - } - - @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { - return new JavaSimpleReaderFactory(); + public String name() { + return this.getClass().toString(); } } -class JavaNoopScanConfigBuilder implements ScanConfigBuilder, ScanConfig { - - private StructType schema; +abstract class JavaSimpleScanBuilder implements ScanBuilder, Scan, Batch { - JavaNoopScanConfigBuilder(StructType schema) { - this.schema = schema; + @Override + public Scan build() { + return this; } @Override - public ScanConfig build() { + public Batch toBatch() { return this; } @Override public StructType readSchema() { - return schema; + return new StructType().add("i", "int").add("j", "int"); + } + + @Override + public PartitionReaderFactory createReaderFactory() { + return new JavaSimpleReaderFactory(); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java index 2cdbba84ec4a..852c4546df88 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java @@ -17,17 +17,17 @@ package test.org.apache.spark.sql.sources.v2; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; -import org.apache.spark.sql.sources.v2.DataSourceV2; import org.apache.spark.sql.sources.v2.DataSourceOptions; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; -public class JavaSimpleDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { +public class JavaSimpleDataSourceV2 implements TableProvider { - class ReadSupport extends JavaSimpleReadSupport { + class MyScanBuilder extends JavaSimpleScanBuilder { @Override - public InputPartition[] planInputPartitions(ScanConfig config) { + public InputPartition[] planInputPartitions() { InputPartition[] partitions = new InputPartition[2]; partitions[0] = new JavaRangeInputPartition(0, 5); partitions[1] = new JavaRangeInputPartition(5, 10); @@ -36,7 +36,12 @@ public InputPartition[] planInputPartitions(ScanConfig config) { } @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); + public Table getTable(DataSourceOptions options) { + return new JavaSimpleBatchTable() { + @Override + public ScanBuilder newScanBuilder(DataSourceOptions options) { + return new MyScanBuilder(); + } + }; } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala index e8f291af13ba..d282193d35d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala @@ -38,18 +38,17 @@ import org.apache.spark.sql.vectorized.ColumnarBatch class DataSourceV2Suite extends QueryTest with SharedSQLContext { import testImplicits._ - private def getScanConfig(query: DataFrame): AdvancedScanConfigBuilder = { + private def getBatch(query: DataFrame): AdvancedBatch = { query.queryExecution.executedPlan.collect { case d: DataSourceV2ScanExec => - d.scanConfig.asInstanceOf[AdvancedScanConfigBuilder] + d.batch.asInstanceOf[AdvancedBatch] }.head } - private def getJavaScanConfig( - query: DataFrame): JavaAdvancedDataSourceV2.AdvancedScanConfigBuilder = { + private def getJavaBatch(query: DataFrame): JavaAdvancedDataSourceV2.AdvancedBatch = { query.queryExecution.executedPlan.collect { case d: DataSourceV2ScanExec => - d.scanConfig.asInstanceOf[JavaAdvancedDataSourceV2.AdvancedScanConfigBuilder] + d.batch.asInstanceOf[JavaAdvancedDataSourceV2.AdvancedBatch] }.head } @@ -73,51 +72,51 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { val q1 = df.select('j) checkAnswer(q1, (0 until 10).map(i => Row(-i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q1) - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + val batch = getBatch(q1) + assert(batch.filters.isEmpty) + assert(batch.requiredSchema.fieldNames === Seq("j")) } else { - val config = getJavaScanConfig(q1) - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + val batch = getJavaBatch(q1) + assert(batch.filters.isEmpty) + assert(batch.requiredSchema.fieldNames === Seq("j")) } val q2 = df.filter('i > 3) checkAnswer(q2, (4 until 10).map(i => Row(i, -i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q2) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i", "j")) + val batch = getBatch(q2) + assert(batch.filters.flatMap(_.references).toSet == Set("i")) + assert(batch.requiredSchema.fieldNames === Seq("i", "j")) } else { - val config = getJavaScanConfig(q2) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i", "j")) + val batch = getJavaBatch(q2) + assert(batch.filters.flatMap(_.references).toSet == Set("i")) + assert(batch.requiredSchema.fieldNames === Seq("i", "j")) } val q3 = df.select('i).filter('i > 6) checkAnswer(q3, (7 until 10).map(i => Row(i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q3) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i")) + val batch = getBatch(q3) + assert(batch.filters.flatMap(_.references).toSet == Set("i")) + assert(batch.requiredSchema.fieldNames === Seq("i")) } else { - val config = getJavaScanConfig(q3) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i")) + val batch = getJavaBatch(q3) + assert(batch.filters.flatMap(_.references).toSet == Set("i")) + assert(batch.requiredSchema.fieldNames === Seq("i")) } val q4 = df.select('j).filter('j < -10) checkAnswer(q4, Nil) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q4) + val batch = getBatch(q4) // 'j < 10 is not supported by the testing data source. - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + assert(batch.filters.isEmpty) + assert(batch.requiredSchema.fieldNames === Seq("j")) } else { - val config = getJavaScanConfig(q4) + val batch = getJavaBatch(q4) // 'j < 10 is not supported by the testing data source. - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + assert(batch.filters.isEmpty) + assert(batch.requiredSchema.fieldNames === Seq("j")) } } } @@ -279,26 +278,26 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { val q1 = df.select('i + 1) checkAnswer(q1, (1 until 11).map(i => Row(i))) - val config1 = getScanConfig(q1) - assert(config1.requiredSchema.fieldNames === Seq("i")) + val batch1 = getBatch(q1) + assert(batch1.requiredSchema.fieldNames === Seq("i")) val q2 = df.select(lit(1)) checkAnswer(q2, (0 until 10).map(i => Row(1))) - val config2 = getScanConfig(q2) - assert(config2.requiredSchema.isEmpty) + val batch2 = getBatch(q2) + assert(batch2.requiredSchema.isEmpty) // 'j === 1 can't be pushed down, but we should still be able do column pruning val q3 = df.filter('j === -1).select('j * 2) checkAnswer(q3, Row(-2)) - val config3 = getScanConfig(q3) - assert(config3.filters.isEmpty) - assert(config3.requiredSchema.fieldNames === Seq("j")) + val batch3 = getBatch(q3) + assert(batch3.filters.isEmpty) + assert(batch3.requiredSchema.fieldNames === Seq("j")) // column pruning should work with other operators. val q4 = df.sort('i).limit(1).select('i + 1) checkAnswer(q4, Row(1)) - val config4 = getScanConfig(q4) - assert(config4.requiredSchema.fieldNames === Seq("i")) + val batch4 = getBatch(q4) + assert(batch4.requiredSchema.fieldNames === Seq("i")) } test("SPARK-23315: get output from canonicalized data source v2 related plans") { @@ -374,10 +373,6 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { case class RangeInputPartition(start: Int, end: Int) extends InputPartition -case class NoopScanConfigBuilder(readSchema: StructType) extends ScanConfigBuilder with ScanConfig { - override def build(): ScanConfig = this -} - object SimpleReaderFactory extends PartitionReaderFactory { override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { val RangeInputPartition(start, end) = partition @@ -396,87 +391,68 @@ object SimpleReaderFactory extends PartitionReaderFactory { } } -abstract class SimpleReadSupport extends BatchReadSupport { - override def fullSchema(): StructType = new StructType().add("i", "int").add("j", "int") +abstract class SimpleBatchTable extends Table with SupportsBatchRead { - override def newScanConfigBuilder(): ScanConfigBuilder = { - NoopScanConfigBuilder(fullSchema()) - } + override def schema(): StructType = new StructType().add("i", "int").add("j", "int") - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - SimpleReaderFactory - } + override def name(): String = this.getClass.toString } +abstract class SimpleScanBuilder extends ScanBuilder + with Batch with Scan { + + override def build(): Scan = this + + override def toBatch: Batch = this -class SimpleSinglePartitionSource extends DataSourceV2 with BatchReadSupportProvider { + override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int") - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def createReaderFactory(): PartitionReaderFactory = SimpleReaderFactory +} + +class SimpleSinglePartitionSource extends TableProvider { + + class MyScanBuilder extends SimpleScanBuilder { + override def planInputPartitions(): Array[InputPartition] = { Array(RangeInputPartition(0, 5)) } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport + override def getTable(options: DataSourceOptions): Table = new SimpleBatchTable { + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + new MyScanBuilder() + } } } // This class is used by pyspark tests. If this class is modified/moved, make sure pyspark // tests still pass. -class SimpleDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { +class SimpleDataSourceV2 extends TableProvider { - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + class MyScanBuilder extends SimpleScanBuilder { + override def planInputPartitions(): Array[InputPartition] = { Array(RangeInputPartition(0, 5), RangeInputPartition(5, 10)) } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport + override def getTable(options: DataSourceOptions): Table = new SimpleBatchTable { + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + new MyScanBuilder() + } } } -class AdvancedDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { - - class ReadSupport extends SimpleReadSupport { - override def newScanConfigBuilder(): ScanConfigBuilder = new AdvancedScanConfigBuilder() +class AdvancedDataSourceV2 extends TableProvider { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val filters = config.asInstanceOf[AdvancedScanConfigBuilder].filters - - val lowerBound = filters.collectFirst { - case GreaterThan("i", v: Int) => v - } - - val res = scala.collection.mutable.ArrayBuffer.empty[InputPartition] - - if (lowerBound.isEmpty) { - res.append(RangeInputPartition(0, 5)) - res.append(RangeInputPartition(5, 10)) - } else if (lowerBound.get < 4) { - res.append(RangeInputPartition(lowerBound.get + 1, 5)) - res.append(RangeInputPartition(5, 10)) - } else if (lowerBound.get < 9) { - res.append(RangeInputPartition(lowerBound.get + 1, 10)) - } - - res.toArray - } - - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - val requiredSchema = config.asInstanceOf[AdvancedScanConfigBuilder].requiredSchema - new AdvancedReaderFactory(requiredSchema) + override def getTable(options: DataSourceOptions): Table = new SimpleBatchTable { + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + new AdvancedScanBuilder() } } - - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } } -class AdvancedScanConfigBuilder extends ScanConfigBuilder with ScanConfig - with SupportsPushDownRequiredColumns with SupportsPushDownFilters { +class AdvancedScanBuilder extends ScanBuilder + with Scan with SupportsPushDownFilters with SupportsPushDownRequiredColumns { var requiredSchema = new StructType().add("i", "int").add("j", "int") var filters = Array.empty[Filter] @@ -498,10 +474,40 @@ class AdvancedScanConfigBuilder extends ScanConfigBuilder with ScanConfig override def pushedFilters(): Array[Filter] = filters - override def build(): ScanConfig = this + override def build(): Scan = this + + override def toBatch: Batch = new AdvancedBatch(filters, requiredSchema) +} + +class AdvancedBatch(val filters: Array[Filter], val requiredSchema: StructType) extends Batch { + + override def planInputPartitions(): Array[InputPartition] = { + val lowerBound = filters.collectFirst { + case GreaterThan("i", v: Int) => v + } + + val res = scala.collection.mutable.ArrayBuffer.empty[InputPartition] + + if (lowerBound.isEmpty) { + res.append(RangeInputPartition(0, 5)) + res.append(RangeInputPartition(5, 10)) + } else if (lowerBound.get < 4) { + res.append(RangeInputPartition(lowerBound.get + 1, 5)) + res.append(RangeInputPartition(5, 10)) + } else if (lowerBound.get < 9) { + res.append(RangeInputPartition(lowerBound.get + 1, 10)) + } + + res.toArray + } + + override def createReaderFactory(): PartitionReaderFactory = { + new AdvancedReaderFactory(requiredSchema) + } } class AdvancedReaderFactory(requiredSchema: StructType) extends PartitionReaderFactory { + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { val RangeInputPartition(start, end) = partition new PartitionReader[InternalRow] { @@ -526,39 +532,47 @@ class AdvancedReaderFactory(requiredSchema: StructType) extends PartitionReaderF } -class SchemaRequiredDataSource extends DataSourceV2 with BatchReadSupportProvider { +class SchemaRequiredDataSource extends TableProvider { - class ReadSupport(val schema: StructType) extends SimpleReadSupport { - override def fullSchema(): StructType = schema + class MyScanBuilder(schema: StructType) extends SimpleScanBuilder { + override def planInputPartitions(): Array[InputPartition] = Array.empty - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = - Array.empty + override def readSchema(): StructType = schema } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { + override def getTable(options: DataSourceOptions): Table = { throw new IllegalArgumentException("requires a user-supplied schema") } - override def createBatchReadSupport( - schema: StructType, options: DataSourceOptions): BatchReadSupport = { - new ReadSupport(schema) + override def getTable(options: DataSourceOptions, schema: StructType): Table = { + val userGivenSchema = schema + new SimpleBatchTable { + override def schema(): StructType = userGivenSchema + + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + new MyScanBuilder(userGivenSchema) + } + } } } -class ColumnarDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { +class ColumnarDataSourceV2 extends TableProvider { - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + class MyScanBuilder extends SimpleScanBuilder { + + override def planInputPartitions(): Array[InputPartition] = { Array(RangeInputPartition(0, 50), RangeInputPartition(50, 90)) } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def createReaderFactory(): PartitionReaderFactory = { ColumnarReaderFactory } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport + override def getTable(options: DataSourceOptions): Table = new SimpleBatchTable { + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + new MyScanBuilder() + } } } @@ -608,21 +622,29 @@ object ColumnarReaderFactory extends PartitionReaderFactory { } -class PartitionAwareDataSource extends DataSourceV2 with BatchReadSupportProvider { +class PartitionAwareDataSource extends TableProvider { + + class MyScanBuilder extends SimpleScanBuilder + with SupportsReportPartitioning{ - class ReadSupport extends SimpleReadSupport with SupportsReportPartitioning { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def planInputPartitions(): Array[InputPartition] = { // Note that we don't have same value of column `a` across partitions. Array( SpecificInputPartition(Array(1, 1, 3), Array(4, 4, 6)), SpecificInputPartition(Array(2, 4, 4), Array(6, 2, 2))) } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def createReaderFactory(): PartitionReaderFactory = { SpecificReaderFactory } - override def outputPartitioning(config: ScanConfig): Partitioning = new MyPartitioning + override def outputPartitioning(): Partitioning = new MyPartitioning + } + + override def getTable(options: DataSourceOptions): Table = new SimpleBatchTable { + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + new MyScanBuilder() + } } class MyPartitioning extends Partitioning { @@ -633,10 +655,6 @@ class PartitionAwareDataSource extends DataSourceV2 with BatchReadSupportProvide case _ => false } } - - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } } case class SpecificInputPartition(i: Array[Int], j: Array[Int]) extends InputPartition @@ -662,7 +680,7 @@ object SpecificReaderFactory extends PartitionReaderFactory { class SchemaReadAttemptException(m: String) extends RuntimeException(m) class SimpleWriteOnlyDataSource extends SimpleWritableDataSource { - override def fullSchema(): StructType = { + override def writeSchema(): StructType = { // This is a bit hacky since this source implements read support but throws // during schema retrieval. Might have to rewrite but it's done // such so for minimised changes. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala index a7dfc2d1deac..82bb4fa33a3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration /** @@ -39,19 +39,16 @@ import org.apache.spark.util.SerializableConfiguration * Each job moves files from `target/_temporary/queryId/` to `target`. */ class SimpleWritableDataSource extends DataSourceV2 - with BatchReadSupportProvider + with TableProvider with BatchWriteSupportProvider with SessionConfigSupport { - protected def fullSchema(): StructType = new StructType().add("i", "long").add("j", "long") + protected def writeSchema(): StructType = new StructType().add("i", "long").add("j", "long") override def keyPrefix: String = "simpleWritableDataSource" - class ReadSupport(path: String, conf: Configuration) extends SimpleReadSupport { - - override def fullSchema(): StructType = SimpleWritableDataSource.this.fullSchema() - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + class MyScanBuilder(path: String, conf: Configuration) extends SimpleScanBuilder { + override def planInputPartitions(): Array[InputPartition] = { val dataPath = new Path(path) val fs = dataPath.getFileSystem(conf) if (fs.exists(dataPath)) { @@ -66,10 +63,24 @@ class SimpleWritableDataSource extends DataSourceV2 } } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def createReaderFactory(): PartitionReaderFactory = { val serializableConf = new SerializableConfiguration(conf) new CSVReaderFactory(serializableConf) } + + override def readSchema(): StructType = writeSchema + } + + override def getTable(options: DataSourceOptions): Table = { + val path = new Path(options.get("path").get()) + val conf = SparkContext.getActive.get.hadoopConfiguration + new SimpleBatchTable { + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + new MyScanBuilder(path.toUri.toString, conf) + } + + override def schema(): StructType = writeSchema + } } class WritSupport(queryId: String, path: String, conf: Configuration) extends BatchWriteSupport { @@ -105,12 +116,6 @@ class SimpleWritableDataSource extends DataSourceV2 } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - val path = new Path(options.get("path").get()) - val conf = SparkContext.getActive.get.hadoopConfiguration - new ReadSupport(path.toUri.toString, conf) - } - override def createBatchWriteSupport( queryId: String, schema: StructType, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala index 93eae292acc2..756092fc7ff5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming.continuous import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} import org.apache.spark.sql._ -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2StreamingScanExec import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream @@ -41,7 +41,7 @@ class ContinuousSuiteBase extends StreamTest { case s: ContinuousExecution => assert(numTriggers >= 2, "must wait for at least 2 triggers to ensure query is initialized") val reader = s.lastExecution.executedPlan.collectFirst { - case DataSourceV2ScanExec(_, _, _, _, r: RateStreamContinuousReadSupport, _) => r + case DataSourceV2StreamingScanExec(_, _, _, _, r: RateStreamContinuousReadSupport, _) => r }.get val deltaMs = numTriggers * 1000 + 300 From c3f27b2437497396913fdec96f085c3626ef4e59 Mon Sep 17 00:00:00 2001 From: Keiji Yoshida Date: Fri, 30 Nov 2018 09:03:46 -0600 Subject: [PATCH 0031/1072] [MINOR][DOCS] Fix typos ## What changes were proposed in this pull request? Fix Typos. This PR is the complete version of https://github.com/apache/spark/pull/23145. ## How was this patch tested? NA Closes #23185 from kjmrknsn/docUpdate. Authored-by: Keiji Yoshida Signed-off-by: Sean Owen --- docs/configuration.md | 2 +- docs/graphx-programming-guide.md | 4 ++-- docs/ml-datasource.md | 2 +- docs/ml-features.md | 8 ++++---- docs/ml-pipeline.md | 2 +- docs/mllib-linear-methods.md | 4 ++-- docs/security.md | 2 +- docs/sparkr.md | 2 +- 8 files changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 04210d855b11..8914bd0310f9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -498,7 +498,7 @@ Apart from these, the following properties are also available, and may be useful
    diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index cb96fd773aa5..ecedeaf958f1 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -522,7 +522,7 @@ val joinedGraph = graph.joinVertices(uniqueCosts, A key step in many graph analytics tasks is aggregating information about the neighborhood of each vertex. -For example, we might want to know the number of followers each user has or the average age of the +For example, we might want to know the number of followers each user has or the average age of the followers of each user. Many iterative graph algorithms (e.g., PageRank, Shortest Path, and connected components) repeatedly aggregate properties of neighboring vertices (e.g., current PageRank Value, shortest path to the source, and smallest reachable vertex id). @@ -700,7 +700,7 @@ a new value for the vertex property, and then send messages to neighboring verti super step. Unlike Pregel, messages are computed in parallel as a function of the edge triplet and the message computation has access to both the source and destination vertex attributes. Vertices that do not receive a message are skipped within a super -step. The Pregel operators terminates iteration and returns the final graph when there are no +step. The Pregel operator terminates iteration and returns the final graph when there are no messages remaining. > Note, unlike more standard Pregel implementations, vertices in GraphX can only send messages to diff --git a/docs/ml-datasource.md b/docs/ml-datasource.md index 15083326240a..35afaef5ad7f 100644 --- a/docs/ml-datasource.md +++ b/docs/ml-datasource.md @@ -5,7 +5,7 @@ displayTitle: Data sources --- In this section, we introduce how to use data source in ML to load data. -Beside some general data sources such as Parquet, CSV, JSON and JDBC, we also provide some specific data sources for ML. +Besides some general data sources such as Parquet, CSV, JSON and JDBC, we also provide some specific data sources for ML. **Table of Contents** diff --git a/docs/ml-features.md b/docs/ml-features.md index 83a211ce02e6..a140bc6e7a22 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -359,7 +359,7 @@ Assume that we have the following DataFrame with columns `id` and `raw`: ~~~~ id | raw ----|---------- - 0 | [I, saw, the, red, baloon] + 0 | [I, saw, the, red, balloon] 1 | [Mary, had, a, little, lamb] ~~~~ @@ -369,7 +369,7 @@ column, we should get the following: ~~~~ id | raw | filtered ----|-----------------------------|-------------------- - 0 | [I, saw, the, red, baloon] | [saw, red, baloon] + 0 | [I, saw, the, red, balloon] | [saw, red, balloon] 1 | [Mary, had, a, little, lamb]|[Mary, little, lamb] ~~~~ @@ -1302,7 +1302,7 @@ need to know vector size, can use that column as an input. To use `VectorSizeHint` a user must set the `inputCol` and `size` parameters. Applying this transformer to a dataframe produces a new dataframe with updated metadata for `inputCol` specifying the vector size. Downstream operations on the resulting dataframe can get this size using the -meatadata. +metadata. `VectorSizeHint` can also take an optional `handleInvalid` parameter which controls its behaviour when the vector column contains nulls or vectors of the wrong size. By default @@ -1310,7 +1310,7 @@ behaviour when the vector column contains nulls or vectors of the wrong size. By also be set to "skip", indicating that rows containing invalid values should be filtered out from the resulting dataframe, or "optimistic", indicating that the column should not be checked for invalid values and all rows should be kept. Note that the use of "optimistic" can cause the -resulting dataframe to be in an inconsistent state, me:aning the metadata for the column +resulting dataframe to be in an inconsistent state, meaning the metadata for the column `VectorSizeHint` was applied to does not match the contents of that column. Users should take care to avoid this kind of inconsistent state. diff --git a/docs/ml-pipeline.md b/docs/ml-pipeline.md index 8c01ccb94c75..0c9c998f6353 100644 --- a/docs/ml-pipeline.md +++ b/docs/ml-pipeline.md @@ -62,7 +62,7 @@ In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [ A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. -Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label." +Columns in a `DataFrame` are named. The code examples below use names such as "text", "features", and "label". ## Pipeline components diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 73f6e206ca54..2879d884162a 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -272,7 +272,7 @@ In `spark.mllib`, the first class $0$ is chosen as the "pivot" class. See Section 4.4 of [The Elements of Statistical Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for references. -Here is an +Here is a [detailed mathematical derivation](http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297). For multiclass classification problems, the algorithm will output a multinomial logistic regression @@ -350,7 +350,7 @@ known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_erro
    -The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. +The following example demonstrates how to load training data, parse it as an RDD of LabeledPoint. The example then uses LinearRegressionWithSGD to build a simple linear model to predict label values. We compute the mean squared error at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). diff --git a/docs/security.md b/docs/security.md index 02d581c6dad9..be4834660fb7 100644 --- a/docs/security.md +++ b/docs/security.md @@ -337,7 +337,7 @@ Configuration for SSL is organized hierarchically. The user can configure the de which will be used for all the supported communication protocols unless they are overwritten by protocol-specific settings. This way the user can easily provide the common settings for all the protocols without disabling the ability to configure each one individually. The following table -describes the the SSL configuration namespaces: +describes the SSL configuration namespaces:
    Reuse Python worker or not. If yes, it will use a fixed number of Python workers, does not need to fork() a Python process for every task. It will be very useful - if there is large broadcast, then the broadcast will not be needed to transferred + if there is a large broadcast, then the broadcast will not need to be transferred from JVM to Python worker for every task.
    diff --git a/docs/sparkr.md b/docs/sparkr.md index 5972435a0e40..0057f05de0ff 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -296,7 +296,7 @@ head(agg(rollup(df, "cyl", "disp", "gear"), avg(df$mpg))) ### Operating on Columns -SparkR also provides a number of functions that can directly applied to columns for data processing and during aggregation. The example below shows the use of basic arithmetic functions. +SparkR also provides a number of functions that can be directly applied to columns for data processing and during aggregation. The example below shows the use of basic arithmetic functions.
    {% highlight r %} From 9b23be2e95fec756066ca0ed3188c3db2602b757 Mon Sep 17 00:00:00 2001 From: schintap Date: Fri, 30 Nov 2018 12:48:56 -0600 Subject: [PATCH 0032/1072] [SPARK-26201] Fix python broadcast with encryption MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Python with rpc and disk encryption enabled along with a python broadcast variable and just read the value back on the driver side the job failed with: Traceback (most recent call last): File "broadcast.py", line 37, in words_new.value File "/pyspark.zip/pyspark/broadcast.py", line 137, in value File "pyspark.zip/pyspark/broadcast.py", line 122, in load_from_path File "pyspark.zip/pyspark/broadcast.py", line 128, in load EOFError: Ran out of input To reproduce use configs: --conf spark.network.crypto.enabled=true --conf spark.io.encryption.enabled=true Code: words_new = sc.broadcast(["scala", "java", "hadoop", "spark", "akka"]) words_new.value print(words_new.value) ## How was this patch tested? words_new = sc.broadcast([“scala”, “java”, “hadoop”, “spark”, “akka”]) textFile = sc.textFile(“README.md”) wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word + words_new.value[1], 1)).reduceByKey(lambda a, b: a+b) count = wordCounts.count() print(count) words_new.value print(words_new.value) Closes #23166 from redsanket/SPARK-26201. Authored-by: schintap Signed-off-by: Thomas Graves --- .../apache/spark/api/python/PythonRDD.scala | 29 ++++++++++++++++--- python/pyspark/broadcast.py | 21 ++++++++++---- python/pyspark/tests/test_broadcast.py | 15 ++++++++++ 3 files changed, 56 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 8b5a7a9aefea..5ed5070558af 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 @@ -660,6 +660,7 @@ private[spark] class PythonBroadcast(@transient var path: String) extends Serial with Logging { private var encryptionServer: PythonServer[Unit] = null + private var decryptionServer: PythonServer[Unit] = null /** * Read data from disks, then copy it to `out` @@ -708,16 +709,36 @@ private[spark] class PythonBroadcast(@transient var path: String) extends Serial override def handleConnection(sock: Socket): Unit = { val env = SparkEnv.get val in = sock.getInputStream() - val dir = new File(Utils.getLocalDir(env.conf)) - val file = File.createTempFile("broadcast", "", dir) - path = file.getAbsolutePath - val out = env.serializerManager.wrapForEncryption(new FileOutputStream(path)) + val abspath = new File(path).getAbsolutePath + val out = env.serializerManager.wrapForEncryption(new FileOutputStream(abspath)) DechunkedInputStream.dechunkAndCopyToOutput(in, out) } } Array(encryptionServer.port, encryptionServer.secret) } + def setupDecryptionServer(): Array[Any] = { + decryptionServer = new PythonServer[Unit]("broadcast-decrypt-server-for-driver") { + override def handleConnection(sock: Socket): Unit = { + val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream())) + Utils.tryWithSafeFinally { + val in = SparkEnv.get.serializerManager.wrapForEncryption(new FileInputStream(path)) + Utils.tryWithSafeFinally { + Utils.copyStream(in, out, false) + } { + in.close() + } + out.flush() + } { + JavaUtils.closeQuietly(out) + } + } + } + Array(decryptionServer.port, decryptionServer.secret) + } + + def waitTillBroadcastDataSent(): Unit = decryptionServer.getResult() + def waitTillDataReceived(): Unit = encryptionServer.getResult() } // scalastyle:on no.finalize diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 1c7f2a7418df..29358b5740e5 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -77,11 +77,12 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=None, # we're on the driver. We want the pickled data to end up in a file (maybe encrypted) f = NamedTemporaryFile(delete=False, dir=sc._temp_dir) self._path = f.name - python_broadcast = sc._jvm.PythonRDD.setupBroadcast(self._path) + self._sc = sc + self._python_broadcast = sc._jvm.PythonRDD.setupBroadcast(self._path) if sc._encryption_enabled: # with encryption, we ask the jvm to do the encryption for us, we send it data # over a socket - port, auth_secret = python_broadcast.setupEncryptionServer() + port, auth_secret = self._python_broadcast.setupEncryptionServer() (encryption_sock_file, _) = local_connect_and_auth(port, auth_secret) broadcast_out = ChunkedStream(encryption_sock_file, 8192) else: @@ -89,12 +90,14 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=None, broadcast_out = f self.dump(value, broadcast_out) if sc._encryption_enabled: - python_broadcast.waitTillDataReceived() - self._jbroadcast = sc._jsc.broadcast(python_broadcast) + self._python_broadcast.waitTillDataReceived() + self._jbroadcast = sc._jsc.broadcast(self._python_broadcast) self._pickle_registry = pickle_registry else: # we're on an executor self._jbroadcast = None + self._sc = None + self._python_broadcast = None if sock_file is not None: # the jvm is doing decryption for us. Read the value # immediately from the sock_file @@ -134,7 +137,15 @@ def value(self): """ Return the broadcasted value """ if not hasattr(self, "_value") and self._path is not None: - self._value = self.load_from_path(self._path) + # we only need to decrypt it here when encryption is enabled and + # if its on the driver, since executor decryption is handled already + if self._sc is not None and self._sc._encryption_enabled: + port, auth_secret = self._python_broadcast.setupDecryptionServer() + (decrypted_sock_file, _) = local_connect_and_auth(port, auth_secret) + self._python_broadcast.waitTillBroadcastDataSent() + return self.load(decrypted_sock_file) + else: + self._value = self.load_from_path(self._path) return self._value def unpersist(self, blocking=False): diff --git a/python/pyspark/tests/test_broadcast.py b/python/pyspark/tests/test_broadcast.py index a98626e8f4bc..11d31d24bb01 100644 --- a/python/pyspark/tests/test_broadcast.py +++ b/python/pyspark/tests/test_broadcast.py @@ -67,6 +67,21 @@ def test_broadcast_with_encryption(self): def test_broadcast_no_encryption(self): self._test_multiple_broadcasts() + def _test_broadcast_on_driver(self, *extra_confs): + conf = SparkConf() + for key, value in extra_confs: + conf.set(key, value) + conf.setMaster("local-cluster[2,1,1024]") + self.sc = SparkContext(conf=conf) + bs = self.sc.broadcast(value=5) + self.assertEqual(5, bs.value) + + def test_broadcast_value_driver_no_encryption(self): + self._test_broadcast_on_driver() + + def test_broadcast_value_driver_encryption(self): + self._test_broadcast_on_driver(("spark.io.encryption.enabled", "true")) + class BroadcastFrameProtocolTest(unittest.TestCase): From 36edbac1c8337a4719f90e4abd58d38738b2e1fb Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 30 Nov 2018 14:23:18 -0800 Subject: [PATCH 0033/1072] [SPARK-26226][SQL] Update query tracker to report timeline for phases ## What changes were proposed in this pull request? This patch changes the query plan tracker added earlier to report phase timeline, rather than just a duration for each phase. This way, we can easily find time that's unaccounted for. ## How was this patch tested? Updated test cases to reflect that. Closes #23183 from rxin/SPARK-26226. Authored-by: Reynold Xin Signed-off-by: gatorsmile --- .../sql/catalyst/QueryPlanningTracker.scala | 45 +++++++++++++++---- .../catalyst/QueryPlanningTrackerSuite.scala | 18 +++++--- .../org/apache/spark/sql/SparkSession.scala | 2 +- .../spark/sql/execution/QueryExecution.scala | 8 ++-- .../QueryPlanningTrackerEndToEndSuite.scala | 15 +------ 5 files changed, 55 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala index 244081cd160b..cd75407c7ee7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala @@ -41,6 +41,13 @@ object QueryPlanningTracker { val OPTIMIZATION = "optimization" val PLANNING = "planning" + /** + * Summary for a rule. + * @param totalTimeNs total amount of time, in nanosecs, spent in this rule. + * @param numInvocations number of times the rule has been invoked. + * @param numEffectiveInvocations number of times the rule has been invoked and + * resulted in a plan change. + */ class RuleSummary( var totalTimeNs: Long, var numInvocations: Long, var numEffectiveInvocations: Long) { @@ -51,6 +58,18 @@ object QueryPlanningTracker { } } + /** + * Summary of a phase, with start time and end time so we can construct a timeline. + */ + class PhaseSummary(val startTimeMs: Long, val endTimeMs: Long) { + + def durationMs: Long = endTimeMs - startTimeMs + + override def toString: String = { + s"PhaseSummary($startTimeMs, $endTimeMs)" + } + } + /** * A thread local variable to implicitly pass the tracker around. This assumes the query planner * is single-threaded, and avoids passing the same tracker context in every function call. @@ -79,15 +98,25 @@ class QueryPlanningTracker { // Use a Java HashMap for less overhead. private val rulesMap = new java.util.HashMap[String, RuleSummary] - // From a phase to time in ns. - private val phaseToTimeNs = new java.util.HashMap[String, Long] + // From a phase to its start time and end time, in ms. + private val phasesMap = new java.util.HashMap[String, PhaseSummary] - /** Measure the runtime of function f, and add it to the time for the specified phase. */ - def measureTime[T](phase: String)(f: => T): T = { - val startTime = System.nanoTime() + /** + * Measure the start and end time of a phase. Note that if this function is called multiple + * times for the same phase, the recorded start time will be the start time of the first call, + * and the recorded end time will be the end time of the last call. + */ + def measurePhase[T](phase: String)(f: => T): T = { + val startTime = System.currentTimeMillis() val ret = f - val timeTaken = System.nanoTime() - startTime - phaseToTimeNs.put(phase, phaseToTimeNs.getOrDefault(phase, 0) + timeTaken) + val endTime = System.currentTimeMillis + + if (phasesMap.containsKey(phase)) { + val oldSummary = phasesMap.get(phase) + phasesMap.put(phase, new PhaseSummary(oldSummary.startTimeMs, endTime)) + } else { + phasesMap.put(phase, new PhaseSummary(startTime, endTime)) + } ret } @@ -114,7 +143,7 @@ class QueryPlanningTracker { def rules: Map[String, RuleSummary] = rulesMap.asScala.toMap - def phases: Map[String, Long] = phaseToTimeNs.asScala.toMap + def phases: Map[String, PhaseSummary] = phasesMap.asScala.toMap /** * Returns the top k most expensive rules (as measured by time). If k is larger than the rules diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala index 120b284a7785..9593a720e424 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala @@ -23,19 +23,23 @@ class QueryPlanningTrackerSuite extends SparkFunSuite { test("phases") { val t = new QueryPlanningTracker - t.measureTime("p1") { + t.measurePhase("p1") { Thread.sleep(1) } - assert(t.phases("p1") > 0) + assert(t.phases("p1").durationMs > 0) assert(!t.phases.contains("p2")) + } - val old = t.phases("p1") + test("multiple measurePhase call") { + val t = new QueryPlanningTracker + t.measurePhase("p1") { Thread.sleep(1) } + val s1 = t.phases("p1") + assert(s1.durationMs > 0) - t.measureTime("p1") { - Thread.sleep(1) - } - assert(t.phases("p1") > old) + t.measurePhase("p1") { Thread.sleep(1) } + val s2 = t.phases("p1") + assert(s2.durationMs > s1.durationMs) } test("rules") { 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 739c6b54b4cb..26272c390668 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 @@ -649,7 +649,7 @@ class SparkSession private( */ def sql(sqlText: String): DataFrame = { val tracker = new QueryPlanningTracker - val plan = tracker.measureTime(QueryPlanningTracker.PARSING) { + val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { sessionState.sqlParser.parsePlan(sqlText) } Dataset.ofRows(self, plan, tracker) 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 cfb5e43207b0..eef5a3f899f5 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 @@ -60,7 +60,7 @@ class QueryExecution( } } - lazy val analyzed: LogicalPlan = tracker.measureTime(QueryPlanningTracker.ANALYSIS) { + lazy val analyzed: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.ANALYSIS) { SparkSession.setActiveSession(sparkSession) sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker) } @@ -71,11 +71,11 @@ class QueryExecution( sparkSession.sharedState.cacheManager.useCachedData(analyzed) } - lazy val optimizedPlan: LogicalPlan = tracker.measureTime(QueryPlanningTracker.OPTIMIZATION) { + lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { sparkSession.sessionState.optimizer.executeAndTrack(withCachedData, tracker) } - lazy val sparkPlan: SparkPlan = tracker.measureTime(QueryPlanningTracker.PLANNING) { + lazy val sparkPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { SparkSession.setActiveSession(sparkSession) // TODO: We use next(), i.e. take the first plan returned by the planner, here for now, // but we will implement to choose the best plan. @@ -84,7 +84,7 @@ class QueryExecution( // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. - lazy val executedPlan: SparkPlan = tracker.measureTime(QueryPlanningTracker.PLANNING) { + lazy val executedPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { prepareForExecution(sparkPlan) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala index 0af4c85400e9..e42177c156ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala @@ -25,12 +25,7 @@ class QueryPlanningTrackerEndToEndSuite extends SharedSQLContext { val df = spark.range(1000).selectExpr("count(*)") df.collect() val tracker = df.queryExecution.tracker - - assert(tracker.phases.size == 3) - assert(tracker.phases("analysis") > 0) - assert(tracker.phases("optimization") > 0) - assert(tracker.phases("planning") > 0) - + assert(tracker.phases.keySet == Set("analysis", "optimization", "planning")) assert(tracker.rules.nonEmpty) } @@ -39,13 +34,7 @@ class QueryPlanningTrackerEndToEndSuite extends SharedSQLContext { df.collect() val tracker = df.queryExecution.tracker - - assert(tracker.phases.size == 4) - assert(tracker.phases("parsing") > 0) - assert(tracker.phases("analysis") > 0) - assert(tracker.phases("optimization") > 0) - assert(tracker.phases("planning") > 0) - + assert(tracker.phases.keySet == Set("parsing", "analysis", "optimization", "planning")) assert(tracker.rules.nonEmpty) } From 8856e9f6a3d5c019fcae45dbbdfa9128cd700e19 Mon Sep 17 00:00:00 2001 From: Shahid Date: Fri, 30 Nov 2018 15:20:05 -0800 Subject: [PATCH 0034/1072] [SPARK-26219][CORE] Executor summary should get updated for failure jobs in the history server UI The root cause of the problem is, whenever the taskEnd event comes after stageCompleted event, execSummary is updating only for live UI. we need to update for history UI too. To see the previous discussion, refer: PR for https://github.com/apache/spark/pull/23038, https://issues.apache.org/jira/browse/SPARK-26100. Added UT. Manually verified Test step to reproduce: ``` bin/spark-shell --master yarn --conf spark.executor.instances=3 sc.parallelize(1 to 10000, 10).map{ x => throw new RuntimeException("Bad executor")}.collect() ``` Open Executors page from the History UI Before patch: ![screenshot from 2018-11-29 22-13-34](https://user-images.githubusercontent.com/23054875/49246338-a21ead00-f43a-11e8-8214-f1020420be52.png) After patch: ![screenshot from 2018-11-30 00-54-49](https://user-images.githubusercontent.com/23054875/49246353-aa76e800-f43a-11e8-98ef-7faecaa7a50e.png) Closes #23181 from shahidki31/executorUpdate. Authored-by: Shahid Signed-off-by: Marcelo Vanzin --- .../spark/status/AppStatusListener.scala | 19 ++-- .../spark/status/AppStatusListenerSuite.scala | 92 +++++++++++-------- 2 files changed, 64 insertions(+), 47 deletions(-) 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 8e845573a903..bd3f58b6182c 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -641,9 +641,14 @@ private[spark] class AppStatusListener( } } - // Force an update on live applications when the number of active tasks reaches 0. This is - // checked in some tests (e.g. SQLTestUtilsBase) so it needs to be reliably up to date. - conditionalLiveUpdate(exec, now, exec.activeTasks == 0) + // Force an update on both live and history applications when the number of active tasks + // reaches 0. This is checked in some tests (e.g. SQLTestUtilsBase) so it needs to be + // reliably up to date. + if (exec.activeTasks == 0) { + update(exec, now) + } else { + maybeUpdate(exec, now) + } } } @@ -1024,14 +1029,6 @@ private[spark] class AppStatusListener( } } - private def conditionalLiveUpdate(entity: LiveEntity, now: Long, condition: Boolean): Unit = { - if (condition) { - liveUpdate(entity, now) - } else { - maybeUpdate(entity, now) - } - } - private def cleanupExecutors(count: Long): Unit = { // Because the limit is on the number of *dead* executors, we need to calculate whether // there are actually enough dead executors to be deleted. 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 7860a0df4bb2..61fec8c1d0e4 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1273,48 +1273,68 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(allJobs.head.numFailedStages == 1) } - test("SPARK-25451: total tasks in the executor summary should match total stage tasks") { - val testConf = conf.clone.set(LIVE_ENTITY_UPDATE_PERIOD, Long.MaxValue) + Seq(true, false).foreach { live => + test(s"Total tasks in the executor summary should match total stage tasks (live = $live)") { - val listener = new AppStatusListener(store, testConf, true) + val testConf = if (live) { + conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, Long.MaxValue) + } else { + conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, -1L) + } - val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") - listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) - listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) + val listener = new AppStatusListener(store, testConf, live) - val tasks = createTasks(4, Array("1", "2")) - tasks.foreach { task => - listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task)) - } + listener.onExecutorAdded(createExecutorAddedEvent(1)) + listener.onExecutorAdded(createExecutorAddedEvent(2)) + val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") + listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) - time += 1 - tasks(0).markFinished(TaskState.FINISHED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", - Success, tasks(0), null)) - time += 1 - tasks(1).markFinished(TaskState.FINISHED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", - Success, tasks(1), null)) + val tasks = createTasks(4, Array("1", "2")) + tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task)) + } - stage.failureReason = Some("Failed") - listener.onStageCompleted(SparkListenerStageCompleted(stage)) - time += 1 - listener.onJobEnd(SparkListenerJobEnd(1, time, JobFailed(new RuntimeException("Bad Executor")))) + time += 1 + tasks(0).markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + Success, tasks(0), null)) + time += 1 + tasks(1).markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + Success, tasks(1), null)) - time += 1 - tasks(2).markFinished(TaskState.FAILED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", - ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), null)) - time += 1 - tasks(3).markFinished(TaskState.FAILED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", - ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), null)) - - val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info) - esummary.foreach { execSummary => - assert(execSummary.failedTasks === 1) - assert(execSummary.succeededTasks === 1) - assert(execSummary.killedTasks === 0) + stage.failureReason = Some("Failed") + listener.onStageCompleted(SparkListenerStageCompleted(stage)) + time += 1 + listener.onJobEnd(SparkListenerJobEnd(1, time, JobFailed( + new RuntimeException("Bad Executor")))) + + time += 1 + tasks(2).markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), null)) + time += 1 + tasks(3).markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), null)) + + val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info) + esummary.foreach { execSummary => + assert(execSummary.failedTasks === 1) + assert(execSummary.succeededTasks === 1) + assert(execSummary.killedTasks === 0) + } + + val allExecutorSummary = store.view(classOf[ExecutorSummaryWrapper]).asScala.map(_.info) + assert(allExecutorSummary.size === 2) + allExecutorSummary.foreach { allExecSummary => + assert(allExecSummary.failedTasks === 1) + assert(allExecSummary.activeTasks === 0) + assert(allExecSummary.completedTasks === 1) + } + store.delete(classOf[ExecutorSummaryWrapper], "1") + store.delete(classOf[ExecutorSummaryWrapper], "2") } } From 6be272b75b4ae3149869e19df193675cc4117763 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 30 Nov 2018 16:23:37 -0800 Subject: [PATCH 0035/1072] [SPARK-25876][K8S] Simplify kubernetes configuration types. There are a few issues with the current configuration types used in the kubernetes backend: - they use type parameters for role-specific specialization, which makes type signatures really noisy throughout the code base. - they break encapsulation by forcing the code that creates the config object to remove the configuration from SparkConf before creating the k8s-specific wrapper. - they don't provide an easy way for tests to have default values for fields they do not use. This change fixes those problems by: - creating a base config type with role-specific specialization using inheritance - encapsulating the logic of parsing SparkConf into k8s-specific views inside the k8s config classes - providing some helper code for tests to easily override just the part of the configs they want. Most of the change relates to the above, especially cleaning up the tests. While doing that, I also made some smaller changes elsewhere: - removed unnecessary type parameters in KubernetesVolumeSpec - simplified the error detection logic in KubernetesVolumeUtils; all the call sites would just throw the first exception collected by that class, since they all called "get" on the "Try" object. Now the unnecessary wrapping is gone and the exception is just thrown where it occurs. - removed a lot of unnecessary mocking from tests. - changed the kerberos-related code so that less logic needs to live in the driver builder. In spirit it should be part of the upcoming work in this series of cleanups, but it made parts of this change simpler. Tested with existing unit tests and integration tests. Author: Marcelo Vanzin Closes #22959 from vanzin/SPARK-25876. --- .../org/apache/spark/deploy/k8s/Config.scala | 17 +- .../spark/deploy/k8s/KubernetesConf.scala | 302 ++++++++---------- .../deploy/k8s/KubernetesVolumeSpec.scala | 10 +- .../deploy/k8s/KubernetesVolumeUtils.scala | 53 +-- .../k8s/features/BasicDriverFeatureStep.scala | 24 +- .../features/BasicExecutorFeatureStep.scala | 29 +- .../features/DriverCommandFeatureStep.scala | 22 +- ...iverKubernetesCredentialsFeatureStep.scala | 6 +- .../features/DriverServiceFeatureStep.scala | 10 +- .../k8s/features/EnvSecretsFeatureStep.scala | 11 +- .../HadoopConfExecutorFeatureStep.scala | 14 +- .../HadoopSparkUserExecutorFeatureStep.scala | 17 +- .../KerberosConfDriverFeatureStep.scala | 113 ++++--- .../KerberosConfExecutorFeatureStep.scala | 21 +- .../k8s/features/LocalDirsFeatureStep.scala | 9 +- .../features/MountSecretsFeatureStep.scala | 13 +- .../features/MountVolumesFeatureStep.scala | 11 +- .../features/PodTemplateConfigMapStep.scala | 5 +- .../hadooputils/HadoopKerberosLogin.scala | 64 ---- ...bernetesHadoopDelegationTokenManager.scala | 37 --- .../submit/KubernetesClientApplication.scala | 61 +--- .../k8s/submit/KubernetesDriverBuilder.scala | 53 ++- .../k8s/KubernetesExecutorBuilder.scala | 36 +-- .../deploy/k8s/KubernetesConfSuite.scala | 71 ++-- .../spark/deploy/k8s/KubernetesTestConf.scala | 138 ++++++++ .../k8s/KubernetesVolumeUtilsSuite.scala | 30 +- .../BasicDriverFeatureStepSuite.scala | 127 ++------ .../BasicExecutorFeatureStepSuite.scala | 103 ++---- .../DriverCommandFeatureStepSuite.scala | 29 +- ...ubernetesCredentialsFeatureStepSuite.scala | 69 +--- .../DriverServiceFeatureStepSuite.scala | 193 ++++------- .../features/EnvSecretsFeatureStepSuite.scala | 32 +- .../features/LocalDirsFeatureStepSuite.scala | 46 +-- .../MountSecretsFeatureStepSuite.scala | 21 +- .../MountVolumesFeatureStepSuite.scala | 56 ++-- .../PodTemplateConfigMapStepSuite.scala | 28 +- .../spark/deploy/k8s/submit/ClientSuite.scala | 47 +-- .../submit/KubernetesDriverBuilderSuite.scala | 204 ++---------- .../k8s/ExecutorPodsAllocatorSuite.scala | 43 +-- .../k8s/KubernetesExecutorBuilderSuite.scala | 114 ++----- 40 files changed, 777 insertions(+), 1512 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 724acd231a6c..1abf2901268f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -60,7 +60,8 @@ private[spark] object Config extends Logging { .doc("Comma separated list of the Kubernetes secrets used " + "to access private image registries.") .stringConf - .createOptional + .toSequence + .createWithDefault(Nil) val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver" @@ -112,16 +113,16 @@ private[spark] object Config extends Logging { .stringConf .createOptional - val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = - ConfigBuilder("spark.kubernetes.executor.podNamePrefix") - .doc("Prefix to use in front of the executor pod names.") + // For testing only. + val KUBERNETES_DRIVER_POD_NAME_PREFIX = + ConfigBuilder("spark.kubernetes.driver.resourceNamePrefix") .internal() .stringConf - .createWithDefault("spark") + .createOptional - val KUBERNETES_PYSPARK_PY_FILES = - ConfigBuilder("spark.kubernetes.python.pyFiles") - .doc("The PyFiles that are distributed via client arguments") + val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = + ConfigBuilder("spark.kubernetes.executor.podNamePrefix") + .doc("Prefix to use in front of the executor pod names.") .internal() .stringConf .createOptional diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index ebb81540bbbb..a06c21b47f15 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -16,93 +16,53 @@ */ package org.apache.spark.deploy.k8s -import scala.collection.mutable +import java.util.Locale import io.fabric8.kubernetes.api.model.{LocalObjectReference, LocalObjectReferenceBuilder, Pod} -import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager import org.apache.spark.deploy.k8s.submit._ -import org.apache.spark.deploy.k8s.submit.KubernetesClientApplication._ import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.util.Utils - -private[spark] sealed trait KubernetesRoleSpecificConf - -/* - * Structure containing metadata for Kubernetes logic that builds a Spark driver. - */ -private[spark] case class KubernetesDriverSpecificConf( - mainAppResource: MainAppResource, - mainClass: String, - appName: String, - appArgs: Seq[String], - pyFiles: Seq[String] = Nil) extends KubernetesRoleSpecificConf { - - require(mainAppResource != null, "Main resource must be provided.") - -} - -/* - * Structure containing metadata for Kubernetes logic that builds a Spark executor. - */ -private[spark] case class KubernetesExecutorSpecificConf( - executorId: String, - driverPod: Option[Pod]) - extends KubernetesRoleSpecificConf - -/* - * Structure containing metadata for HADOOP_CONF_DIR customization - */ -private[spark] case class HadoopConfSpec( - hadoopConfDir: Option[String], - hadoopConfigMapName: Option[String]) - /** * Structure containing metadata for Kubernetes logic to build Spark pods. */ -private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( - sparkConf: SparkConf, - roleSpecificConf: T, - appResourceNamePrefix: String, - appId: String, - roleLabels: Map[String, String], - roleAnnotations: Map[String, String], - roleSecretNamesToMountPaths: Map[String, String], - roleSecretEnvNamesToKeyRefs: Map[String, String], - roleEnvs: Map[String, String], - roleVolumes: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]], - hadoopConfSpec: Option[HadoopConfSpec]) { +private[spark] abstract class KubernetesConf(val sparkConf: SparkConf) { - def hadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" + val resourceNamePrefix: String + def labels: Map[String, String] + def environment: Map[String, String] + def annotations: Map[String, String] + def secretEnvNamesToKeyRefs: Map[String, String] + def secretNamesToMountPaths: Map[String, String] + def volumes: Seq[KubernetesVolumeSpec] - def krbConfigMapName: String = s"$appResourceNamePrefix-krb5-file" + def appName: String = get("spark.app.name", "spark") - def tokenManager(conf: SparkConf, hConf: Configuration): KubernetesHadoopDelegationTokenManager = - new KubernetesHadoopDelegationTokenManager(conf, hConf) + def hadoopConfigMapName: String = s"$resourceNamePrefix-hadoop-config" - def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) + def krbConfigMapName: String = s"$resourceNamePrefix-krb5-file" - def imagePullPolicy(): String = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) + def namespace: String = get(KUBERNETES_NAMESPACE) - def imagePullSecrets(): Seq[LocalObjectReference] = { + def imagePullPolicy: String = get(CONTAINER_IMAGE_PULL_POLICY) + + def imagePullSecrets: Seq[LocalObjectReference] = { sparkConf .get(IMAGE_PULL_SECRETS) - .map(_.split(",")) - .getOrElse(Array.empty[String]) - .map(_.trim) .map { secret => new LocalObjectReferenceBuilder().withName(secret).build() } } - def nodeSelector(): Map[String, String] = + def nodeSelector: Map[String, String] = KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) + def contains(config: ConfigEntry[_]): Boolean = sparkConf.contains(config) + def get[T](config: ConfigEntry[T]): T = sparkConf.get(config) def get(conf: String): String = sparkConf.get(conf) @@ -112,125 +72,139 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def getOption(key: String): Option[String] = sparkConf.getOption(key) } +private[spark] class KubernetesDriverConf( + sparkConf: SparkConf, + val appId: String, + val mainAppResource: MainAppResource, + val mainClass: String, + val appArgs: Array[String], + val pyFiles: Seq[String]) + extends KubernetesConf(sparkConf) { + + override val resourceNamePrefix: String = { + val custom = if (Utils.isTesting) get(KUBERNETES_DRIVER_POD_NAME_PREFIX) else None + custom.getOrElse(KubernetesConf.getResourceNamePrefix(appName)) + } + + override def labels: Map[String, String] = { + val presetLabels = Map( + SPARK_APP_ID_LABEL -> appId, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) + + presetLabels.keys.foreach { key => + require( + !driverCustomLabels.contains(key), + s"Label with key $key is not allowed as it is reserved for Spark bookkeeping operations.") + } + + driverCustomLabels ++ presetLabels + } + + override def environment: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_ENV_PREFIX) + } + + override def annotations: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) + } + + override def secretNamesToMountPaths: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) + } + + override def secretEnvNamesToKeyRefs: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX) + } + + override def volumes: Seq[KubernetesVolumeSpec] = { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_DRIVER_VOLUMES_PREFIX) + } +} + +private[spark] class KubernetesExecutorConf( + sparkConf: SparkConf, + val appId: String, + val executorId: String, + val driverPod: Option[Pod]) + extends KubernetesConf(sparkConf) { + + override val resourceNamePrefix: String = { + get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX).getOrElse( + KubernetesConf.getResourceNamePrefix(appName)) + } + + override def labels: Map[String, String] = { + val presetLabels = Map( + SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> appId, + SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) + + val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) + + presetLabels.keys.foreach { key => + require( + !executorCustomLabels.contains(key), + s"Custom executor labels cannot contain $key as it is reserved for Spark.") + } + + executorCustomLabels ++ presetLabels + } + + override def environment: Map[String, String] = sparkConf.getExecutorEnv.toMap + + override def annotations: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) + } + + override def secretNamesToMountPaths: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) + } + + override def secretEnvNamesToKeyRefs: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX) + } + + override def volumes: Seq[KubernetesVolumeSpec] = { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX) + } + +} + private[spark] object KubernetesConf { def createDriverConf( sparkConf: SparkConf, - appName: String, - appResourceNamePrefix: String, appId: String, mainAppResource: MainAppResource, mainClass: String, appArgs: Array[String], - maybePyFiles: Option[String], - hadoopConfDir: Option[String]): KubernetesConf[KubernetesDriverSpecificConf] = { - val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) - require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), "Label with key " + - s"$SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping " + - "operations.") - require(!driverCustomLabels.contains(SPARK_ROLE_LABEL), "Label with key " + - s"$SPARK_ROLE_LABEL is not allowed as it is reserved for Spark bookkeeping " + - "operations.") - val driverLabels = driverCustomLabels ++ Map( - SPARK_APP_ID_LABEL -> appId, - SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - val driverAnnotations = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) - val driverSecretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) - val driverSecretEnvNamesToKeyRefs = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX) - val driverEnvs = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_ENV_PREFIX) - val driverVolumes = KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_DRIVER_VOLUMES_PREFIX).map(_.get) - // Also parse executor volumes in order to verify configuration - // before the driver pod is created - KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX).map(_.get) - - val hadoopConfigMapName = sparkConf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) - KubernetesUtils.requireNandDefined( - hadoopConfDir, - hadoopConfigMapName, - "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + - "as the creation of an additional ConfigMap, when one is already specified is extraneous" ) - val hadoopConfSpec = - if (hadoopConfDir.isDefined || hadoopConfigMapName.isDefined) { - Some(HadoopConfSpec(hadoopConfDir, hadoopConfigMapName)) - } else { - None - } - val pyFiles = maybePyFiles.map(Utils.stringToSeq).getOrElse(Nil) + maybePyFiles: Option[String]): KubernetesDriverConf = { + // Parse executor volumes in order to verify configuration before the driver pod is created. + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX) - - KubernetesConf( - sparkConf.clone(), - KubernetesDriverSpecificConf(mainAppResource, mainClass, appName, appArgs, pyFiles), - appResourceNamePrefix, - appId, - driverLabels, - driverAnnotations, - driverSecretNamesToMountPaths, - driverSecretEnvNamesToKeyRefs, - driverEnvs, - driverVolumes, - hadoopConfSpec) + val pyFiles = maybePyFiles.map(Utils.stringToSeq).getOrElse(Nil) + new KubernetesDriverConf(sparkConf.clone(), appId, mainAppResource, mainClass, appArgs, + pyFiles) } def createExecutorConf( sparkConf: SparkConf, executorId: String, appId: String, - driverPod: Option[Pod]): KubernetesConf[KubernetesExecutorSpecificConf] = { - val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) - require( - !executorCustomLabels.contains(SPARK_APP_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") - require( - !executorCustomLabels.contains(SPARK_EXECUTOR_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + - " Spark.") - require( - !executorCustomLabels.contains(SPARK_ROLE_LABEL), - s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.") - val executorLabels = Map( - SPARK_EXECUTOR_ID_LABEL -> executorId, - SPARK_APP_ID_LABEL -> appId, - SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ - executorCustomLabels - val executorAnnotations = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) - val executorMountSecrets = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) - val executorEnvSecrets = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX) - val executorEnv = sparkConf.getExecutorEnv.toMap - val executorVolumes = KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX).map(_.get) - - // If no prefix is defined then we are in pure client mode - // (not the one used by cluster mode inside the container) - val appResourceNamePrefix = { - if (sparkConf.getOption(KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key).isEmpty) { - getResourceNamePrefix(getAppName(sparkConf)) - } else { - sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) - } - } + driverPod: Option[Pod]): KubernetesExecutorConf = { + new KubernetesExecutorConf(sparkConf.clone(), appId, executorId, driverPod) + } - KubernetesConf( - sparkConf.clone(), - KubernetesExecutorSpecificConf(executorId, driverPod), - appResourceNamePrefix, - appId, - executorLabels, - executorAnnotations, - executorMountSecrets, - executorEnvSecrets, - executorEnv, - executorVolumes, - None) + def getResourceNamePrefix(appName: String): String = { + val launchTime = System.currentTimeMillis() + s"$appName-$launchTime" + .trim + .toLowerCase(Locale.ROOT) + .replaceAll("\\s+", "-") + .replaceAll("\\.", "-") + .replaceAll("[^a-z0-9\\-]", "") + .replaceAll("-+", "-") } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala index 1a214fad9661..0ebe8fd26015 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala @@ -18,12 +18,10 @@ package org.apache.spark.deploy.k8s private[spark] sealed trait KubernetesVolumeSpecificConf -private[spark] case class KubernetesHostPathVolumeConf( - hostPath: String) +private[spark] case class KubernetesHostPathVolumeConf(hostPath: String) extends KubernetesVolumeSpecificConf -private[spark] case class KubernetesPVCVolumeConf( - claimName: String) +private[spark] case class KubernetesPVCVolumeConf(claimName: String) extends KubernetesVolumeSpecificConf private[spark] case class KubernetesEmptyDirVolumeConf( @@ -31,9 +29,9 @@ private[spark] case class KubernetesEmptyDirVolumeConf( sizeLimit: Option[String]) extends KubernetesVolumeSpecificConf -private[spark] case class KubernetesVolumeSpec[T <: KubernetesVolumeSpecificConf]( +private[spark] case class KubernetesVolumeSpec( volumeName: String, mountPath: String, mountSubPath: String, mountReadOnly: Boolean, - volumeConf: T) + volumeConf: KubernetesVolumeSpecificConf) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala index 155326469235..c0c4f86f1a6a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala @@ -16,10 +16,6 @@ */ package org.apache.spark.deploy.k8s -import java.util.NoSuchElementException - -import scala.util.{Failure, Success, Try} - import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ @@ -31,9 +27,7 @@ private[spark] object KubernetesVolumeUtils { * @param prefix the given property name prefix * @return a Map storing with volume name as key and spec as value */ - def parseVolumesWithPrefix( - sparkConf: SparkConf, - prefix: String): Iterable[Try[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]]] = { + def parseVolumesWithPrefix(sparkConf: SparkConf, prefix: String): Seq[KubernetesVolumeSpec] = { val properties = sparkConf.getAllWithPrefix(prefix).toMap getVolumeTypesAndNames(properties).map { case (volumeType, volumeName) => @@ -41,17 +35,13 @@ private[spark] object KubernetesVolumeUtils { val readOnlyKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_MOUNT_READONLY_KEY" val subPathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_MOUNT_SUBPATH_KEY" - for { - path <- properties.getTry(pathKey) - volumeConf <- parseVolumeSpecificConf(properties, volumeType, volumeName) - } yield KubernetesVolumeSpec( + KubernetesVolumeSpec( volumeName = volumeName, - mountPath = path, + mountPath = properties(pathKey), mountSubPath = properties.get(subPathKey).getOrElse(""), mountReadOnly = properties.get(readOnlyKey).exists(_.toBoolean), - volumeConf = volumeConf - ) - } + volumeConf = parseVolumeSpecificConf(properties, volumeType, volumeName)) + }.toSeq } /** @@ -61,9 +51,7 @@ private[spark] object KubernetesVolumeUtils { * @param properties flat mapping of property names to values * @return Set[(volumeType, volumeName)] */ - private def getVolumeTypesAndNames( - properties: Map[String, String] - ): Set[(String, String)] = { + private def getVolumeTypesAndNames(properties: Map[String, String]): Set[(String, String)] = { properties.keys.flatMap { k => k.split('.').toList match { case tpe :: name :: _ => Some((tpe, name)) @@ -73,40 +61,25 @@ private[spark] object KubernetesVolumeUtils { } private def parseVolumeSpecificConf( - options: Map[String, String], - volumeType: String, - volumeName: String): Try[KubernetesVolumeSpecificConf] = { + options: Map[String, String], + volumeType: String, + volumeName: String): KubernetesVolumeSpecificConf = { volumeType match { case KUBERNETES_VOLUMES_HOSTPATH_TYPE => val pathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_PATH_KEY" - for { - path <- options.getTry(pathKey) - } yield KubernetesHostPathVolumeConf(path) + KubernetesHostPathVolumeConf(options(pathKey)) case KUBERNETES_VOLUMES_PVC_TYPE => val claimNameKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_CLAIM_NAME_KEY" - for { - claimName <- options.getTry(claimNameKey) - } yield KubernetesPVCVolumeConf(claimName) + KubernetesPVCVolumeConf(options(claimNameKey)) case KUBERNETES_VOLUMES_EMPTYDIR_TYPE => val mediumKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_MEDIUM_KEY" val sizeLimitKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_SIZE_LIMIT_KEY" - Success(KubernetesEmptyDirVolumeConf(options.get(mediumKey), options.get(sizeLimitKey))) + KubernetesEmptyDirVolumeConf(options.get(mediumKey), options.get(sizeLimitKey)) case _ => - Failure(new RuntimeException(s"Kubernetes Volume type `$volumeType` is not supported")) - } - } - - /** - * Convenience wrapper to accumulate key lookup errors - */ - implicit private class MapOps[A, B](m: Map[A, B]) { - def getTry(key: A): Try[B] = { - m - .get(key) - .fold[Try[B]](Failure(new NoSuchElementException(key.toString)))(Success(_)) + throw new IllegalArgumentException(s"Kubernetes Volume type `$volumeType` is not supported") } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 5ddf73cb16a6..d8cf3653d322 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -30,13 +30,12 @@ import org.apache.spark.internal.config._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils -private[spark] class BasicDriverFeatureStep( - conf: KubernetesConf[KubernetesDriverSpecificConf]) +private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) extends KubernetesFeatureConfigStep { private val driverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(s"${conf.appResourceNamePrefix}-driver") + .getOrElse(s"${conf.resourceNamePrefix}-driver") private val driverContainerImage = conf .get(DRIVER_CONTAINER_IMAGE) @@ -52,8 +51,8 @@ private[spark] class BasicDriverFeatureStep( // The memory overhead factor to use. If the user has not set it, then use a different // value for non-JVM apps. This value is propagated to executors. private val overheadFactor = - if (conf.roleSpecificConf.mainAppResource.isInstanceOf[NonJVMResource]) { - if (conf.sparkConf.contains(MEMORY_OVERHEAD_FACTOR)) { + if (conf.mainAppResource.isInstanceOf[NonJVMResource]) { + if (conf.contains(MEMORY_OVERHEAD_FACTOR)) { conf.get(MEMORY_OVERHEAD_FACTOR) } else { NON_JVM_MEMORY_OVERHEAD_FACTOR @@ -68,8 +67,7 @@ private[spark] class BasicDriverFeatureStep( private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB override def configurePod(pod: SparkPod): SparkPod = { - val driverCustomEnvs = conf.roleEnvs - .toSeq + val driverCustomEnvs = conf.environment.toSeq .map { env => new EnvVarBuilder() .withName(env._1) @@ -96,7 +94,7 @@ private[spark] class BasicDriverFeatureStep( val driverContainer = new ContainerBuilder(pod.container) .withName(Option(pod.container.getName).getOrElse(DEFAULT_DRIVER_CONTAINER_NAME)) .withImage(driverContainerImage) - .withImagePullPolicy(conf.imagePullPolicy()) + .withImagePullPolicy(conf.imagePullPolicy) .addNewPort() .withName(DRIVER_PORT_NAME) .withContainerPort(driverPort) @@ -130,13 +128,13 @@ private[spark] class BasicDriverFeatureStep( val driverPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(driverPodName) - .addToLabels(conf.roleLabels.asJava) - .addToAnnotations(conf.roleAnnotations.asJava) + .addToLabels(conf.labels.asJava) + .addToAnnotations(conf.annotations.asJava) .endMetadata() .editOrNewSpec() .withRestartPolicy("Never") - .addToNodeSelector(conf.nodeSelector().asJava) - .addToImagePullSecrets(conf.imagePullSecrets(): _*) + .addToNodeSelector(conf.nodeSelector.asJava) + .addToImagePullSecrets(conf.imagePullSecrets: _*) .endSpec() .build() @@ -147,7 +145,7 @@ private[spark] class BasicDriverFeatureStep( val additionalProps = mutable.Map( KUBERNETES_DRIVER_POD_NAME.key -> driverPodName, "spark.app.id" -> conf.appId, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> conf.appResourceNamePrefix, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> conf.resourceNamePrefix, KUBERNETES_DRIVER_SUBMIT_CHECK.key -> "true", MEMORY_OVERHEAD_FACTOR.key -> overheadFactor.toString) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 7f397e6e84fa..8bf315248388 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -29,8 +29,7 @@ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils -private[spark] class BasicExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) +private[spark] class BasicExecutorFeatureStep(kubernetesConf: KubernetesExecutorConf) extends KubernetesFeatureConfigStep { // Consider moving some of these fields to KubernetesConf or KubernetesExecutorSpecificConf @@ -42,7 +41,7 @@ private[spark] class BasicExecutorFeatureStep( .sparkConf .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) - private val executorPodNamePrefix = kubernetesConf.appResourceNamePrefix + private val executorPodNamePrefix = kubernetesConf.resourceNamePrefix private val driverUrl = RpcEndpointAddress( kubernetesConf.get("spark.driver.host"), @@ -76,7 +75,7 @@ private[spark] class BasicExecutorFeatureStep( private val executorLimitCores = kubernetesConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES) override def configurePod(pod: SparkPod): SparkPod = { - val name = s"$executorPodNamePrefix-exec-${kubernetesConf.roleSpecificConf.executorId}" + val name = s"$executorPodNamePrefix-exec-${kubernetesConf.executorId}" // hostname must be no longer than 63 characters, so take the last 63 characters of the pod // name as the hostname. This preserves uniqueness since the end of name contains @@ -98,7 +97,7 @@ private[spark] class BasicExecutorFeatureStep( .get(EXECUTOR_JAVA_OPTIONS) .map { opts => val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId, - kubernetesConf.roleSpecificConf.executorId) + kubernetesConf.executorId) val delimitedOpts = Utils.splitCommandString(subsOpts) delimitedOpts.zipWithIndex.map { case (opt, index) => @@ -112,8 +111,8 @@ private[spark] class BasicExecutorFeatureStep( (ENV_APPLICATION_ID, kubernetesConf.appId), // This is to set the SPARK_CONF_DIR to be /opt/spark/conf (ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL), - (ENV_EXECUTOR_ID, kubernetesConf.roleSpecificConf.executorId)) ++ - kubernetesConf.roleEnvs) + (ENV_EXECUTOR_ID, kubernetesConf.executorId)) ++ + kubernetesConf.environment) .map(env => new EnvVarBuilder() .withName(env._1) .withValue(env._2) @@ -138,7 +137,7 @@ private[spark] class BasicExecutorFeatureStep( val executorContainer = new ContainerBuilder(pod.container) .withName(Option(pod.container.getName).getOrElse(DEFAULT_EXECUTOR_CONTAINER_NAME)) .withImage(executorContainerImage) - .withImagePullPolicy(kubernetesConf.imagePullPolicy()) + .withImagePullPolicy(kubernetesConf.imagePullPolicy) .editOrNewResources() .addToRequests("memory", executorMemoryQuantity) .addToLimits("memory", executorMemoryQuantity) @@ -158,27 +157,27 @@ private[spark] class BasicExecutorFeatureStep( .endResources() .build() }.getOrElse(executorContainer) - val driverPod = kubernetesConf.roleSpecificConf.driverPod - val ownerReference = driverPod.map(pod => + val ownerReference = kubernetesConf.driverPod.map { pod => new OwnerReferenceBuilder() .withController(true) .withApiVersion(pod.getApiVersion) .withKind(pod.getKind) .withName(pod.getMetadata.getName) .withUid(pod.getMetadata.getUid) - .build()) + .build() + } val executorPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(name) - .addToLabels(kubernetesConf.roleLabels.asJava) - .addToAnnotations(kubernetesConf.roleAnnotations.asJava) + .addToLabels(kubernetesConf.labels.asJava) + .addToAnnotations(kubernetesConf.annotations.asJava) .addToOwnerReferences(ownerReference.toSeq: _*) .endMetadata() .editOrNewSpec() .withHostname(hostname) .withRestartPolicy("Never") - .addToNodeSelector(kubernetesConf.nodeSelector().asJava) - .addToImagePullSecrets(kubernetesConf.imagePullSecrets(): _*) + .addToNodeSelector(kubernetesConf.nodeSelector.asJava) + .addToImagePullSecrets(kubernetesConf.imagePullSecrets: _*) .endSpec() .build() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala index 8b8f0d01d49f..76b4ec98d494 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala @@ -32,13 +32,11 @@ import org.apache.spark.util.Utils * Creates the driver command for running the user app, and propagates needed configuration so * executors can also find the app code. */ -private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDriverSpecificConf]) +private[spark] class DriverCommandFeatureStep(conf: KubernetesDriverConf) extends KubernetesFeatureConfigStep { - private val driverConf = conf.roleSpecificConf - override def configurePod(pod: SparkPod): SparkPod = { - driverConf.mainAppResource match { + conf.mainAppResource match { case JavaMainAppResource(_) => configureForJava(pod) @@ -51,7 +49,7 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDri } override def getAdditionalPodSystemProperties(): Map[String, String] = { - driverConf.mainAppResource match { + conf.mainAppResource match { case JavaMainAppResource(res) => res.map(additionalJavaProperties).getOrElse(Map.empty) @@ -71,10 +69,10 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDri } private def configureForPython(pod: SparkPod, res: String): SparkPod = { - val maybePythonFiles = if (driverConf.pyFiles.nonEmpty) { + val maybePythonFiles = if (conf.pyFiles.nonEmpty) { // Delineation by ":" is to append the PySpark Files to the PYTHONPATH // of the respective PySpark pod - val resolved = KubernetesUtils.resolveFileUrisAndPath(driverConf.pyFiles) + val resolved = KubernetesUtils.resolveFileUrisAndPath(conf.pyFiles) Some(new EnvVarBuilder() .withName(ENV_PYSPARK_FILES) .withValue(resolved.mkString(":")) @@ -85,7 +83,7 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDri val pythonEnvs = Seq(new EnvVarBuilder() .withName(ENV_PYSPARK_MAJOR_PYTHON_VERSION) - .withValue(conf.sparkConf.get(PYSPARK_MAJOR_PYTHON_VERSION)) + .withValue(conf.get(PYSPARK_MAJOR_PYTHON_VERSION)) .build()) ++ maybePythonFiles @@ -105,9 +103,9 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDri new ContainerBuilder(pod.container) .addToArgs("driver") .addToArgs("--properties-file", SPARK_CONF_PATH) - .addToArgs("--class", driverConf.mainClass) + .addToArgs("--class", conf.mainClass) .addToArgs(resource) - .addToArgs(driverConf.appArgs: _*) + .addToArgs(conf.appArgs: _*) } private def additionalJavaProperties(resource: String): Map[String, String] = { @@ -116,7 +114,7 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDri private def additionalPythonProperties(resource: String): Map[String, String] = { resourceType(APP_RESOURCE_TYPE_PYTHON) ++ - mergeFileList("spark.files", Seq(resource) ++ driverConf.pyFiles) + mergeFileList("spark.files", Seq(resource) ++ conf.pyFiles) } private def additionalRProperties(resource: String): Map[String, String] = { @@ -124,7 +122,7 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDri } private def mergeFileList(key: String, filesToAdd: Seq[String]): Map[String, String] = { - val existing = Utils.stringToSeq(conf.sparkConf.get(key, "")) + val existing = Utils.stringToSeq(conf.get(key, "")) Map(key -> (existing ++ filesToAdd).distinct.mkString(",")) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala index ff5ad6673b30..795ca49a3c87 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf[_]) +private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { // TODO clean up this class, and credentials in general. See also SparkKubernetesClientFactory. // We should use a struct to hold all creds-related fields. A lot of the code is very repetitive. @@ -66,7 +66,7 @@ private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: Kube clientCertDataBase64.isDefined private val driverCredentialsSecretName = - s"${kubernetesConf.appResourceNamePrefix}-kubernetes-credentials" + s"${kubernetesConf.resourceNamePrefix}-kubernetes-credentials" override def configurePod(pod: SparkPod): SparkPod = { if (!shouldMountSecret) { @@ -122,7 +122,7 @@ private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: Kube val redactedTokens = kubernetesConf.sparkConf.getAll .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)) .toMap - .mapValues( _ => "") + .map { case (k, v) => (k, "") } redactedTokens ++ resolvedMountedCaCertFile.map { file => Map( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala index f2d7bbd08f30..42305457f4ff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala @@ -20,13 +20,13 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{HasMetadata, ServiceBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesDriverConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging import org.apache.spark.util.{Clock, SystemClock} private[spark] class DriverServiceFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf], + kubernetesConf: KubernetesDriverConf, clock: Clock = new SystemClock) extends KubernetesFeatureConfigStep with Logging { import DriverServiceFeatureStep._ @@ -38,7 +38,7 @@ private[spark] class DriverServiceFeatureStep( s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be " + "managed via a Kubernetes service.") - private val preferredServiceName = s"${kubernetesConf.appResourceNamePrefix}$DRIVER_SVC_POSTFIX" + private val preferredServiceName = s"${kubernetesConf.resourceNamePrefix}$DRIVER_SVC_POSTFIX" private val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { preferredServiceName } else { @@ -58,7 +58,7 @@ private[spark] class DriverServiceFeatureStep( override def configurePod(pod: SparkPod): SparkPod = pod override def getAdditionalPodSystemProperties(): Map[String, String] = { - val driverHostname = s"$resolvedServiceName.${kubernetesConf.namespace()}.svc" + val driverHostname = s"$resolvedServiceName.${kubernetesConf.namespace}.svc" Map(DRIVER_HOST_KEY -> driverHostname, "spark.driver.port" -> driverPort.toString, org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key -> @@ -72,7 +72,7 @@ private[spark] class DriverServiceFeatureStep( .endMetadata() .withNewSpec() .withClusterIP("None") - .withSelector(kubernetesConf.roleLabels.asJava) + .withSelector(kubernetesConf.labels.asJava) .addNewPort() .withName(DRIVER_PORT_NAME) .withPort(driverPort) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala index 03ff7d48420f..d78f04dcc40e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala @@ -20,14 +20,13 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -private[spark] class EnvSecretsFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class EnvSecretsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { val addedEnvSecrets = kubernetesConf - .roleSecretEnvNamesToKeyRefs + .secretEnvNamesToKeyRefs .map{ case (envName, keyRef) => // Keyref parts val keyRefParts = keyRef.split(":") @@ -50,8 +49,4 @@ private[spark] class EnvSecretsFeatureStep( .build() SparkPod(pod.pod, containerWithEnvVars) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala index fd09de2a918a..bca66759d586 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala @@ -16,9 +16,7 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil import org.apache.spark.internal.Logging @@ -28,21 +26,15 @@ import org.apache.spark.internal.Logging * containing Hadoop config files mounted as volumes and an ENV variable * pointed to the mounted file directory. */ -private[spark] class HadoopConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) +private[spark] class HadoopConfExecutorFeatureStep(conf: KubernetesExecutorConf) extends KubernetesFeatureConfigStep with Logging { override def configurePod(pod: SparkPod): SparkPod = { - val sparkConf = kubernetesConf.sparkConf - val hadoopConfDirCMapName = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) + val hadoopConfDirCMapName = conf.getOption(HADOOP_CONFIG_MAP_NAME) require(hadoopConfDirCMapName.isDefined, "Ensure that the env `HADOOP_CONF_DIR` is defined either in the client or " + " using pre-existing ConfigMaps") logInfo("HADOOP_CONF_DIR defined") HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala index 5b6a6d5a7db4..e34211076319 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala @@ -16,28 +16,19 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging /** * This step is responsible for setting ENV_SPARK_USER when HADOOP_FILES are detected * however, this step would not be run if Kerberos is enabled, as Kerberos sets SPARK_USER */ -private[spark] class HadoopSparkUserExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { +private[spark] class HadoopSparkUserExecutorFeatureStep(conf: KubernetesExecutorConf) + extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) + val sparkUserName = conf.get(KERBEROS_SPARK_USER_NAME) HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index ce47933b7f70..c6d5a866fa7b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -16,40 +16,43 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata +import io.fabric8.kubernetes.api.model.{HasMetadata, Secret, SecretBuilder} +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesDriverConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils._ -import org.apache.spark.internal.Logging +import org.apache.spark.deploy.security.HadoopDelegationTokenManager /** * Runs the necessary Hadoop-based logic based on Kerberos configs and the presence of the * HADOOP_CONF_DIR. This runs various bootstrap methods defined in HadoopBootstrapUtil. */ -private[spark] class KerberosConfDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - require(kubernetesConf.hadoopConfSpec.isDefined, - "Ensure that HADOOP_CONF_DIR is defined either via env or a pre-defined ConfigMap") - private val hadoopConfDirSpec = kubernetesConf.hadoopConfSpec.get - private val conf = kubernetesConf.sparkConf - private val principal = conf.get(org.apache.spark.internal.config.PRINCIPAL) - private val keytab = conf.get(org.apache.spark.internal.config.KEYTAB) - private val existingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val existingSecretItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) - private val krb5File = conf.get(KUBERNETES_KERBEROS_KRB5_FILE) - private val krb5CMap = conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) - private val kubeTokenManager = kubernetesConf.tokenManager(conf, - SparkHadoopUtil.get.newConfiguration(conf)) +private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDriverConf) + extends KubernetesFeatureConfigStep { + + private val hadoopConfDir = Option(kubernetesConf.sparkConf.getenv(ENV_HADOOP_CONF_DIR)) + private val hadoopConfigMapName = kubernetesConf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) + KubernetesUtils.requireNandDefined( + hadoopConfDir, + hadoopConfigMapName, + "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + + "as the creation of an additional ConfigMap, when one is already specified is extraneous") + + private val principal = kubernetesConf.get(org.apache.spark.internal.config.PRINCIPAL) + private val keytab = kubernetesConf.get(org.apache.spark.internal.config.KEYTAB) + private val existingSecretName = kubernetesConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val existingSecretItemKey = kubernetesConf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val krb5File = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + private val krb5CMap = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf) + private val tokenManager = new HadoopDelegationTokenManager(kubernetesConf.sparkConf, hadoopConf) private val isKerberosEnabled = - (hadoopConfDirSpec.hadoopConfDir.isDefined && kubeTokenManager.isSecurityEnabled) || - (hadoopConfDirSpec.hadoopConfigMapName.isDefined && - (krb5File.isDefined || krb5CMap.isDefined)) + (hadoopConfDir.isDefined && UserGroupInformation.isSecurityEnabled) || + (hadoopConfigMapName.isDefined && (krb5File.isDefined || krb5CMap.isDefined)) require(keytab.isEmpty || isKerberosEnabled, "You must enable Kerberos support if you are specifying a Kerberos Keytab") @@ -76,11 +79,11 @@ private[spark] class KerberosConfDriverFeatureStep( "If a secret storing a Kerberos Delegation Token is specified you must also" + " specify the item-key where the data is stored") - private val hadoopConfigurationFiles = hadoopConfDirSpec.hadoopConfDir.map { hConfDir => + private val hadoopConfigurationFiles = hadoopConfDir.map { hConfDir => HadoopBootstrapUtil.getHadoopConfFiles(hConfDir) } private val newHadoopConfigMapName = - if (hadoopConfDirSpec.hadoopConfigMapName.isEmpty) { + if (hadoopConfigMapName.isEmpty) { Some(kubernetesConf.hadoopConfigMapName) } else { None @@ -95,23 +98,24 @@ private[spark] class KerberosConfDriverFeatureStep( dtSecret = None, dtSecretName = secretName, dtSecretItemKey = secretItemKey, - jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) + jobUserName = UserGroupInformation.getCurrentUser.getShortUserName) }).orElse( if (isKerberosEnabled) { - Some(HadoopKerberosLogin.buildSpec( - conf, - kubernetesConf.appResourceNamePrefix, - kubeTokenManager)) + Some(buildKerberosSpec()) } else { None } ) override def configurePod(pod: SparkPod): SparkPod = { + if (!isKerberosEnabled) { + return pod + } + val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - hadoopConfDirSpec.hadoopConfDir, + hadoopConfDir, newHadoopConfigMapName, - hadoopConfDirSpec.hadoopConfigMapName, + hadoopConfigMapName, pod) kerberosConfSpec.map { hSpec => HadoopBootstrapUtil.bootstrapKerberosPod( @@ -124,11 +128,15 @@ private[spark] class KerberosConfDriverFeatureStep( hadoopBasedSparkPod) }.getOrElse( HadoopBootstrapUtil.bootstrapSparkUserPod( - kubeTokenManager.getCurrentUser.getShortUserName, + UserGroupInformation.getCurrentUser.getShortUserName, hadoopBasedSparkPod)) } override def getAdditionalPodSystemProperties(): Map[String, String] = { + if (!isKerberosEnabled) { + return Map.empty + } + val resolvedConfValues = kerberosConfSpec.map { hSpec => Map(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, @@ -136,13 +144,16 @@ private[spark] class KerberosConfDriverFeatureStep( KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.krbConfigMapName)) }.getOrElse( Map(KERBEROS_SPARK_USER_NAME -> - kubeTokenManager.getCurrentUser.getShortUserName)) + UserGroupInformation.getCurrentUser.getShortUserName)) Map(HADOOP_CONFIG_MAP_NAME -> - hadoopConfDirSpec.hadoopConfigMapName.getOrElse( - kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues + hadoopConfigMapName.getOrElse(kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + if (!isKerberosEnabled) { + return Seq.empty + } + val hadoopConfConfigMap = for { hName <- newHadoopConfigMapName hFiles <- hadoopConfigurationFiles @@ -162,4 +173,34 @@ private[spark] class KerberosConfDriverFeatureStep( krb5ConfigMap.toSeq ++ kerberosDTSecret.toSeq } + + private def buildKerberosSpec(): KerberosConfigSpec = { + // The JobUserUGI will be taken fom the Local Ticket Cache or via keytab+principal + // The login happens in the SparkSubmit so login logic is not necessary to include + val jobUserUGI = UserGroupInformation.getCurrentUser + val creds = jobUserUGI.getCredentials + tokenManager.obtainDelegationTokens(creds) + val tokenData = SparkHadoopUtil.get.serialize(creds) + require(tokenData.nonEmpty, "Did not obtain any delegation tokens") + val newSecretName = + s"${kubernetesConf.resourceNamePrefix}-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME" + val secretDT = + new SecretBuilder() + .withNewMetadata() + .withName(newSecretName) + .endMetadata() + .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(tokenData)) + .build() + KerberosConfigSpec( + dtSecret = Some(secretDT), + dtSecretName = newSecretName, + dtSecretItemKey = KERBEROS_SECRET_KEY, + jobUserName = jobUserUGI.getShortUserName) + } + + private case class KerberosConfigSpec( + dtSecret: Option[Secret], + dtSecretName: String, + dtSecretItemKey: String, + jobUserName: String) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala index 06a88b6c229f..32bb6a5d2bcb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala @@ -16,38 +16,29 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil import org.apache.spark.internal.Logging /** * This step is responsible for mounting the DT secret for the executors */ -private[spark] class KerberosConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) +private[spark] class KerberosConfExecutorFeatureStep(conf: KubernetesExecutorConf) extends KubernetesFeatureConfigStep with Logging { - private val sparkConf = kubernetesConf.sparkConf - private val maybeKrb5CMap = sparkConf.getOption(KRB5_CONFIG_MAP_NAME) + private val maybeKrb5CMap = conf.getOption(KRB5_CONFIG_MAP_NAME) require(maybeKrb5CMap.isDefined, "HADOOP_CONF_DIR ConfigMap not found") override def configurePod(pod: SparkPod): SparkPod = { logInfo(s"Mounting Resources for Kerberos") HadoopBootstrapUtil.bootstrapKerberosPod( - sparkConf.get(KERBEROS_DT_SECRET_NAME), - sparkConf.get(KERBEROS_DT_SECRET_KEY), - sparkConf.get(KERBEROS_SPARK_USER_NAME), + conf.get(KERBEROS_DT_SECRET_NAME), + conf.get(KERBEROS_DT_SECRET_KEY), + conf.get(KERBEROS_SPARK_USER_NAME), None, None, maybeKrb5CMap, pod) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty[HasMetadata] } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala index be386e119d46..19ed2df5551d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala @@ -16,16 +16,15 @@ */ package org.apache.spark.deploy.k8s.features -import java.nio.file.Paths import java.util.UUID import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, VolumeBuilder, VolumeMountBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ private[spark] class LocalDirsFeatureStep( - conf: KubernetesConf[_ <: KubernetesRoleSpecificConf], + conf: KubernetesConf, defaultLocalDir: String = s"/var/data/spark-${UUID.randomUUID}") extends KubernetesFeatureConfigStep { @@ -73,8 +72,4 @@ private[spark] class LocalDirsFeatureStep( .build() SparkPod(podWithLocalDirVolumes, containerWithLocalDirVolumeMounts) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala index 97fa9499b2ed..f4e1a3a32672 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala @@ -18,14 +18,13 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, VolumeBuilder, VolumeMountBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -private[spark] class MountSecretsFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class MountSecretsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { val addedVolumes = kubernetesConf - .roleSecretNamesToMountPaths + .secretNamesToMountPaths .keys .map(secretName => new VolumeBuilder() @@ -40,7 +39,7 @@ private[spark] class MountSecretsFeatureStep( .endSpec() .build() val addedVolumeMounts = kubernetesConf - .roleSecretNamesToMountPaths + .secretNamesToMountPaths .map { case (secretName, mountPath) => new VolumeMountBuilder() @@ -54,9 +53,5 @@ private[spark] class MountSecretsFeatureStep( SparkPod(podWithVolumes, containerWithMounts) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty - private def secretVolumeName(secretName: String): String = s"$secretName-volume" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala index 1473a7d3ee7f..8548e7057cdf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala @@ -20,12 +20,11 @@ import io.fabric8.kubernetes.api.model._ import org.apache.spark.deploy.k8s._ -private[spark] class MountVolumesFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class MountVolumesFeatureStep(conf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - val (volumeMounts, volumes) = constructVolumes(kubernetesConf.roleVolumes).unzip + val (volumeMounts, volumes) = constructVolumes(conf.volumes).unzip val podWithVolumes = new PodBuilder(pod.pod) .editSpec() @@ -40,12 +39,8 @@ private[spark] class MountVolumesFeatureStep( SparkPod(podWithVolumes, containerWithVolumeMounts) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty - private def constructVolumes( - volumeSpecs: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]] + volumeSpecs: Iterable[KubernetesVolumeSpec] ): Iterable[(VolumeMount, Volume)] = { volumeSpecs.map { spec => val volumeMount = new VolumeMountBuilder() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala index 28e2d1726ae2..09dcf93a54f8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala @@ -22,12 +22,11 @@ import java.nio.charset.StandardCharsets import com.google.common.io.Files import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, HasMetadata, PodBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -private[spark] class PodTemplateConfigMapStep( - conf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class PodTemplateConfigMapStep(conf: KubernetesConf) extends KubernetesFeatureConfigStep { def configurePod(pod: SparkPod): SparkPod = { val podWithVolume = new PodBuilder(pod.pod) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala deleted file mode 100644 index 0022d8f242a7..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadooputils - -import io.fabric8.kubernetes.api.model.SecretBuilder -import org.apache.commons.codec.binary.Base64 - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager - -/** - * This logic does all the heavy lifting for Delegation Token creation. This step - * assumes that the job user has either specified a principal and keytab or ran - * $kinit before running spark-submit. By running UGI.getCurrentUser we are able - * to obtain the current user, either signed in via $kinit or keytab. With the - * Job User principal you then retrieve the delegation token from the NameNode - * and store values in DelegationToken. Lastly, the class puts the data into - * a secret. All this is defined in a KerberosConfigSpec. - */ -private[spark] object HadoopKerberosLogin { - def buildSpec( - submissionSparkConf: SparkConf, - kubernetesResourceNamePrefix: String, - tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { - // The JobUserUGI will be taken fom the Local Ticket Cache or via keytab+principal - // The login happens in the SparkSubmit so login logic is not necessary to include - val jobUserUGI = tokenManager.getCurrentUser - val originalCredentials = jobUserUGI.getCredentials - tokenManager.obtainDelegationTokens(originalCredentials) - - val tokenData = SparkHadoopUtil.get.serialize(originalCredentials) - - val initialTokenDataKeyName = KERBEROS_SECRET_KEY - val newSecretName = s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME" - val secretDT = - new SecretBuilder() - .withNewMetadata() - .withName(newSecretName) - .endMetadata() - .addToData(initialTokenDataKeyName, Base64.encodeBase64String(tokenData)) - .build() - KerberosConfigSpec( - dtSecret = Some(secretDT), - dtSecretName = newSecretName, - dtSecretItemKey = initialTokenDataKeyName, - jobUserName = jobUserUGI.getShortUserName) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala deleted file mode 100644 index 3e98d5811d83..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala +++ /dev/null @@ -1,37 +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.deploy.k8s.security - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.UserGroupInformation - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.security.HadoopDelegationTokenManager - -/** - * Adds Kubernetes-specific functionality to HadoopDelegationTokenManager. - */ -private[spark] class KubernetesHadoopDelegationTokenManager( - _sparkConf: SparkConf, - _hadoopConf: Configuration) - extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf) { - - def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser - def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled - -} 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 543d6b16d6ae..70a93c968795 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 @@ -17,19 +17,17 @@ package org.apache.spark.deploy.k8s.submit import java.io.StringWriter -import java.util.{Collections, Locale, Properties, UUID} import java.util.{Collections, UUID} import java.util.Properties import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.hadoop.security.UserGroupInformation import scala.collection.mutable import scala.util.control.NonFatal import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkApplication -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging @@ -47,8 +45,7 @@ private[spark] case class ClientArguments( mainAppResource: MainAppResource, mainClass: String, driverArgs: Array[String], - maybePyFiles: Option[String], - hadoopConfigDir: Option[String]) + maybePyFiles: Option[String]) private[spark] object ClientArguments { @@ -82,8 +79,7 @@ private[spark] object ClientArguments { mainAppResource, mainClass.get, driverArgs.toArray, - maybePyFiles, - sys.env.get(ENV_HADOOP_CONF_DIR)) + maybePyFiles) } } @@ -92,27 +88,24 @@ private[spark] object ClientArguments { * watcher that monitors and logs the application status. Waits for the application to terminate if * spark.kubernetes.submission.waitAppCompletion is true. * + * @param conf The kubernetes driver config. * @param builder Responsible for building the base driver pod based on a composition of * implemented features. - * @param kubernetesConf application configuration * @param kubernetesClient the client to talk to the Kubernetes API server * @param waitForAppCompletion a flag indicating whether the client should wait for the application * to complete - * @param appName the application name * @param watcher a watcher that monitors and logs the application status */ private[spark] class Client( + conf: KubernetesDriverConf, builder: KubernetesDriverBuilder, - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf], kubernetesClient: KubernetesClient, waitForAppCompletion: Boolean, - appName: String, - watcher: LoggingPodStatusWatcher, - kubernetesResourceNamePrefix: String) extends Logging { + watcher: LoggingPodStatusWatcher) extends Logging { def run(): Unit = { - val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf) - val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map" + val resolvedDriverSpec = builder.buildFromFeatures(conf) + val configMapName = s"${conf.resourceNamePrefix}-driver-conf-map" val configMap = buildConfigMap(configMapName, resolvedDriverSpec.systemProperties) // The include of the ENV_VAR for "SPARK_CONF_DIR" is to allow for the // Spark command builder to pickup on the Java Options present in the ConfigMap @@ -155,11 +148,11 @@ private[spark] class Client( } if (waitForAppCompletion) { - logInfo(s"Waiting for application $appName to finish...") + logInfo(s"Waiting for application ${conf.appName} to finish...") watcher.awaitCompletion() - logInfo(s"Application $appName finished.") + logInfo(s"Application ${conf.appName} finished.") } else { - logInfo(s"Deployed Spark application $appName into Kubernetes.") + logInfo(s"Deployed Spark application ${conf.appName} into Kubernetes.") } } } @@ -216,19 +209,13 @@ private[spark] class KubernetesClientApplication extends SparkApplication { // a unique app ID (captured by spark.app.id) in the format below. val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) - val kubernetesResourceNamePrefix = KubernetesClientApplication.getResourceNamePrefix(appName) - sparkConf.set(KUBERNETES_PYSPARK_PY_FILES, clientArguments.maybePyFiles.getOrElse("")) val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, - appName, - kubernetesResourceNamePrefix, kubernetesAppId, clientArguments.mainAppResource, clientArguments.mainClass, clientArguments.driverArgs, - clientArguments.maybePyFiles, - clientArguments.hadoopConfigDir) - val namespace = kubernetesConf.namespace() + clientArguments.maybePyFiles) // The master URL has been checked for validity already in SparkSubmit. // We just need to get rid of the "k8s://" prefix here. val master = KubernetesUtils.parseMasterUrl(sparkConf.get("spark.master")) @@ -238,36 +225,18 @@ private[spark] class KubernetesClientApplication extends SparkApplication { Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, - Some(namespace), + Some(kubernetesConf.namespace), KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, sparkConf, None, None)) { kubernetesClient => val client = new Client( - KubernetesDriverBuilder(kubernetesClient, kubernetesConf.sparkConf), kubernetesConf, + KubernetesDriverBuilder(kubernetesClient, kubernetesConf.sparkConf), kubernetesClient, waitForAppCompletion, - appName, - watcher, - kubernetesResourceNamePrefix) + watcher) client.run() } } } - -private[spark] object KubernetesClientApplication { - - def getAppName(conf: SparkConf): String = conf.getOption("spark.app.name").getOrElse("spark") - - def getResourceNamePrefix(appName: String): String = { - val launchTime = System.currentTimeMillis() - s"$appName-$launchTime" - .trim - .toLowerCase(Locale.ROOT) - .replaceAll("\\s+", "-") - .replaceAll("\\.", "-") - .replaceAll("[^a-z0-9\\-]", "") - .replaceAll("-+", "-") - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index 167fb402cd40..a5ad9729aee9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -21,57 +21,46 @@ import java.io.File import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.{Config, KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.features._ private[spark] class KubernetesDriverBuilder( - provideBasicStep: (KubernetesConf[KubernetesDriverSpecificConf]) => BasicDriverFeatureStep = + provideBasicStep: (KubernetesDriverConf => BasicDriverFeatureStep) = new BasicDriverFeatureStep(_), - provideCredentialsStep: (KubernetesConf[KubernetesDriverSpecificConf]) - => DriverKubernetesCredentialsFeatureStep = + provideCredentialsStep: (KubernetesDriverConf => DriverKubernetesCredentialsFeatureStep) = new DriverKubernetesCredentialsFeatureStep(_), - provideServiceStep: (KubernetesConf[KubernetesDriverSpecificConf]) => DriverServiceFeatureStep = + provideServiceStep: (KubernetesDriverConf => DriverServiceFeatureStep) = new DriverServiceFeatureStep(_), - provideSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountSecretsFeatureStep) = + provideSecretsStep: (KubernetesConf => MountSecretsFeatureStep) = new MountSecretsFeatureStep(_), - provideEnvSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => EnvSecretsFeatureStep) = + provideEnvSecretsStep: (KubernetesConf => EnvSecretsFeatureStep) = new EnvSecretsFeatureStep(_), - provideLocalDirsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => LocalDirsFeatureStep = + provideLocalDirsStep: (KubernetesConf => LocalDirsFeatureStep) = new LocalDirsFeatureStep(_), - provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountVolumesFeatureStep) = + provideVolumesStep: (KubernetesConf => MountVolumesFeatureStep) = new MountVolumesFeatureStep(_), - provideDriverCommandStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => DriverCommandFeatureStep) = + provideDriverCommandStep: (KubernetesDriverConf => DriverCommandFeatureStep) = new DriverCommandFeatureStep(_), - provideHadoopGlobalStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => KerberosConfDriverFeatureStep) = - new KerberosConfDriverFeatureStep(_), - providePodTemplateConfigMapStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => PodTemplateConfigMapStep) = - new PodTemplateConfigMapStep(_), - provideInitialPod: () => SparkPod = () => SparkPod.initialPod()) { + provideHadoopGlobalStep: (KubernetesDriverConf => KerberosConfDriverFeatureStep) = + new KerberosConfDriverFeatureStep(_), + providePodTemplateConfigMapStep: (KubernetesConf => PodTemplateConfigMapStep) = + new PodTemplateConfigMapStep(_), + provideInitialPod: () => SparkPod = () => SparkPod.initialPod) { - def buildFromFeatures( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]): KubernetesDriverSpec = { + def buildFromFeatures(kubernetesConf: KubernetesDriverConf): KubernetesDriverSpec = { val baseFeatures = Seq( provideBasicStep(kubernetesConf), provideCredentialsStep(kubernetesConf), provideServiceStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) - val secretFeature = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { + val secretFeature = if (kubernetesConf.secretNamesToMountPaths.nonEmpty) { Seq(provideSecretsStep(kubernetesConf)) } else Nil - val envSecretFeature = if (kubernetesConf.roleSecretEnvNamesToKeyRefs.nonEmpty) { + val envSecretFeature = if (kubernetesConf.secretEnvNamesToKeyRefs.nonEmpty) { Seq(provideEnvSecretsStep(kubernetesConf)) } else Nil - val volumesFeature = if (kubernetesConf.roleVolumes.nonEmpty) { + val volumesFeature = if (kubernetesConf.volumes.nonEmpty) { Seq(provideVolumesStep(kubernetesConf)) } else Nil val podTemplateFeature = if ( @@ -81,14 +70,12 @@ private[spark] class KubernetesDriverBuilder( val driverCommandStep = provideDriverCommandStep(kubernetesConf) - val maybeHadoopConfigStep = - kubernetesConf.hadoopConfSpec.map { _ => - provideHadoopGlobalStep(kubernetesConf)} + val hadoopConfigStep = Some(provideHadoopGlobalStep(kubernetesConf)) val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ Seq(driverCommandStep) ++ secretFeature ++ envSecretFeature ++ volumesFeature ++ - maybeHadoopConfigStep.toSeq ++ podTemplateFeature + hadoopConfigStep ++ podTemplateFeature var spec = KubernetesDriverSpec( provideInitialPod(), diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index fc41a4770bce..d24ff0d1e660 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -26,50 +26,38 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ private[spark] class KubernetesExecutorBuilder( - provideBasicStep: (KubernetesConf [KubernetesExecutorSpecificConf]) - => BasicExecutorFeatureStep = + provideBasicStep: (KubernetesExecutorConf => BasicExecutorFeatureStep) = new BasicExecutorFeatureStep(_), - provideSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => MountSecretsFeatureStep = + provideSecretsStep: (KubernetesConf => MountSecretsFeatureStep) = new MountSecretsFeatureStep(_), - provideEnvSecretsStep: - (KubernetesConf[_ <: KubernetesRoleSpecificConf] => EnvSecretsFeatureStep) = + provideEnvSecretsStep: (KubernetesConf => EnvSecretsFeatureStep) = new EnvSecretsFeatureStep(_), - provideLocalDirsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => LocalDirsFeatureStep = + provideLocalDirsStep: (KubernetesConf => LocalDirsFeatureStep) = new LocalDirsFeatureStep(_), - provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountVolumesFeatureStep) = + provideVolumesStep: (KubernetesConf => MountVolumesFeatureStep) = new MountVolumesFeatureStep(_), - provideHadoopConfStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopConfExecutorFeatureStep) = + provideHadoopConfStep: (KubernetesExecutorConf => HadoopConfExecutorFeatureStep) = new HadoopConfExecutorFeatureStep(_), - provideKerberosConfStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => KerberosConfExecutorFeatureStep) = + provideKerberosConfStep: (KubernetesExecutorConf => KerberosConfExecutorFeatureStep) = new KerberosConfExecutorFeatureStep(_), - provideHadoopSparkUserStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopSparkUserExecutorFeatureStep) = + provideHadoopSparkUserStep: (KubernetesExecutorConf => HadoopSparkUserExecutorFeatureStep) = new HadoopSparkUserExecutorFeatureStep(_), provideInitialPod: () => SparkPod = () => SparkPod.initialPod()) { - def buildFromFeatures( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { + def buildFromFeatures(kubernetesConf: KubernetesExecutorConf): SparkPod = { val sparkConf = kubernetesConf.sparkConf val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) val maybeDTSecretName = sparkConf.getOption(KERBEROS_DT_SECRET_NAME) val maybeDTDataItem = sparkConf.getOption(KERBEROS_DT_SECRET_KEY) val baseFeatures = Seq(provideBasicStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) - val secretFeature = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { + val secretFeature = if (kubernetesConf.secretNamesToMountPaths.nonEmpty) { Seq(provideSecretsStep(kubernetesConf)) } else Nil - val secretEnvFeature = if (kubernetesConf.roleSecretEnvNamesToKeyRefs.nonEmpty) { + val secretEnvFeature = if (kubernetesConf.secretEnvNamesToKeyRefs.nonEmpty) { Seq(provideEnvSecretsStep(kubernetesConf)) } else Nil - val volumesFeature = if (kubernetesConf.roleVolumes.nonEmpty) { + val volumesFeature = if (kubernetesConf.volumes.nonEmpty) { Seq(provideVolumesStep(kubernetesConf)) } else Nil diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index 41ca8d186c17..f4d40b0b3590 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -26,10 +26,6 @@ import org.apache.spark.deploy.k8s.submit._ class KubernetesConfSuite extends SparkFunSuite { - private val APP_NAME = "test-app" - private val RESOURCE_NAME_PREFIX = "prefix" - private val APP_ID = "test-id" - private val MAIN_CLASS = "test-class" private val APP_ARGS = Array("arg1", "arg2") private val CUSTOM_LABELS = Map( "customLabel1Key" -> "customLabel1Value", @@ -49,26 +45,6 @@ class KubernetesConfSuite extends SparkFunSuite { private val DRIVER_POD = new PodBuilder().build() private val EXECUTOR_ID = "executor-id" - test("Basic driver translated fields.") { - val sparkConf = new SparkConf(false) - val conf = KubernetesConf.createDriverConf( - sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource = JavaMainAppResource(None), - MAIN_CLASS, - APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) - assert(conf.appId === APP_ID) - assert(conf.sparkConf.getAll.toMap === sparkConf.getAll.toMap) - assert(conf.appResourceNamePrefix === RESOURCE_NAME_PREFIX) - assert(conf.roleSpecificConf.appName === APP_NAME) - assert(conf.roleSpecificConf.mainClass === MAIN_CLASS) - assert(conf.roleSpecificConf.appArgs === APP_ARGS) - } - test("Resolve driver labels, annotations, secret mount paths, envs, and memory overhead") { val sparkConf = new SparkConf(false) .set(MEMORY_OVERHEAD_FACTOR, 0.3) @@ -90,22 +66,19 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createDriverConf( sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource = JavaMainAppResource(None), - MAIN_CLASS, + KubernetesTestConf.APP_ID, + JavaMainAppResource(None), + KubernetesTestConf.MAIN_CLASS, APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) - assert(conf.roleLabels === Map( - SPARK_APP_ID_LABEL -> APP_ID, + None) + assert(conf.labels === Map( + SPARK_APP_ID_LABEL -> KubernetesTestConf.APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ CUSTOM_LABELS) - assert(conf.roleAnnotations === CUSTOM_ANNOTATIONS) - assert(conf.roleSecretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) - assert(conf.roleSecretEnvNamesToKeyRefs === SECRET_ENV_VARS) - assert(conf.roleEnvs === CUSTOM_ENVS) + assert(conf.annotations === CUSTOM_ANNOTATIONS) + assert(conf.secretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) + assert(conf.secretEnvNamesToKeyRefs === SECRET_ENV_VARS) + assert(conf.environment === CUSTOM_ENVS) assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.3) } @@ -113,20 +86,20 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createExecutorConf( new SparkConf(false), EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.roleSpecificConf.executorId === EXECUTOR_ID) - assert(conf.roleSpecificConf.driverPod.get === DRIVER_POD) + assert(conf.executorId === EXECUTOR_ID) + assert(conf.driverPod.get === DRIVER_POD) } test("Image pull secrets.") { val conf = KubernetesConf.createExecutorConf( new SparkConf(false) - .set(IMAGE_PULL_SECRETS, "my-secret-1,my-secret-2 "), + .set(IMAGE_PULL_SECRETS, Seq("my-secret-1", "my-secret-2 ")), EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.imagePullSecrets() === + assert(conf.imagePullSecrets === Seq( new LocalObjectReferenceBuilder().withName("my-secret-1").build(), new LocalObjectReferenceBuilder().withName("my-secret-2").build())) @@ -150,14 +123,14 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createExecutorConf( sparkConf, EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.roleLabels === Map( + assert(conf.labels === Map( SPARK_EXECUTOR_ID_LABEL -> EXECUTOR_ID, - SPARK_APP_ID_LABEL -> APP_ID, + SPARK_APP_ID_LABEL -> KubernetesTestConf.APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ CUSTOM_LABELS) - assert(conf.roleAnnotations === CUSTOM_ANNOTATIONS) - assert(conf.roleSecretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) - assert(conf.roleSecretEnvNamesToKeyRefs === SECRET_ENV_VARS) + assert(conf.annotations === CUSTOM_ANNOTATIONS) + assert(conf.secretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) + assert(conf.secretEnvNamesToKeyRefs === SECRET_ENV_VARS) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala new file mode 100644 index 000000000000..1d77a6d18152 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.k8s + +import io.fabric8.kubernetes.api.model.Pod + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.{JavaMainAppResource, MainAppResource} + +/** + * Builder methods for KubernetesConf that allow easy control over what to return for a few + * properties. For use with tests instead of having to mock specific properties. + */ +object KubernetesTestConf { + + val APP_ID = "appId" + val MAIN_CLASS = "mainClass" + val RESOURCE_PREFIX = "prefix" + val EXECUTOR_ID = "1" + + private val DEFAULT_CONF = new SparkConf(false) + + // scalastyle:off argcount + def createDriverConf( + sparkConf: SparkConf = DEFAULT_CONF, + appId: String = APP_ID, + mainAppResource: MainAppResource = JavaMainAppResource(None), + mainClass: String = MAIN_CLASS, + appArgs: Array[String] = Array.empty, + pyFiles: Seq[String] = Nil, + resourceNamePrefix: Option[String] = None, + labels: Map[String, String] = Map.empty, + environment: Map[String, String] = Map.empty, + annotations: Map[String, String] = Map.empty, + secretEnvNamesToKeyRefs: Map[String, String] = Map.empty, + secretNamesToMountPaths: Map[String, String] = Map.empty, + volumes: Seq[KubernetesVolumeSpec] = Seq.empty): KubernetesDriverConf = { + val conf = sparkConf.clone() + + resourceNamePrefix.foreach { prefix => + conf.set(KUBERNETES_DRIVER_POD_NAME_PREFIX, prefix) + } + setPrefixedConfigs(conf, KUBERNETES_DRIVER_LABEL_PREFIX, labels) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_ENV_PREFIX, environment) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_ANNOTATION_PREFIX, annotations) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_SECRETS_PREFIX, secretNamesToMountPaths) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX, secretEnvNamesToKeyRefs) + setVolumeSpecs(conf, KUBERNETES_DRIVER_VOLUMES_PREFIX, volumes) + + new KubernetesDriverConf(conf, appId, mainAppResource, mainClass, appArgs, pyFiles) + } + // scalastyle:on argcount + + def createExecutorConf( + sparkConf: SparkConf = DEFAULT_CONF, + driverPod: Option[Pod] = None, + labels: Map[String, String] = Map.empty, + environment: Map[String, String] = Map.empty, + annotations: Map[String, String] = Map.empty, + secretEnvNamesToKeyRefs: Map[String, String] = Map.empty, + secretNamesToMountPaths: Map[String, String] = Map.empty, + volumes: Seq[KubernetesVolumeSpec] = Seq.empty): KubernetesExecutorConf = { + val conf = sparkConf.clone() + + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_LABEL_PREFIX, labels) + setPrefixedConfigs(conf, "spark.executorEnv.", environment) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, annotations) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_SECRETS_PREFIX, secretNamesToMountPaths) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX, secretEnvNamesToKeyRefs) + setVolumeSpecs(conf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX, volumes) + + new KubernetesExecutorConf(conf, APP_ID, EXECUTOR_ID, driverPod) + } + + private def setPrefixedConfigs( + conf: SparkConf, + prefix: String, + values: Map[String, String]): Unit = { + values.foreach { case (k, v) => + conf.set(s"${prefix}$k", v) + } + } + + private def setVolumeSpecs( + conf: SparkConf, + prefix: String, + volumes: Seq[KubernetesVolumeSpec]): Unit = { + def key(vtype: String, vname: String, subkey: String): String = { + s"${prefix}$vtype.$vname.$subkey" + } + + volumes.foreach { case spec => + val (vtype, configs) = spec.volumeConf match { + case KubernetesHostPathVolumeConf(path) => + (KUBERNETES_VOLUMES_HOSTPATH_TYPE, + Map(KUBERNETES_VOLUMES_OPTIONS_PATH_KEY -> path)) + + case KubernetesPVCVolumeConf(claimName) => + (KUBERNETES_VOLUMES_PVC_TYPE, + Map(KUBERNETES_VOLUMES_OPTIONS_CLAIM_NAME_KEY -> claimName)) + + case KubernetesEmptyDirVolumeConf(medium, sizeLimit) => + val mconf = medium.map { m => (KUBERNETES_VOLUMES_OPTIONS_MEDIUM_KEY, m) }.toMap + val lconf = sizeLimit.map { l => (KUBERNETES_VOLUMES_OPTIONS_SIZE_LIMIT_KEY, l) }.toMap + (KUBERNETES_VOLUMES_EMPTYDIR_TYPE, mconf ++ lconf) + } + + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_PATH_KEY), spec.mountPath) + if (spec.mountSubPath.nonEmpty) { + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_SUBPATH_KEY), + spec.mountSubPath) + } + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_READONLY_KEY), + spec.mountReadOnly.toString) + configs.foreach { case (k, v) => + conf.set(key(vtype, spec.volumeName, k), v) + } + } + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala index de79a58a3a75..c0790898e097 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala @@ -25,7 +25,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.hostPath.volumeName.mount.readOnly", "true") sparkConf.set("test.hostPath.volumeName.options.path", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountReadOnly === true) @@ -39,7 +39,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.emptyDir.volumeName.mount.readOnly", "true") sparkConf.set("test.emptyDir.volumeName.mount.subPath", "subPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountSubPath === "subPath") @@ -51,7 +51,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.persistentVolumeClaim.volumeName.mount.readOnly", "true") sparkConf.set("test.persistentVolumeClaim.volumeName.options.claimName", "claimeName") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountReadOnly === true) @@ -66,7 +66,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.emptyDir.volumeName.options.medium", "medium") sparkConf.set("test.emptyDir.volumeName.options.sizeLimit", "5G") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountReadOnly === true) @@ -79,7 +79,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.emptyDir.volumeName.mount.path", "/path") sparkConf.set("test.emptyDir.volumeName.mount.readOnly", "true") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountReadOnly === true) @@ -92,27 +92,29 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.hostPath.volumeName.mount.path", "/path") sparkConf.set("test.hostPath.volumeName.options.path", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.mountReadOnly === false) } - test("Gracefully fails on missing mount key") { + test("Fails on missing mount key") { val sparkConf = new SparkConf(false) sparkConf.set("test.emptyDir.volumeName.mnt.path", "/path") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head - assert(volumeSpec.isFailure === true) - assert(volumeSpec.failed.get.getMessage === "emptyDir.volumeName.mount.path") + val e = intercept[NoSuchElementException] { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.") + } + assert(e.getMessage.contains("emptyDir.volumeName.mount.path")) } - test("Gracefully fails on missing option key") { + test("Fails on missing option key") { val sparkConf = new SparkConf(false) sparkConf.set("test.hostPath.volumeName.mount.path", "/path") sparkConf.set("test.hostPath.volumeName.mount.readOnly", "true") sparkConf.set("test.hostPath.volumeName.options.pth", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head - assert(volumeSpec.isFailure === true) - assert(volumeSpec.failed.get.getMessage === "hostPath.volumeName.options.path") + val e = intercept[NoSuchElementException] { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.") + } + assert(e.getMessage.contains("hostPath.volumeName.options.path")) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 1e7dfbeffdb2..e4951bc1e69e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerPort, ContainerPortBuilder, LocalObjectReferenceBuilder} import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ @@ -30,32 +30,17 @@ import org.apache.spark.ui.SparkUI class BasicDriverFeatureStepSuite extends SparkFunSuite { - private val APP_ID = "spark-app-id" - private val RESOURCE_NAME_PREFIX = "spark" private val DRIVER_LABELS = Map("labelkey" -> "labelvalue") private val CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent" - private val APP_NAME = "spark-test" - private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" - private val PY_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" - private val APP_ARGS = Array("arg1", "arg2", "\"arg 3\"") - private val CUSTOM_ANNOTATION_KEY = "customAnnotation" - private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" - private val DRIVER_ANNOTATIONS = Map(CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE) - private val DRIVER_CUSTOM_ENV1 = "customDriverEnv1" - private val DRIVER_CUSTOM_ENV2 = "customDriverEnv2" + private val DRIVER_ANNOTATIONS = Map("customAnnotation" -> "customAnnotationValue") private val DRIVER_ENVS = Map( - DRIVER_CUSTOM_ENV1 -> DRIVER_CUSTOM_ENV1, - DRIVER_CUSTOM_ENV2 -> DRIVER_CUSTOM_ENV2) + "customDriverEnv1" -> "customDriverEnv2", + "customDriverEnv2" -> "customDriverEnv2") private val TEST_IMAGE_PULL_SECRETS = Seq("my-secret-1", "my-secret-2") private val TEST_IMAGE_PULL_SECRET_OBJECTS = TEST_IMAGE_PULL_SECRETS.map { secret => new LocalObjectReferenceBuilder().withName(secret).build() } - private val emptyDriverSpecificConf = KubernetesDriverSpecificConf( - JavaMainAppResource(None), - APP_NAME, - MAIN_CLASS, - APP_ARGS) test("Check the pod respects all configurations from the user.") { val sparkConf = new SparkConf() @@ -65,19 +50,12 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .set(DRIVER_MEMORY.key, "256M") .set(DRIVER_MEMORY_OVERHEAD, 200L) .set(CONTAINER_IMAGE, "spark-driver:latest") - .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS.mkString(",")) - val kubernetesConf = KubernetesConf( - sparkConf, - emptyDriverSpecificConf, - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) + .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS) + val kubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS, + environment = DRIVER_ENVS, + annotations = DRIVER_ANNOTATIONS) val featureStep = new BasicDriverFeatureStep(kubernetesConf) val basePod = SparkPod.initialPod() @@ -99,10 +77,11 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val envs = configuredPod.container .getEnv .asScala - .map(env => (env.getName, env.getValue)) + .map { env => (env.getName, env.getValue) } .toMap - assert(envs(DRIVER_CUSTOM_ENV1) === DRIVER_ENVS(DRIVER_CUSTOM_ENV1)) - assert(envs(DRIVER_CUSTOM_ENV2) === DRIVER_ENVS(DRIVER_CUSTOM_ENV2)) + DRIVER_ENVS.foreach { case (k, v) => + assert(envs(v) === v) + } assert(configuredPod.pod.getSpec().getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) @@ -122,13 +101,15 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val driverPodMetadata = configuredPod.pod.getMetadata assert(driverPodMetadata.getName === "spark-driver-pod") - assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS) + DRIVER_LABELS.foreach { case (k, v) => + assert(driverPodMetadata.getLabels.get(k) === v) + } assert(driverPodMetadata.getAnnotations.asScala === DRIVER_ANNOTATIONS) assert(configuredPod.pod.getSpec.getRestartPolicy === "Never") val expectedSparkConf = Map( KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", - "spark.app.id" -> APP_ID, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX, + "spark.app.id" -> KubernetesTestConf.APP_ID, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> kubernetesConf.resourceNamePrefix, "spark.kubernetes.submitInDriver" -> "true", MEMORY_OVERHEAD_FACTOR.key -> MEMORY_OVERHEAD_FACTOR.defaultValue.get.toString) assert(featureStep.getAdditionalPodSystemProperties() === expectedSparkConf) @@ -141,39 +122,10 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val pythonSparkConf = new SparkConf() .set(DRIVER_MEMORY.key, "4g") .set(CONTAINER_IMAGE, "spark-driver-py:latest") - val javaKubernetesConf = KubernetesConf( - javaSparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - APP_NAME, - PY_MAIN_CLASS, - APP_ARGS), - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) - - val pythonKubernetesConf = KubernetesConf( - pythonSparkConf, - KubernetesDriverSpecificConf( - PythonMainAppResource(""), - APP_NAME, - PY_MAIN_CLASS, - APP_ARGS), - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) + val javaKubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = javaSparkConf) + val pythonKubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = pythonSparkConf, + mainAppResource = PythonMainAppResource("")) val javaFeatureStep = new BasicDriverFeatureStep(javaKubernetesConf) val pythonFeatureStep = new BasicDriverFeatureStep(pythonKubernetesConf) val basePod = SparkPod.initialPod() @@ -191,25 +143,14 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .setJars(allJars) .set("spark.files", allFiles.mkString(",")) .set(CONTAINER_IMAGE, "spark-driver:latest") - val kubernetesConf = KubernetesConf( - sparkConf, - emptyDriverSpecificConf, - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val step = new BasicDriverFeatureStep(kubernetesConf) val additionalProperties = step.getAdditionalPodSystemProperties() val expectedSparkConf = Map( KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", - "spark.app.id" -> APP_ID, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX, + "spark.app.id" -> KubernetesTestConf.APP_ID, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> kubernetesConf.resourceNamePrefix, "spark.kubernetes.submitInDriver" -> "true", "spark.jars" -> "/opt/spark/jar1.jar,hdfs:///opt/spark/jar2.jar", "spark.files" -> "https://localhost:9000/file1.txt,/opt/spark/file2.txt", @@ -234,19 +175,9 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .set(CONTAINER_IMAGE, "spark-driver:latest") .set(DRIVER_MEMORY.key, s"${driverMem.toInt}m") factor.foreach { value => sparkConf.set(MEMORY_OVERHEAD_FACTOR, value) } - val driverConf = emptyDriverSpecificConf.copy(mainAppResource = resource) - val conf = KubernetesConf( - sparkConf, - driverConf, - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + mainAppResource = resource) val step = new BasicDriverFeatureStep(conf) val pod = step.configurePod(SparkPod.initialPod()) val mem = pod.container.getResources.getRequests.get("memory").getAmount() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index e9a16aab6ccc..d6003c977937 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -19,20 +19,18 @@ package org.apache.spark.deploy.k8s.features import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ -import org.mockito.MockitoAnnotations -import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} +import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.config._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -class BasicExecutorFeatureStepSuite - extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { +class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { - private val APP_ID = "app-id" private val DRIVER_HOSTNAME = "localhost" private val DRIVER_PORT = 7098 private val DRIVER_ADDRESS = RpcEndpointAddress( @@ -45,7 +43,6 @@ class BasicExecutorFeatureStepSuite private val RESOURCE_NAME_PREFIX = "base" private val EXECUTOR_IMAGE = "executor-image" private val LABELS = Map("label1key" -> "label1value") - private val ANNOTATIONS = Map("annotation1key" -> "annotation1value") private val TEST_IMAGE_PULL_SECRETS = Seq("my-1secret-1", "my-secret-2") private val TEST_IMAGE_PULL_SECRET_OBJECTS = TEST_IMAGE_PULL_SECRETS.map { secret => @@ -66,37 +63,35 @@ class BasicExecutorFeatureStepSuite private var baseConf: SparkConf = _ before { - MockitoAnnotations.initMocks(this) baseConf = new SparkConf() .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, RESOURCE_NAME_PREFIX) .set(CONTAINER_IMAGE, EXECUTOR_IMAGE) .set(KUBERNETES_DRIVER_SUBMIT_CHECK, true) - .set("spark.driver.host", DRIVER_HOSTNAME) + .set(DRIVER_HOST_ADDRESS, DRIVER_HOSTNAME) .set("spark.driver.port", DRIVER_PORT.toString) - .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS.mkString(",")) + .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS) .set("spark.kubernetes.resource.type", "java") } + private def newExecutorConf( + environment: Map[String, String] = Map.empty): KubernetesExecutorConf = { + KubernetesTestConf.createExecutorConf( + sparkConf = baseConf, + driverPod = Some(DRIVER_POD), + labels = LABELS, + environment = environment) + } + test("basic executor pod has reasonable defaults") { - val step = new BasicExecutorFeatureStep( - KubernetesConf( - baseConf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + val step = new BasicExecutorFeatureStep(newExecutorConf()) val executor = step.configurePod(SparkPod.initialPod()) // The executor pod name and default labels. assert(executor.pod.getMetadata.getName === s"$RESOURCE_NAME_PREFIX-exec-1") - assert(executor.pod.getMetadata.getLabels.asScala === LABELS) + LABELS.foreach { case (k, v) => + assert(executor.pod.getMetadata.getLabels.get(k) === v) + } assert(executor.pod.getSpec.getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) // There is exactly 1 container with no volume mounts and default memory limits. @@ -116,43 +111,18 @@ class BasicExecutorFeatureStepSuite } test("executor pod hostnames get truncated to 63 characters") { - val conf = baseConf.clone() val longPodNamePrefix = "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple" - val step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - longPodNamePrefix, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + baseConf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, longPodNamePrefix) + val step = new BasicExecutorFeatureStep(newExecutorConf()) assert(step.configurePod(SparkPod.initialPod()).pod.getSpec.getHostname.length === 63) } test("classpath and extra java options get translated into environment variables") { - val conf = baseConf.clone() - conf.set(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS, "foo=bar") - conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") - - val step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map("qux" -> "quux"), - Nil, - hadoopConfSpec = None)) + baseConf.set(EXECUTOR_JAVA_OPTIONS, "foo=bar") + baseConf.set(EXECUTOR_CLASS_PATH, "bar=baz") + val kconf = newExecutorConf(environment = Map("qux" -> "quux")) + val step = new BasicExecutorFeatureStep(kconf) val executor = step.configurePod(SparkPod.initialPod()) checkEnv(executor, @@ -163,23 +133,10 @@ class BasicExecutorFeatureStepSuite } test("test executor pyspark memory") { - val conf = baseConf.clone() - conf.set("spark.kubernetes.resource.type", "python") - conf.set(org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY, 42L) - - val step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + baseConf.set("spark.kubernetes.resource.type", "python") + baseConf.set(PYSPARK_EXECUTOR_MEMORY, 42L) + + val step = new BasicExecutorFeatureStep(newExecutorConf()) val executor = step.configurePod(SparkPod.initialPod()) // This is checking that basic executor + executorMemory = 1408 + 42 = 1450 assert(executor.container.getResources.getRequests.get("memory").getAmount === "1450Mi") @@ -199,7 +156,7 @@ class BasicExecutorFeatureStepSuite ENV_DRIVER_URL -> DRIVER_ADDRESS.toString, ENV_EXECUTOR_CORES -> "1", ENV_EXECUTOR_MEMORY -> "1g", - ENV_APPLICATION_ID -> APP_ID, + ENV_APPLICATION_ID -> KubernetesTestConf.APP_ID, ENV_SPARK_CONF_DIR -> SPARK_CONF_DIR_INTERNAL, ENV_EXECUTOR_POD_IP -> null) ++ additionalEnvVars diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala index 30672952aaf6..f74ac928028c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala @@ -27,8 +27,6 @@ import org.apache.spark.util.Utils class DriverCommandFeatureStepSuite extends SparkFunSuite { - private val MAIN_CLASS = "mainClass" - test("java resource") { val mainResource = "local:///main.jar" val spec = applyFeatureStep( @@ -37,7 +35,7 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { assert(spec.pod.container.getArgs.asScala === List( "driver", "--properties-file", SPARK_CONF_PATH, - "--class", MAIN_CLASS, + "--class", KubernetesTestConf.MAIN_CLASS, "spark-internal", "5", "7")) val jars = Utils.stringToSeq(spec.systemProperties("spark.jars")) @@ -55,7 +53,7 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { assert(spec.pod.container.getArgs.asScala === List( "driver", "--properties-file", SPARK_CONF_PATH, - "--class", MAIN_CLASS, + "--class", KubernetesTestConf.MAIN_CLASS, "/main.py")) val envs = spec.pod.container.getEnv.asScala .map { env => (env.getName, env.getValue) } @@ -86,7 +84,7 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { assert(spec.pod.container.getArgs.asScala === List( "driver", "--properties-file", SPARK_CONF_PATH, - "--class", MAIN_CLASS, + "--class", KubernetesTestConf.MAIN_CLASS, "/main.py", "5", "7", "9")) val envs = spec.pod.container.getEnv.asScala @@ -112,7 +110,7 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { assert(spec.pod.container.getArgs.asScala === List( "driver", "--properties-file", SPARK_CONF_PATH, - "--class", MAIN_CLASS, + "--class", KubernetesTestConf.MAIN_CLASS, "/main.R", "5", "7", "9")) } @@ -121,20 +119,11 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { conf: SparkConf = new SparkConf(false), appArgs: Array[String] = Array(), pyFiles: Seq[String] = Nil): KubernetesDriverSpec = { - val driverConf = new KubernetesDriverSpecificConf( - resource, MAIN_CLASS, "appName", appArgs, pyFiles = pyFiles) - val kubernetesConf = KubernetesConf( - conf, - driverConf, - "resource-prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = conf, + mainAppResource = resource, + appArgs = appArgs, + pyFiles = pyFiles) val step = new DriverCommandFeatureStep(kubernetesConf) val pod = step.configurePod(SparkPod.initialPod()) val props = step.getAdditionalPodSystemProperties() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala index 36c6616a87b0..7d8e9296a6cb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala @@ -18,51 +18,25 @@ package org.apache.spark.deploy.k8s.features import java.io.File +import scala.collection.JavaConverters._ + import com.google.common.base.Charsets import com.google.common.io.{BaseEncoding, Files} -import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, Secret} -import org.mockito.{Mock, MockitoAnnotations} -import org.scalatest.BeforeAndAfter -import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.Secret import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.util.Utils -class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { +class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite { - private val KUBERNETES_RESOURCE_NAME_PREFIX = "spark" - private val APP_ID = "k8s-app" - private var credentialsTempDirectory: File = _ + private val credentialsTempDirectory = Utils.createTempDir() private val BASE_DRIVER_POD = SparkPod.initialPod() - @Mock - private var driverSpecificConf: KubernetesDriverSpecificConf = _ - - before { - MockitoAnnotations.initMocks(this) - credentialsTempDirectory = Utils.createTempDir() - } - - after { - credentialsTempDirectory.delete() - } - test("Don't set any credentials") { - val kubernetesConf = KubernetesConf( - new SparkConf(false), - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf() val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalPodSystemProperties().isEmpty) @@ -83,19 +57,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .set( s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", "/mnt/secrets/my-ca.pem") - val kubernetesConf = KubernetesConf( - submissionSparkConf, - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) - + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalKubernetesResources().isEmpty) @@ -122,18 +84,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .set( s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", caCertFile.getAbsolutePath) - val kubernetesConf = KubernetesConf( - submissionSparkConf, - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) val resolvedProperties = kubernetesCredentialsStep.getAdditionalPodSystemProperties() val expectedSparkConf = Map( @@ -153,7 +104,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .head .asInstanceOf[Secret] assert(credentialsSecret.getMetadata.getName === - s"$KUBERNETES_RESOURCE_NAME_PREFIX-kubernetes-credentials") + s"${kubernetesConf.resourceNamePrefix}-kubernetes-credentials") val decodedSecretData = credentialsSecret.getData.asScala.map { data => (data._1, new String(BaseEncoding.base64().decode(data._2), Charsets.UTF_8)) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala index 3c46667c3042..045278939dff 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala @@ -16,24 +16,19 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.Service -import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Mockito.when -import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.Service + import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.JavaMainAppResource -import org.apache.spark.util.Clock - -class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { +import org.apache.spark.internal.config._ +import org.apache.spark.util.ManualClock - private val SHORT_RESOURCE_NAME_PREFIX = - "a" * (DriverServiceFeatureStep.MAX_SERVICE_NAME_LENGTH - - DriverServiceFeatureStep.DRIVER_SVC_POSTFIX.length) +class DriverServiceFeatureStepSuite extends SparkFunSuite { private val LONG_RESOURCE_NAME_PREFIX = "a" * (DriverServiceFeatureStep.MAX_SERVICE_NAME_LENGTH - @@ -42,34 +37,14 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { "label1key" -> "label1value", "label2key" -> "label2value") - @Mock - private var clock: Clock = _ - - private var sparkConf: SparkConf = _ - - before { - MockitoAnnotations.initMocks(this) - sparkConf = new SparkConf(false) - } - test("Headless service has a port for the driver RPC and the block manager.") { - sparkConf = sparkConf + val sparkConf = new SparkConf(false) .set("spark.driver.port", "9000") - .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + .set(DRIVER_BLOCK_MANAGER_PORT, 8080) + val kconf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) assert(configurationStep.configurePod(SparkPod.initialPod()) === SparkPod.initialPod()) assert(configurationStep.getAdditionalKubernetesResources().size === 1) assert(configurationStep.getAdditionalKubernetesResources().head.isInstanceOf[Service]) @@ -80,50 +55,28 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { verifyService( 9000, 8080, - s"$SHORT_RESOURCE_NAME_PREFIX${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", + s"${kconf.resourceNamePrefix}${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", driverService) } test("Hostname and ports are set according to the service name.") { - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf - .set("spark.driver.port", "9000") - .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) - .set(KUBERNETES_NAMESPACE, "my-namespace"), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) - val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + - DriverServiceFeatureStep.DRIVER_SVC_POSTFIX + val sparkConf = new SparkConf(false) + .set("spark.driver.port", "9000") + .set(DRIVER_BLOCK_MANAGER_PORT, 8080) + .set(KUBERNETES_NAMESPACE, "my-namespace") + val kconf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) + val expectedServiceName = kconf.resourceNamePrefix + DriverServiceFeatureStep.DRIVER_SVC_POSTFIX val expectedHostName = s"$expectedServiceName.my-namespace.svc" val additionalProps = configurationStep.getAdditionalPodSystemProperties() verifySparkConfHostNames(additionalProps, expectedHostName) } test("Ports should resolve to defaults in SparkConf and in the service.") { - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + val kconf = KubernetesTestConf.createDriverConf(labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) val resolvedService = configurationStep .getAdditionalKubernetesResources() .head @@ -131,30 +84,23 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { verifyService( DEFAULT_DRIVER_PORT, DEFAULT_BLOCKMANAGER_PORT, - s"$SHORT_RESOURCE_NAME_PREFIX${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", + s"${kconf.resourceNamePrefix}${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", resolvedService) val additionalProps = configurationStep.getAdditionalPodSystemProperties() assert(additionalProps("spark.driver.port") === DEFAULT_DRIVER_PORT.toString) - assert(additionalProps(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key) - === DEFAULT_BLOCKMANAGER_PORT.toString) + assert(additionalProps(DRIVER_BLOCK_MANAGER_PORT.key) === DEFAULT_BLOCKMANAGER_PORT.toString) } test("Long prefixes should switch to using a generated name.") { - when(clock.getTimeMillis()).thenReturn(10000) + val clock = new ManualClock() + clock.setTime(10000) + val sparkConf = new SparkConf(false) + .set(KUBERNETES_NAMESPACE, "my-namespace") val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf.set(KUBERNETES_NAMESPACE, "my-namespace"), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None), + KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + resourceNamePrefix = Some(LONG_RESOURCE_NAME_PREFIX), + labels = DRIVER_LABELS), clock) val driverService = configurationStep .getAdditionalKubernetesResources() @@ -168,56 +114,27 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("Disallow bind address and driver host to be set explicitly.") { - try { - new DriverServiceFeatureStep( - KubernetesConf( - sparkConf.set(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS, "host"), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None), - clock) - fail("The driver bind address should not be allowed.") - } catch { - case e: Throwable => - assert(e.getMessage === - s"requirement failed: ${DriverServiceFeatureStep.DRIVER_BIND_ADDRESS_KEY} is" + - " not supported in Kubernetes mode, as the driver's bind address is managed" + - " and set to the driver pod's IP address.") + val sparkConf = new SparkConf(false) + .set(DRIVER_BIND_ADDRESS, "host") + .set("spark.app.name", LONG_RESOURCE_NAME_PREFIX) + val e1 = intercept[IllegalArgumentException] { + new DriverServiceFeatureStep(KubernetesTestConf.createDriverConf(sparkConf = sparkConf)) } - sparkConf.remove(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS) - sparkConf.set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, "host") - try { - new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None), - clock) - fail("The driver host address should not be allowed.") - } catch { - case e: Throwable => - assert(e.getMessage === - s"requirement failed: ${DriverServiceFeatureStep.DRIVER_HOST_KEY} is" + - " not supported in Kubernetes mode, as the driver's hostname will be managed via" + - " a Kubernetes service.") + assert(e1.getMessage === + s"requirement failed: ${DriverServiceFeatureStep.DRIVER_BIND_ADDRESS_KEY} is" + + " not supported in Kubernetes mode, as the driver's bind address is managed" + + " and set to the driver pod's IP address.") + + sparkConf.remove(DRIVER_BIND_ADDRESS) + sparkConf.set(DRIVER_HOST_ADDRESS, "host") + + val e2 = intercept[IllegalArgumentException] { + new DriverServiceFeatureStep(KubernetesTestConf.createDriverConf(sparkConf = sparkConf)) } + assert(e2.getMessage === + s"requirement failed: ${DriverServiceFeatureStep.DRIVER_HOST_KEY} is" + + " not supported in Kubernetes mode, as the driver's hostname will be managed via" + + " a Kubernetes service.") } private def verifyService( @@ -227,7 +144,9 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { service: Service): Unit = { assert(service.getMetadata.getName === expectedServiceName) assert(service.getSpec.getClusterIP === "None") - assert(service.getSpec.getSelector.asScala === DRIVER_LABELS) + DRIVER_LABELS.foreach { case (k, v) => + assert(service.getSpec.getSelector.get(k) === v) + } assert(service.getSpec.getPorts.size() === 2) val driverServicePorts = service.getSpec.getPorts.asScala assert(driverServicePorts.head.getName === DRIVER_PORT_NAME) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala index 3d253079c3ce..045552611106 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala @@ -16,12 +16,12 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.PodBuilder +import scala.collection.JavaConverters._ -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s._ -class EnvSecretsFeatureStepSuite extends SparkFunSuite{ +class EnvSecretsFeatureStepSuite extends SparkFunSuite { private val KEY_REF_NAME_FOO = "foo" private val KEY_REF_NAME_BAR = "bar" private val KEY_REF_KEY_FOO = "key_foo" @@ -34,28 +34,14 @@ class EnvSecretsFeatureStepSuite extends SparkFunSuite{ val envVarsToKeys = Map( ENV_NAME_BAR -> s"${KEY_REF_NAME_BAR}:${KEY_REF_KEY_BAR}", ENV_NAME_FOO -> s"${KEY_REF_NAME_FOO}:${KEY_REF_KEY_FOO}") - val sparkConf = new SparkConf(false) - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesExecutorSpecificConf("1", Some(new PodBuilder().build())), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - Map.empty, - envVarsToKeys, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf( + secretEnvNamesToKeyRefs = envVarsToKeys) val step = new EnvSecretsFeatureStep(kubernetesConf) - val driverContainerWithEnvSecrets = step.configurePod(baseDriverPod).container - - val expectedVars = - Seq(s"${ENV_NAME_BAR}", s"${ENV_NAME_FOO}") - - expectedVars.foreach { envName => - assert(KubernetesFeaturesTestUtils.containerHasEnvVar(driverContainerWithEnvSecrets, envName)) + val container = step.configurePod(baseDriverPod).container + val containerEnvKeys = container.getEnv.asScala.map { v => v.getName }.toSet + envVarsToKeys.keys.foreach { envName => + assert(containerEnvKeys.contains(envName)) } } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala index 894d824999aa..8f34ce5c6b94 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala @@ -17,45 +17,19 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.{EnvVarBuilder, VolumeBuilder, VolumeMountBuilder} -import org.mockito.Mockito -import org.scalatest._ -import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.util.SparkConfWithEnv -class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { +class LocalDirsFeatureStepSuite extends SparkFunSuite { private val defaultLocalDir = "/var/data/default-local-dir" - private var sparkConf: SparkConf = _ - private var kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf] = _ - - before { - val realSparkConf = new SparkConf(false) - sparkConf = Mockito.spy(realSparkConf) - kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "app-name", - "main", - Seq.empty), - "resource", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) - } test("Resolve to default local dir if neither env nor configuration are set") { - Mockito.doReturn(null).when(sparkConf).get("spark.local.dir") - Mockito.doReturn(null).when(sparkConf).getenv("SPARK_LOCAL_DIRS") - val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) + val stepUnderTest = new LocalDirsFeatureStep(KubernetesTestConf.createDriverConf(), + defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 1) assert(configuredPod.pod.getSpec.getVolumes.get(0) === @@ -79,8 +53,9 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("Use configured local dirs split on comma if provided.") { - Mockito.doReturn("/var/data/my-local-dir-1,/var/data/my-local-dir-2") - .when(sparkConf).getenv("SPARK_LOCAL_DIRS") + val sparkConf = new SparkConfWithEnv(Map( + "SPARK_LOCAL_DIRS" -> "/var/data/my-local-dir-1,/var/data/my-local-dir-2")) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 2) @@ -116,9 +91,8 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("Use tmpfs to back default local dir") { - Mockito.doReturn(null).when(sparkConf).get("spark.local.dir") - Mockito.doReturn(null).when(sparkConf).getenv("SPARK_LOCAL_DIRS") - Mockito.doReturn(true).when(sparkConf).get(KUBERNETES_LOCAL_DIRS_TMPFS) + val sparkConf = new SparkConf(false).set(KUBERNETES_LOCAL_DIRS_TMPFS, true) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 1) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala index 1555f6a9c652..22f6d26c4d0d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala @@ -16,10 +16,8 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.PodBuilder - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SecretVolumeUtils, SparkPod} +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SecretVolumeUtils, SparkPod} class MountSecretsFeatureStepSuite extends SparkFunSuite { @@ -32,19 +30,8 @@ class MountSecretsFeatureStepSuite extends SparkFunSuite { val secretNamesToMountPaths = Map( SECRET_FOO -> SECRET_MOUNT_PATH, SECRET_BAR -> SECRET_MOUNT_PATH) - val sparkConf = new SparkConf(false) - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesExecutorSpecificConf("1", Some(new PodBuilder().build())), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - secretNamesToMountPaths, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createExecutorConf( + secretNamesToMountPaths = secretNamesToMountPaths) val step = new MountSecretsFeatureStep(kubernetesConf) val driverPodWithSecretsMounted = step.configurePod(baseDriverPod).pod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala index aadbf16897f4..e6f1dd640e3e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala @@ -16,29 +16,12 @@ */ package org.apache.spark.deploy.k8s.features +import scala.collection.JavaConverters._ + import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.submit.JavaMainAppResource class MountVolumesFeatureStepSuite extends SparkFunSuite { - private val sparkConf = new SparkConf(false) - private val emptyKubernetesConf = KubernetesConf( - sparkConf = sparkConf, - roleSpecificConf = KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "app-name", - "main", - Seq.empty), - appResourceNamePrefix = "resource", - appId = "app-id", - roleLabels = Map.empty, - roleAnnotations = Map.empty, - roleSecretNamesToMountPaths = Map.empty, - roleSecretEnvNamesToKeyRefs = Map.empty, - roleEnvs = Map.empty, - roleVolumes = Nil, - hadoopConfSpec = None) - test("Mounts hostPath volumes") { val volumeConf = KubernetesVolumeSpec( "testVolume", @@ -47,7 +30,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { false, KubernetesHostPathVolumeConf("/hostPath/tmp") ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -67,7 +50,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { true, KubernetesPVCVolumeConf("pvcClaim") ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -89,7 +72,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { false, KubernetesEmptyDirVolumeConf(Some("Memory"), Some("6G")) ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -111,7 +94,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { false, KubernetesEmptyDirVolumeConf(None, None) ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -140,8 +123,8 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { true, KubernetesPVCVolumeConf("pvcClaim") ) - val volumesConf = hpVolumeConf :: pvcVolumeConf :: Nil - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumesConf) + val kubernetesConf = KubernetesTestConf.createDriverConf( + volumes = Seq(hpVolumeConf, pvcVolumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -157,7 +140,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { false, KubernetesEmptyDirVolumeConf(None, None) ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -176,7 +159,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { true, KubernetesPVCVolumeConf("pvcClaim") ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -206,19 +189,18 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { true, KubernetesEmptyDirVolumeConf(None, None) ) - val kubernetesConf = emptyKubernetesConf.copy( - roleVolumes = emptyDirSpec :: pvcSpec :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(emptyDirSpec, pvcSpec)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size() === 2) - val mounts = configuredPod.container.getVolumeMounts - assert(mounts.size() === 2) - assert(mounts.get(0).getName === "testEmptyDir") - assert(mounts.get(0).getMountPath === "/tmp/foo") - assert(mounts.get(0).getSubPath === "foo") - assert(mounts.get(1).getName === "testPVC") - assert(mounts.get(1).getMountPath === "/tmp/bar") - assert(mounts.get(1).getSubPath === "bar") + val mounts = configuredPod.container.getVolumeMounts.asScala.sortBy(_.getName()) + assert(mounts.size === 2) + assert(mounts(0).getName === "testEmptyDir") + assert(mounts(0).getMountPath === "/tmp/foo") + assert(mounts(0).getSubPath === "foo") + assert(mounts(1).getName === "testPVC") + assert(mounts(1).getMountPath === "/tmp/bar") + assert(mounts(1).getSubPath === "bar") } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala index 370948c9502e..7295b82ca479 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala @@ -20,40 +20,22 @@ import java.io.{File, PrintWriter} import java.nio.file.Files import io.fabric8.kubernetes.api.model.ConfigMap -import org.mockito.Mockito import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.submit.JavaMainAppResource class PodTemplateConfigMapStepSuite extends SparkFunSuite with BeforeAndAfter { - private var sparkConf: SparkConf = _ - private var kubernetesConf : KubernetesConf[_ <: KubernetesRoleSpecificConf] = _ + private var kubernetesConf : KubernetesConf = _ private var templateFile: File = _ before { - sparkConf = Mockito.mock(classOf[SparkConf]) - kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "app-name", - "main", - Seq.empty), - "resource", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) templateFile = Files.createTempFile("pod-template", "yml").toFile templateFile.deleteOnExit() - Mockito.doReturn(Option(templateFile.getAbsolutePath)).when(sparkConf) - .get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) + + val sparkConf = new SparkConf(false) + .set(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, templateFile.getAbsolutePath) + kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) } test("Mounts executor template volume if config specified") { 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 08f28758ef48..e9c05fef6f5d 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 @@ -24,8 +24,8 @@ import org.mockito.Mockito.{doReturn, verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.mockito.MockitoSugar._ -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ @@ -37,10 +37,6 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { private val KUBERNETES_RESOURCE_PREFIX = "resource-example" private val POD_NAME = "driver" private val CONTAINER_NAME = "container" - private val APP_ID = "app-id" - private val APP_NAME = "app" - private val MAIN_CLASS = "main" - private val APP_ARGS = Seq("arg1", "arg2") private val RESOLVED_JAVA_OPTIONS = Map( "conf1key" -> "conf1value", "conf2key" -> "conf2value") @@ -122,28 +118,15 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var resourceList: RESOURCE_LIST = _ - private var kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf] = _ - - private var sparkConf: SparkConf = _ + private var kconf: KubernetesDriverConf = _ private var createdPodArgumentCaptor: ArgumentCaptor[Pod] = _ private var createdResourcesArgumentCaptor: ArgumentCaptor[HasMetadata] = _ before { MockitoAnnotations.initMocks(this) - sparkConf = new SparkConf(false) - kubernetesConf = KubernetesConf[KubernetesDriverSpecificConf]( - sparkConf, - KubernetesDriverSpecificConf(JavaMainAppResource(None), MAIN_CLASS, APP_NAME, APP_ARGS), - KUBERNETES_RESOURCE_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) - when(driverBuilder.buildFromFeatures(kubernetesConf)).thenReturn(BUILT_KUBERNETES_SPEC) + kconf = KubernetesTestConf.createDriverConf( + resourceNamePrefix = Some(KUBERNETES_RESOURCE_PREFIX)) + when(driverBuilder.buildFromFeatures(kconf)).thenReturn(BUILT_KUBERNETES_SPEC) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(POD_NAME)).thenReturn(namedPods) @@ -158,26 +141,22 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { test("The client should configure the pod using the builder.") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, false, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() verify(podOperations).create(FULL_EXPECTED_POD) } test("The client should create Kubernetes resources") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, false, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues assert(otherCreatedResources.size === 2) @@ -197,13 +176,11 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { test("Waiting for app completion should stall on the watcher") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, true, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() verify(loggingPodStatusWatcher).awaitCompletion() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index 3708864592d7..7e7dc4763c2e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -83,48 +83,21 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { _ => templateVolumeStep) test("Apply fundamental steps all the time.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(Some("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf() validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, CREDENTIALS_STEP_TYPE, SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + DRIVER_CMD_STEP_TYPE, + HADOOP_GLOBAL_STEP_TYPE) } test("Apply secrets step if secrets are present.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map("secret" -> "secretMountPath"), - Map("EnvName" -> "SecretName:secretKey"), - Map.empty, - Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf( + secretEnvNamesToKeyRefs = Map("EnvName" -> "SecretName:secretKey"), + secretNamesToMountPaths = Map("secret" -> "secretMountPath")) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -133,7 +106,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { LOCAL_DIRS_STEP_TYPE, SECRETS_STEP_TYPE, ENV_SECRETS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + DRIVER_CMD_STEP_TYPE, + HADOOP_GLOBAL_STEP_TYPE) } test("Apply volumes step if mounts are present.") { @@ -143,22 +117,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { "", false, KubernetesHostPathVolumeConf("/path")) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - volumeSpec :: Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeSpec)) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -166,7 +125,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, MOUNT_VOLUMES_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + DRIVER_CMD_STEP_TYPE, + HADOOP_GLOBAL_STEP_TYPE) } test("Apply volumes step if a mount subpath is present.") { @@ -176,22 +136,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { "foo", false, KubernetesHostPathVolumeConf("/path")) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - volumeSpec :: Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeSpec)) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -199,89 +144,14 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, MOUNT_VOLUMES_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) - } - - test("Apply template volume step if executor template is present.") { - val sparkConf = spy(new SparkConf(false)) - doReturn(Option("filename")).when(sparkConf) - .get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) - val conf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(Some("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - TEMPLATE_VOLUME_STEP_TYPE) - } - - test("Apply HadoopSteps if HADOOP_CONF_DIR is defined.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = Some( - HadoopConfSpec( - Some("/var/hadoop-conf"), - None))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, DRIVER_CMD_STEP_TYPE, HADOOP_GLOBAL_STEP_TYPE) } - test("Apply HadoopSteps if HADOOP_CONF ConfigMap is defined.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = Some( - HadoopConfSpec( - None, - Some("pre-defined-configMapName")))) + test("Apply template volume step if executor template is present.") { + val sparkConf = new SparkConf(false) + .set(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, "filename") + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -289,12 +159,16 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) + HADOOP_GLOBAL_STEP_TYPE, + TEMPLATE_VOLUME_STEP_TYPE) } private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) : Unit = { - assert(resolvedSpec.systemProperties.size === stepTypes.size) + val addedProperties = resolvedSpec.systemProperties + .filter { case (k, _) => !k.startsWith("spark.") } + .toMap + assert(addedProperties.keys.toSet === stepTypes.toSet) stepTypes.foreach { stepType => assert(resolvedSpec.pod.pod.getMetadata.getLabels.get(stepType) === stepType) assert(resolvedSpec.driverKubernetesResources.containsSlice( @@ -314,22 +188,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(CONTAINER_IMAGE, "spark-driver:latest") .set(KUBERNETES_DRIVER_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = new KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(Some("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val driverSpec = KubernetesDriverBuilder .apply(kubernetesClient, sparkConf) .buildFromFeatures(kubernetesConf) @@ -346,22 +205,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(CONTAINER_IMAGE, "spark-driver:latest") .set(KUBERNETES_DRIVER_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = new KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(Some("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val exception = intercept[SparkException] { KubernetesDriverBuilder .apply(kubernetesClient, sparkConf) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 2f984e5d8980..ddf9f67a0727 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -27,7 +27,7 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ @@ -79,7 +79,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) when(driverPodOperations.get).thenReturn(driverPod) - when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) + when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]))) .thenAnswer(executorPodAnswer()) snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() waitForExecutorPodsClock = new ManualClock(0L) @@ -147,44 +147,9 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private def executorPodAnswer(): Answer[SparkPod] = { new Answer[SparkPod] { override def answer(invocation: InvocationOnMock): SparkPod = { - val k8sConf = invocation.getArgumentAt( - 0, classOf[KubernetesConf[KubernetesExecutorSpecificConf]]) - executorPodWithId(k8sConf.roleSpecificConf.executorId.toInt) + val k8sConf = invocation.getArgumentAt(0, classOf[KubernetesExecutorConf]) + executorPodWithId(k8sConf.executorId.toInt) } } } - - private def kubernetesConfWithCorrectFields(): KubernetesConf[KubernetesExecutorSpecificConf] = - Matchers.argThat(new ArgumentMatcher[KubernetesConf[KubernetesExecutorSpecificConf]] { - override def matches(argument: scala.Any): Boolean = { - if (!argument.isInstanceOf[KubernetesConf[_]]) { - false - } else { - val k8sConf = argument.asInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] - val executorSpecificConf = k8sConf.roleSpecificConf - // TODO: HADOOP_CONF_DIR - val expectedK8sConf = KubernetesConf.createExecutorConf( - conf, - executorSpecificConf.executorId, - TEST_SPARK_APP_ID, - Some(driverPod)) - - // Set prefixes to a common string since KUBERNETES_EXECUTOR_POD_NAME_PREFIX - // has not be set for the tests and thus KubernetesConf will use a random - // string for the prefix, based on the app name, and this comparison here will fail. - val k8sConfCopy = k8sConf - .copy(appResourceNamePrefix = "") - .copy(sparkConf = conf) - val expectedK8sConfCopy = expectedK8sConf - .copy(appResourceNamePrefix = "") - .copy(sparkConf = conf) - - k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && - // Since KubernetesConf.createExecutorConf clones the SparkConf object, force - // deep equality comparison for the SparkConf object and use object equality - // comparison on all other fields. - k8sConfCopy == expectedK8sConfCopy - } - } - }) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index a59f6d072023..b6a75b15af85 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.scheduler.cluster.k8s +import scala.collection.JavaConverters._ + import io.fabric8.kubernetes.api.model.{Config => _, _} import io.fabric8.kubernetes.client.KubernetesClient import org.mockito.Mockito.{mock, never, verify} @@ -25,6 +27,7 @@ import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ import org.apache.spark.deploy.k8s.submit.PodBuilderSuiteUtils +import org.apache.spark.util.SparkConfWithEnv class KubernetesExecutorBuilderSuite extends SparkFunSuite { private val BASIC_STEP_TYPE = "basic" @@ -64,37 +67,15 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { _ => hadoopSparkUser) test("Basic steps are consistently applied.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - None) + val conf = KubernetesTestConf.createExecutorConf() validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) } test("Apply secrets step if secrets are present.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map("secret" -> "secretMountPath"), - Map("secret-name" -> "secret-key"), - Map.empty, - Nil, - None) + val conf = KubernetesTestConf.createExecutorConf( + secretEnvNamesToKeyRefs = Map("secret-name" -> "secret-key"), + secretNamesToMountPaths = Map("secret" -> "secretMountPath")) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -110,19 +91,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { "", false, KubernetesHostPathVolumeConf("/checkpoint")) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - volumeSpec :: Nil, - None) + val conf = KubernetesTestConf.createExecutorConf( + volumes = Seq(volumeSpec)) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -132,25 +102,10 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { test("Apply basicHadoop step if HADOOP_CONF_DIR is defined") { // HADOOP_DELEGATION_TOKEN - val HADOOP_CREDS_PREFIX = "spark.security.credentials." - val HADOOPFS_PROVIDER = s"$HADOOP_CREDS_PREFIX.hadoopfs.enabled" - val conf = KubernetesConf( - new SparkConf(false) + val conf = KubernetesTestConf.createExecutorConf( + sparkConf = new SparkConfWithEnv(Map("HADOOP_CONF_DIR" -> "/var/hadoop-conf")) .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") - .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") - .set(KERBEROS_SPARK_USER_NAME, "spark-user") - .set(HADOOPFS_PROVIDER, "true"), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Some(HadoopConfSpec(Some("/var/hadoop-conf"), None))) + .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name")) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -160,24 +115,13 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { } test("Apply kerberos step if DT secrets created") { - val conf = KubernetesConf( - new SparkConf(false) + val conf = KubernetesTestConf.createExecutorConf( + sparkConf = new SparkConf(false) .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") .set(KERBEROS_SPARK_USER_NAME, "spark-user") .set(KERBEROS_DT_SECRET_NAME, "dt-secret") - .set(KERBEROS_DT_SECRET_KEY, "dt-key"), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Some(HadoopConfSpec(None, Some("pre-defined-onfigMapName")))) + .set(KERBEROS_DT_SECRET_KEY, "dt-key" )) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -187,10 +131,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { } private def validateStepTypesApplied(resolvedPod: SparkPod, stepTypes: String*): Unit = { - assert(resolvedPod.pod.getMetadata.getLabels.size === stepTypes.size) - stepTypes.foreach { stepType => - assert(resolvedPod.pod.getMetadata.getLabels.get(stepType) === stepType) - } + assert(resolvedPod.pod.getMetadata.getLabels.asScala.keys.toSet === stepTypes.toSet) } test("Starts with empty executor pod if template is not specified") { @@ -205,25 +146,14 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { .set("spark.driver.host", "https://driver.host.com") .set(Config.CONTAINER_IMAGE, "spark-executor:latest") .set(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder() - .withNewMetadata() + val kubernetesConf = KubernetesTestConf.createExecutorConf( + sparkConf = sparkConf, + driverPod = Some(new PodBuilder() + .withNewMetadata() .withName("driver") .endMetadata() - .build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) - val sparkPod = KubernetesExecutorBuilder - .apply(kubernetesClient, sparkConf) + .build())) + val sparkPod = KubernetesExecutorBuilder(kubernetesClient, sparkConf) .buildFromFeatures(kubernetesConf) PodBuilderSuiteUtils.verifyPodWithSupportedFeatures(sparkPod) } From 28d33744076abd8bf7955eefcbdeef4849a99c40 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Sat, 1 Dec 2018 10:37:03 +0800 Subject: [PATCH 0036/1072] [SPARK-23647][PYTHON][SQL] Adds more types for hint in pyspark Signed-off-by: DylanGuedes ## What changes were proposed in this pull request? Addition of float, int and list hints for `pyspark.sql` Hint. ## How was this patch tested? I did manual tests following the same principles used in the Scala version, and also added unit tests. Closes #20788 from DylanGuedes/jira-21030. Authored-by: DylanGuedes Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/dataframe.py | 6 ++++-- python/pyspark/sql/tests/test_dataframe.py | 13 +++++++++++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index b8833a39078b..1b1092c409be 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -485,10 +485,12 @@ 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) for p in parameters: - if not isinstance(p, str): + if not isinstance(p, allowed_types): raise TypeError( - "all parameters should be str, got {0} of type {1}".format(p, type(p))) + "all parameters should be in {0}, got {1} of type {2}".format( + allowed_types, p, type(p))) jdf = self._jdf.hint(name, self._jseq(parameters)) return DataFrame(jdf, self.sql_ctx) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 908d400e0009..65edf593c300 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -375,6 +375,19 @@ def test_generic_hints(self): plan = df1.join(df2.hint("broadcast"), "id")._jdf.queryExecution().executedPlan() self.assertEqual(1, plan.toString().count("BroadcastHashJoin")) + # add tests for SPARK-23647 (test more types for hint) + def test_extended_hint_types(self): + from pyspark.sql import DataFrame + + df = self.spark.range(10e10).toDF("id") + such_a_nice_list = ["itworks1", "itworks2", "itworks3"] + hinted_df = df.hint("my awesome hint", 1.2345, "what", such_a_nice_list) + logical_plan = hinted_df._jdf.queryExecution().logical() + + self.assertEqual(1, logical_plan.toString().count("1.2345")) + self.assertEqual(1, logical_plan.toString().count("what")) + self.assertEqual(3, logical_plan.toString().count("itworks")) + def test_sample(self): self.assertRaisesRegexp( TypeError, From 2f6e88fecb455a02c4c08c41290e2f338e979543 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 30 Nov 2018 23:14:05 -0800 Subject: [PATCH 0037/1072] [SPARK-26189][R] Fix unionAll doc in SparkR ## What changes were proposed in this pull request? Fix unionAll doc in SparkR ## How was this patch tested? Manually ran test Author: Huaxin Gao Closes #23161 from huaxingao/spark-26189. --- R/pkg/R/DataFrame.R | 20 ++++++++++++++++---- R/pkg/R/generics.R | 2 +- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 745bb3e15932..24ed449f2a7d 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2730,13 +2730,25 @@ setMethod("union", dataFrame(unioned) }) -#' Return a new SparkDataFrame containing the union of rows +#' Return a new SparkDataFrame containing the union of rows. #' -#' This is an alias for `union`. +#' This is an alias for \code{union}. #' -#' @rdname union -#' @name unionAll +#' @param x a SparkDataFrame. +#' @param y a SparkDataFrame. +#' @return A SparkDataFrame containing the result of the unionAll operation. +#' @family SparkDataFrame functions #' @aliases unionAll,SparkDataFrame,SparkDataFrame-method +#' @rdname unionAll +#' @name unionAll +#' @seealso \link{union} +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) +#' unionAllDF <- unionAll(df1, df2) +#' } #' @note unionAll since 1.4.0 setMethod("unionAll", signature(x = "SparkDataFrame", y = "SparkDataFrame"), diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 9d8c24c686c7..eed76465221c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -631,7 +631,7 @@ setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) #' @rdname union setGeneric("union", function(x, y) { standardGeneric("union") }) -#' @rdname union +#' @rdname unionAll setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) #' @rdname unionByName From 327ac83f5cf33c84775a95442862bea56d8a0005 Mon Sep 17 00:00:00 2001 From: caoxuewen Date: Sat, 1 Dec 2018 16:34:11 +0800 Subject: [PATCH 0038/1072] [SPARK-26180][CORE][TEST] Reuse withTempDir function to the SparkCore test case ## What changes were proposed in this pull request? Currently, the common `withTempDir` function is used in Spark SQL test cases. To handle `val dir = Utils. createTempDir()` and `Utils. deleteRecursively (dir)`. Unfortunately, the `withTempDir` function cannot be used in the Spark Core test case. This PR Sharing `withTempDir` function in Spark Sql and SparkCore to clean up SparkCore test cases. thanks. ## How was this patch tested? N / A Closes #23151 from heary-cao/withCreateTempDir. Authored-by: caoxuewen Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/CheckpointSuite.scala | 5 +- .../apache/spark/ContextCleanerSuite.scala | 97 ++-- .../scala/org/apache/spark/FileSuite.scala | 19 +- .../org/apache/spark/SparkContextSuite.scala | 315 +++++----- .../org/apache/spark/SparkFunSuite.scala | 12 +- .../api/python/PythonBroadcastSuite.scala | 5 +- .../spark/deploy/SparkSubmitSuite.scala | 539 +++++++++--------- .../history/FsHistoryProviderSuite.scala | 89 +-- .../history/HistoryServerArgumentsSuite.scala | 8 +- .../master/PersistenceEngineSuite.scala | 5 +- .../input/WholeTextFileInputFormatSuite.scala | 6 +- .../WholeTextFileRecordReaderSuite.scala | 62 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 5 +- .../org/apache/spark/rpc/RpcEnvSuite.scala | 113 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 22 +- ...putCommitCoordinatorIntegrationSuite.scala | 5 +- .../serializer/KryoSerializerSuite.scala | 42 +- .../apache/spark/storage/DiskStoreSuite.scala | 1 - .../util/PeriodicRDDCheckpointerSuite.scala | 52 +- .../org/apache/spark/util/UtilsSuite.scala | 222 ++++---- .../apache/spark/sql/test/SQLTestUtils.scala | 100 ++-- .../spark/sql/hive/client/VersionsSuite.scala | 9 - .../spark/streaming/TestSuiteBase.scala | 12 - 23 files changed, 858 insertions(+), 887 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 48408ccc8f81..6d9e47cfd00f 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -586,8 +586,7 @@ object CheckpointSuite { class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { test("checkpoint compression") { - val checkpointDir = Utils.createTempDir() - try { + withTempDir { checkpointDir => val conf = new SparkConf() .set("spark.checkpoint.compress", "true") .set("spark.ui.enabled", "false") @@ -616,8 +615,6 @@ class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { // Verify that the compressed content can be read back assert(rdd.collect().toSeq === (1 to 20)) - } finally { - Utils.deleteRecursively(checkpointDir) } } } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 6724af952505..1fcc975ab39a 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -207,54 +207,55 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { } test("automatically cleanup normal checkpoint") { - val checkpointDir = Utils.createTempDir() - checkpointDir.delete() - var rdd = newPairRDD() - sc.setCheckpointDir(checkpointDir.toString) - rdd.checkpoint() - rdd.cache() - rdd.collect() - var rddId = rdd.id - - // Confirm the checkpoint directory exists - assert(ReliableRDDCheckpointData.checkpointPath(sc, rddId).isDefined) - val path = ReliableRDDCheckpointData.checkpointPath(sc, rddId).get - val fs = path.getFileSystem(sc.hadoopConfiguration) - assert(fs.exists(path)) - - // the checkpoint is not cleaned by default (without the configuration set) - var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Seq(rddId)) - rdd = null // Make RDD out of scope, ok if collected earlier - runGC() - postGCTester.assertCleanup() - assert(!fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) - - // Verify that checkpoints are NOT cleaned up if the config is not enabled - sc.stop() - val conf = new SparkConf() - .setMaster("local[2]") - .setAppName("cleanupCheckpoint") - .set("spark.cleaner.referenceTracking.cleanCheckpoints", "false") - sc = new SparkContext(conf) - rdd = newPairRDD() - sc.setCheckpointDir(checkpointDir.toString) - rdd.checkpoint() - rdd.cache() - rdd.collect() - rddId = rdd.id - - // Confirm the checkpoint directory exists - assert(fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) - - // Reference rdd to defeat any early collection by the JVM - rdd.count() - - // Test that GC causes checkpoint data cleanup after dereferencing the RDD - postGCTester = new CleanerTester(sc, Seq(rddId)) - rdd = null // Make RDD out of scope - runGC() - postGCTester.assertCleanup() - assert(fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) + withTempDir { checkpointDir => + checkpointDir.delete() + var rdd = newPairRDD() + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + var rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(ReliableRDDCheckpointData.checkpointPath(sc, rddId).isDefined) + val path = ReliableRDDCheckpointData.checkpointPath(sc, rddId).get + val fs = path.getFileSystem(sc.hadoopConfiguration) + assert(fs.exists(path)) + + // the checkpoint is not cleaned by default (without the configuration set) + var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Seq(rddId)) + rdd = null // Make RDD out of scope, ok if collected earlier + runGC() + postGCTester.assertCleanup() + assert(!fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) + + // Verify that checkpoints are NOT cleaned up if the config is not enabled + sc.stop() + val conf = new SparkConf() + .setMaster("local[2]") + .setAppName("cleanupCheckpoint") + .set("spark.cleaner.referenceTracking.cleanCheckpoints", "false") + sc = new SparkContext(conf) + rdd = newPairRDD() + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) + + // Reference rdd to defeat any early collection by the JVM + rdd.count() + + // Test that GC causes checkpoint data cleanup after dereferencing the RDD + postGCTester = new CleanerTester(sc, Seq(rddId)) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) + } } test("automatically clean up local checkpoint") { diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index df04a5ea1d99..983a7917e8aa 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -306,17 +306,18 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { .set("spark.files.openCostInBytes", "0") .set("spark.default.parallelism", "1")) - val tempDir = Utils.createTempDir() - val tempDirPath = tempDir.getAbsolutePath + withTempDir { tempDir => + val tempDirPath = tempDir.getAbsolutePath - for (i <- 0 until 8) { - val tempFile = new File(tempDir, s"part-0000$i") - Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", tempFile, - StandardCharsets.UTF_8) - } + for (i <- 0 until 8) { + val tempFile = new File(tempDir, s"part-0000$i") + Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", tempFile, + StandardCharsets.UTF_8) + } - for (p <- Seq(1, 2, 8)) { - assert(sc.binaryFiles(tempDirPath, minPartitions = p).getNumPartitions === p) + for (p <- Seq(1, 2, 8)) { + assert(sc.binaryFiles(tempDirPath, minPartitions = p).getNumPartitions === p) + } } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 79192f3f3c92..ec4c7efb5835 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -116,56 +116,57 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test("basic case for addFile and listFiles") { - val dir = Utils.createTempDir() - - val file1 = File.createTempFile("someprefix1", "somesuffix1", dir) - val absolutePath1 = file1.getAbsolutePath - - val file2 = File.createTempFile("someprefix2", "somesuffix2", dir) - val relativePath = file2.getParent + "/../" + file2.getParentFile.getName + "/" + file2.getName - val absolutePath2 = file2.getAbsolutePath - - try { - Files.write("somewords1", file1, StandardCharsets.UTF_8) - Files.write("somewords2", file2, StandardCharsets.UTF_8) - val length1 = file1.length() - val length2 = file2.length() - - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addFile(file1.getAbsolutePath) - sc.addFile(relativePath) - sc.parallelize(Array(1), 1).map(x => { - val gotten1 = new File(SparkFiles.get(file1.getName)) - val gotten2 = new File(SparkFiles.get(file2.getName)) - if (!gotten1.exists()) { - throw new SparkException("file doesn't exist : " + absolutePath1) - } - if (!gotten2.exists()) { - throw new SparkException("file doesn't exist : " + absolutePath2) - } + withTempDir { dir => + val file1 = File.createTempFile("someprefix1", "somesuffix1", dir) + val absolutePath1 = file1.getAbsolutePath + + val file2 = File.createTempFile("someprefix2", "somesuffix2", dir) + val relativePath = file2.getParent + "/../" + file2.getParentFile.getName + + "/" + file2.getName + val absolutePath2 = file2.getAbsolutePath + + try { + Files.write("somewords1", file1, StandardCharsets.UTF_8) + Files.write("somewords2", file2, StandardCharsets.UTF_8) + val length1 = file1.length() + val length2 = file2.length() + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(file1.getAbsolutePath) + sc.addFile(relativePath) + sc.parallelize(Array(1), 1).map(x => { + val gotten1 = new File(SparkFiles.get(file1.getName)) + val gotten2 = new File(SparkFiles.get(file2.getName)) + if (!gotten1.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath1) + } + if (!gotten2.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath2) + } - if (length1 != gotten1.length()) { - throw new SparkException( - s"file has different length $length1 than added file ${gotten1.length()} : " + - absolutePath1) - } - if (length2 != gotten2.length()) { - throw new SparkException( - s"file has different length $length2 than added file ${gotten2.length()} : " + - absolutePath2) - } + if (length1 != gotten1.length()) { + throw new SparkException( + s"file has different length $length1 than added file ${gotten1.length()} : " + + absolutePath1) + } + if (length2 != gotten2.length()) { + throw new SparkException( + s"file has different length $length2 than added file ${gotten2.length()} : " + + absolutePath2) + } - if (absolutePath1 == gotten1.getAbsolutePath) { - throw new SparkException("file should have been copied :" + absolutePath1) - } - if (absolutePath2 == gotten2.getAbsolutePath) { - throw new SparkException("file should have been copied : " + absolutePath2) - } - x - }).count() - assert(sc.listFiles().filter(_.contains("somesuffix1")).size == 1) - } finally { - sc.stop() + if (absolutePath1 == gotten1.getAbsolutePath) { + throw new SparkException("file should have been copied :" + absolutePath1) + } + if (absolutePath2 == gotten2.getAbsolutePath) { + throw new SparkException("file should have been copied : " + absolutePath2) + } + x + }).count() + assert(sc.listFiles().filter(_.contains("somesuffix1")).size == 1) + } finally { + sc.stop() + } } } @@ -202,51 +203,51 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test("addFile recursive works") { - val pluto = Utils.createTempDir() - val neptune = Utils.createTempDir(pluto.getAbsolutePath) - val saturn = Utils.createTempDir(neptune.getAbsolutePath) - val alien1 = File.createTempFile("alien", "1", neptune) - val alien2 = File.createTempFile("alien", "2", saturn) - - try { - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addFile(neptune.getAbsolutePath, true) - sc.parallelize(Array(1), 1).map(x => { - val sep = File.separator - if (!new File(SparkFiles.get(neptune.getName + sep + alien1.getName)).exists()) { - throw new SparkException("can't access file under root added directory") - } - if (!new File(SparkFiles.get(neptune.getName + sep + saturn.getName + sep + alien2.getName)) - .exists()) { - throw new SparkException("can't access file in nested directory") - } - if (new File(SparkFiles.get(pluto.getName + sep + neptune.getName + sep + alien1.getName)) - .exists()) { - throw new SparkException("file exists that shouldn't") - } - x - }).count() - } finally { - sc.stop() + withTempDir { pluto => + val neptune = Utils.createTempDir(pluto.getAbsolutePath) + val saturn = Utils.createTempDir(neptune.getAbsolutePath) + val alien1 = File.createTempFile("alien", "1", neptune) + val alien2 = File.createTempFile("alien", "2", saturn) + + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(neptune.getAbsolutePath, true) + sc.parallelize(Array(1), 1).map(x => { + val sep = File.separator + if (!new File(SparkFiles.get(neptune.getName + sep + alien1.getName)).exists()) { + throw new SparkException("can't access file under root added directory") + } + if (!new File(SparkFiles.get( + neptune.getName + sep + saturn.getName + sep + alien2.getName)).exists()) { + throw new SparkException("can't access file in nested directory") + } + if (new File(SparkFiles.get( + pluto.getName + sep + neptune.getName + sep + alien1.getName)).exists()) { + throw new SparkException("file exists that shouldn't") + } + x + }).count() + } finally { + sc.stop() + } } } test("addFile recursive can't add directories by default") { - val dir = Utils.createTempDir() - - try { - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - intercept[SparkException] { - sc.addFile(dir.getAbsolutePath) + withTempDir { dir => + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + intercept[SparkException] { + sc.addFile(dir.getAbsolutePath) + } + } finally { + sc.stop() } - } finally { - sc.stop() } } test("cannot call addFile with different paths that have the same filename") { - val dir = Utils.createTempDir() - try { + withTempDir { dir => val subdir1 = new File(dir, "subdir1") val subdir2 = new File(dir, "subdir2") assert(subdir1.mkdir()) @@ -267,8 +268,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc.addFile(file2.getAbsolutePath) } assert(getAddedFileContents() === "old") - } finally { - Utils.deleteRecursively(dir) } } @@ -296,30 +295,33 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test("add jar with invalid path") { - val tmpDir = Utils.createTempDir() - val tmpJar = File.createTempFile("test", ".jar", tmpDir) + withTempDir { tmpDir => + val tmpJar = File.createTempFile("test", ".jar", tmpDir) - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addJar(tmpJar.getAbsolutePath) + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(tmpJar.getAbsolutePath) - // Invalid jar path will only print the error log, will not add to file server. - sc.addJar("dummy.jar") - sc.addJar("") - sc.addJar(tmpDir.getAbsolutePath) + // Invalid jar path will only print the error log, will not add to file server. + sc.addJar("dummy.jar") + sc.addJar("") + sc.addJar(tmpDir.getAbsolutePath) - assert(sc.listJars().size == 1) - assert(sc.listJars().head.contains(tmpJar.getName)) + assert(sc.listJars().size == 1) + assert(sc.listJars().head.contains(tmpJar.getName)) + } } test("SPARK-22585 addJar argument without scheme is interpreted literally without url decoding") { - val tmpDir = new File(Utils.createTempDir(), "host%3A443") - tmpDir.mkdirs() - val tmpJar = File.createTempFile("t%2F", ".jar", tmpDir) + withTempDir { dir => + val tmpDir = new File(dir, "host%3A443") + tmpDir.mkdirs() + val tmpJar = File.createTempFile("t%2F", ".jar", tmpDir) - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test") - sc.addJar(tmpJar.getAbsolutePath) - assert(sc.listJars().size === 1) + sc.addJar(tmpJar.getAbsolutePath) + assert(sc.listJars().size === 1) + } } test("Cancelling job group should not cause SparkContext to shutdown (SPARK-6414)") { @@ -340,60 +342,61 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("Comma separated paths for newAPIHadoopFile/wholeTextFiles/binaryFiles (SPARK-7155)") { // Regression test for SPARK-7155 // dir1 and dir2 are used for wholeTextFiles and binaryFiles - val dir1 = Utils.createTempDir() - val dir2 = Utils.createTempDir() - - val dirpath1 = dir1.getAbsolutePath - val dirpath2 = dir2.getAbsolutePath - - // file1 and file2 are placed inside dir1, they are also used for - // textFile, hadoopFile, and newAPIHadoopFile - // file3, file4 and file5 are placed inside dir2, they are used for - // textFile, hadoopFile, and newAPIHadoopFile as well - val file1 = new File(dir1, "part-00000") - val file2 = new File(dir1, "part-00001") - val file3 = new File(dir2, "part-00000") - val file4 = new File(dir2, "part-00001") - val file5 = new File(dir2, "part-00002") - - val filepath1 = file1.getAbsolutePath - val filepath2 = file2.getAbsolutePath - val filepath3 = file3.getAbsolutePath - val filepath4 = file4.getAbsolutePath - val filepath5 = file5.getAbsolutePath - - - try { - // Create 5 text files. - Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, - StandardCharsets.UTF_8) - Files.write("someline1 in file2\nsomeline2 in file2", file2, StandardCharsets.UTF_8) - Files.write("someline1 in file3", file3, StandardCharsets.UTF_8) - Files.write("someline1 in file4\nsomeline2 in file4", file4, StandardCharsets.UTF_8) - Files.write("someline1 in file2\nsomeline2 in file5", file5, StandardCharsets.UTF_8) - - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - - // Test textFile, hadoopFile, and newAPIHadoopFile for file1 and file2 - assert(sc.textFile(filepath1 + "," + filepath2).count() == 5L) - assert(sc.hadoopFile(filepath1 + "," + filepath2, - classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) - assert(sc.newAPIHadoopFile(filepath1 + "," + filepath2, - classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) - - // Test textFile, hadoopFile, and newAPIHadoopFile for file3, file4, and file5 - assert(sc.textFile(filepath3 + "," + filepath4 + "," + filepath5).count() == 5L) - assert(sc.hadoopFile(filepath3 + "," + filepath4 + "," + filepath5, - classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) - assert(sc.newAPIHadoopFile(filepath3 + "," + filepath4 + "," + filepath5, - classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) - - // Test wholeTextFiles, and binaryFiles for dir1 and dir2 - assert(sc.wholeTextFiles(dirpath1 + "," + dirpath2).count() == 5L) - assert(sc.binaryFiles(dirpath1 + "," + dirpath2).count() == 5L) - - } finally { - sc.stop() + withTempDir { dir1 => + withTempDir { dir2 => + val dirpath1 = dir1.getAbsolutePath + val dirpath2 = dir2.getAbsolutePath + + // file1 and file2 are placed inside dir1, they are also used for + // textFile, hadoopFile, and newAPIHadoopFile + // file3, file4 and file5 are placed inside dir2, they are used for + // textFile, hadoopFile, and newAPIHadoopFile as well + val file1 = new File(dir1, "part-00000") + val file2 = new File(dir1, "part-00001") + val file3 = new File(dir2, "part-00000") + val file4 = new File(dir2, "part-00001") + val file5 = new File(dir2, "part-00002") + + val filepath1 = file1.getAbsolutePath + val filepath2 = file2.getAbsolutePath + val filepath3 = file3.getAbsolutePath + val filepath4 = file4.getAbsolutePath + val filepath5 = file5.getAbsolutePath + + + try { + // Create 5 text files. + Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, + StandardCharsets.UTF_8) + Files.write("someline1 in file2\nsomeline2 in file2", file2, StandardCharsets.UTF_8) + Files.write("someline1 in file3", file3, StandardCharsets.UTF_8) + Files.write("someline1 in file4\nsomeline2 in file4", file4, StandardCharsets.UTF_8) + Files.write("someline1 in file2\nsomeline2 in file5", file5, StandardCharsets.UTF_8) + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + + // Test textFile, hadoopFile, and newAPIHadoopFile for file1 and file2 + assert(sc.textFile(filepath1 + "," + filepath2).count() == 5L) + assert(sc.hadoopFile(filepath1 + "," + filepath2, + classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + assert(sc.newAPIHadoopFile(filepath1 + "," + filepath2, + classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + + // Test textFile, hadoopFile, and newAPIHadoopFile for file3, file4, and file5 + assert(sc.textFile(filepath3 + "," + filepath4 + "," + filepath5).count() == 5L) + assert(sc.hadoopFile(filepath3 + "," + filepath4 + "," + filepath5, + classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + assert(sc.newAPIHadoopFile(filepath3 + "," + filepath4 + "," + filepath5, + classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + + // Test wholeTextFiles, and binaryFiles for dir1 and dir2 + assert(sc.wholeTextFiles(dirpath1 + "," + dirpath2).count() == 5L) + assert(sc.binaryFiles(dirpath1 + "," + dirpath2).count() == 5L) + + } finally { + sc.stop() + } + } } } diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 31289026b002..dad24d7c01b8 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -23,7 +23,7 @@ import java.io.File import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} import org.apache.spark.internal.Logging -import org.apache.spark.util.AccumulatorContext +import org.apache.spark.util.{AccumulatorContext, Utils} /** * Base abstract class for all unit tests in Spark for handling common functionality. @@ -106,4 +106,14 @@ abstract class SparkFunSuite } } + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + */ + protected def withTempDir(f: File => Unit): Unit = { + val dir = Utils.createTempDir() + try f(dir) finally { + Utils.deleteRecursively(dir) + } + } } 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 b38a3667abee..7407a656dbfc 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 @@ -31,7 +31,6 @@ import org.apache.spark.util.Utils // a PythonBroadcast: class PythonBroadcastSuite extends SparkFunSuite with Matchers with SharedSparkContext { test("PythonBroadcast can be serialized with Kryo (SPARK-4882)") { - val tempDir = Utils.createTempDir() val broadcastedString = "Hello, world!" def assertBroadcastIsValid(broadcast: PythonBroadcast): Unit = { val source = Source.fromFile(broadcast.path) @@ -39,7 +38,7 @@ class PythonBroadcastSuite extends SparkFunSuite with Matchers with SharedSparkC source.close() contents should be (broadcastedString) } - try { + withTempDir { tempDir => val broadcastDataFile: File = { val file = new File(tempDir, "broadcastData") val printWriter = new PrintWriter(file) @@ -53,8 +52,6 @@ class PythonBroadcastSuite extends SparkFunSuite with Matchers with SharedSparkC val deserializedBroadcast = Utils.clone[PythonBroadcast](broadcast, new KryoSerializer(conf).newInstance()) assertBroadcastIsValid(deserializedBroadcast) - } finally { - Utils.deleteRecursively(tempDir) } } } 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 c093789244bf..a8973d1b60f8 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -494,13 +494,11 @@ class SparkSubmitSuite } test("launch simple application with spark-submit with redaction") { - val testDir = Utils.createTempDir() - testDir.deleteOnExit() - val testDirPath = new Path(testDir.getAbsolutePath()) val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) - try { + withTempDir { testDir => + val testDirPath = new Path(testDir.getAbsolutePath()) val args = Seq( "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", @@ -519,8 +517,6 @@ class SparkSubmitSuite Source.fromInputStream(logData).getLines().foreach { line => assert(!line.contains("secret_password")) } - } finally { - Utils.deleteRecursively(testDir) } } @@ -614,108 +610,112 @@ class SparkSubmitSuite assert(new File(rScriptDir).exists) // compile a small jar containing a class that will be called from R code. - val tempDir = Utils.createTempDir() - val srcDir = new File(tempDir, "sparkrtest") - srcDir.mkdirs() - val excSource = new JavaSourceFromString(new File(srcDir, "DummyClass").toURI.getPath, - """package sparkrtest; + withTempDir { tempDir => + val srcDir = new File(tempDir, "sparkrtest") + srcDir.mkdirs() + val excSource = new JavaSourceFromString(new File(srcDir, "DummyClass").toURI.getPath, + """package sparkrtest; | |public class DummyClass implements java.io.Serializable { | public static String helloWorld(String arg) { return "Hello " + arg; } | public static int addStuff(int arg1, int arg2) { return arg1 + arg2; } |} - """.stripMargin) - val excFile = TestUtils.createCompiledClass("DummyClass", srcDir, excSource, Seq.empty) - val jarFile = new File(tempDir, "sparkRTestJar-%s.jar".format(System.currentTimeMillis())) - val jarURL = TestUtils.createJar(Seq(excFile), jarFile, directoryPrefix = Some("sparkrtest")) + """. + stripMargin) + val excFile = TestUtils.createCompiledClass("DummyClass", srcDir, excSource, Seq.empty) + val jarFile = new File(tempDir, "sparkRTestJar-%s.jar".format(System.currentTimeMillis())) + val jarURL = TestUtils.createJar(Seq(excFile), jarFile, directoryPrefix = Some("sparkrtest")) - val args = Seq( - "--name", "testApp", - "--master", "local", - "--jars", jarURL.toString, - "--verbose", - "--conf", "spark.ui.enabled=false", - rScriptDir) - runSparkSubmit(args) + val args = Seq( + "--name", "testApp", + "--master", "local", + "--jars", jarURL.toString, + "--verbose", + "--conf", "spark.ui.enabled=false", + rScriptDir) + runSparkSubmit(args) + } } test("resolves command line argument paths correctly") { - val dir = Utils.createTempDir() - val archive = Paths.get(dir.toPath.toString, "single.zip") - Files.createFile(archive) - val jars = "/jar1,/jar2" - val files = "local:/file1,file2" - val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" - val pyFiles = "py-file1,py-file2" - - // Test jars and files - val clArgs = Seq( - "--master", "local", - "--class", "org.SomeClass", - "--jars", jars, - "--files", files, - "thejar.jar") - val appArgs = new SparkSubmitArguments(clArgs) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - appArgs.jars should be (Utils.resolveURIs(jars)) - appArgs.files should be (Utils.resolveURIs(files)) - conf.get("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar")) - conf.get("spark.files") should be (Utils.resolveURIs(files)) - - // Test files and archives (Yarn) - val clArgs2 = Seq( - "--master", "yarn", - "--class", "org.SomeClass", - "--files", files, - "--archives", archives, - "thejar.jar" - ) - val appArgs2 = new SparkSubmitArguments(clArgs2) - val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) - appArgs2.files should be (Utils.resolveURIs(files)) - appArgs2.archives should fullyMatch regex ("file:/archive1,file:.*#archive3") - conf2.get("spark.yarn.dist.files") should be (Utils.resolveURIs(files)) - conf2.get("spark.yarn.dist.archives") should fullyMatch regex - ("file:/archive1,file:.*#archive3") - - // Test python files - val clArgs3 = Seq( - "--master", "local", - "--py-files", pyFiles, - "--conf", "spark.pyspark.driver.python=python3.4", - "--conf", "spark.pyspark.python=python3.5", - "mister.py" - ) - val appArgs3 = new SparkSubmitArguments(clArgs3) - val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) - appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles)) - conf3.get("spark.submit.pyFiles") should be ( - PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) - conf3.get(PYSPARK_DRIVER_PYTHON.key) should be ("python3.4") - conf3.get(PYSPARK_PYTHON.key) should be ("python3.5") + withTempDir { dir => + val archive = Paths.get(dir.toPath.toString, "single.zip") + Files.createFile(archive) + val jars = "/jar1,/jar2" + val files = "local:/file1,file2" + val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" + val pyFiles = "py-file1,py-file2" + + // Test jars and files + val clArgs = Seq( + "--master", "local", + "--class", "org.SomeClass", + "--jars", jars, + "--files", files, + "thejar.jar") + val appArgs = new SparkSubmitArguments(clArgs) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) + appArgs.jars should be(Utils.resolveURIs(jars)) + appArgs.files should be(Utils.resolveURIs(files)) + conf.get("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) + conf.get("spark.files") should be(Utils.resolveURIs(files)) + + // Test files and archives (Yarn) + val clArgs2 = Seq( + "--master", "yarn", + "--class", "org.SomeClass", + "--files", files, + "--archives", archives, + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) + appArgs2.files should be(Utils.resolveURIs(files)) + appArgs2.archives should fullyMatch regex ("file:/archive1,file:.*#archive3") + conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) + conf2.get("spark.yarn.dist.archives") should fullyMatch regex + ("file:/archive1,file:.*#archive3") + + // Test python files + val clArgs3 = Seq( + "--master", "local", + "--py-files", pyFiles, + "--conf", "spark.pyspark.driver.python=python3.4", + "--conf", "spark.pyspark.python=python3.5", + "mister.py" + ) + val appArgs3 = new SparkSubmitArguments(clArgs3) + val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) + appArgs3.pyFiles should be(Utils.resolveURIs(pyFiles)) + conf3.get("spark.submit.pyFiles") should be( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + conf3.get(PYSPARK_DRIVER_PYTHON.key) should be("python3.4") + conf3.get(PYSPARK_PYTHON.key) should be("python3.5") + } } test("ambiguous archive mapping results in error message") { - val dir = Utils.createTempDir() - val archive1 = Paths.get(dir.toPath.toString, "first.zip") - val archive2 = Paths.get(dir.toPath.toString, "second.zip") - Files.createFile(archive1) - Files.createFile(archive2) - val jars = "/jar1,/jar2" - val files = "local:/file1,file2" - val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" - val pyFiles = "py-file1,py-file2" - - // Test files and archives (Yarn) - val clArgs2 = Seq( - "--master", "yarn", - "--class", "org.SomeClass", - "--files", files, - "--archives", archives, - "thejar.jar" - ) + withTempDir { dir => + val archive1 = Paths.get(dir.toPath.toString, "first.zip") + val archive2 = Paths.get(dir.toPath.toString, "second.zip") + Files.createFile(archive1) + Files.createFile(archive2) + val jars = "/jar1,/jar2" + val files = "local:/file1,file2" + val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" + val pyFiles = "py-file1,py-file2" + + // Test files and archives (Yarn) + val clArgs2 = Seq( + "--master", "yarn", + "--class", "org.SomeClass", + "--files", files, + "--archives", archives, + "thejar.jar" + ) - testPrematureExit(clArgs2.toArray, "resolves ambiguously to multiple files") + testPrematureExit(clArgs2.toArray, "resolves ambiguously to multiple files") + } } test("resolves config paths correctly") { @@ -724,77 +724,77 @@ class SparkSubmitSuite val archives = "file:/archive1,archive2" // spark.yarn.dist.archives val pyFiles = "py-file1,py-file2" // spark.submit.pyFiles - val tmpDir = Utils.createTempDir() - - // Test jars and files - val f1 = File.createTempFile("test-submit-jars-files", "", tmpDir) - val writer1 = new PrintWriter(f1) - writer1.println("spark.jars " + jars) - writer1.println("spark.files " + files) - writer1.close() - val clArgs = Seq( - "--master", "local", - "--class", "org.SomeClass", - "--properties-file", f1.getPath, - "thejar.jar" - ) - val appArgs = new SparkSubmitArguments(clArgs) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - conf.get("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) - conf.get("spark.files") should be(Utils.resolveURIs(files)) - - // Test files and archives (Yarn) - val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir) - val writer2 = new PrintWriter(f2) - writer2.println("spark.yarn.dist.files " + files) - writer2.println("spark.yarn.dist.archives " + archives) - writer2.close() - val clArgs2 = Seq( - "--master", "yarn", - "--class", "org.SomeClass", - "--properties-file", f2.getPath, - "thejar.jar" - ) - val appArgs2 = new SparkSubmitArguments(clArgs2) - val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) - conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) - conf2.get("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) - - // Test python files - val f3 = File.createTempFile("test-submit-python-files", "", tmpDir) - val writer3 = new PrintWriter(f3) - writer3.println("spark.submit.pyFiles " + pyFiles) - writer3.close() - val clArgs3 = Seq( - "--master", "local", - "--properties-file", f3.getPath, - "mister.py" - ) - val appArgs3 = new SparkSubmitArguments(clArgs3) - val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) - conf3.get("spark.submit.pyFiles") should be( - PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) - - // Test remote python files - val hadoopConf = new Configuration() - updateConfWithFakeS3Fs(hadoopConf) - val f4 = File.createTempFile("test-submit-remote-python-files", "", tmpDir) - val pyFile1 = File.createTempFile("file1", ".py", tmpDir) - val pyFile2 = File.createTempFile("file2", ".py", tmpDir) - val writer4 = new PrintWriter(f4) - val remotePyFiles = s"s3a://${pyFile1.getAbsolutePath},s3a://${pyFile2.getAbsolutePath}" - writer4.println("spark.submit.pyFiles " + remotePyFiles) - writer4.close() - val clArgs4 = Seq( - "--master", "yarn", - "--deploy-mode", "cluster", - "--properties-file", f4.getPath, - "hdfs:///tmp/mister.py" - ) - val appArgs4 = new SparkSubmitArguments(clArgs4) - val (_, _, conf4, _) = submit.prepareSubmitEnvironment(appArgs4, conf = Some(hadoopConf)) - // Should not format python path for yarn cluster mode - conf4.get("spark.submit.pyFiles") should be(Utils.resolveURIs(remotePyFiles)) + withTempDir { tmpDir => + // Test jars and files + val f1 = File.createTempFile("test-submit-jars-files", "", tmpDir) + val writer1 = new PrintWriter(f1) + writer1.println("spark.jars " + jars) + writer1.println("spark.files " + files) + writer1.close() + val clArgs = Seq( + "--master", "local", + "--class", "org.SomeClass", + "--properties-file", f1.getPath, + "thejar.jar" + ) + val appArgs = new SparkSubmitArguments(clArgs) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) + conf.get("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) + conf.get("spark.files") should be(Utils.resolveURIs(files)) + + // Test files and archives (Yarn) + val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir) + val writer2 = new PrintWriter(f2) + writer2.println("spark.yarn.dist.files " + files) + writer2.println("spark.yarn.dist.archives " + archives) + writer2.close() + val clArgs2 = Seq( + "--master", "yarn", + "--class", "org.SomeClass", + "--properties-file", f2.getPath, + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) + conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) + conf2.get("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) + + // Test python files + val f3 = File.createTempFile("test-submit-python-files", "", tmpDir) + val writer3 = new PrintWriter(f3) + writer3.println("spark.submit.pyFiles " + pyFiles) + writer3.close() + val clArgs3 = Seq( + "--master", "local", + "--properties-file", f3.getPath, + "mister.py" + ) + val appArgs3 = new SparkSubmitArguments(clArgs3) + val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) + conf3.get("spark.submit.pyFiles") should be( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + + // Test remote python files + val hadoopConf = new Configuration() + updateConfWithFakeS3Fs(hadoopConf) + val f4 = File.createTempFile("test-submit-remote-python-files", "", tmpDir) + val pyFile1 = File.createTempFile("file1", ".py", tmpDir) + val pyFile2 = File.createTempFile("file2", ".py", tmpDir) + val writer4 = new PrintWriter(f4) + val remotePyFiles = s"s3a://${pyFile1.getAbsolutePath},s3a://${pyFile2.getAbsolutePath}" + writer4.println("spark.submit.pyFiles " + remotePyFiles) + writer4.close() + val clArgs4 = Seq( + "--master", "yarn", + "--deploy-mode", "cluster", + "--properties-file", f4.getPath, + "hdfs:///tmp/mister.py" + ) + val appArgs4 = new SparkSubmitArguments(clArgs4) + val (_, _, conf4, _) = submit.prepareSubmitEnvironment(appArgs4, conf = Some(hadoopConf)) + // Should not format python path for yarn cluster mode + conf4.get("spark.submit.pyFiles") should be(Utils.resolveURIs(remotePyFiles)) + } } test("user classpath first in driver") { @@ -828,46 +828,50 @@ class SparkSubmitSuite } test("support glob path") { - val tmpJarDir = Utils.createTempDir() - val jar1 = TestUtils.createJarWithFiles(Map("test.resource" -> "1"), tmpJarDir) - val jar2 = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpJarDir) - - val tmpFileDir = Utils.createTempDir() - val file1 = File.createTempFile("tmpFile1", "", tmpFileDir) - val file2 = File.createTempFile("tmpFile2", "", tmpFileDir) - - val tmpPyFileDir = Utils.createTempDir() - val pyFile1 = File.createTempFile("tmpPy1", ".py", tmpPyFileDir) - val pyFile2 = File.createTempFile("tmpPy2", ".egg", tmpPyFileDir) - - val tmpArchiveDir = Utils.createTempDir() - val archive1 = File.createTempFile("archive1", ".zip", tmpArchiveDir) - val archive2 = File.createTempFile("archive2", ".zip", tmpArchiveDir) - - val tempPyFile = File.createTempFile("tmpApp", ".py") - tempPyFile.deleteOnExit() - - val args = Seq( - "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), - "--name", "testApp", - "--master", "yarn", - "--deploy-mode", "client", - "--jars", s"${tmpJarDir.getAbsolutePath}/*.jar", - "--files", s"${tmpFileDir.getAbsolutePath}/tmpFile*", - "--py-files", s"${tmpPyFileDir.getAbsolutePath}/tmpPy*", - "--archives", s"${tmpArchiveDir.getAbsolutePath}/*.zip", - tempPyFile.toURI().toString()) - - val appArgs = new SparkSubmitArguments(args) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - conf.get("spark.yarn.dist.jars").split(",").toSet should be - (Set(jar1.toURI.toString, jar2.toURI.toString)) - conf.get("spark.yarn.dist.files").split(",").toSet should be - (Set(file1.toURI.toString, file2.toURI.toString)) - conf.get("spark.yarn.dist.pyFiles").split(",").toSet should be - (Set(pyFile1.getAbsolutePath, pyFile2.getAbsolutePath)) - conf.get("spark.yarn.dist.archives").split(",").toSet should be - (Set(archive1.toURI.toString, archive2.toURI.toString)) + withTempDir { tmpJarDir => + withTempDir { tmpFileDir => + withTempDir { tmpPyFileDir => + withTempDir { tmpArchiveDir => + val jar1 = TestUtils.createJarWithFiles(Map("test.resource" -> "1"), tmpJarDir) + val jar2 = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpJarDir) + + val file1 = File.createTempFile("tmpFile1", "", tmpFileDir) + val file2 = File.createTempFile("tmpFile2", "", tmpFileDir) + + val pyFile1 = File.createTempFile("tmpPy1", ".py", tmpPyFileDir) + val pyFile2 = File.createTempFile("tmpPy2", ".egg", tmpPyFileDir) + + val archive1 = File.createTempFile("archive1", ".zip", tmpArchiveDir) + val archive2 = File.createTempFile("archive2", ".zip", tmpArchiveDir) + + val tempPyFile = File.createTempFile("tmpApp", ".py") + tempPyFile.deleteOnExit() + + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--jars", s"${tmpJarDir.getAbsolutePath}/*.jar", + "--files", s"${tmpFileDir.getAbsolutePath}/tmpFile*", + "--py-files", s"${tmpPyFileDir.getAbsolutePath}/tmpPy*", + "--archives", s"${tmpArchiveDir.getAbsolutePath}/*.zip", + tempPyFile.toURI().toString()) + + val appArgs = new SparkSubmitArguments(args) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) + conf.get("spark.yarn.dist.jars").split(",").toSet should be + (Set(jar1.toURI.toString, jar2.toURI.toString)) + conf.get("spark.yarn.dist.files").split(",").toSet should be + (Set(file1.toURI.toString, file2.toURI.toString)) + conf.get("spark.yarn.dist.pyFiles").split(",").toSet should be + (Set(pyFile1.getAbsolutePath, pyFile2.getAbsolutePath)) + conf.get("spark.yarn.dist.archives").split(",").toSet should be + (Set(archive1.toURI.toString, archive2.toURI.toString)) + } + } + } + } } // scalastyle:on println @@ -985,37 +989,38 @@ class SparkSubmitSuite val hadoopConf = new Configuration() updateConfWithFakeS3Fs(hadoopConf) - val tmpDir = Utils.createTempDir() - val file = File.createTempFile("tmpFile", "", tmpDir) - val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) - val mainResource = File.createTempFile("tmpPy", ".py", tmpDir) - val tmpJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir) - val tmpJarPath = s"s3a://${new File(tmpJar.toURI).getAbsolutePath}" + withTempDir { tmpDir => + val file = File.createTempFile("tmpFile", "", tmpDir) + val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) + val mainResource = File.createTempFile("tmpPy", ".py", tmpDir) + val tmpJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir) + val tmpJarPath = s"s3a://${new File(tmpJar.toURI).getAbsolutePath}" - val args = Seq( - "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), - "--name", "testApp", - "--master", "yarn", - "--deploy-mode", "client", - "--jars", tmpJarPath, - "--files", s"s3a://${file.getAbsolutePath}", - "--py-files", s"s3a://${pyFile.getAbsolutePath}", - s"s3a://$mainResource" + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--jars", tmpJarPath, + "--files", s"s3a://${file.getAbsolutePath}", + "--py-files", s"s3a://${pyFile.getAbsolutePath}", + s"s3a://$mainResource" ) - val appArgs = new SparkSubmitArguments(args) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) + val appArgs = new SparkSubmitArguments(args) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) - // All the resources should still be remote paths, so that YARN client will not upload again. - conf.get("spark.yarn.dist.jars") should be (tmpJarPath) - conf.get("spark.yarn.dist.files") should be (s"s3a://${file.getAbsolutePath}") - conf.get("spark.yarn.dist.pyFiles") should be (s"s3a://${pyFile.getAbsolutePath}") + // All the resources should still be remote paths, so that YARN client will not upload again. + conf.get("spark.yarn.dist.jars") should be(tmpJarPath) + conf.get("spark.yarn.dist.files") should be(s"s3a://${file.getAbsolutePath}") + conf.get("spark.yarn.dist.pyFiles") should be(s"s3a://${pyFile.getAbsolutePath}") - // Local repl jars should be a local path. - conf.get("spark.repl.local.jars") should (startWith("file:")) + // Local repl jars should be a local path. + conf.get("spark.repl.local.jars") should (startWith("file:")) - // local py files should not be a URI format. - conf.get("spark.submit.pyFiles") should (startWith("/")) + // local py files should not be a URI format. + conf.get("spark.submit.pyFiles") should (startWith("/")) + } } test("download remote resource if it is not supported by yarn service") { @@ -1095,18 +1100,13 @@ class SparkSubmitSuite } private def forConfDir(defaults: Map[String, String]) (f: String => Unit) = { - val tmpDir = Utils.createTempDir() - - val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") - val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf), StandardCharsets.UTF_8) - for ((key, value) <- defaults) writer.write(s"$key $value\n") - - writer.close() - - try { + withTempDir { tmpDir => + val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") + val writer = + new OutputStreamWriter(new FileOutputStream(defaultsConf), StandardCharsets.UTF_8) + for ((key, value) <- defaults) writer.write(s"$key $value\n") + writer.close() f(tmpDir.getAbsolutePath) - } finally { - Utils.deleteRecursively(tmpDir) } } @@ -1134,39 +1134,40 @@ class SparkSubmitSuite val hadoopConf = new Configuration() updateConfWithFakeS3Fs(hadoopConf) - val tmpDir = Utils.createTempDir() - val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) + withTempDir { tmpDir => + val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) - val args = Seq( - "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), - "--name", "testApp", - "--master", "yarn", - "--deploy-mode", "client", - "--py-files", s"s3a://${pyFile.getAbsolutePath}", - "spark-internal" - ) + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--py-files", s"s3a://${pyFile.getAbsolutePath}", + "spark-internal" + ) - val appArgs = new SparkSubmitArguments(args) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) + val appArgs = new SparkSubmitArguments(args) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) - conf.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}") - conf.get("spark.submit.pyFiles") should (startWith("/")) + conf.get(PY_FILES.key) should be(s"s3a://${pyFile.getAbsolutePath}") + conf.get("spark.submit.pyFiles") should (startWith("/")) - // Verify "spark.submit.pyFiles" - val args1 = Seq( - "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), - "--name", "testApp", - "--master", "yarn", - "--deploy-mode", "client", - "--conf", s"spark.submit.pyFiles=s3a://${pyFile.getAbsolutePath}", - "spark-internal" - ) + // Verify "spark.submit.pyFiles" + val args1 = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--conf", s"spark.submit.pyFiles=s3a://${pyFile.getAbsolutePath}", + "spark-internal" + ) - val appArgs1 = new SparkSubmitArguments(args1) - val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1, conf = Some(hadoopConf)) + val appArgs1 = new SparkSubmitArguments(args1) + val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1, conf = Some(hadoopConf)) - conf1.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}") - conf1.get("spark.submit.pyFiles") should (startWith("/")) + conf1.get(PY_FILES.key) should be(s"s3a://${pyFile.getAbsolutePath}") + conf1.get("spark.submit.pyFiles") should (startWith("/")) + } } test("handles natural line delimiters in --properties-file and --conf uniformly") { 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 527c654a7cd6..c1ae27aa940f 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 @@ -767,53 +767,54 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("clean up stale app information") { - val storeDir = Utils.createTempDir() - val conf = createTestConf().set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) - val clock = new ManualClock() - val provider = spy(new FsHistoryProvider(conf, clock)) - val appId = "new1" - - // Write logs for two app attempts. - clock.advance(1) - val attempt1 = newLogFile(appId, Some("1"), inProgress = false) - writeFile(attempt1, true, None, - SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("1")), - SparkListenerJobStart(0, 1L, Nil, null), - SparkListenerApplicationEnd(5L) + withTempDir { storeDir => + val conf = createTestConf().set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) + val clock = new ManualClock() + val provider = spy(new FsHistoryProvider(conf, clock)) + val appId = "new1" + + // Write logs for two app attempts. + clock.advance(1) + val attempt1 = newLogFile(appId, Some("1"), inProgress = false) + writeFile(attempt1, true, None, + SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("1")), + SparkListenerJobStart(0, 1L, Nil, null), + SparkListenerApplicationEnd(5L) ) - val attempt2 = newLogFile(appId, Some("2"), inProgress = false) - writeFile(attempt2, true, None, - SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("2")), - SparkListenerJobStart(0, 1L, Nil, null), - SparkListenerApplicationEnd(5L) + val attempt2 = newLogFile(appId, Some("2"), inProgress = false) + writeFile(attempt2, true, None, + SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("2")), + SparkListenerJobStart(0, 1L, Nil, null), + SparkListenerApplicationEnd(5L) ) - updateAndCheck(provider) { list => - assert(list.size === 1) - assert(list(0).id === appId) - assert(list(0).attempts.size === 2) - } - - // Load the app's UI. - val ui = provider.getAppUI(appId, Some("1")) - assert(ui.isDefined) - - // Delete the underlying log file for attempt 1 and rescan. The UI should go away, but since - // attempt 2 still exists, listing data should be there. - clock.advance(1) - attempt1.delete() - updateAndCheck(provider) { list => - assert(list.size === 1) - assert(list(0).id === appId) - assert(list(0).attempts.size === 1) - } - assert(!ui.get.valid) - assert(provider.getAppUI(appId, None) === None) + updateAndCheck(provider) { list => + assert(list.size === 1) + assert(list(0).id === appId) + assert(list(0).attempts.size === 2) + } - // Delete the second attempt's log file. Now everything should go away. - clock.advance(1) - attempt2.delete() - updateAndCheck(provider) { list => - assert(list.isEmpty) + // Load the app's UI. + val ui = provider.getAppUI(appId, Some("1")) + assert(ui.isDefined) + + // Delete the underlying log file for attempt 1 and rescan. The UI should go away, but since + // attempt 2 still exists, listing data should be there. + clock.advance(1) + attempt1.delete() + updateAndCheck(provider) { list => + assert(list.size === 1) + assert(list(0).id === appId) + assert(list(0).attempts.size === 1) + } + assert(!ui.get.valid) + assert(provider.getAppUI(appId, None) === None) + + // Delete the second attempt's log file. Now everything should go away. + clock.advance(1) + attempt2.delete() + updateAndCheck(provider) { list => + assert(list.isEmpty) + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala index 37954826af90..e89733a144cf 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala @@ -41,18 +41,14 @@ class HistoryServerArgumentsSuite extends SparkFunSuite { } test("Properties File Arguments Parsing --properties-file") { - val tmpDir = Utils.createTempDir() - val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) - try { + withTempDir { tmpDir => + val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) Files.write("spark.test.CustomPropertyA blah\n" + "spark.test.CustomPropertyB notblah\n", outFile, UTF_8) val argStrings = Array("--properties-file", outFile.getAbsolutePath) val hsa = new HistoryServerArguments(conf, argStrings) assert(conf.get("spark.test.CustomPropertyA") === "blah") assert(conf.get("spark.test.CustomPropertyB") === "notblah") - } finally { - Utils.deleteRecursively(tmpDir) } } - } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala index 62fe0eaedfd2..30278655dbe0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala @@ -31,14 +31,11 @@ import org.apache.spark.util.Utils class PersistenceEngineSuite extends SparkFunSuite { test("FileSystemPersistenceEngine") { - val dir = Utils.createTempDir() - try { + withTempDir { dir => val conf = new SparkConf() testPersistenceEngine(conf, serializer => new FileSystemPersistenceEngine(dir.getAbsolutePath, serializer) ) - } finally { - Utils.deleteRecursively(dir) } } diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala index 817dc082b7d3..576ca1613f75 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala @@ -59,9 +59,7 @@ class WholeTextFileInputFormatSuite extends SparkFunSuite with BeforeAndAfterAll test("for small files minimum split size per node and per rack should be less than or equal to " + "maximum split size.") { - var dir : File = null; - try { - dir = Utils.createTempDir() + withTempDir { dir => logInfo(s"Local disk address is ${dir.toString}.") // Set the minsize per node and rack to be larger than the size of the input file. @@ -75,8 +73,6 @@ class WholeTextFileInputFormatSuite extends SparkFunSuite with BeforeAndAfterAll } // ensure spark job runs successfully without exceptions from the CombineFileInputFormat assert(sc.wholeTextFiles(dir.toString).count == 3) - } finally { - Utils.deleteRecursively(dir) } } } diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index ddf73d637063..47552916adb2 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -89,52 +89,50 @@ class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAl * 3) Does the contents be the same. */ test("Correctness of WholeTextFileRecordReader.") { - val dir = Utils.createTempDir() - logInfo(s"Local disk address is ${dir.toString}.") + withTempDir { dir => + logInfo(s"Local disk address is ${dir.toString}.") - WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => - createNativeFile(dir, filename, contents, false) - } + WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => + createNativeFile(dir, filename, contents, false) + } - val res = sc.wholeTextFiles(dir.toString, 3).collect() + val res = sc.wholeTextFiles(dir.toString, 3).collect() - assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, - "Number of files read out does not fit with the actual value.") + assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, + "Number of files read out does not fit with the actual value.") - for ((filename, contents) <- res) { - val shortName = filename.split('/').last - assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), - s"Missing file name $filename.") - assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, - s"file $filename contents can not match.") + for ((filename, contents) <- res) { + val shortName = filename.split('/').last + assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), + s"Missing file name $filename.") + assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, + s"file $filename contents can not match.") + } } - - Utils.deleteRecursively(dir) } test("Correctness of WholeTextFileRecordReader with GzipCodec.") { - val dir = Utils.createTempDir() - logInfo(s"Local disk address is ${dir.toString}.") + withTempDir { dir => + logInfo(s"Local disk address is ${dir.toString}.") - WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => - createNativeFile(dir, filename, contents, true) - } + WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => + createNativeFile(dir, filename, contents, true) + } - val res = sc.wholeTextFiles(dir.toString, 3).collect() + val res = sc.wholeTextFiles(dir.toString, 3).collect() - assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, - "Number of files read out does not fit with the actual value.") + assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, + "Number of files read out does not fit with the actual value.") - for ((filename, contents) <- res) { - val shortName = filename.split('/').last.split('.')(0) + for ((filename, contents) <- res) { + val shortName = filename.split('/').last.split('.')(0) - assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), - s"Missing file name $filename.") - assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, - s"file $filename contents can not match.") + assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), + s"Missing file name $filename.") + assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, + s"file $filename contents can not match.") + } } - - Utils.deleteRecursively(dir) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 0ec359d1c94f..945b09441ea9 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -470,15 +470,12 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { } test("zero-partition RDD") { - val emptyDir = Utils.createTempDir() - try { + withTempDir { emptyDir => val file = sc.textFile(emptyDir.getAbsolutePath) assert(file.partitions.isEmpty) assert(file.collect().toList === Nil) // Test that a shuffle on the file works, because this used to be a bug assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) - } finally { - Utils.deleteRecursively(emptyDir) } } diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index a799b1cfb076..5cb2b561d6bc 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -822,63 +822,66 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } test("file server") { - val conf = new SparkConf() - val tempDir = Utils.createTempDir() - val file = new File(tempDir, "file") - Files.write(UUID.randomUUID().toString(), file, UTF_8) - val fileWithSpecialChars = new File(tempDir, "file name") - Files.write(UUID.randomUUID().toString(), fileWithSpecialChars, UTF_8) - val empty = new File(tempDir, "empty") - Files.write("", empty, UTF_8); - val jar = new File(tempDir, "jar") - Files.write(UUID.randomUUID().toString(), jar, UTF_8) - - val dir1 = new File(tempDir, "dir1") - assert(dir1.mkdir()) - val subFile1 = new File(dir1, "file1") - Files.write(UUID.randomUUID().toString(), subFile1, UTF_8) - - val dir2 = new File(tempDir, "dir2") - assert(dir2.mkdir()) - val subFile2 = new File(dir2, "file2") - Files.write(UUID.randomUUID().toString(), subFile2, UTF_8) - - val fileUri = env.fileServer.addFile(file) - val fileWithSpecialCharsUri = env.fileServer.addFile(fileWithSpecialChars) - val emptyUri = env.fileServer.addFile(empty) - val jarUri = env.fileServer.addJar(jar) - val dir1Uri = env.fileServer.addDirectory("/dir1", dir1) - val dir2Uri = env.fileServer.addDirectory("/dir2", dir2) - - // Try registering directories with invalid names. - Seq("/files", "/jars").foreach { uri => - intercept[IllegalArgumentException] { - env.fileServer.addDirectory(uri, dir1) - } - } + withTempDir { tempDir => + withTempDir { destDir => + val conf = new SparkConf() + + val file = new File(tempDir, "file") + Files.write(UUID.randomUUID().toString(), file, UTF_8) + val fileWithSpecialChars = new File(tempDir, "file name") + Files.write(UUID.randomUUID().toString(), fileWithSpecialChars, UTF_8) + val empty = new File(tempDir, "empty") + Files.write("", empty, UTF_8); + val jar = new File(tempDir, "jar") + Files.write(UUID.randomUUID().toString(), jar, UTF_8) + + val dir1 = new File(tempDir, "dir1") + assert(dir1.mkdir()) + val subFile1 = new File(dir1, "file1") + Files.write(UUID.randomUUID().toString(), subFile1, UTF_8) + + val dir2 = new File(tempDir, "dir2") + assert(dir2.mkdir()) + val subFile2 = new File(dir2, "file2") + Files.write(UUID.randomUUID().toString(), subFile2, UTF_8) + + val fileUri = env.fileServer.addFile(file) + val fileWithSpecialCharsUri = env.fileServer.addFile(fileWithSpecialChars) + val emptyUri = env.fileServer.addFile(empty) + val jarUri = env.fileServer.addJar(jar) + val dir1Uri = env.fileServer.addDirectory("/dir1", dir1) + val dir2Uri = env.fileServer.addDirectory("/dir2", dir2) + + // Try registering directories with invalid names. + Seq("/files", "/jars").foreach { uri => + intercept[IllegalArgumentException] { + env.fileServer.addDirectory(uri, dir1) + } + } - val destDir = Utils.createTempDir() - val sm = new SecurityManager(conf) - val hc = SparkHadoopUtil.get.conf - - val files = Seq( - (file, fileUri), - (fileWithSpecialChars, fileWithSpecialCharsUri), - (empty, emptyUri), - (jar, jarUri), - (subFile1, dir1Uri + "/file1"), - (subFile2, dir2Uri + "/file2")) - files.foreach { case (f, uri) => - val destFile = new File(destDir, f.getName()) - Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) - assert(Files.equal(f, destFile)) - } + val sm = new SecurityManager(conf) + val hc = SparkHadoopUtil.get.conf + + val files = Seq( + (file, fileUri), + (fileWithSpecialChars, fileWithSpecialCharsUri), + (empty, emptyUri), + (jar, jarUri), + (subFile1, dir1Uri + "/file1"), + (subFile2, dir2Uri + "/file2")) + files.foreach { case (f, uri) => + val destFile = new File(destDir, f.getName()) + Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) + assert(Files.equal(f, destFile)) + } - // Try to download files that do not exist. - Seq("files", "jars", "dir1").foreach { root => - intercept[Exception] { - val uri = env.address.toSparkURL + s"/$root/doesNotExist" - Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) + // Try to download files that do not exist. + Seq("files", "jars", "dir1").foreach { root => + intercept[Exception] { + val uri = env.address.toSparkURL + s"/$root/doesNotExist" + Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) + } + } } } } 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 5f4ffa151d19..ed6a3d93b312 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2831,18 +2831,22 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("SPARK-23207: reliable checkpoint can avoid rollback (checkpointed before)") { - sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) - val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) - shuffleMapRdd.checkpoint() - shuffleMapRdd.doCheckpoint() - assertResultStageNotRollbacked(shuffleMapRdd) + withTempDir { dir => + sc.setCheckpointDir(dir.getCanonicalPath) + val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) + shuffleMapRdd.checkpoint() + shuffleMapRdd.doCheckpoint() + assertResultStageNotRollbacked(shuffleMapRdd) + } } test("SPARK-23207: reliable checkpoint fail to rollback (checkpointing now)") { - sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) - val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) - shuffleMapRdd.checkpoint() - assertResultStageFailToRollback(shuffleMapRdd) + withTempDir { dir => + sc.setCheckpointDir(dir.getCanonicalPath) + val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) + shuffleMapRdd.checkpoint() + assertResultStageFailToRollback(shuffleMapRdd) + } } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala index d6ff5bb33055..848f70293553 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala @@ -49,11 +49,8 @@ class OutputCommitCoordinatorIntegrationSuite test("exception thrown in OutputCommitter.commitTask()") { // Regression test for SPARK-10381 failAfter(Span(60, Seconds)) { - val tempDir = Utils.createTempDir() - try { + withTempDir { tempDir => sc.parallelize(1 to 4, 2).map(_.toString).saveAsTextFile(tempDir.getAbsolutePath + "/out") - } finally { - Utils.deleteRecursively(tempDir) } } } 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 a7eed4b6a8b8..467e49026a02 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -369,27 +369,27 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } test("SPARK-12222: deserialize RoaringBitmap throw Buffer underflow exception") { - val dir = Utils.createTempDir() - val tmpfile = dir.toString + "/RoaringBitmap" - val outStream = new FileOutputStream(tmpfile) - val output = new KryoOutput(outStream) - val bitmap = new RoaringBitmap - bitmap.add(1) - bitmap.add(3) - bitmap.add(5) - // Ignore Kryo because it doesn't use writeObject - bitmap.serialize(new KryoOutputObjectOutputBridge(null, output)) - output.flush() - output.close() - - val inStream = new FileInputStream(tmpfile) - val input = new KryoInput(inStream) - val ret = new RoaringBitmap - // Ignore Kryo because it doesn't use readObject - ret.deserialize(new KryoInputObjectInputBridge(null, input)) - input.close() - assert(ret == bitmap) - Utils.deleteRecursively(dir) + withTempDir { dir => + val tmpfile = dir.toString + "/RoaringBitmap" + val outStream = new FileOutputStream(tmpfile) + val output = new KryoOutput(outStream) + val bitmap = new RoaringBitmap + bitmap.add(1) + bitmap.add(3) + bitmap.add(5) + // Ignore Kryo because it doesn't use writeObject + bitmap.serialize(new KryoOutputObjectOutputBridge(null, output)) + output.flush() + output.close() + + val inStream = new FileInputStream(tmpfile) + val input = new KryoInput(inStream) + val ret = new RoaringBitmap + // Ignore Kryo because it doesn't use readObject + ret.deserialize(new KryoInputObjectInputBridge(null, input)) + input.close() + assert(ret == bitmap) + } } test("KryoOutputObjectOutputBridge.writeObject and KryoInputObjectInputBridge.readObject") { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala index eec961a49110..959cf58fa053 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala @@ -132,7 +132,6 @@ class DiskStoreSuite extends SparkFunSuite { } test("block data encryption") { - val testDir = Utils.createTempDir() val testData = new Array[Byte](128 * 1024) new Random().nextBytes(testData) diff --git a/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala b/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala index f9e1b791c86e..e48f0014fbbd 100644 --- a/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala @@ -50,34 +50,34 @@ class PeriodicRDDCheckpointerSuite extends SparkFunSuite with SharedSparkContext } test("Checkpointing") { - val tempDir = Utils.createTempDir() - val path = tempDir.toURI.toString - val checkpointInterval = 2 - var rddsToCheck = Seq.empty[RDDToCheck] - sc.setCheckpointDir(path) - val rdd1 = createRDD(sc) - val checkpointer = new PeriodicRDDCheckpointer[Double](checkpointInterval, rdd1.sparkContext) - checkpointer.update(rdd1) - rdd1.count() - rddsToCheck = rddsToCheck :+ RDDToCheck(rdd1, 1) - checkCheckpoint(rddsToCheck, 1, checkpointInterval) - - var iteration = 2 - while (iteration < 9) { - val rdd = createRDD(sc) - checkpointer.update(rdd) - rdd.count() - rddsToCheck = rddsToCheck :+ RDDToCheck(rdd, iteration) - checkCheckpoint(rddsToCheck, iteration, checkpointInterval) - iteration += 1 - } + withTempDir { tempDir => + val path = tempDir.toURI.toString + val checkpointInterval = 2 + var rddsToCheck = Seq.empty[RDDToCheck] + sc.setCheckpointDir(path) + val rdd1 = createRDD(sc) + val checkpointer = + new PeriodicRDDCheckpointer[Double](checkpointInterval, rdd1.sparkContext) + checkpointer.update(rdd1) + rdd1.count() + rddsToCheck = rddsToCheck :+ RDDToCheck(rdd1, 1) + checkCheckpoint(rddsToCheck, 1, checkpointInterval) + + var iteration = 2 + while (iteration < 9) { + val rdd = createRDD(sc) + checkpointer.update(rdd) + rdd.count() + rddsToCheck = rddsToCheck :+ RDDToCheck(rdd, iteration) + checkCheckpoint(rddsToCheck, iteration, checkpointInterval) + iteration += 1 + } - checkpointer.deleteAllCheckpoints() - rddsToCheck.foreach { rdd => - confirmCheckpointRemoved(rdd.rdd) + checkpointer.deleteAllCheckpoints() + rddsToCheck.foreach { rdd => + confirmCheckpointRemoved(rdd.rdd) + } } - - Utils.deleteRecursively(tempDir) } } 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 f5e912b50d1a..901a724da8a1 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -295,31 +295,30 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { private val workerConf = new SparkConf() def testOffsetBytes(isCompressed: Boolean): Unit = { - val tmpDir2 = Utils.createTempDir() - val suffix = getSuffix(isCompressed) - val f1Path = tmpDir2 + "/f1" + suffix - writeLogFile(f1Path, "1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(StandardCharsets.UTF_8)) - val f1Length = Utils.getFileLength(new File(f1Path), workerConf) + withTempDir { tmpDir2 => + val suffix = getSuffix(isCompressed) + val f1Path = tmpDir2 + "/f1" + suffix + writeLogFile(f1Path, "1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(StandardCharsets.UTF_8)) + val f1Length = Utils.getFileLength(new File(f1Path), workerConf) - // Read first few bytes - assert(Utils.offsetBytes(f1Path, f1Length, 0, 5) === "1\n2\n3") + // Read first few bytes + assert(Utils.offsetBytes(f1Path, f1Length, 0, 5) === "1\n2\n3") - // Read some middle bytes - assert(Utils.offsetBytes(f1Path, f1Length, 4, 11) === "3\n4\n5\n6") + // Read some middle bytes + assert(Utils.offsetBytes(f1Path, f1Length, 4, 11) === "3\n4\n5\n6") - // Read last few bytes - assert(Utils.offsetBytes(f1Path, f1Length, 12, 18) === "7\n8\n9\n") + // Read last few bytes + assert(Utils.offsetBytes(f1Path, f1Length, 12, 18) === "7\n8\n9\n") - // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(f1Path, f1Length, -5, 5) === "1\n2\n3") + // Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(f1Path, f1Length, -5, 5) === "1\n2\n3") - // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(f1Path, f1Length, 12, 22) === "7\n8\n9\n") + // Read some nonexistent bytes at the end + assert(Utils.offsetBytes(f1Path, f1Length, 12, 22) === "7\n8\n9\n") - // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(f1Path, f1Length, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") - - Utils.deleteRecursively(tmpDir2) + // Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(f1Path, f1Length, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + } } test("reading offset bytes of a file") { @@ -331,41 +330,41 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { } def testOffsetBytesMultipleFiles(isCompressed: Boolean): Unit = { - val tmpDir = Utils.createTempDir() - val suffix = getSuffix(isCompressed) - val files = (1 to 3).map(i => new File(tmpDir, i.toString + suffix)) :+ new File(tmpDir, "4") - writeLogFile(files(0).getAbsolutePath, "0123456789".getBytes(StandardCharsets.UTF_8)) - writeLogFile(files(1).getAbsolutePath, "abcdefghij".getBytes(StandardCharsets.UTF_8)) - writeLogFile(files(2).getAbsolutePath, "ABCDEFGHIJ".getBytes(StandardCharsets.UTF_8)) - writeLogFile(files(3).getAbsolutePath, "9876543210".getBytes(StandardCharsets.UTF_8)) - val fileLengths = files.map(Utils.getFileLength(_, workerConf)) - - // Read first few bytes in the 1st file - assert(Utils.offsetBytes(files, fileLengths, 0, 5) === "01234") + withTempDir { tmpDir => + val suffix = getSuffix(isCompressed) + val files = (1 to 3).map(i => + new File(tmpDir, i.toString + suffix)) :+ new File(tmpDir, "4") + writeLogFile(files(0).getAbsolutePath, "0123456789".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(1).getAbsolutePath, "abcdefghij".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(2).getAbsolutePath, "ABCDEFGHIJ".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(3).getAbsolutePath, "9876543210".getBytes(StandardCharsets.UTF_8)) + val fileLengths = files.map(Utils.getFileLength(_, workerConf)) - // Read bytes within the 1st file - assert(Utils.offsetBytes(files, fileLengths, 5, 8) === "567") + // Read first few bytes in the 1st file + assert(Utils.offsetBytes(files, fileLengths, 0, 5) === "01234") - // Read bytes across 1st and 2nd file - assert(Utils.offsetBytes(files, fileLengths, 8, 18) === "89abcdefgh") + // Read bytes within the 1st file + assert(Utils.offsetBytes(files, fileLengths, 5, 8) === "567") - // Read bytes across 1st, 2nd and 3rd file - assert(Utils.offsetBytes(files, fileLengths, 5, 24) === "56789abcdefghijABCD") + // Read bytes across 1st and 2nd file + assert(Utils.offsetBytes(files, fileLengths, 8, 18) === "89abcdefgh") - // Read bytes across 3rd and 4th file - assert(Utils.offsetBytes(files, fileLengths, 25, 35) === "FGHIJ98765") + // Read bytes across 1st, 2nd and 3rd file + assert(Utils.offsetBytes(files, fileLengths, 5, 24) === "56789abcdefghijABCD") - // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(files, fileLengths, -5, 18) === "0123456789abcdefgh") + // Read bytes across 3rd and 4th file + assert(Utils.offsetBytes(files, fileLengths, 25, 35) === "FGHIJ98765") - // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(files, fileLengths, 18, 45) === "ijABCDEFGHIJ9876543210") + // Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(files, fileLengths, -5, 18) === "0123456789abcdefgh") - // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(files, fileLengths, -5, 45) === - "0123456789abcdefghijABCDEFGHIJ9876543210") + // Read some nonexistent bytes at the end + assert(Utils.offsetBytes(files, fileLengths, 18, 45) === "ijABCDEFGHIJ9876543210") - Utils.deleteRecursively(tmpDir) + // Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(files, fileLengths, -5, 45) === + "0123456789abcdefghijABCDEFGHIJ9876543210") + } } test("reading offset bytes across multiple files") { @@ -427,27 +426,28 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("doesDirectoryContainFilesNewerThan") { // create some temporary directories and files - val parent: File = Utils.createTempDir() - // The parent directory has two child directories - val child1: File = Utils.createTempDir(parent.getCanonicalPath) - val child2: File = Utils.createTempDir(parent.getCanonicalPath) - val child3: File = Utils.createTempDir(child1.getCanonicalPath) - // set the last modified time of child1 to 30 secs old - child1.setLastModified(System.currentTimeMillis() - (1000 * 30)) - - // although child1 is old, child2 is still new so return true - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) - - child2.setLastModified(System.currentTimeMillis - (1000 * 30)) - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) - - parent.setLastModified(System.currentTimeMillis - (1000 * 30)) - // although parent and its immediate children are new, child3 is still old - // we expect a full recursive search for new files. - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) - - child3.setLastModified(System.currentTimeMillis - (1000 * 30)) - assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + withTempDir { parent => + // The parent directory has two child directories + val child1: File = Utils.createTempDir(parent.getCanonicalPath) + val child2: File = Utils.createTempDir(parent.getCanonicalPath) + val child3: File = Utils.createTempDir(child1.getCanonicalPath) + // set the last modified time of child1 to 30 secs old + child1.setLastModified(System.currentTimeMillis() - (1000 * 30)) + + // although child1 is old, child2 is still new so return true + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + child2.setLastModified(System.currentTimeMillis - (1000 * 30)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + parent.setLastModified(System.currentTimeMillis - (1000 * 30)) + // although parent and its immediate children are new, child3 is still old + // we expect a full recursive search for new files. + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + child3.setLastModified(System.currentTimeMillis - (1000 * 30)) + assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + } } test("resolveURI") { @@ -608,9 +608,8 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { } test("loading properties from file") { - val tmpDir = Utils.createTempDir() - val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) - try { + withTempDir { tmpDir => + val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) System.setProperty("spark.test.fileNameLoadB", "2") Files.write("spark.test.fileNameLoadA true\n" + "spark.test.fileNameLoadB 1\n", outFile, StandardCharsets.UTF_8) @@ -621,8 +620,6 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { val sparkConf = new SparkConf assert(sparkConf.getBoolean("spark.test.fileNameLoadA", false) === true) assert(sparkConf.getInt("spark.test.fileNameLoadB", 1) === 2) - } finally { - Utils.deleteRecursively(tmpDir) } } @@ -638,52 +635,53 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { } test("fetch hcfs dir") { - val tempDir = Utils.createTempDir() - val sourceDir = new File(tempDir, "source-dir") - sourceDir.mkdir() - val innerSourceDir = Utils.createTempDir(root = sourceDir.getPath) - val sourceFile = File.createTempFile("someprefix", "somesuffix", innerSourceDir) - val targetDir = new File(tempDir, "target-dir") - Files.write("some text", sourceFile, StandardCharsets.UTF_8) - - val path = - if (Utils.isWindows) { - new Path("file:/" + sourceDir.getAbsolutePath.replace("\\", "/")) - } else { - new Path("file://" + sourceDir.getAbsolutePath) - } - val conf = new Configuration() - val fs = Utils.getHadoopFileSystem(path.toString, conf) + withTempDir { tempDir => + val sourceDir = new File(tempDir, "source-dir") + sourceDir.mkdir() + val innerSourceDir = Utils.createTempDir(root = sourceDir.getPath) + val sourceFile = File.createTempFile("someprefix", "somesuffix", innerSourceDir) + val targetDir = new File(tempDir, "target-dir") + Files.write("some text", sourceFile, StandardCharsets.UTF_8) + + val path = + if (Utils.isWindows) { + new Path("file:/" + sourceDir.getAbsolutePath.replace("\\", "/")) + } else { + new Path("file://" + sourceDir.getAbsolutePath) + } + val conf = new Configuration() + val fs = Utils.getHadoopFileSystem(path.toString, conf) - assert(!targetDir.isDirectory()) - Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) - assert(targetDir.isDirectory()) + assert(!targetDir.isDirectory()) + Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) + assert(targetDir.isDirectory()) - // Copy again to make sure it doesn't error if the dir already exists. - Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) + // Copy again to make sure it doesn't error if the dir already exists. + Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) - val destDir = new File(targetDir, sourceDir.getName()) - assert(destDir.isDirectory()) + val destDir = new File(targetDir, sourceDir.getName()) + assert(destDir.isDirectory()) - val destInnerDir = new File(destDir, innerSourceDir.getName) - assert(destInnerDir.isDirectory()) + val destInnerDir = new File(destDir, innerSourceDir.getName) + assert(destInnerDir.isDirectory()) - val destInnerFile = new File(destInnerDir, sourceFile.getName) - assert(destInnerFile.isFile()) + val destInnerFile = new File(destInnerDir, sourceFile.getName) + assert(destInnerFile.isFile()) - val filePath = - if (Utils.isWindows) { - new Path("file:/" + sourceFile.getAbsolutePath.replace("\\", "/")) - } else { - new Path("file://" + sourceFile.getAbsolutePath) - } - val testFileDir = new File(tempDir, "test-filename") - val testFileName = "testFName" - val testFilefs = Utils.getHadoopFileSystem(filePath.toString, conf) - Utils.fetchHcfsFile(filePath, testFileDir, testFilefs, new SparkConf(), - conf, false, Some(testFileName)) - val newFileName = new File(testFileDir, testFileName) - assert(newFileName.isFile()) + val filePath = + if (Utils.isWindows) { + new Path("file:/" + sourceFile.getAbsolutePath.replace("\\", "/")) + } else { + new Path("file://" + sourceFile.getAbsolutePath) + } + val testFileDir = new File(tempDir, "test-filename") + val testFileName = "testFName" + val testFilefs = Utils.getHadoopFileSystem(filePath.toString, conf) + Utils.fetchHcfsFile(filePath, testFileDir, testFilefs, new SparkConf(), + conf, false, Some(testFileName)) + val newFileName = new File(testFileDir, testFileName) + assert(newFileName.isFile()) + } } test("shutdown hook manager") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 23419493e536..85963ec4ca69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -66,6 +66,17 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with } } + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + */ + protected override def withTempDir(f: File => Unit): Unit = { + super.withTempDir { dir => + f(dir) + waitForTasksToFinish() + } + } + /** * A helper function for turning off/on codegen. */ @@ -143,43 +154,6 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with test(name) { runOnThread() } } } -} - -/** - * Helper trait that can be extended by all external SQL test suites. - * - * This allows subclasses to plugin a custom `SQLContext`. - * To use implicit methods, import `testImplicits._` instead of through the `SQLContext`. - * - * Subclasses should *not* create `SQLContext`s in the test suite constructor, which is - * prone to leaving multiple overlapping [[org.apache.spark.SparkContext]]s in the same JVM. - */ -private[sql] trait SQLTestUtilsBase - extends Eventually - with BeforeAndAfterAll - with SQLTestData - with PlanTestBase { self: Suite => - - protected def sparkContext = spark.sparkContext - - // Shorthand for running a query using our SQLContext - protected lazy val sql = spark.sql _ - - /** - * A helper object for importing SQL implicits. - * - * Note that the alternative of importing `spark.implicits._` is not possible here. - * This is because we create the `SQLContext` immediately before the first test is run, - * but the implicits import is needed in the constructor. - */ - protected object testImplicits extends SQLImplicits { - protected override def _sqlContext: SQLContext = self.spark.sqlContext - } - - protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { - SparkSession.setActiveSession(spark) - super.withSQLConf(pairs: _*)(f) - } /** * Copy file in jar's resource to a temp file, then pass it to `f`. @@ -206,21 +180,6 @@ private[sql] trait SQLTestUtilsBase } } - /** - * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` - * returns. - * - * @todo Probably this method should be moved to a more general place - */ - protected def withTempDir(f: File => Unit): Unit = { - val dir = Utils.createTempDir().getCanonicalFile - try f(dir) finally { - // wait for all tasks to finish before deleting files - waitForTasksToFinish() - Utils.deleteRecursively(dir) - } - } - /** * Creates the specified number of temporary directories, which is then passed to `f` and will be * deleted after `f` returns. @@ -233,6 +192,43 @@ private[sql] trait SQLTestUtilsBase files.foreach(Utils.deleteRecursively) } } +} + +/** + * Helper trait that can be extended by all external SQL test suites. + * + * This allows subclasses to plugin a custom `SQLContext`. + * To use implicit methods, import `testImplicits._` instead of through the `SQLContext`. + * + * Subclasses should *not* create `SQLContext`s in the test suite constructor, which is + * prone to leaving multiple overlapping [[org.apache.spark.SparkContext]]s in the same JVM. + */ +private[sql] trait SQLTestUtilsBase + extends Eventually + with BeforeAndAfterAll + with SQLTestData + with PlanTestBase { self: Suite => + + protected def sparkContext = spark.sparkContext + + // Shorthand for running a query using our SQLContext + protected lazy val sql = spark.sql _ + + /** + * A helper object for importing SQL implicits. + * + * Note that the alternative of importing `spark.implicits._` is not possible here. + * This is because we create the `SQLContext` immediately before the first test is run, + * but the implicits import is needed in the constructor. + */ + protected object testImplicits extends SQLImplicits { + protected override def _sqlContext: SQLContext = self.spark.sqlContext + } + + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + SparkSession.setActiveSession(spark) + super.withSQLConf(pairs: _*)(f) + } /** * Drops functions after calling `f`. A function is represented by (functionName, isTemporary). 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 dc96ec416afd..218bd18e5dc9 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 @@ -55,15 +55,6 @@ class VersionsSuite extends SparkFunSuite with Logging { import HiveClientBuilder.buildClient - /** - * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` - * returns. - */ - protected def withTempDir(f: File => Unit): Unit = { - val dir = Utils.createTempDir().getCanonicalFile - try f(dir) finally Utils.deleteRecursively(dir) - } - /** * Drops table `tableName` after calling `f`. */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index ada494eb897f..6a0f523e4b49 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -557,16 +557,4 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { verifyOutput[W](output.toSeq, expectedOutput, useSet) } } - - /** - * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` - * returns. - * (originally from `SqlTestUtils`.) - * @todo Probably this method should be moved to a more general place - */ - protected def withTempDir(f: File => Unit): Unit = { - val dir = Utils.createTempDir().getCanonicalFile - try f(dir) finally Utils.deleteRecursively(dir) - } - } From 1abfbda7eb9bd855d70ba64fc137ecc101e1d8b0 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 1 Dec 2018 07:06:18 -0600 Subject: [PATCH 0039/1072] [SPARK-26212][BUILD][TEST-MAVEN] Upgrade maven version to 3.6.0 ## What changes were proposed in this pull request? This PR updates maven version from 3.5.4 to 3.6.0. The release note of the 3.6.0 is [here](https://maven.apache.org/docs/3.6.0/release-notes.html). From [the release note of the 3.6.0](https://maven.apache.org/docs/3.6.0/release-notes.html), the followings are new features: 1. There had been issues related to the project discoverytime which has been increased in previous version which influenced some of our users. 1. The output in the reactor summary has been improved. 1. There was an issue related to the classpath ordering. ## How was this patch tested? Existing tests Closes #23177 from kiszk/SPARK-26212. Authored-by: Kazuaki Ishizaki Signed-off-by: Sean Owen --- dev/appveyor-install-dependencies.ps1 | 2 +- docs/building-spark.md | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index cc68ffb90d87..7c7bdd623477 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -81,7 +81,7 @@ if (!(Test-Path $tools)) { # ========================== Maven Push-Location $tools -$mavenVer = "3.5.4" +$mavenVer = "3.6.0" Start-FileDownload "https://archive.apache.org/dist/maven/maven-3/$mavenVer/binaries/apache-maven-$mavenVer-bin.zip" "maven.zip" # extract diff --git a/docs/building-spark.md b/docs/building-spark.md index dfcd53c48e85..55695f35931c 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -12,7 +12,7 @@ redirect_from: "building-with-maven.html" ## Apache Maven The Maven-based build is the build of reference for Apache Spark. -Building Spark using Maven requires Maven 3.5.4 and Java 8. +Building Spark using Maven requires Maven 3.6.0 and Java 8. Note that support for Java 7 was removed as of Spark 2.2.0. ### Setting up Maven's Memory Usage diff --git a/pom.xml b/pom.xml index dfc3c540dc18..61321a145070 100644 --- a/pom.xml +++ b/pom.xml @@ -114,7 +114,7 @@ 1.8 ${java.version} ${java.version} - 3.5.4 + 3.6.0 spark 1.7.16 1.2.17 From 60e4239a1e3506d342099981b6e3b3b8431a203e Mon Sep 17 00:00:00 2001 From: liuxian Date: Sat, 1 Dec 2018 07:11:31 -0600 Subject: [PATCH 0040/1072] [MINOR][DOC] Correct some document description errors ## What changes were proposed in this pull request? Correct some document description errors. ## How was this patch tested? N/A Closes #23162 from 10110346/docerror. Authored-by: liuxian Signed-off-by: Sean Owen --- .../org/apache/spark/internal/config/package.scala | 10 +++++----- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 6 +++--- 2 files changed, 8 insertions(+), 8 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 9cc48f637500..646b3881a79b 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 @@ -281,7 +281,7 @@ package object config { private[spark] val LISTENER_BUS_EVENT_QUEUE_CAPACITY = ConfigBuilder("spark.scheduler.listenerbus.eventqueue.capacity") .intConf - .checkValue(_ > 0, "The capacity of listener bus event queue must not be negative") + .checkValue(_ > 0, "The capacity of listener bus event queue must be positive") .createWithDefault(10000) private[spark] val LISTENER_BUS_METRICS_MAX_LISTENER_CLASSES_TIMED = @@ -430,8 +430,8 @@ package object config { .doc("The chunk size in bytes during writing out the bytes of ChunkedByteBuffer.") .bytesConf(ByteUnit.BYTE) .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, - "The chunk size during writing out the bytes of" + - " ChunkedByteBuffer should not larger than Int.MaxValue - 15.") + "The chunk size during writing out the bytes of ChunkedByteBuffer should" + + s" be less than or equal to ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") .createWithDefault(64 * 1024 * 1024) private[spark] val CHECKPOINT_COMPRESS = @@ -503,7 +503,7 @@ package object config { "made in creating intermediate shuffle files.") .bytesConf(ByteUnit.KiB) .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, - s"The file buffer size must be greater than 0 and less than" + + s"The file buffer size must be positive and less than or equal to" + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") @@ -513,7 +513,7 @@ package object config { "is written in unsafe shuffle writer. In KiB unless otherwise specified.") .bytesConf(ByteUnit.KiB) .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, - s"The buffer size must be greater than 0 and less than" + + s"The buffer size must be positive and less than or equal to" + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") 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 f1c845bc9450..c4f00d723c25 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 @@ -326,7 +326,7 @@ object SQLConf { "factor as the estimated data size, in case the data is compressed in the file and lead to" + " a heavily underestimated result.") .doubleConf - .checkValue(_ > 0, "the value of fileDataSizeFactor must be larger than 0") + .checkValue(_ > 0, "the value of fileDataSizeFactor must be greater than 0") .createWithDefault(1.0) val PARQUET_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.parquet.mergeSchema") @@ -673,7 +673,7 @@ object SQLConf { val BUCKETING_MAX_BUCKETS = buildConf("spark.sql.sources.bucketing.maxBuckets") .doc("The maximum number of buckets allowed. Defaults to 100000") .intConf - .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be larger than 0") + .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be greater than 0") .createWithDefault(100000) val CROSS_JOINS_ENABLED = buildConf("spark.sql.crossJoin.enabled") @@ -1154,7 +1154,7 @@ object SQLConf { .internal() .doc("The number of bins when generating histograms.") .intConf - .checkValue(num => num > 1, "The number of bins must be larger than 1.") + .checkValue(num => num > 1, "The number of bins must be greater than 1.") .createWithDefault(254) val PERCENTILE_ACCURACY = From 55c96858107739dd768abea1dff88bd970e47e9f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 1 Dec 2018 16:22:38 -0800 Subject: [PATCH 0041/1072] [SPARK-26226][SQL] Track optimization phase for streaming queries ## What changes were proposed in this pull request? In an earlier PR, we missed measuring the optimization phase time for streaming queries. This patch adds it. ## How was this patch tested? Given this is a debugging feature, and it is very convoluted to add tests to verify the phase is set properly, I am not introducing a streaming specific test. Closes #23193 from rxin/SPARK-26226-1. Authored-by: Reynold Xin Signed-off-by: gatorsmile --- .../spark/sql/execution/streaming/IncrementalExecution.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 fad287e28877..a73e88c19ba9 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 @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy} +import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.expressions.{CurrentBatchTimestamp, ExpressionWithRandomSeed} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, HashPartitioning, SinglePartition} @@ -73,7 +74,8 @@ class IncrementalExecution( * Walk the optimized logical plan and replace CurrentBatchTimestamp * with the desired literal */ - override lazy val optimizedPlan: LogicalPlan = { + override + lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { sparkSession.sessionState.optimizer.execute(withCachedData) transformAllExpressions { case ts @ CurrentBatchTimestamp(timestamp, _, _) => logInfo(s"Current batch timestamp = $timestamp") From cbb9bb96d292d6e738f2f33637fb1c9715b167ac Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 1 Dec 2018 16:24:06 -0800 Subject: [PATCH 0042/1072] [SPARK-26241][SQL] Add queryId to IncrementalExecution ## What changes were proposed in this pull request? This is a small change for better debugging: to pass query uuid in IncrementalExecution, when we look at the QueryExecution in isolation to trace back the query. ## How was this patch tested? N/A - just add some field for better debugging. Closes #23192 from rxin/SPARK-26241. Authored-by: Reynold Xin Signed-off-by: gatorsmile --- .../scala/org/apache/spark/sql/execution/command/commands.scala | 2 +- .../spark/sql/execution/streaming/IncrementalExecution.scala | 1 + .../spark/sql/execution/streaming/MicroBatchExecution.scala | 1 + .../execution/streaming/continuous/ContinuousExecution.scala | 1 + 4 files changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index ab40936eb3cc..754a3316ffb7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -154,7 +154,7 @@ case class ExplainCommand( // output mode does not matter since there is no `Sink`. new IncrementalExecution( sparkSession, logicalPlan, OutputMode.Append(), "", - UUID.randomUUID, 0, OffsetSeqMetadata(0, 0)) + UUID.randomUUID, UUID.randomUUID, 0, OffsetSeqMetadata(0, 0)) } else { sparkSession.sessionState.executePlan(logicalPlan) } 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 a73e88c19ba9..af52af0d1d7e 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 @@ -42,6 +42,7 @@ class IncrementalExecution( logicalPlan: LogicalPlan, val outputMode: OutputMode, val checkpointLocation: String, + val queryId: UUID, val runId: UUID, val currentBatchId: Long, val offsetSeqMetadata: OffsetSeqMetadata) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 5defca391a35..64e09edf27f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -521,6 +521,7 @@ class MicroBatchExecution( triggerLogicalPlan, outputMode, checkpointFile("state"), + id, runId, currentBatchId, offsetSeqMetadata) 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 af23c5cd3d80..4d42428fd189 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 @@ -199,6 +199,7 @@ class ContinuousExecution( withSink, outputMode, checkpointFile("state"), + id, runId, currentBatchId, offsetSeqMetadata) From 17fdca7c1bab94e6e54b25807344b06a78780cf6 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sun, 2 Dec 2018 10:22:22 +0800 Subject: [PATCH 0043/1072] [SPARK-26211][SQL][TEST][FOLLOW-UP] Combine test cases for `In` and `InSet`. ## What changes were proposed in this pull request? This is a follow pr of #23176. `In` and `InSet` are semantically equal, so the tests for `In` should pass with `InSet`, and vice versa. This combines those test cases. ## How was this patch tested? The combined tests and existing tests. Closes #23187 from ueshin/issues/SPARK-26211/in_inset_tests. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan --- .../catalyst/expressions/PredicateSuite.scala | 160 ++++++++---------- 1 file changed, 66 insertions(+), 94 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index 3b60d1d88b3c..0f63717f9daf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -124,34 +124,43 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { (null, false, null) :: (null, null, null) :: Nil) - test("basic IN predicate test") { - checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), Seq(Literal(1), + private def checkInAndInSet(in: In, expected: Any): Unit = { + // expecting all in.list are Literal or NonFoldableLiteral. + checkEvaluation(in, expected) + checkEvaluation(InSet(in.value, HashSet() ++ in.list.map(_.eval())), expected) + } + + test("basic IN/INSET predicate test") { + checkInAndInSet(In(NonFoldableLiteral.create(null, IntegerType), Seq(Literal(1), Literal(2))), null) - checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), + checkInAndInSet(In(NonFoldableLiteral.create(null, IntegerType), Seq(NonFoldableLiteral.create(null, IntegerType))), null) - checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), Seq.empty), null) - checkEvaluation(In(Literal(1), Seq.empty), false) - checkEvaluation(In(Literal(1), Seq(NonFoldableLiteral.create(null, IntegerType))), null) - checkEvaluation(In(Literal(1), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), + checkInAndInSet(In(NonFoldableLiteral.create(null, IntegerType), Seq.empty), null) + checkInAndInSet(In(Literal(1), Seq.empty), false) + checkInAndInSet(In(Literal(1), Seq(NonFoldableLiteral.create(null, IntegerType))), null) + checkInAndInSet(In(Literal(1), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), true) - checkEvaluation(In(Literal(2), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), + checkInAndInSet(In(Literal(2), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), null) - checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) - checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) - checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) + checkInAndInSet(In(Literal(1), Seq(Literal(1), Literal(2))), true) + checkInAndInSet(In(Literal(2), Seq(Literal(1), Literal(2))), true) + checkInAndInSet(In(Literal(3), Seq(Literal(1), Literal(2))), false) + checkEvaluation( And(In(Literal(1), Seq(Literal(1), Literal(2))), In(Literal(2), Seq(Literal(1), Literal(2)))), true) + checkEvaluation( + And(InSet(Literal(1), HashSet(1, 2)), InSet(Literal(2), Set(1, 2))), + true) val ns = NonFoldableLiteral.create(null, StringType) - checkEvaluation(In(ns, Seq(Literal("1"), Literal("2"))), null) - checkEvaluation(In(ns, Seq(ns)), null) - checkEvaluation(In(Literal("a"), Seq(ns)), null) - checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("^Ba*n"), ns)), true) - checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^Ba*n"))), true) - checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^n"))), false) - + checkInAndInSet(In(ns, Seq(Literal("1"), Literal("2"))), null) + checkInAndInSet(In(ns, Seq(ns)), null) + checkInAndInSet(In(Literal("a"), Seq(ns)), null) + checkInAndInSet(In(Literal("^Ba*n"), Seq(Literal("^Ba*n"), ns)), true) + checkInAndInSet(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^Ba*n"))), true) + checkInAndInSet(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^n"))), false) } test("IN with different types") { @@ -187,11 +196,12 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { } else { false } - checkEvaluation(In(input(0), input.slice(1, 10)), expected) + checkInAndInSet(In(input(0), input.slice(1, 10)), expected) } val atomicTypes = DataTypeTestUtils.atomicTypes.filter { t => - RandomDataGenerator.forType(t).isDefined && !t.isInstanceOf[DecimalType] + RandomDataGenerator.forType(t).isDefined && + !t.isInstanceOf[DecimalType] && !t.isInstanceOf[BinaryType] } ++ Seq(DecimalType.USER_DEFAULT) val atomicArrayTypes = atomicTypes.map(ArrayType(_, containsNull = true)) @@ -252,93 +262,55 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { assert(ctx.inlinedMutableStates.isEmpty) } - test("INSET") { - val hS = HashSet[Any]() + 1 + 2 - val nS = HashSet[Any]() + 1 + 2 + null - val one = Literal(1) - val two = Literal(2) - val three = Literal(3) - val nl = Literal(null) - checkEvaluation(InSet(one, hS), true) - checkEvaluation(InSet(two, hS), true) - checkEvaluation(InSet(two, nS), true) - checkEvaluation(InSet(three, hS), false) - checkEvaluation(InSet(three, nS), null) - checkEvaluation(InSet(nl, hS), null) - checkEvaluation(InSet(nl, nS), null) - - val primitiveTypes = Seq(IntegerType, FloatType, DoubleType, StringType, ByteType, ShortType, - LongType, BooleanType, DecimalType.USER_DEFAULT, TimestampType) - primitiveTypes.foreach { t => - val dataGen = RandomDataGenerator.forType(t, nullable = true).get - val inputData = Seq.fill(10) { - val value = dataGen.apply() - value match { - case d: Double if d.isNaN => 0.0d - case f: Float if f.isNaN => 0.0f - case _ => value - } - } - val input = inputData.map(Literal(_)) - val expected = if (inputData(0) == null) { - null - } else if (inputData.slice(1, 10).contains(inputData(0))) { - true - } else if (inputData.slice(1, 10).contains(null)) { - null - } else { - false - } - checkEvaluation(InSet(input(0), inputData.slice(1, 10).toSet), expected) - } - } - - test("INSET: binary") { - val hS = HashSet[Any]() + Array(1.toByte, 2.toByte) + Array(3.toByte) - val nS = HashSet[Any]() + Array(1.toByte, 2.toByte) + Array(3.toByte) + null + test("IN/INSET: binary") { val onetwo = Literal(Array(1.toByte, 2.toByte)) val three = Literal(Array(3.toByte)) val threefour = Literal(Array(3.toByte, 4.toByte)) - val nl = Literal(null, onetwo.dataType) - checkEvaluation(InSet(onetwo, hS), true) - checkEvaluation(InSet(three, hS), true) - checkEvaluation(InSet(three, nS), true) - checkEvaluation(InSet(threefour, hS), false) - checkEvaluation(InSet(threefour, nS), null) - checkEvaluation(InSet(nl, hS), null) - checkEvaluation(InSet(nl, nS), null) + val nl = NonFoldableLiteral.create(null, onetwo.dataType) + val hS = Seq(Literal(Array(1.toByte, 2.toByte)), Literal(Array(3.toByte))) + val nS = Seq(Literal(Array(1.toByte, 2.toByte)), Literal(Array(3.toByte)), + NonFoldableLiteral.create(null, onetwo.dataType)) + checkInAndInSet(In(onetwo, hS), true) + checkInAndInSet(In(three, hS), true) + checkInAndInSet(In(three, nS), true) + checkInAndInSet(In(threefour, hS), false) + checkInAndInSet(In(threefour, nS), null) + checkInAndInSet(In(nl, hS), null) + checkInAndInSet(In(nl, nS), null) } - test("INSET: struct") { - val hS = HashSet[Any]() + Literal.create((1, "a")).value + Literal.create((2, "b")).value - val nS = HashSet[Any]() + Literal.create((1, "a")).value + Literal.create((2, "b")).value + null + test("IN/INSET: struct") { val oneA = Literal.create((1, "a")) val twoB = Literal.create((2, "b")) val twoC = Literal.create((2, "c")) - val nl = Literal(null, oneA.dataType) - checkEvaluation(InSet(oneA, hS), true) - checkEvaluation(InSet(twoB, hS), true) - checkEvaluation(InSet(twoB, nS), true) - checkEvaluation(InSet(twoC, hS), false) - checkEvaluation(InSet(twoC, nS), null) - checkEvaluation(InSet(nl, hS), null) - checkEvaluation(InSet(nl, nS), null) + val nl = NonFoldableLiteral.create(null, oneA.dataType) + val hS = Seq(Literal.create((1, "a")), Literal.create((2, "b"))) + val nS = Seq(Literal.create((1, "a")), Literal.create((2, "b")), + NonFoldableLiteral.create(null, oneA.dataType)) + checkInAndInSet(In(oneA, hS), true) + checkInAndInSet(In(twoB, hS), true) + checkInAndInSet(In(twoB, nS), true) + checkInAndInSet(In(twoC, hS), false) + checkInAndInSet(In(twoC, nS), null) + checkInAndInSet(In(nl, hS), null) + checkInAndInSet(In(nl, nS), null) } - test("INSET: array") { - val hS = HashSet[Any]() + Literal.create(Seq(1, 2)).value + Literal.create(Seq(3)).value - val nS = HashSet[Any]() + Literal.create(Seq(1, 2)).value + Literal.create(Seq(3)).value + null + test("IN/INSET: array") { val onetwo = Literal.create(Seq(1, 2)) val three = Literal.create(Seq(3)) val threefour = Literal.create(Seq(3, 4)) - val nl = Literal(null, onetwo.dataType) - checkEvaluation(InSet(onetwo, hS), true) - checkEvaluation(InSet(three, hS), true) - checkEvaluation(InSet(three, nS), true) - checkEvaluation(InSet(threefour, hS), false) - checkEvaluation(InSet(threefour, nS), null) - checkEvaluation(InSet(nl, hS), null) - checkEvaluation(InSet(nl, nS), null) + val nl = NonFoldableLiteral.create(null, onetwo.dataType) + val hS = Seq(Literal.create(Seq(1, 2)), Literal.create(Seq(3))) + val nS = Seq(Literal.create(Seq(1, 2)), Literal.create(Seq(3)), + NonFoldableLiteral.create(null, onetwo.dataType)) + checkInAndInSet(In(onetwo, hS), true) + checkInAndInSet(In(three, hS), true) + checkInAndInSet(In(three, nS), true) + checkInAndInSet(In(threefour, hS), false) + checkInAndInSet(In(threefour, nS), null) + checkInAndInSet(In(nl, hS), null) + checkInAndInSet(In(nl, nS), null) } private case class MyStruct(a: Long, b: String) From 3e46e3ccd58d0a2d445dff58a52ab1966ce133e8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 2 Dec 2018 10:29:25 +0800 Subject: [PATCH 0044/1072] [SPARK-26161][SQL] Ignore empty files in load ## What changes were proposed in this pull request? In the PR, I propose filtering out all empty files inside of `FileSourceScanExec` and exclude them from file splits. It should reduce overhead of opening and reading files without any data, and as consequence datasources will not produce empty partitions for such files. ## How was this patch tested? Added a test which creates an empty and non-empty files. If empty files are ignored in load, Text datasource in the `wholetext` mode must create only one partition for non-empty file. Closes #23130 from MaxGekk/ignore-empty-files. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../spark/sql/execution/DataSourceScanExec.scala | 4 ++-- .../sql/execution/datasources/json/JsonSuite.scala | 3 +-- .../apache/spark/sql/sources/SaveLoadSuite.scala | 13 +++++++++++++ 3 files changed, 16 insertions(+), 4 deletions(-) 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 4faa27c2c1e2..b29d5c76c5f3 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 @@ -368,7 +368,7 @@ case class FileSourceScanExec( logInfo(s"Planning with ${bucketSpec.numBuckets} buckets") val filesGroupedToBuckets = selectedPartitions.flatMap { p => - p.files.map { f => + p.files.filter(_.getLen > 0).map { f => val hosts = getBlockHosts(getBlockLocations(f), 0, f.getLen) PartitionedFile(p.values, f.getPath.toUri.toString, 0, f.getLen, hosts) } @@ -418,7 +418,7 @@ case class FileSourceScanExec( s"open cost is considered as scanning $openCostInBytes bytes.") val splitFiles = selectedPartitions.flatMap { partition => - partition.files.flatMap { file => + partition.files.filter(_.getLen > 0).flatMap { file => val blockLocations = getBlockLocations(file) if (fsRelation.fileFormat.isSplitable( fsRelation.sparkSession, fsRelation.options, file.getPath)) { 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 ee5176e23e34..9d23161c1f24 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 @@ -1842,7 +1842,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val path = dir.getCanonicalPath primitiveFieldAndType .toDF("value") - .repartition(1) .write .text(path) @@ -1910,7 +1909,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { F.count($"dummy").as("valid"), F.count($"_corrupt_record").as("corrupt"), F.count("*").as("count")) - checkAnswer(counts, Row(1, 4, 6)) // null row for empty file + checkAnswer(counts, Row(1, 4, 6)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index 12779b46bfe8..048e4b80c72a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.sources import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} import org.scalatest.BeforeAndAfter @@ -142,4 +144,15 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA assert(e.contains(s"Partition column `$unknown` not found in schema $schemaCatalog")) } } + + test("skip empty files in non bucketed read") { + withTempDir { dir => + val path = dir.getCanonicalPath + Files.write(Paths.get(path, "empty"), Array.empty[Byte]) + Files.write(Paths.get(path, "notEmpty"), "a".getBytes(StandardCharsets.UTF_8)) + val readback = spark.read.option("wholetext", true).text(path) + + assert(readback.rdd.getNumPartitions === 1) + } + } } From 39617cb2c0c433494e9f17fcd4e49c6300a9c4b0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 2 Dec 2018 10:46:17 +0800 Subject: [PATCH 0045/1072] [SPARK-26216][SQL] Do not use case class as public API (UserDefinedFunction) ## What changes were proposed in this pull request? It's a bad idea to use case class as public API, as it has a very wide surface. For example, the `copy` method, its fields, the companion object, etc. For a particular case, `UserDefinedFunction`. It has a private constructor, and I believe we only want users to access a few methods:`apply`, `nullable`, `asNonNullable`, etc. However, all its fields, and `copy` method, and the companion object are public unexpectedly. As a result, we made many tricks to work around the binary compatibility issues. This PR proposes to only make interfaces public, and hide implementations behind with a private class. Now `UserDefinedFunction` is a pure trait, and the concrete implementation is `SparkUserDefinedFunction`, which is private. Changing class to interface is not binary compatible(but source compatible), so 3.0 is a good chance to do it. This is the first PR to go with this direction. If it's accepted, I'll create a umbrella JIRA and fix all the public case classes. ## How was this patch tested? existing tests. Closes #23178 from cloud-fan/udf. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- docs/sql-migration-guide-upgrade.md | 2 + project/MimaExcludes.scala | 6 +- .../sql/expressions/UserDefinedFunction.scala | 119 ++++++++---------- .../org/apache/spark/sql/functions.scala | 2 +- 4 files changed, 63 insertions(+), 66 deletions(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index e48125a0972b..787f4bcbbea8 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -31,6 +31,8 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.execution.setCommandRejectsSparkConfs` to `false`. + - Spark applications which are built with Spark version 2.4 and prior, and call methods of `UserDefinedFunction`, need to be re-compiled with Spark 3.0, as they are not binary compatible with Spark 3.0. + ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index fcef424c330f..1c83cf5860c5 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -227,7 +227,11 @@ object MimaExcludes { case ReversedMissingMethodProblem(meth) => !meth.owner.fullName.startsWith("org.apache.spark.sql.sources.v2") case _ => true - } + }, + + // [SPARK-26216][SQL] Do not use case class as public API (UserDefinedFunction) + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.expressions.UserDefinedFunction$"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.sql.expressions.UserDefinedFunction") ) // Exclude rules for 2.4.x 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 58a942afe28c..f88e0e0f299d 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 @@ -38,25 +38,14 @@ import org.apache.spark.sql.types.DataType * @since 1.3.0 */ @Stable -case class UserDefinedFunction protected[sql] ( - f: AnyRef, - dataType: DataType, - inputTypes: Option[Seq[DataType]]) { - - private var _nameOption: Option[String] = None - private var _nullable: Boolean = true - private var _deterministic: Boolean = true - - // This is a `var` instead of in the constructor for backward compatibility of this case class. - // TODO: revisit this case class in Spark 3.0, and narrow down the public surface. - private[sql] var nullableTypes: Option[Seq[Boolean]] = None +sealed trait UserDefinedFunction { /** * Returns true when the UDF can return a nullable value. * * @since 2.3.0 */ - def nullable: Boolean = _nullable + def nullable: Boolean /** * Returns true iff the UDF is deterministic, i.e. the UDF produces the same output given the same @@ -64,7 +53,7 @@ case class UserDefinedFunction protected[sql] ( * * @since 2.3.0 */ - def deterministic: Boolean = _deterministic + def deterministic: Boolean /** * Returns an expression that invokes the UDF, using the given arguments. @@ -72,80 +61,83 @@ case class UserDefinedFunction protected[sql] ( * @since 1.3.0 */ @scala.annotation.varargs - def apply(exprs: Column*): Column = { + def apply(exprs: Column*): Column + + /** + * Updates UserDefinedFunction with a given name. + * + * @since 2.3.0 + */ + def withName(name: String): UserDefinedFunction + + /** + * Updates UserDefinedFunction to non-nullable. + * + * @since 2.3.0 + */ + def asNonNullable(): UserDefinedFunction + + /** + * Updates UserDefinedFunction to nondeterministic. + * + * @since 2.3.0 + */ + def asNondeterministic(): UserDefinedFunction +} + +private[sql] case class SparkUserDefinedFunction( + f: AnyRef, + dataType: DataType, + inputTypes: Option[Seq[DataType]], + nullableTypes: Option[Seq[Boolean]], + name: Option[String] = None, + nullable: Boolean = true, + deterministic: Boolean = true) extends UserDefinedFunction { + + @scala.annotation.varargs + override def apply(exprs: Column*): Column = { // TODO: make sure this class is only instantiated through `SparkUserDefinedFunction.create()` // and `nullableTypes` is always set. - if (nullableTypes.isEmpty) { - nullableTypes = Some(ScalaReflection.getParameterTypeNullability(f)) - } if (inputTypes.isDefined) { assert(inputTypes.get.length == nullableTypes.get.length) } + val inputsNullSafe = nullableTypes.getOrElse { + ScalaReflection.getParameterTypeNullability(f) + } + Column(ScalaUDF( f, dataType, exprs.map(_.expr), - nullableTypes.get, + inputsNullSafe, inputTypes.getOrElse(Nil), - udfName = _nameOption, - nullable = _nullable, - udfDeterministic = _deterministic)) - } - - private def copyAll(): UserDefinedFunction = { - val udf = copy() - udf._nameOption = _nameOption - udf._nullable = _nullable - udf._deterministic = _deterministic - udf.nullableTypes = nullableTypes - udf + udfName = name, + nullable = nullable, + udfDeterministic = deterministic)) } - /** - * Updates UserDefinedFunction with a given name. - * - * @since 2.3.0 - */ - def withName(name: String): UserDefinedFunction = { - val udf = copyAll() - udf._nameOption = Option(name) - udf + override def withName(name: String): UserDefinedFunction = { + copy(name = Option(name)) } - /** - * Updates UserDefinedFunction to non-nullable. - * - * @since 2.3.0 - */ - def asNonNullable(): UserDefinedFunction = { + override def asNonNullable(): UserDefinedFunction = { if (!nullable) { this } else { - val udf = copyAll() - udf._nullable = false - udf + copy(nullable = false) } } - /** - * Updates UserDefinedFunction to nondeterministic. - * - * @since 2.3.0 - */ - def asNondeterministic(): UserDefinedFunction = { - if (!_deterministic) { + override def asNondeterministic(): UserDefinedFunction = { + if (!deterministic) { this } else { - val udf = copyAll() - udf._deterministic = false - udf + copy(deterministic = false) } } } -// We have to use a name different than `UserDefinedFunction` here, to avoid breaking the binary -// compatibility of the auto-generate UserDefinedFunction object. private[sql] object SparkUserDefinedFunction { def create( @@ -157,8 +149,7 @@ private[sql] object SparkUserDefinedFunction { } else { Some(inputSchemas.map(_.get.dataType)) } - val udf = new UserDefinedFunction(f, dataType, inputTypes) - udf.nullableTypes = Some(inputSchemas.map(_.map(_.nullable).getOrElse(true))) - udf + val nullableTypes = Some(inputSchemas.map(_.map(_.nullable).getOrElse(true))) + SparkUserDefinedFunction(f, dataType, inputTypes, nullableTypes) } } 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 efa8f8526387..33186f778d86 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 @@ -4259,7 +4259,7 @@ object functions { def udf(f: AnyRef, dataType: DataType): UserDefinedFunction = { // TODO: should call SparkUserDefinedFunction.create() instead but inputSchemas is currently // unavailable. We may need to create type-safe overloaded versions of udf() methods. - new UserDefinedFunction(f, dataType, inputTypes = None) + SparkUserDefinedFunction(f, dataType, inputTypes = None, nullableTypes = None) } /** From 031bd80e4f943d64a1171856978022fac320de5d Mon Sep 17 00:00:00 2001 From: lichaoqun Date: Sun, 2 Dec 2018 10:55:17 +0800 Subject: [PATCH 0046/1072] [SPARK-26195][SQL] Correct exception messages in some classes ## What changes were proposed in this pull request? UnsupportedOperationException messages are not the same with method name.This PR correct these messages. ## How was this patch tested? NA Closes #23154 from lcqzte10192193/wid-lcq-1127. Authored-by: lichaoqun Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/catalyst/analysis/unresolved.scala | 4 ++-- .../apache/spark/sql/catalyst/expressions/Expression.scala | 2 +- .../apache/spark/sql/catalyst/expressions/generators.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 36cad3cf7478..d44b42134f86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -204,10 +204,10 @@ case class UnresolvedGenerator(name: FunctionIdentifier, children: Seq[Expressio throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + throw new UnsupportedOperationException(s"Cannot generate code for expression: $this") override def terminate(): TraversableOnce[InternalRow] = - throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + throw new UnsupportedOperationException(s"Cannot terminate expression: $this") } case class UnresolvedFunction( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 2ecec61adb0a..c89c2272be75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -288,7 +288,7 @@ trait Unevaluable extends Expression { throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + throw new UnsupportedOperationException(s"Cannot generate code for expression: $this") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index d6e67b9ac3d1..9c74fdf6c9a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -258,7 +258,7 @@ case class GeneratorOuter(child: Generator) extends UnaryExpression with Generat throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + throw new UnsupportedOperationException(s"Cannot generate code for expression: $this") override def elementSchema: StructType = child.elementSchema From c7d95ccedf593edf9fda9ecaf8d0b4dda451440d Mon Sep 17 00:00:00 2001 From: Koert Kuipers Date: Sun, 2 Dec 2018 17:38:25 +0800 Subject: [PATCH 0047/1072] [SPARK-26208][SQL] add headers to empty csv files when header=true ## What changes were proposed in this pull request? Add headers to empty csv files when header=true, because otherwise these files are invalid when reading. ## How was this patch tested? Added test for roundtrip of empty dataframe to csv file with headers and back in CSVSuite Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23173 from koertkuipers/feat-empty-csv-with-header. Authored-by: Koert Kuipers Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/csv/UnivocityGenerator.scala | 9 ++++---- .../datasources/csv/CSVFileFormat.scala | 22 ++++++++++++------- .../execution/datasources/csv/CSVSuite.scala | 13 +++++++++++ 3 files changed, 31 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala index 1218f9242afe..2ab376c0ac20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala @@ -32,7 +32,6 @@ class UnivocityGenerator( private val writerSettings = options.asWriterSettings writerSettings.setHeaders(schema.fieldNames: _*) private val gen = new CsvWriter(writer, writerSettings) - private var printHeader = options.headerFlag // A `ValueConverter` is responsible for converting a value of an `InternalRow` to `String`. // When the value is null, this converter should not be called. @@ -72,15 +71,15 @@ class UnivocityGenerator( values } + def writeHeaders(): Unit = { + gen.writeHeaders() + } + /** * Writes a single InternalRow to CSV using Univocity. */ def write(row: InternalRow): Unit = { - if (printHeader) { - gen.writeHeaders() - } gen.writeRow(convertRow(row): _*) - printHeader = false } def writeToString(row: InternalRow): String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 4c5a1d327023..f7d8a9e1042d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -171,15 +171,21 @@ private[csv] class CsvOutputWriter( private var univocityGenerator: Option[UnivocityGenerator] = None - override def write(row: InternalRow): Unit = { - val gen = univocityGenerator.getOrElse { - val charset = Charset.forName(params.charset) - val os = CodecStreams.createOutputStreamWriter(context, new Path(path), charset) - val newGen = new UnivocityGenerator(dataSchema, os, params) - univocityGenerator = Some(newGen) - newGen - } + if (params.headerFlag) { + val gen = getGen() + gen.writeHeaders() + } + private def getGen(): UnivocityGenerator = univocityGenerator.getOrElse { + val charset = Charset.forName(params.charset) + val os = CodecStreams.createOutputStreamWriter(context, new Path(path), charset) + val newGen = new UnivocityGenerator(dataSchema, os, params) + univocityGenerator = Some(newGen) + newGen + } + + override def write(row: InternalRow): Unit = { + val gen = getGen() gen.write(row) } 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 e14e8d49db5c..bc950f2418d3 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 @@ -1987,6 +1987,19 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te assert(errMsg2.contains("'lineSep' can contain only 1 character")) } + test("SPARK-26208: write and read empty data to csv file with headers") { + withTempPath { path => + val df1 = spark.range(10).repartition(2).filter(_ < 0).map(_.toString).toDF + // we have 2 partitions but they are both empty and will be filtered out upon writing + // thanks to SPARK-23271 one new empty partition will be inserted + df1.write.format("csv").option("header", true).save(path.getAbsolutePath) + val df2 = spark.read.format("csv").option("header", true).option("inferSchema", false) + .load(path.getAbsolutePath) + assert(df1.schema === df2.schema) + checkAnswer(df1, df2) + } + } + test("do not produce empty files for empty partitions") { withTempPath { dir => val path = dir.getCanonicalPath From 9cda9a892d03f60a76cd5d9b4546e72c50962c85 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 2 Dec 2018 17:41:08 +0800 Subject: [PATCH 0048/1072] [SPARK-26080][PYTHON] Skips Python resource limit on Windows in Python worker ## What changes were proposed in this pull request? `resource` package is a Unix specific package. See https://docs.python.org/2/library/resource.html and https://docs.python.org/3/library/resource.html. Note that we document Windows support: > Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). This should be backported into branch-2.4 to restore Windows support in Spark 2.4.1. ## How was this patch tested? Manually mocking the changed logics. Closes #23055 from HyukjinKwon/SPARK-26080. Lead-authored-by: hyukjinkwon Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- docs/configuration.md | 2 ++ python/pyspark/worker.py | 19 ++++++++++++------- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 8914bd0310f9..9abbb3f63490 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -190,6 +190,8 @@ of the most common options to set are: and it is up to the application to avoid exceeding the overhead memory space shared with other non-JVM processes. When PySpark is run in YARN or Kubernetes, this memory is added to executor resource requests. + + NOTE: Python memory usage may not be limited on platforms that do not support resource limiting, such as Windows.
    diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8c59f1f999f1..953b468e9651 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -22,7 +22,12 @@ import os import sys import time -import resource +# 'resource' is a Unix specific module. +has_resource_module = True +try: + import resource +except ImportError: + has_resource_module = False import socket import traceback @@ -268,9 +273,9 @@ def main(infile, outfile): # set up memory limits memory_limit_mb = int(os.environ.get('PYSPARK_EXECUTOR_MEMORY_MB', "-1")) - total_memory = resource.RLIMIT_AS - try: - if memory_limit_mb > 0: + if memory_limit_mb > 0 and has_resource_module: + total_memory = resource.RLIMIT_AS + try: (soft_limit, hard_limit) = resource.getrlimit(total_memory) msg = "Current mem limits: {0} of max {1}\n".format(soft_limit, hard_limit) print(msg, file=sys.stderr) @@ -283,9 +288,9 @@ def main(infile, outfile): print(msg, file=sys.stderr) resource.setrlimit(total_memory, (new_limit, new_limit)) - except (resource.error, OSError, ValueError) as e: - # not all systems support resource limits, so warn instead of failing - print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr) + except (resource.error, OSError, ValueError) as e: + # not all systems support resource limits, so warn instead of failing + print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr) # initialize global state taskContext = None From 676bbb2446af1f281b8f76a5428b7ba75b7588b3 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 2 Dec 2018 08:52:01 -0600 Subject: [PATCH 0049/1072] [SPARK-26198][SQL] Fix Metadata serialize null values throw NPE ## What changes were proposed in this pull request? How to reproduce this issue: ```scala scala> val meta = new org.apache.spark.sql.types.MetadataBuilder().putNull("key").build().json java.lang.NullPointerException at org.apache.spark.sql.types.Metadata$.org$apache$spark$sql$types$Metadata$$toJsonValue(Metadata.scala:196) at org.apache.spark.sql.types.Metadata$$anonfun$1.apply(Metadata.scala:180) ``` This pr fix `NullPointerException` when `Metadata` serialize `null` values. ## How was this patch tested? unit tests Closes #23164 from wangyum/SPARK-26198. Authored-by: Yuming Wang Signed-off-by: Sean Owen --- .../src/main/scala/org/apache/spark/sql/types/Metadata.scala | 2 ++ .../scala/org/apache/spark/sql/types/MetadataSuite.scala | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 4979aced145c..b6a859b75c37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -190,6 +190,8 @@ object Metadata { JBool(x) case x: String => JString(x) + case null => + JNull case x: Metadata => toJsonValue(x.map) case other => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala index 210e65708170..b4aeac562d2b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala @@ -26,6 +26,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getString("key") === "value") assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getString("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -36,6 +37,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getLong("key") === 12) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getLong("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -46,6 +48,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getDouble("key") === 12) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getDouble("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -56,6 +59,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getBoolean("key") === true) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getBoolean("no_such_key")) intercept[ClassCastException](meta.getString("key")) } @@ -69,6 +73,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.getLong("key") === 0) assert(meta.getBoolean("key") === false) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getLong("no_such_key")) } } From bfa3d32f7719cd4bfb2c161fe4a6bd3eea148158 Mon Sep 17 00:00:00 2001 From: caoxuewen Date: Mon, 3 Dec 2018 16:18:22 +0800 Subject: [PATCH 0050/1072] [SPARK-26117][FOLLOW-UP][SQL] throw SparkOutOfMemoryError intead of SparkException in UnsafeHashedRelation ## What changes were proposed in this pull request? When build hash Map with one row of data and run out of memory, we should throw a SparkOutOfMemoryError exception, which is more accurate than SparkException. this PR fix it. ## How was this patch tested? N / A Closes #23190 from heary-cao/throwUnsafeHashedRelation. Authored-by: caoxuewen Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/joins/HashedRelation.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 86eb47a70f1a..e8c01d46a84c 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 @@ -24,7 +24,7 @@ import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.{SparkConf, SparkEnv, SparkException} import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED -import org.apache.spark.memory.{MemoryConsumer, StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.memory._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode @@ -316,7 +316,9 @@ private[joins] object UnsafeHashedRelation { row.getBaseObject, row.getBaseOffset, row.getSizeInBytes) if (!success) { binaryMap.free() - throw new SparkException("There is no enough memory to build hash map") + // scalastyle:off throwerror + throw new SparkOutOfMemoryError("There is no enough memory to build hash map") + // scalastyle:on throwerror } } } From 11e5f1bcd49eec8ab4225d6e68a051b5c6a21cb2 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 3 Dec 2018 18:25:38 +0800 Subject: [PATCH 0051/1072] [SPARK-26151][SQL] Return partial results for bad CSV records ## What changes were proposed in this pull request? In the PR, I propose to change behaviour of `UnivocityParser` and `FailureSafeParser`, and return all fields that were parsed and converted to expected types successfully instead of just returning a row with all `null`s for a bad input in the `PERMISSIVE` mode. For example, for CSV line `0,2013-111-11 12:13:14` and DDL schema `a int, b timestamp`, new result is `Row(0, null)`. ## How was this patch tested? It was checked by existing tests from `CsvSuite` and `CsvFunctionsSuite`. Closes #23120 from MaxGekk/failuresafe-partial-result. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/csv/UnivocityParser.scala | 27 ++++++++++--------- .../sql/catalyst/util/FailureSafeParser.scala | 21 ++++++--------- .../apache/spark/sql/CsvFunctionsSuite.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 6 ++--- 4 files changed, 27 insertions(+), 29 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 85e129224c91..8fff4b0781b1 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 @@ -243,21 +243,24 @@ class UnivocityParser( () => getPartialResult(), new RuntimeException("Malformed CSV record")) } else { - try { - // When the length of the returned tokens is identical to the length of the parsed schema, - // we just need to convert the tokens that correspond to the required columns. - var i = 0 - while (i < requiredSchema.length) { + // When the length of the returned tokens is identical to the length of the parsed schema, + // we just need to convert the tokens that correspond to the required columns. + var badRecordException: Option[Throwable] = None + var i = 0 + while (i < requiredSchema.length) { + try { row(i) = valueConverters(i).apply(getToken(tokens, i)) - i += 1 + } catch { + case NonFatal(e) => + badRecordException = badRecordException.orElse(Some(e)) } + i += 1 + } + + if (badRecordException.isEmpty) { row - } catch { - case NonFatal(e) => - // For corrupted records with the number of tokens same as the schema, - // CSV reader doesn't support partial results. All fields other than the field - // configured by `columnNameOfCorruptRecord` are set to `null`. - throw BadRecordException(() => getCurrentInput, () => None, e) + } else { + throw BadRecordException(() => getCurrentInput, () => Some(row), badRecordException.get) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala index 76745b11c84c..4baf052bfe56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala @@ -33,26 +33,21 @@ class FailureSafeParser[IN]( private val corruptFieldIndex = schema.getFieldIndex(columnNameOfCorruptRecord) private val actualSchema = StructType(schema.filterNot(_.name == columnNameOfCorruptRecord)) private val resultRow = new GenericInternalRow(schema.length) - private val nullResult = new GenericInternalRow(schema.length) // This function takes 2 parameters: an optional partial result, and the bad record. If the given // schema doesn't contain a field for corrupted record, we just return the partial result or a // row with all fields null. If the given schema contains a field for corrupted record, we will // set the bad record to this field, and set other fields according to the partial result or null. private val toResultRow: (Option[InternalRow], () => UTF8String) => InternalRow = { - if (corruptFieldIndex.isDefined) { - (row, badRecord) => { - var i = 0 - while (i < actualSchema.length) { - val from = actualSchema(i) - resultRow(schema.fieldIndex(from.name)) = row.map(_.get(i, from.dataType)).orNull - i += 1 - } - resultRow(corruptFieldIndex.get) = badRecord() - resultRow + (row, badRecord) => { + var i = 0 + while (i < actualSchema.length) { + val from = actualSchema(i) + resultRow(schema.fieldIndex(from.name)) = row.map(_.get(i, from.dataType)).orNull + i += 1 } - } else { - (row, _) => row.getOrElse(nullResult) + corruptFieldIndex.foreach(index => resultRow(index) = badRecord()) + resultRow } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index 1c359ce1d201..537d13b1bc8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -60,7 +60,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSQLContext { "mode" -> "Permissive", "columnNameOfCorruptRecord" -> columnNameOfCorruptRecord))) checkAnswer(df2, Seq( - Row(Row(null, null, "0,2013-111-11 12:13:14")), + Row(Row(0, null, "0,2013-111-11 12:13:14")), Row(Row(1, java.sql.Date.valueOf("1983-08-04"), null)))) } 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 bc950f2418d3..c9273193b642 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 @@ -1116,7 +1116,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te .schema(schema) .csv(testFile(valueMalformedFile)) checkAnswer(df1, - Row(null, null) :: + Row(0, null) :: Row(1, java.sql.Date.valueOf("1983-08-04")) :: Nil) @@ -1131,7 +1131,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te .schema(schemaWithCorrField1) .csv(testFile(valueMalformedFile)) checkAnswer(df2, - Row(null, null, "0,2013-111-11 12:13:14") :: + Row(0, null, "0,2013-111-11 12:13:14") :: Row(1, java.sql.Date.valueOf("1983-08-04"), null) :: Nil) @@ -1148,7 +1148,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te .schema(schemaWithCorrField2) .csv(testFile(valueMalformedFile)) checkAnswer(df3, - Row(null, "0,2013-111-11 12:13:14", null) :: + Row(0, "0,2013-111-11 12:13:14", null) :: Row(1, null, java.sql.Date.valueOf("1983-08-04")) :: Nil) From b569ba53f4b650c03bd11def7c7f7589ceff61eb Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 3 Dec 2018 19:53:45 +0800 Subject: [PATCH 0052/1072] [SPARK-26230][SQL] FileIndex: if case sensitive, validate partitions with original column names ## What changes were proposed in this pull request? Partition column name is required to be unique under the same directory. The following paths are invalid partitioned directory: ``` hdfs://host:9000/path/a=1 hdfs://host:9000/path/b=2 ``` If case sensitive, the following paths should be invalid too: ``` hdfs://host:9000/path/a=1 hdfs://host:9000/path/A=2 ``` Since column 'a' and 'A' are different, and it is wrong to use either one as the column name in partition schema. Also, there is a `TODO` comment in the code. Currently the Spark doesn't validate such case when `CASE_SENSITIVE` enabled. This PR is to resolve the problem. ## How was this patch tested? Add unit test Closes #23186 from gengliangwang/SPARK-26230. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../datasources/PartitioningUtils.scala | 14 +++++--- .../datasources/FileIndexSuite.scala | 32 ++++++++++++++++++- 2 files changed, 40 insertions(+), 6 deletions(-) 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 9d2c9ba0c1a5..d66cb09bda0c 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 @@ -155,7 +155,8 @@ object PartitioningUtils { "root directory of the table. If there are multiple root directories, " + "please load them separately and then union them.") - val resolvedPartitionValues = resolvePartitions(pathsWithPartitionValues, timeZone) + val resolvedPartitionValues = + resolvePartitions(pathsWithPartitionValues, caseSensitive, timeZone) // Creates the StructType which represents the partition columns. val fields = { @@ -345,15 +346,18 @@ object PartitioningUtils { */ def resolvePartitions( pathsWithPartitionValues: Seq[(Path, PartitionValues)], + caseSensitive: Boolean, timeZone: TimeZone): Seq[PartitionValues] = { if (pathsWithPartitionValues.isEmpty) { Seq.empty } else { - // TODO: Selective case sensitivity. - val distinctPartColNames = - pathsWithPartitionValues.map(_._2.columnNames.map(_.toLowerCase())).distinct + val partColNames = if (caseSensitive) { + pathsWithPartitionValues.map(_._2.columnNames) + } else { + pathsWithPartitionValues.map(_._2.columnNames.map(_.toLowerCase())) + } assert( - distinctPartColNames.size == 1, + partColNames.distinct.size == 1, listConflictingPartitionColumns(pathsWithPartitionValues)) // Resolves possible type conflicts for each column 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 fdb0511f01a2..ec552f7ddf47 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 @@ -52,7 +52,7 @@ class FileIndexSuite extends SharedSQLContext { test("SPARK-26188: don't infer data types of partition columns if user specifies schema") { withTempDir { dir => - val partitionDirectory = new File(dir, s"a=4d") + val partitionDirectory = new File(dir, "a=4d") partitionDirectory.mkdir() val file = new File(partitionDirectory, "text.txt") stringToFile(file, "text") @@ -65,6 +65,36 @@ class FileIndexSuite extends SharedSQLContext { } } + test("SPARK-26230: if case sensitive, validate partitions with original column names") { + withTempDir { dir => + val partitionDirectory = new File(dir, "a=1") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val partitionDirectory2 = new File(dir, "A=2") + partitionDirectory2.mkdir() + val file2 = new File(partitionDirectory2, "text.txt") + stringToFile(file2, "text") + val path = new Path(dir.getCanonicalPath) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, None) + val partitionValues = fileIndex.partitionSpec().partitions.map(_.values) + assert(partitionValues.length == 2) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val msg = intercept[AssertionError] { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, None) + fileIndex.partitionSpec() + }.getMessage + assert(msg.contains("Conflicting partition column names detected")) + assert("Partition column name list #[0-1]: A".r.findFirstIn(msg).isDefined) + assert("Partition column name list #[0-1]: a".r.findFirstIn(msg).isDefined) + } + } + } + test("InMemoryFileIndex: input paths are converted to qualified paths") { withTempDir { dir => val file = new File(dir, "text.txt") From eebb940edb0c89eef88e1deb0fc0ae1c7a24bc3d Mon Sep 17 00:00:00 2001 From: pgandhi Date: Mon, 3 Dec 2018 07:53:21 -0600 Subject: [PATCH 0053/1072] [SPARK-26253][WEBUI] Task Summary Metrics Table on Stage Page shows empty table when no data is present Task Summary Metrics Table on Stage Page shows empty table when no data is present instead of showing a message. ## What changes were proposed in this pull request? Added a custom message to show on the task summary metrics table as well as executor summary table when no data is present. ## How was this patch tested? **Before:** ![49335550-29277d00-f615-11e8-8e62-a953e76bcebf](https://user-images.githubusercontent.com/22228190/49361520-425a2780-f702-11e8-8df4-08862ab6ceb8.png) **After:** screen shot 2018-12-03 at 1 56 09 pm Closes #23205 from pgandhi999/SPARK-26253. Authored-by: pgandhi Signed-off-by: Sean Owen --- .../resources/org/apache/spark/ui/static/stagepage.js | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 4c83ec7e95ab..564467487e84 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -221,7 +221,10 @@ function createDataTableForTaskSummaryMetricsTable(taskSummaryMetricsTable) { "searching": false, "order": [[0, "asc"]], "bSort": false, - "bAutoWidth": false + "bAutoWidth": false, + "oLanguage": { + "sEmptyTable": "No tasks have reported metrics yet" + } }; taskSummaryMetricsDataTable = $(taskMetricsTable).DataTable(taskConf); } @@ -426,7 +429,10 @@ $(document).ready(function () { } ], "order": [[0, "asc"]], - "bAutoWidth": false + "bAutoWidth": false, + "oLanguage": { + "sEmptyTable": "No data to show yet" + } } var executorSummaryTableSelector = $("#summary-executor-table").DataTable(executorSummaryConf); From 8534d753ecb21ea64ffbaefb5eaca38ba0464c6d Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 3 Dec 2018 23:54:26 +0800 Subject: [PATCH 0054/1072] [SPARK-26181][SQL] the `hasMinMaxStats` method of `ColumnStatsMap` is not correct ## What changes were proposed in this pull request? For now the `hasMinMaxStats` will return the same as `hasCountStats`, which is obviously not as expected. ## How was this patch tested? Existing tests. Closes #23152 from adrian-wang/minmaxstats. Authored-by: Daoyuan Wang Signed-off-by: Wenchen Fan --- .../statsEstimation/FilterEstimation.scala | 14 +++++++--- .../FilterEstimationSuite.scala | 27 +++++++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 14 ++++++++++ 3 files changed, 52 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala index 5a3eeefaedb1..2c5beef43f52 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala @@ -670,6 +670,14 @@ case class FilterEstimation(plan: Filter) extends Logging { logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft) return None case _ => + if (!colStatsMap.hasMinMaxStats(attrLeft)) { + logDebug("[CBO] No min/max statistics for " + attrLeft) + return None + } + if (!colStatsMap.hasMinMaxStats(attrRight)) { + logDebug("[CBO] No min/max statistics for " + attrRight) + return None + } } val colStatLeft = colStatsMap(attrLeft) @@ -879,13 +887,13 @@ case class ColumnStatsMap(originalMap: AttributeMap[ColumnStat]) { } def hasCountStats(a: Attribute): Boolean = - get(a).map(_.hasCountStats).getOrElse(false) + get(a).exists(_.hasCountStats) def hasDistinctCount(a: Attribute): Boolean = - get(a).map(_.distinctCount.isDefined).getOrElse(false) + get(a).exists(_.distinctCount.isDefined) def hasMinMaxStats(a: Attribute): Boolean = - get(a).map(_.hasCountStats).getOrElse(false) + get(a).exists(_.hasMinMaxStats) /** * Gets column stat for the given attribute. Prefer the column stat in updatedMap than that in diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala index 47bfa6256958..b0a47e783512 100755 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.plans.LeftOuter import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.{ColumnStatsMap, FilterEstimation} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ @@ -821,6 +822,32 @@ class FilterEstimationSuite extends StatsEstimationTestBase { expectedRowCount = 3) } + test("ColumnStatsMap tests") { + val attrNoDistinct = AttributeReference("att_without_distinct", IntegerType)() + val attrNoCount = AttributeReference("att_without_count", BooleanType)() + val attrNoMinMax = AttributeReference("att_without_min_max", DateType)() + val colStatNoDistinct = ColumnStat(distinctCount = None, min = Some(1), max = Some(10), + nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) + val colStatNoCount = ColumnStat(distinctCount = Some(2), min = Some(false), max = Some(true), + nullCount = None, avgLen = Some(1), maxLen = Some(1)) + val colStatNoMinMax = ColumnStat(distinctCount = Some(1), min = None, max = None, + nullCount = Some(1), avgLen = None, maxLen = None) + val columnStatsMap = ColumnStatsMap(AttributeMap(Seq( + attrNoDistinct -> colStatNoDistinct, + attrNoCount -> colStatNoCount, + attrNoMinMax -> colStatNoMinMax + ))) + assert(!columnStatsMap.hasDistinctCount(attrNoDistinct)) + assert(columnStatsMap.hasDistinctCount(attrNoCount)) + assert(columnStatsMap.hasDistinctCount(attrNoMinMax)) + assert(!columnStatsMap.hasCountStats(attrNoDistinct)) + assert(!columnStatsMap.hasCountStats(attrNoCount)) + assert(columnStatsMap.hasCountStats(attrNoMinMax)) + assert(columnStatsMap.hasMinMaxStats(attrNoDistinct)) + assert(columnStatsMap.hasMinMaxStats(attrNoCount)) + assert(!columnStatsMap.hasMinMaxStats(attrNoMinMax)) + } + private def childStatsTestPlan(outList: Seq[Attribute], tableRowCount: BigInt): StatsTestPlan = { StatsTestPlan( outputList = outList, 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 dfcde8cc0d39..fab2a27cdef1 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 @@ -2276,4 +2276,18 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + + test("SPARK-26181 hasMinMaxStats method of ColumnStatsMap is not correct") { + withSQLConf(SQLConf.CBO_ENABLED.key -> "true") { + withTable("all_null") { + sql("create table all_null (attr1 int, attr2 int)") + sql("insert into all_null values (null, null)") + sql("analyze table all_null compute statistics for columns attr1, attr2") + // check if the stats can be calculated without Cast exception. + sql("select * from all_null where attr1 < 1").queryExecution.stringWithStats + sql("select * from all_null where attr1 < attr2").queryExecution.stringWithStats + } + } + } + } From 6e4e70fe7bc3e103b8538748511261bb43cf3548 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 3 Dec 2018 10:02:15 -0600 Subject: [PATCH 0055/1072] [SPARK-26235][CORE] Change log level for ClassNotFoundException/NoClassDefFoundError in SparkSubmit to Error ## What changes were proposed in this pull request? In my local setup, I set log4j root category as ERROR (https://stackoverflow.com/questions/27781187/how-to-stop-info-messages-displaying-on-spark-console , first item show up if we google search "set spark log level".) When I run such command ``` spark-submit --class foo bar.jar ``` Nothing shows up, and the script exits. After quick investigation, I think the log level for ClassNotFoundException/NoClassDefFoundError in SparkSubmit should be ERROR instead of WARN. Since the whole process exit because of the exception/error. Before https://github.com/apache/spark/pull/20925, the message is not controlled by `log4j.rootCategory`. ## How was this patch tested? Manual check. Closes #23189 from gengliangwang/changeLogLevel. Authored-by: Gengliang Wang Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/deploy/SparkSubmit.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 324f6f8894d3..d4055cb6c585 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -813,14 +813,14 @@ private[spark] class SparkSubmit extends Logging { mainClass = Utils.classForName(childMainClass) } catch { case e: ClassNotFoundException => - logWarning(s"Failed to load $childMainClass.", e) + logError(s"Failed to load class $childMainClass.") if (childMainClass.contains("thriftserver")) { logInfo(s"Failed to load main class $childMainClass.") logInfo("You need to build Spark with -Phive and -Phive-thriftserver.") } throw new SparkUserAppException(CLASS_NOT_FOUND_EXIT_STATUS) case e: NoClassDefFoundError => - logWarning(s"Failed to load $childMainClass: ${e.getMessage()}") + logError(s"Failed to load $childMainClass: ${e.getMessage()}") if (e.getMessage.contains("org/apache/hadoop/hive")) { logInfo(s"Failed to load hive class.") logInfo("You need to build Spark with -Phive and -Phive-thriftserver.") @@ -915,6 +915,8 @@ object SparkSubmit extends CommandLineUtils with Logging { override protected def logInfo(msg: => String): Unit = self.logInfo(msg) override protected def logWarning(msg: => String): Unit = self.logWarning(msg) + + override protected def logError(msg: => String): Unit = self.logError(msg) } } @@ -922,6 +924,8 @@ object SparkSubmit extends CommandLineUtils with Logging { override protected def logWarning(msg: => String): Unit = printMessage(s"Warning: $msg") + override protected def logError(msg: => String): Unit = printMessage(s"Error: $msg") + override def doSubmit(args: Array[String]): Unit = { try { super.doSubmit(args) From 5e5b9f2ee0b4d8470197b404906fbd245c28f8ac Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Mon, 3 Dec 2018 10:03:51 -0600 Subject: [PATCH 0056/1072] [SPARK-26177] Config change followup to [] Automated formatting for Scala code Let's keep this open for a while to see if other configuration tweaks are suggested ## What changes were proposed in this pull request? Formatting configuration changes following up https://github.com/apache/spark/pull/23148 ## How was this patch tested? ./dev/scalafmt Closes #23182 from koeninger/scalafmt-config. Authored-by: cody koeninger Signed-off-by: Sean Owen --- dev/.scalafmt.conf | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dev/.scalafmt.conf b/dev/.scalafmt.conf index def67e026982..9a8813e3b3ee 100644 --- a/dev/.scalafmt.conf +++ b/dev/.scalafmt.conf @@ -19,6 +19,10 @@ align = none align.openParenDefnSite = false align.openParenCallSite = false align.tokens = [] +optIn = { + configStyleArguments = false +} +danglingParentheses = false docstrings = JavaDoc maxColumn = 98 From 04046e5432acb1132fa567f2230723bc1a92a482 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 4 Dec 2018 00:05:15 +0800 Subject: [PATCH 0057/1072] [SPARK-25498][SQL] InterpretedMutableProjection should handle UnsafeRow ## What changes were proposed in this pull request? Since `AggregationIterator` uses `MutableProjection` for `UnsafeRow`, `InterpretedMutableProjection` needs to handle `UnsafeRow` as buffer internally for fixed-length types only. ## How was this patch tested? Run 'SQLQueryTestSuite' with the interpreted mode. Closes #22512 from maropu/InterpreterTest. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/InternalRow.scala | 22 +++++ .../InterpretedMutableProjection.scala | 23 +++++- .../expressions/ExpressionEvalHelper.scala | 11 +++ .../expressions/MutableProjectionSuite.scala | 81 +++++++++++++++++++ .../expressions/UnsafeRowConverterSuite.scala | 15 +--- .../sql-tests/inputs/change-column.sql | 1 + .../test/resources/sql-tests/inputs/udaf.sql | 3 + .../sql-tests/results/change-column.sql.out | 10 ++- .../resources/sql-tests/results/udaf.sql.out | 18 ++++- .../apache/spark/sql/SQLQueryTestSuite.scala | 27 ++++++- 10 files changed, 192 insertions(+), 19 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala index e49c10be6be4..bdab407688a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala @@ -157,4 +157,26 @@ object InternalRow { getValueNullSafe } } + + /** + * Returns a writer for an `InternalRow` with given data type. + */ + def getWriter(ordinal: Int, dt: DataType): (InternalRow, Any) => Unit = dt match { + case BooleanType => (input, v) => input.setBoolean(ordinal, v.asInstanceOf[Boolean]) + case ByteType => (input, v) => input.setByte(ordinal, v.asInstanceOf[Byte]) + case ShortType => (input, v) => input.setShort(ordinal, v.asInstanceOf[Short]) + case IntegerType | DateType => (input, v) => input.setInt(ordinal, v.asInstanceOf[Int]) + case LongType | TimestampType => (input, v) => input.setLong(ordinal, v.asInstanceOf[Long]) + case FloatType => (input, v) => input.setFloat(ordinal, v.asInstanceOf[Float]) + case DoubleType => (input, v) => input.setDouble(ordinal, v.asInstanceOf[Double]) + case DecimalType.Fixed(precision, _) => + (input, v) => input.setDecimal(ordinal, v.asInstanceOf[Decimal], precision) + case udt: UserDefinedType[_] => getWriter(ordinal, udt.sqlType) + case NullType => (input, _) => input.setNullAt(ordinal) + case StringType => (input, v) => input.update(ordinal, v.asInstanceOf[UTF8String].copy()) + case _: StructType => (input, v) => input.update(ordinal, v.asInstanceOf[InternalRow].copy()) + case _: ArrayType => (input, v) => input.update(ordinal, v.asInstanceOf[ArrayData].copy()) + case _: MapType => (input, v) => input.update(ordinal, v.asInstanceOf[MapData].copy()) + case _ => (input, v) => input.update(ordinal, v) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala index 0654108cea28..122a564da61b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala @@ -49,10 +49,31 @@ class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mutable def currentValue: InternalRow = mutableRow override def target(row: InternalRow): MutableProjection = { + // If `mutableRow` is `UnsafeRow`, `MutableProjection` accepts fixed-length types only + require(!row.isInstanceOf[UnsafeRow] || + validExprs.forall { case (e, _) => UnsafeRow.isFixedLength(e.dataType) }, + "MutableProjection cannot use UnsafeRow for output data types: " + + validExprs.map(_._1.dataType).filterNot(UnsafeRow.isFixedLength) + .map(_.catalogString).mkString(", ")) mutableRow = row this } + private[this] val fieldWriters: Array[Any => Unit] = validExprs.map { case (e, i) => + val writer = InternalRow.getWriter(i, e.dataType) + if (!e.nullable) { + (v: Any) => writer(mutableRow, v) + } else { + (v: Any) => { + if (v == null) { + mutableRow.setNullAt(i) + } else { + writer(mutableRow, v) + } + } + } + }.toArray + override def apply(input: InternalRow): InternalRow = { var i = 0 while (i < validExprs.length) { @@ -64,7 +85,7 @@ class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mutable i = 0 while (i < validExprs.length) { val (_, ordinal) = validExprs(i) - mutableRow(ordinal) = buffer(ordinal) + fieldWriters(i)(buffer(ordinal)) i += 1 } mutableRow diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index eb33325d0b31..a7282e1b1cad 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -456,4 +456,15 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa diff < eps * math.min(absX, absY) } } + + def testBothCodegenAndInterpreted(name: String)(f: => Unit): Unit = { + val modes = Seq(CodegenObjectFactoryMode.CODEGEN_ONLY, CodegenObjectFactoryMode.NO_CODEGEN) + for (fallbackMode <- modes) { + test(s"$name with $fallbackMode") { + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { + f + } + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala new file mode 100644 index 000000000000..2db1c3b98819 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval + +class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { + + val fixedLengthTypes = Array[DataType]( + BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, + DateType, TimestampType) + + val variableLengthTypes = Array( + StringType, DecimalType.defaultConcreteType, CalendarIntervalType, BinaryType, + ArrayType(StringType), MapType(IntegerType, StringType), + StructType.fromDDL("a INT, b STRING"), ObjectType(classOf[java.lang.Integer])) + + def createMutableProjection(dataTypes: Array[DataType]): MutableProjection = { + MutableProjection.create(dataTypes.zipWithIndex.map(x => BoundReference(x._2, x._1, true))) + } + + testBothCodegenAndInterpreted("fixed-length types") { + val inputRow = InternalRow.fromSeq(Seq(true, 3.toByte, 15.toShort, -83, 129L, 1.0f, 5.0, 1, 2L)) + val proj = createMutableProjection(fixedLengthTypes) + assert(proj(inputRow) === inputRow) + } + + testBothCodegenAndInterpreted("unsafe buffer") { + val inputRow = InternalRow.fromSeq(Seq(false, 1.toByte, 9.toShort, -18, 53L, 3.2f, 7.8, 4, 9L)) + val numBytes = UnsafeRow.calculateBitSetWidthInBytes(fixedLengthTypes.length) + val unsafeBuffer = UnsafeRow.createFromByteArray(numBytes, fixedLengthTypes.length) + val proj = createMutableProjection(fixedLengthTypes) + val projUnsafeRow = proj.target(unsafeBuffer)(inputRow) + assert(FromUnsafeProjection.apply(fixedLengthTypes)(projUnsafeRow) === inputRow) + } + + testBothCodegenAndInterpreted("variable-length types") { + val proj = createMutableProjection(variableLengthTypes) + val scalaValues = Seq("abc", BigDecimal(10), CalendarInterval.fromString("interval 1 day"), + Array[Byte](1, 2), Array("123", "456"), Map(1 -> "a", 2 -> "b"), Row(1, "a"), + new java.lang.Integer(5)) + val inputRow = InternalRow.fromSeq(scalaValues.zip(variableLengthTypes).map { + case (v, dataType) => CatalystTypeConverters.createToCatalystConverter(dataType)(v) + }) + val projRow = proj(inputRow) + variableLengthTypes.zipWithIndex.foreach { case (dataType, index) => + val toScala = CatalystTypeConverters.createToScalaConverter(dataType) + assert(toScala(projRow.get(index, dataType)) === toScala(inputRow.get(index, dataType))) + } + } + + test("unsupported types for unsafe buffer") { + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString) { + val proj = createMutableProjection(Array(StringType)) + val errMsg = intercept[IllegalArgumentException] { + proj.target(new UnsafeRow(1)) + }.getMessage + assert(errMsg.contains("MutableProjection cannot use UnsafeRow for output data types:")) + } + } +} 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 5a646d9a850a..268372b5d050 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 @@ -26,26 +26,15 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, LongType, _} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.UTF8String -class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestBase { +class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestBase + with ExpressionEvalHelper { private def roundedSize(size: Int) = ByteArrayMethods.roundNumberOfBytesToNearestWord(size) - private def testBothCodegenAndInterpreted(name: String)(f: => Unit): Unit = { - val modes = Seq(CodegenObjectFactoryMode.CODEGEN_ONLY, CodegenObjectFactoryMode.NO_CODEGEN) - for (fallbackMode <- modes) { - test(s"$name with $fallbackMode") { - withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { - f - } - } - } - } - testBothCodegenAndInterpreted("basic conversion with only primitive types") { val factory = UnsafeProjection val fieldTypes: Array[DataType] = Array(LongType, LongType, IntegerType) diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 2909024e4c9f..6f5ac221ce79 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -54,3 +54,4 @@ ALTER TABLE partition_table CHANGE COLUMN c c INT COMMENT 'this is column C'; -- DROP TEST TABLE DROP TABLE test_change; DROP TABLE partition_table; +DROP VIEW global_temp.global_temp_view; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql index 2183ba23afc3..58613a1325df 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql @@ -11,3 +11,6 @@ SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1; CREATE FUNCTION udaf1 AS 'test.non.existent.udaf'; SELECT default.udaf1(int_col1) as udaf1 from t1; + +DROP FUNCTION myDoubleAvg; +DROP FUNCTION udaf1; diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index ff1ecbcc44c2..114617873af4 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 33 +-- Number of queries: 34 -- !query 0 @@ -313,3 +313,11 @@ DROP TABLE partition_table struct<> -- !query 32 output + + +-- !query 33 +DROP VIEW global_temp.global_temp_view +-- !query 33 schema +struct<> +-- !query 33 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out index 87824ab81cdf..f4455bb71757 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 6 +-- Number of queries: 8 -- !query 0 @@ -52,3 +52,19 @@ struct<> -- !query 5 output org.apache.spark.sql.AnalysisException Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 7 + + +-- !query 6 +DROP FUNCTION myDoubleAvg +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +DROP FUNCTION udaf1 +-- !query 7 schema +struct<> +-- !query 7 output + 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 6ca3ac596e5f..fd180ce2380a 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 @@ -22,11 +22,13 @@ import java.util.{Locale, TimeZone} import scala.util.control.NonFatal +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation 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} import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeTableCommand} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType @@ -140,6 +142,12 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { val input = fileToString(new File(testCase.inputFile)) val (comments, code) = input.split("\n").partition(_.startsWith("--")) + + // Runs all the tests on both codegen-only and interpreter modes + val codegenConfigSets = Array(CODEGEN_ONLY, NO_CODEGEN).map { + case codegenFactoryMode => + Array(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenFactoryMode.toString) + } val configSets = { val configLines = comments.filter(_.startsWith("--SET")).map(_.substring(5)) val configs = configLines.map(_.split(",").map { confAndValue => @@ -148,12 +156,25 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { }) // When we are regenerating the golden files we don't need to run all the configs as they // all need to return the same result - if (regenerateGoldenFiles && configs.nonEmpty) { - configs.take(1) + if (regenerateGoldenFiles) { + if (configs.nonEmpty) { + configs.take(1) + } else { + Array.empty[Array[(String, String)]] + } } else { - configs + if (configs.nonEmpty) { + codegenConfigSets.flatMap { codegenConfig => + configs.map { config => + config ++ codegenConfig + } + } + } else { + codegenConfigSets + } } } + // List of SQL queries to run // note: this is not a robust way to split queries using semicolon, but works for now. val queries = code.mkString("\n").split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq From 0c2935b01def8a5f631851999d9c2d57b63763e6 Mon Sep 17 00:00:00 2001 From: Stavros Kontopoulos Date: Mon, 3 Dec 2018 09:02:47 -0800 Subject: [PATCH 0058/1072] [SPARK-25515][K8S] Adds a config option to keep executor pods for debugging ## What changes were proposed in this pull request? Keeps K8s executor resources present if case of failure or normal termination. Introduces a new boolean config option: `spark.kubernetes.deleteExecutors`, with default value set to true. The idea is to update Spark K8s backend structures but leave the resources around. The assumption is that since entries are not removed from the `removedExecutorsCache` we are immune to updates that refer to the the executor resources previously removed. The only delete operation not touched is the one in the `doKillExecutors` method. Reason is right now we dont support [blacklisting](https://issues.apache.org/jira/browse/SPARK-23485) and dynamic allocation with Spark on K8s. In both cases in the future we might want to handle these scenarios although its more complicated. More tests can be added if approach is approved. ## How was this patch tested? Manually by running a Spark job and verifying pods are not deleted. Closes #23136 from skonto/keep_pods. Authored-by: Stavros Kontopoulos Signed-off-by: Yinan Li --- docs/running-on-kubernetes.md | 7 +++++++ .../org/apache/spark/deploy/k8s/Config.scala | 7 +++++++ .../cluster/k8s/ExecutorPodsAllocator.scala | 15 ++++++++++----- .../k8s/ExecutorPodsLifecycleManager.scala | 8 ++++++-- .../k8s/KubernetesClusterSchedulerBackend.scala | 15 ++++++++++----- .../k8s/ExecutorPodsLifecycleManagerSuite.scala | 14 +++++++++++++- 6 files changed, 53 insertions(+), 13 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 5639253d52f5..3172b1bca8f0 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -944,6 +944,13 @@ specific to Spark on Kubernetes. spark.kubernetes.executor.podTemplateFile=/path/to/executor-pod-template.yaml` + + + + +
    spark.kubernetes.executor.deleteOnTerminationtrue + Specify whether executor pods should be deleted in case of failure or normal termination. +
    #### Pod template properties diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 1abf2901268f..e8bf16df190e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -282,6 +282,13 @@ private[spark] object Config extends Logging { val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." + val KUBERNETES_DELETE_EXECUTORS = + ConfigBuilder("spark.kubernetes.executor.deleteOnTermination") + .doc("If set to false then executor pods will not be deleted in case " + + "of failure or normal termination.") + .booleanConf + .createWithDefault(true) + val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 77bb9c3fcc9f..ef4cbdf162c6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -51,6 +51,8 @@ private[spark] class ExecutorPodsAllocator( private val kubernetesDriverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) + private val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) + private val driverPod = kubernetesDriverPodName .map(name => Option(kubernetesClient.pods() .withName(name) @@ -86,11 +88,14 @@ private[spark] class ExecutorPodsAllocator( s" cluster after $podCreationTimeout milliseconds despite the fact that a" + " previous allocation attempt tried to create it. The executor may have been" + " deleted but the application missed the deletion event.") - Utils.tryLogNonFatalError { - kubernetesClient - .pods() - .withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString) - .delete() + + if (shouldDeleteExecutors) { + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString) + .delete() + } } newlyCreatedExecutors -= execId } else { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index 77a1d6cfae3b..95e1ba8362a0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -30,7 +30,7 @@ import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.util.Utils private[spark] class ExecutorPodsLifecycleManager( - conf: SparkConf, + val conf: SparkConf, kubernetesClient: KubernetesClient, snapshotsStore: ExecutorPodsSnapshotsStore, // Use a best-effort to track which executors have been removed already. It's not generally @@ -43,6 +43,8 @@ private[spark] class ExecutorPodsLifecycleManager( private val eventProcessingInterval = conf.get(KUBERNETES_EXECUTOR_EVENT_PROCESSING_INTERVAL) + private lazy val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) + def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { snapshotsStore.addSubscriber(eventProcessingInterval) { onNewSnapshots(schedulerBackend, _) @@ -112,7 +114,9 @@ private[spark] class ExecutorPodsLifecycleManager( schedulerBackend: KubernetesClusterSchedulerBackend, execIdsRemovedInRound: mutable.Set[Long]): Unit = { removeExecutorFromSpark(schedulerBackend, podState, execId) - removeExecutorFromK8s(podState.pod) + if (shouldDeleteExecutors) { + removeExecutorFromK8s(podState.pod) + } execIdsRemovedInRound += execId } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index fa6dc2c479bb..6356b5864580 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -21,6 +21,7 @@ import java.util.concurrent.ExecutorService import io.fabric8.kubernetes.client.KubernetesClient import scala.concurrent.{ExecutionContext, Future} +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.rpc.{RpcAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} @@ -51,6 +52,8 @@ private[spark] class KubernetesClusterSchedulerBackend( private val initialExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(conf) + private val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) + // Allow removeExecutor to be accessible by ExecutorPodsLifecycleEventHandler private[k8s] def doRemoveExecutor(executorId: String, reason: ExecutorLossReason): Unit = { removeExecutor(executorId, reason) @@ -82,11 +85,13 @@ private[spark] class KubernetesClusterSchedulerBackend( pollEvents.stop() } - Utils.tryLogNonFatalError { - kubernetesClient.pods() - .withLabel(SPARK_APP_ID_LABEL, applicationId()) - .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) - .delete() + if (shouldDeleteExecutors) { + Utils.tryLogNonFatalError { + kubernetesClient.pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId()) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .delete() + } } Utils.tryLogNonFatalError { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala index 3995b2afe7c4..7411f8f9d69e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Matchers.any -import org.mockito.Mockito.{mock, times, verify, when} +import org.mockito.Mockito.{mock, never, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter @@ -30,6 +30,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.deploy.k8s.KubernetesUtils._ import org.apache.spark.scheduler.ExecutorExited @@ -100,6 +101,17 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) } + test("Keep executor pods in k8s if configured.") { + val failedPod = failedExecutorWithoutDeletion(1) + eventHandlerUnderTest.conf.set(Config.KUBERNETES_DELETE_EXECUTORS, false) + snapshotsStore.updatePod(failedPod) + snapshotsStore.notifySubscribers() + val msg = exitReasonMessage(1, failedPod) + val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) + verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) + verify(podOperations, never()).delete() + } + private def exitReasonMessage(failedExecutorId: Int, failedPod: Pod): String = { val reason = Option(failedPod.getStatus.getReason) val message = Option(failedPod.getStatus.getMessage) From 187bb7d008872e812aaa6590c89121bfa50e97d3 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 3 Dec 2018 13:54:09 -0800 Subject: [PATCH 0059/1072] [SPARK-25957][FOLLOWUP] Build python docker image in sbt build too. docker-image-tool.sh requires explicit argument to create the python image now; do that from the sbt integration tests target too. Closes #23172 from vanzin/SPARK-25957.followup. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- project/SparkBuild.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bb834bc483f1..a0946a9ad665 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -494,7 +494,13 @@ object KubernetesIntegrationTests { dockerBuild := { if (shouldBuildImage) { val dockerTool = s"$sparkHome/bin/docker-image-tool.sh" - val cmd = Seq(dockerTool, "-m", "-t", imageTag.value, "build") + val bindingsDir = s"$sparkHome/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings" + val cmd = Seq(dockerTool, "-m", + "-t", imageTag.value, + "-p", s"$bindingsDir/python/Dockerfile", + "-R", s"$bindingsDir/R/Dockerfile", + "build" + ) val ec = Process(cmd).! if (ec != 0) { throw new IllegalStateException(s"Process '${cmd.mkString(" ")}' exited with $ec.") From 518a3d10c87bb6d7d442eba7265fc026aa54473e Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 3 Dec 2018 14:03:10 -0800 Subject: [PATCH 0060/1072] [SPARK-26033][SPARK-26034][PYTHON][FOLLOW-UP] Small cleanup and deduplication in ml/mllib tests ## What changes were proposed in this pull request? This PR is a small follow up that puts some logic and functions into smaller scope and make it localized, and deduplicate. ## How was this patch tested? Manually tested. Jenkins tests as well. Closes #23200 from HyukjinKwon/followup-SPARK-26034-SPARK-26033. Authored-by: Hyukjin Kwon Signed-off-by: Bryan Cutler --- python/pyspark/ml/tests/test_linalg.py | 44 +++++++------ python/pyspark/mllib/tests/test_algorithms.py | 8 +-- python/pyspark/mllib/tests/test_linalg.py | 62 ++++++++----------- python/pyspark/testing/mllibutils.py | 5 -- 4 files changed, 51 insertions(+), 68 deletions(-) diff --git a/python/pyspark/ml/tests/test_linalg.py b/python/pyspark/ml/tests/test_linalg.py index 71cad5d7f5ad..995bc35e4ca8 100644 --- a/python/pyspark/ml/tests/test_linalg.py +++ b/python/pyspark/ml/tests/test_linalg.py @@ -20,25 +20,17 @@ from numpy import arange, array, array_equal, inf, ones, tile, zeros +from pyspark.serializers import PickleSerializer from pyspark.ml.linalg import DenseMatrix, DenseVector, MatrixUDT, SparseMatrix, SparseVector, \ Vector, VectorUDT, Vectors -from pyspark.testing.mllibutils import make_serializer, MLlibTestCase +from pyspark.testing.mllibutils import MLlibTestCase from pyspark.sql import Row -ser = make_serializer() - - -def _squared_distance(a, b): - if isinstance(a, Vector): - return a.squared_distance(b) - else: - return b.squared_distance(a) - - class VectorTests(MLlibTestCase): def _test_serialize(self, v): + ser = PickleSerializer() self.assertEqual(v, ser.loads(ser.dumps(v))) jvec = self.sc._jvm.org.apache.spark.ml.python.MLSerDe.loads(bytearray(ser.dumps(v))) nv = ser.loads(bytes(self.sc._jvm.org.apache.spark.ml.python.MLSerDe.dumps(jvec))) @@ -77,24 +69,30 @@ def test_dot(self): self.assertEqual(7.0, sv.dot(arr)) def test_squared_distance(self): + def squared_distance(a, b): + if isinstance(a, Vector): + return a.squared_distance(b) + else: + return b.squared_distance(a) + sv = SparseVector(4, {1: 1, 3: 2}) dv = DenseVector(array([1., 2., 3., 4.])) lst = DenseVector([4, 3, 2, 1]) lst1 = [4, 3, 2, 1] arr = pyarray.array('d', [0, 2, 1, 3]) narr = array([0, 2, 1, 3]) - self.assertEqual(15.0, _squared_distance(sv, dv)) - self.assertEqual(25.0, _squared_distance(sv, lst)) - self.assertEqual(20.0, _squared_distance(dv, lst)) - self.assertEqual(15.0, _squared_distance(dv, sv)) - self.assertEqual(25.0, _squared_distance(lst, sv)) - self.assertEqual(20.0, _squared_distance(lst, dv)) - self.assertEqual(0.0, _squared_distance(sv, sv)) - self.assertEqual(0.0, _squared_distance(dv, dv)) - self.assertEqual(0.0, _squared_distance(lst, lst)) - self.assertEqual(25.0, _squared_distance(sv, lst1)) - self.assertEqual(3.0, _squared_distance(sv, arr)) - self.assertEqual(3.0, _squared_distance(sv, narr)) + self.assertEqual(15.0, squared_distance(sv, dv)) + self.assertEqual(25.0, squared_distance(sv, lst)) + self.assertEqual(20.0, squared_distance(dv, lst)) + self.assertEqual(15.0, squared_distance(dv, sv)) + self.assertEqual(25.0, squared_distance(lst, sv)) + self.assertEqual(20.0, squared_distance(lst, dv)) + self.assertEqual(0.0, squared_distance(sv, sv)) + self.assertEqual(0.0, squared_distance(dv, dv)) + self.assertEqual(0.0, squared_distance(lst, lst)) + self.assertEqual(25.0, squared_distance(sv, lst1)) + self.assertEqual(3.0, squared_distance(sv, arr)) + self.assertEqual(3.0, squared_distance(sv, narr)) def test_hash(self): v1 = DenseVector([0.0, 1.0, 0.0, 5.5]) diff --git a/python/pyspark/mllib/tests/test_algorithms.py b/python/pyspark/mllib/tests/test_algorithms.py index cc3b64b1cb28..21a2d64087bc 100644 --- a/python/pyspark/mllib/tests/test_algorithms.py +++ b/python/pyspark/mllib/tests/test_algorithms.py @@ -26,10 +26,8 @@ from pyspark.mllib.fpm import FPGrowth from pyspark.mllib.recommendation import Rating from pyspark.mllib.regression import LabeledPoint -from pyspark.testing.mllibutils import make_serializer, MLlibTestCase - - -ser = make_serializer() +from pyspark.serializers import PickleSerializer +from pyspark.testing.mllibutils import MLlibTestCase class ListTests(MLlibTestCase): @@ -265,6 +263,7 @@ def test_regression(self): class ALSTests(MLlibTestCase): def test_als_ratings_serialize(self): + ser = PickleSerializer() r = Rating(7, 1123, 3.14) jr = self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.loads(bytearray(ser.dumps(r))) nr = ser.loads(bytes(self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.dumps(jr))) @@ -273,6 +272,7 @@ def test_als_ratings_serialize(self): self.assertAlmostEqual(r.rating, nr.rating, 2) def test_als_ratings_id_long_error(self): + ser = PickleSerializer() r = Rating(1205640308657491975, 50233468418, 1.0) # rating user id exceeds max int value, should fail when pickled self.assertRaises(Py4JJavaError, self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.loads, diff --git a/python/pyspark/mllib/tests/test_linalg.py b/python/pyspark/mllib/tests/test_linalg.py index d0ebd9bc3db7..f26e28d1744d 100644 --- a/python/pyspark/mllib/tests/test_linalg.py +++ b/python/pyspark/mllib/tests/test_linalg.py @@ -22,33 +22,18 @@ from numpy import array, array_equal, zeros, arange, tile, ones, inf import pyspark.ml.linalg as newlinalg +from pyspark.serializers import PickleSerializer from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector, \ DenseMatrix, SparseMatrix, Vectors, Matrices, MatrixUDT from pyspark.mllib.regression import LabeledPoint -from pyspark.testing.mllibutils import make_serializer, MLlibTestCase - -_have_scipy = False -try: - import scipy.sparse - _have_scipy = True -except: - # No SciPy, but that's okay, we'll skip those tests - pass - - -ser = make_serializer() - - -def _squared_distance(a, b): - if isinstance(a, Vector): - return a.squared_distance(b) - else: - return b.squared_distance(a) +from pyspark.testing.mllibutils import MLlibTestCase +from pyspark.testing.utils import have_scipy class VectorTests(MLlibTestCase): def _test_serialize(self, v): + ser = PickleSerializer() self.assertEqual(v, ser.loads(ser.dumps(v))) jvec = self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.loads(bytearray(ser.dumps(v))) nv = ser.loads(bytes(self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.dumps(jvec))) @@ -87,24 +72,30 @@ def test_dot(self): self.assertEqual(7.0, sv.dot(arr)) def test_squared_distance(self): + def squared_distance(a, b): + if isinstance(a, Vector): + return a.squared_distance(b) + else: + return b.squared_distance(a) + sv = SparseVector(4, {1: 1, 3: 2}) dv = DenseVector(array([1., 2., 3., 4.])) lst = DenseVector([4, 3, 2, 1]) lst1 = [4, 3, 2, 1] arr = pyarray.array('d', [0, 2, 1, 3]) narr = array([0, 2, 1, 3]) - self.assertEqual(15.0, _squared_distance(sv, dv)) - self.assertEqual(25.0, _squared_distance(sv, lst)) - self.assertEqual(20.0, _squared_distance(dv, lst)) - self.assertEqual(15.0, _squared_distance(dv, sv)) - self.assertEqual(25.0, _squared_distance(lst, sv)) - self.assertEqual(20.0, _squared_distance(lst, dv)) - self.assertEqual(0.0, _squared_distance(sv, sv)) - self.assertEqual(0.0, _squared_distance(dv, dv)) - self.assertEqual(0.0, _squared_distance(lst, lst)) - self.assertEqual(25.0, _squared_distance(sv, lst1)) - self.assertEqual(3.0, _squared_distance(sv, arr)) - self.assertEqual(3.0, _squared_distance(sv, narr)) + self.assertEqual(15.0, squared_distance(sv, dv)) + self.assertEqual(25.0, squared_distance(sv, lst)) + self.assertEqual(20.0, squared_distance(dv, lst)) + self.assertEqual(15.0, squared_distance(dv, sv)) + self.assertEqual(25.0, squared_distance(lst, sv)) + self.assertEqual(20.0, squared_distance(lst, dv)) + self.assertEqual(0.0, squared_distance(sv, sv)) + self.assertEqual(0.0, squared_distance(dv, dv)) + self.assertEqual(0.0, squared_distance(lst, lst)) + self.assertEqual(25.0, squared_distance(sv, lst1)) + self.assertEqual(3.0, squared_distance(sv, arr)) + self.assertEqual(3.0, squared_distance(sv, narr)) def test_hash(self): v1 = DenseVector([0.0, 1.0, 0.0, 5.5]) @@ -466,7 +457,7 @@ def test_infer_schema(self): raise ValueError("Expected a matrix but got type %r" % type(m)) -@unittest.skipIf(not _have_scipy, "SciPy not installed") +@unittest.skipIf(not have_scipy, "SciPy not installed") class SciPyTests(MLlibTestCase): """ @@ -476,6 +467,8 @@ class SciPyTests(MLlibTestCase): def test_serialize(self): from scipy.sparse import lil_matrix + + ser = PickleSerializer() lil = lil_matrix((4, 1)) lil[1, 0] = 1 lil[3, 0] = 2 @@ -621,13 +614,10 @@ def test_regression(self): if __name__ == "__main__": from pyspark.mllib.tests.test_linalg import * - if not _have_scipy: - print("NOTE: Skipping SciPy tests as it does not seem to be installed") + try: import xmlrunner testRunner = xmlrunner.XMLTestRunner(output='target/test-reports') except ImportError: testRunner = None unittest.main(testRunner=testRunner, verbosity=2) - if not _have_scipy: - print("NOTE: SciPy tests were skipped as it does not seem to be installed") diff --git a/python/pyspark/testing/mllibutils.py b/python/pyspark/testing/mllibutils.py index 25f1bba8d37a..c09fb50482e4 100644 --- a/python/pyspark/testing/mllibutils.py +++ b/python/pyspark/testing/mllibutils.py @@ -18,14 +18,9 @@ import unittest from pyspark import SparkContext -from pyspark.serializers import PickleSerializer from pyspark.sql import SparkSession -def make_serializer(): - return PickleSerializer() - - class MLlibTestCase(unittest.TestCase): def setUp(self): self.sc = SparkContext('local[4]', "MLlib tests") From a24e1a126c55fc06f5867c0e5e5b0ee71201e018 Mon Sep 17 00:00:00 2001 From: Stavros Kontopoulos Date: Mon, 3 Dec 2018 14:57:18 -0800 Subject: [PATCH 0061/1072] [SPARK-26256][K8S] Fix labels for pod deletion ## What changes were proposed in this pull request? Adds proper labels when deleting executor pods. ## How was this patch tested? Manually with tests. Closes #23209 from skonto/fix-deletion-labels. Authored-by: Stavros Kontopoulos Signed-off-by: Marcelo Vanzin --- .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 2 ++ .../cluster/k8s/ExecutorPodsAllocatorSuite.scala | 10 +++++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index ef4cbdf162c6..2f0f949566d6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -93,6 +93,8 @@ private[spark] class ExecutorPodsAllocator( Utils.tryLogNonFatalError { kubernetesClient .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString) .delete() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index ddf9f67a0727..303e24b8f497 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -138,7 +138,15 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.notifySubscribers() snapshotsStore.replaceSnapshot(Seq.empty[Pod]) waitForExecutorPodsClock.setTime(podCreationTimeout + 1) - when(podOperations.withLabel(SPARK_EXECUTOR_ID_LABEL, "1")).thenReturn(labeledPods) + when(podOperations + .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(podOperations) + when(podOperations + withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_EXECUTOR_ID_LABEL, "1")) + .thenReturn(labeledPods) snapshotsStore.notifySubscribers() verify(labeledPods).delete() verify(podOperations).create(podWithAttachedContainerForId(2)) From 0889fbaf959e25ebb79e691692a02a93962727d0 Mon Sep 17 00:00:00 2001 From: Qi Shao Date: Mon, 3 Dec 2018 15:36:41 -0800 Subject: [PATCH 0062/1072] [SPARK-26083][K8S] Add Copy pyspark into corresponding dir cmd in pyspark Dockerfile When I try to run `./bin/pyspark` cmd in a pod in Kubernetes(image built without change from pyspark Dockerfile), I'm getting an error: ``` $SPARK_HOME/bin/pyspark --deploy-mode client --master k8s://https://$KUBERNETES_SERVICE_HOST:$KUBERNETES_SERVICE_PORT_HTTPS ... Python 2.7.15 (default, Aug 22 2018, 13:24:18) [GCC 6.4.0] on linux2 Type "help", "copyright", "credits" or "license" for more information. Could not open PYTHONSTARTUP IOError: [Errno 2] No such file or directory: '/opt/spark/python/pyspark/shell.py' ``` This is because `pyspark` folder doesn't exist under `/opt/spark/python/` ## What changes were proposed in this pull request? Added `COPY python/pyspark ${SPARK_HOME}/python/pyspark` to pyspark Dockerfile to resolve issue above. ## How was this patch tested? Google Kubernetes Engine Closes #23037 from AzureQ/master. Authored-by: Qi Shao Signed-off-by: Marcelo Vanzin --- bin/docker-image-tool.sh | 1 + .../docker/src/main/dockerfiles/spark/bindings/python/Dockerfile | 1 + 2 files changed, 2 insertions(+) diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index fbf9c9e448fd..4f66137eb1c7 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -107,6 +107,7 @@ function create_dev_build_context {( "$PYSPARK_CTX/kubernetes/dockerfiles" mkdir "$PYSPARK_CTX/python" cp -r "python/lib" "$PYSPARK_CTX/python/lib" + cp -r "python/pyspark" "$PYSPARK_CTX/python/pyspark" local R_CTX="$CTX_DIR/sparkr" mkdir -p "$R_CTX/kubernetes" diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile index de1a0617b1cc..36b91eb9a3aa 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -38,6 +38,7 @@ RUN apk add --no-cache python && \ # Removed the .cache to save space rm -r /root/.cache +COPY python/pyspark ${SPARK_HOME}/python/pyspark COPY python/lib ${SPARK_HOME}/python/lib ENV PYTHONPATH ${SPARK_HOME}/python/lib/pyspark.zip:${SPARK_HOME}/python/lib/py4j-*.zip From f7af4a1965b1052d3c77505ab1b660a294757bed Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 4 Dec 2018 12:14:38 +0800 Subject: [PATCH 0063/1072] [SPARK-25498][SQL][FOLLOW-UP] Return an empty config set when regenerating the golden files ## What changes were proposed in this pull request? This pr is to return an empty config set when regenerating the golden files in `SQLQueryTestSuite`. This is the follow-up of #22512. ## How was this patch tested? N/A Closes #23212 from maropu/SPARK-25498-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/SQLQueryTestSuite.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) 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 fd180ce2380a..cf4585bf7ac6 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 @@ -154,14 +154,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { val (conf, value) = confAndValue.span(_ != '=') conf.trim -> value.substring(1).trim }) - // When we are regenerating the golden files we don't need to run all the configs as they + // When we are regenerating the golden files, we don't need to set any config as they // all need to return the same result if (regenerateGoldenFiles) { - if (configs.nonEmpty) { - configs.take(1) - } else { - Array.empty[Array[(String, String)]] - } + Array.empty[Array[(String, String)]] } else { if (configs.nonEmpty) { codegenConfigSets.flatMap { codegenConfig => From b4dea313c45042e4094d14ebdeb8ad27be4cc695 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Mon, 3 Dec 2018 23:00:02 -0800 Subject: [PATCH 0064/1072] [SPARK-25573] Combine resolveExpression and resolve in the Analyzer ## What changes were proposed in this pull request? Currently in the Analyzer, we have two methods 1) Resolve 2)ResolveExpressions that are called at different code paths to resolve attributes, column ordinal and extract value expressions. ~~In this PR, we combine the two into one method to make sure, there is only one method that is tasked with resolving the attributes.~~ Update the description of the methods and use better names to make it easier to know when to make use of one method vs the other. ## How was this patch tested? Existing tests. Closes #22899 from dilipbiswal/SPARK-25573-final. Authored-by: Dilip Biswal Signed-off-by: gatorsmile --- .../sql/catalyst/analysis/Analyzer.scala | 97 +++++++++++++------ 1 file changed, 66 insertions(+), 31 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 b977fa07db5c..777053168a05 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 @@ -883,21 +883,38 @@ class Analyzer( } } - private def resolve(e: Expression, q: LogicalPlan): Expression = e match { - case f: LambdaFunction if !f.bound => f - case u @ UnresolvedAttribute(nameParts) => - // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = - withPosition(u) { - q.resolveChildren(nameParts, resolver) - .orElse(resolveLiteralFunction(nameParts, u, q)) - .getOrElse(u) - } - logDebug(s"Resolving $u to $result") - result - case UnresolvedExtractValue(child, fieldExpr) if child.resolved => - ExtractValue(child, fieldExpr, resolver) - case _ => e.mapChildren(resolve(_, q)) + /** + * Resolves the attribute and extract value expressions(s) by traversing the + * input expression in top down manner. The traversal is done in top-down manner as + * we need to skip over unbound lamda function expression. The lamda expressions are + * resolved in a different rule [[ResolveLambdaVariables]] + * + * Example : + * SELECT transform(array(1, 2, 3), (x, i) -> x + i)" + * + * In the case above, x and i are resolved as lamda variables in [[ResolveLambdaVariables]] + * + * Note : In this routine, the unresolved attributes are resolved from the input plan's + * children attributes. + */ + private def resolveExpressionTopDown(e: Expression, q: LogicalPlan): Expression = { + if (e.resolved) return e + e match { + case f: LambdaFunction if !f.bound => f + case u @ UnresolvedAttribute(nameParts) => + // Leave unchanged if resolution fails. Hopefully will be resolved next round. + val result = + withPosition(u) { + q.resolveChildren(nameParts, resolver) + .orElse(resolveLiteralFunction(nameParts, u, q)) + .getOrElse(u) + } + logDebug(s"Resolving $u to $result") + result + case UnresolvedExtractValue(child, fieldExpr) if child.resolved => + ExtractValue(child, fieldExpr, resolver) + case _ => e.mapChildren(resolveExpressionTopDown(_, q)) + } } def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { @@ -936,7 +953,7 @@ class Analyzer( // we still have chance to resolve it based on its descendants case s @ Sort(ordering, global, child) if child.resolved && !s.resolved => val newOrdering = - ordering.map(order => resolveExpression(order, child).asInstanceOf[SortOrder]) + ordering.map(order => resolveExpressionBottomUp(order, child).asInstanceOf[SortOrder]) Sort(newOrdering, global, child) // A special case for Generate, because the output of Generate should not be resolved by @@ -944,7 +961,7 @@ class Analyzer( case g @ Generate(generator, _, _, _, _, _) if generator.resolved => g case g @ Generate(generator, join, outer, qualifier, output, child) => - val newG = resolveExpression(generator, child, throws = true) + val newG = resolveExpressionBottomUp(generator, child, throws = true) if (newG.fastEquals(generator)) { g } else { @@ -959,11 +976,11 @@ class Analyzer( // `AppendColumns`, because `AppendColumns`'s serializer might produce conflict attribute // names leading to ambiguous references exception. case a @ Aggregate(groupingExprs, aggExprs, appendColumns: AppendColumns) => - a.mapExpressions(resolve(_, appendColumns)) + a.mapExpressions(resolveExpressionTopDown(_, appendColumns)) case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") - q.mapExpressions(resolve(_, q)) + q.mapExpressions(resolveExpressionTopDown(_, q)) } def newAliases(expressions: Seq[NamedExpression]): Seq[NamedExpression] = { @@ -1060,7 +1077,22 @@ class Analyzer( func.map(wrapper) } - protected[sql] def resolveExpression( + /** + * Resolves the attribute, column value and extract value expressions(s) by traversing the + * input expression in bottom-up manner. In order to resolve the nested complex type fields + * correctly, this function makes use of `throws` parameter to control when to raise an + * AnalysisException. + * + * Example : + * SELECT a.b FROM t ORDER BY b[0].d + * + * In the above example, in b needs to be resolved before d can be resolved. Given we are + * doing a bottom up traversal, it will first attempt to resolve d and fail as b has not + * been resolved yet. If `throws` is false, this function will handle the exception by + * returning the original attribute. In this case `d` will be resolved in subsequent passes + * after `b` is resolved. + */ + protected[sql] def resolveExpressionBottomUp( expr: Expression, plan: LogicalPlan, throws: Boolean = false): Expression = { @@ -1073,11 +1105,14 @@ class Analyzer( expr transformUp { case GetColumnByOrdinal(ordinal, _) => plan.output(ordinal) case u @ UnresolvedAttribute(nameParts) => - withPosition(u) { - plan.resolve(nameParts, resolver) - .orElse(resolveLiteralFunction(nameParts, u, plan)) - .getOrElse(u) - } + val result = + withPosition(u) { + plan.resolve(nameParts, resolver) + .orElse(resolveLiteralFunction(nameParts, u, plan)) + .getOrElse(u) + } + logDebug(s"Resolving $u to $result") + result case UnresolvedExtractValue(child, fieldName) if child.resolved => ExtractValue(child, fieldName, resolver) } @@ -1223,7 +1258,7 @@ class Analyzer( plan match { case p: Project => // Resolving expressions against current plan. - val maybeResolvedExprs = exprs.map(resolveExpression(_, p)) + val maybeResolvedExprs = exprs.map(resolveExpressionBottomUp(_, p)) // Recursively resolving expressions on the child of current plan. val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, p.child) // If some attributes used by expressions are resolvable only on the rewritten child @@ -1232,7 +1267,7 @@ class Analyzer( (newExprs, Project(p.projectList ++ missingAttrs, newChild)) case a @ Aggregate(groupExprs, aggExprs, child) => - val maybeResolvedExprs = exprs.map(resolveExpression(_, a)) + val maybeResolvedExprs = exprs.map(resolveExpressionBottomUp(_, a)) val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, child) val missingAttrs = (AttributeSet(newExprs) -- a.outputSet).intersect(newChild.outputSet) if (missingAttrs.forall(attr => groupExprs.exists(_.semanticEquals(attr)))) { @@ -1244,20 +1279,20 @@ class Analyzer( } case g: Generate => - val maybeResolvedExprs = exprs.map(resolveExpression(_, g)) + val maybeResolvedExprs = exprs.map(resolveExpressionBottomUp(_, g)) val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, g.child) (newExprs, g.copy(unrequiredChildIndex = Nil, child = newChild)) // For `Distinct` and `SubqueryAlias`, we can't recursively resolve and add attributes // via its children. case u: UnaryNode if !u.isInstanceOf[Distinct] && !u.isInstanceOf[SubqueryAlias] => - val maybeResolvedExprs = exprs.map(resolveExpression(_, u)) + val maybeResolvedExprs = exprs.map(resolveExpressionBottomUp(_, u)) val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, u.child) (newExprs, u.withNewChildren(Seq(newChild))) // For other operators, we can't recursively resolve and add attributes via its children. case other => - (exprs.map(resolveExpression(_, other)), other) + (exprs.map(resolveExpressionBottomUp(_, other)), other) } } } @@ -2387,7 +2422,7 @@ class Analyzer( } validateTopLevelTupleFields(deserializer, inputs) - val resolved = resolveExpression( + val resolved = resolveExpressionBottomUp( deserializer, LocalRelation(inputs), throws = true) val result = resolved transformDown { case UnresolvedMapObjects(func, inputData, cls) if inputData.resolved => From 26128484228089c74517cd15cef0bb4166a4186f Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 4 Dec 2018 20:20:29 +0800 Subject: [PATCH 0065/1072] [SPARK-25374][SQL] SafeProjection supports fallback to an interpreted mode ## What changes were proposed in this pull request? In SPARK-23711, we have implemented the expression fallback logic to an interpreted mode. So, this pr fixed code to support the same fallback mode in `SafeProjection` based on `CodeGeneratorWithInterpretedFallback`. ## How was this patch tested? Add tests in `CodeGeneratorWithInterpretedFallbackSuite` and `UnsafeRowConverterSuite`. Closes #22468 from maropu/SPARK-25374-3. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../catalyst/encoders/ExpressionEncoder.scala | 2 +- .../InterpretedSafeProjection.scala | 125 ++++++++++++++++++ .../sql/catalyst/expressions/Projection.scala | 34 +++-- .../expressions/CodeGenerationSuite.scala | 2 +- ...eneratorWithInterpretedFallbackSuite.scala | 15 +++ .../expressions/ExpressionEvalHelper.scala | 4 +- .../expressions/MutableProjectionSuite.scala | 2 +- .../expressions/UnsafeRowConverterSuite.scala | 89 ++++++++++++- .../DeclarativeAggregateEvaluator.scala | 11 +- .../codegen/GeneratedProjectionSuite.scala | 8 +- .../util/ArrayDataIndexedSeqSuite.scala | 4 +- .../org/apache/spark/sql/types/TestUDT.scala | 61 +++++++++ .../spark/sql/FileBasedDataSourceSuite.scala | 4 +- .../spark/sql/UserDefinedTypeSuite.scala | 105 +++++---------- .../datasources/json/JsonSuite.scala | 4 +- .../datasources/orc/OrcQuerySuite.scala | 4 +- .../execution/AggregationQuerySuite.scala | 2 +- .../execution/ObjectHashAggregateSuite.scala | 4 +- .../sql/sources/HadoopFsRelationTest.scala | 2 +- 19 files changed, 371 insertions(+), 111 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedSafeProjection.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/types/TestUDT.scala 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 589e215c55e4..fbf0bd68b958 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 @@ -302,7 +302,7 @@ case class ExpressionEncoder[T]( private lazy val inputRow = new GenericInternalRow(1) @transient - private lazy val constructProjection = GenerateSafeProjection.generate(deserializer :: Nil) + private lazy val constructProjection = SafeProjection.create(deserializer :: Nil) /** * Returns a new set (with unique ids) of [[NamedExpression]] that represent the serialized form diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedSafeProjection.scala new file mode 100644 index 000000000000..70789dac1d87 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedSafeProjection.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} +import org.apache.spark.sql.types._ + + +/** + * An interpreted version of a safe projection. + * + * @param expressions that produces the resulting fields. These expressions must be bound + * to a schema. + */ +class InterpretedSafeProjection(expressions: Seq[Expression]) extends Projection { + + private[this] val mutableRow = new SpecificInternalRow(expressions.map(_.dataType)) + + private[this] val exprsWithWriters = expressions.zipWithIndex.filter { + case (NoOp, _) => false + case _ => true + }.map { case (e, i) => + val converter = generateSafeValueConverter(e.dataType) + val writer = InternalRow.getWriter(i, e.dataType) + val f = if (!e.nullable) { + (v: Any) => writer(mutableRow, converter(v)) + } else { + (v: Any) => { + if (v == null) { + mutableRow.setNullAt(i) + } else { + writer(mutableRow, converter(v)) + } + } + } + (e, f) + } + + private def generateSafeValueConverter(dt: DataType): Any => Any = dt match { + case ArrayType(elemType, _) => + val elementConverter = generateSafeValueConverter(elemType) + v => { + val arrayValue = v.asInstanceOf[ArrayData] + val result = new Array[Any](arrayValue.numElements()) + arrayValue.foreach(elemType, (i, e) => { + result(i) = elementConverter(e) + }) + new GenericArrayData(result) + } + + case st: StructType => + val fieldTypes = st.fields.map(_.dataType) + val fieldConverters = fieldTypes.map(generateSafeValueConverter) + v => { + val row = v.asInstanceOf[InternalRow] + val ar = new Array[Any](row.numFields) + var idx = 0 + while (idx < row.numFields) { + ar(idx) = fieldConverters(idx)(row.get(idx, fieldTypes(idx))) + idx += 1 + } + new GenericInternalRow(ar) + } + + case MapType(keyType, valueType, _) => + lazy val keyConverter = generateSafeValueConverter(keyType) + lazy val valueConverter = generateSafeValueConverter(valueType) + v => { + val mapValue = v.asInstanceOf[MapData] + val keys = mapValue.keyArray().toArray[Any](keyType) + val values = mapValue.valueArray().toArray[Any](valueType) + val convertedKeys = keys.map(keyConverter) + val convertedValues = values.map(valueConverter) + ArrayBasedMapData(convertedKeys, convertedValues) + } + + case udt: UserDefinedType[_] => + generateSafeValueConverter(udt.sqlType) + + case _ => identity + } + + override def apply(row: InternalRow): InternalRow = { + var i = 0 + while (i < exprsWithWriters.length) { + val (expr, writer) = exprsWithWriters(i) + writer(expr.eval(row)) + i += 1 + } + mutableRow + } +} + +/** + * Helper functions for creating an [[InterpretedSafeProjection]]. + */ +object InterpretedSafeProjection { + + /** + * Returns an [[SafeProjection]] for given sequence of bound Expressions. + */ + def createProjection(exprs: Seq[Expression]): Projection = { + // We need to make sure that we do not reuse stateful expressions. + val cleanedExpressions = exprs.map(_.transform { + case s: Stateful => s.freshCopy() + }) + new InterpretedSafeProjection(cleanedExpressions) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 792646cf9f10..b48f7ba655b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -169,26 +169,40 @@ object UnsafeProjection /** * A projection that could turn UnsafeRow into GenericInternalRow */ -object FromUnsafeProjection { +object SafeProjection extends CodeGeneratorWithInterpretedFallback[Seq[Expression], Projection] { + + override protected def createCodeGeneratedObject(in: Seq[Expression]): Projection = { + GenerateSafeProjection.generate(in) + } + + override protected def createInterpretedObject(in: Seq[Expression]): Projection = { + InterpretedSafeProjection.createProjection(in) + } /** - * Returns a Projection for given StructType. + * Returns a SafeProjection for given StructType. */ - def apply(schema: StructType): Projection = { - apply(schema.fields.map(_.dataType)) + def create(schema: StructType): Projection = create(schema.fields.map(_.dataType)) + + /** + * Returns a SafeProjection for given Array of DataTypes. + */ + def create(fields: Array[DataType]): Projection = { + createObject(fields.zipWithIndex.map(x => new BoundReference(x._2, x._1, true))) } /** - * Returns an UnsafeProjection for given Array of DataTypes. + * Returns a SafeProjection for given sequence of Expressions (bounded). */ - def apply(fields: Seq[DataType]): Projection = { - create(fields.zipWithIndex.map(x => new BoundReference(x._2, x._1, true))) + def create(exprs: Seq[Expression]): Projection = { + createObject(exprs) } /** - * Returns a Projection for given sequence of Expressions (bounded). + * Returns a SafeProjection for given sequence of Expressions, which will be bound to + * `inputSchema`. */ - private def create(exprs: Seq[Expression]): Projection = { - GenerateSafeProjection.generate(exprs) + def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): Projection = { + create(toBoundExprs(exprs, inputSchema)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 7843003a4aac..7e6fe5b4e206 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -251,7 +251,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { UTF8String.fromString("c")) assert(unsafeRow.getStruct(3, 1).getStruct(0, 2).getInt(1) === 3) - val fromUnsafe = FromUnsafeProjection(schema) + val fromUnsafe = SafeProjection.create(schema) val internalRow2 = fromUnsafe(unsafeRow) assert(internalRow === internalRow2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGeneratorWithInterpretedFallbackSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGeneratorWithInterpretedFallbackSuite.scala index 6ea3b05ff9c1..da5bddb0c09f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGeneratorWithInterpretedFallbackSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGeneratorWithInterpretedFallbackSuite.scala @@ -106,4 +106,19 @@ class CodeGeneratorWithInterpretedFallbackSuite extends SparkFunSuite with PlanT assert(proj(input).toSeq(StructType.fromDDL("c0 int, c1 int")) === expected) } } + + test("SPARK-25374 Correctly handles NoOp in SafeProjection") { + val exprs = Seq(Add(BoundReference(0, IntegerType, nullable = true), Literal.create(1)), NoOp) + val input = InternalRow.fromSeq(1 :: 1 :: Nil) + val expected = 2 :: null :: Nil + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenOnly) { + val proj = SafeProjection.createObject(exprs) + assert(proj(input).toSeq(StructType.fromDDL("c0 int, c1 int")) === expected) + } + + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> noCodegen) { + val proj = SafeProjection.createObject(exprs) + assert(proj(input).toSeq(StructType.fromDDL("c0 int, c1 int")) === expected) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index a7282e1b1cad..b4fd170467d8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -321,8 +321,8 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa GenerateUnsafeProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), expression) plan.initialize(0) - actual = FromUnsafeProjection(expression.dataType :: Nil)( - plan(inputRow)).get(0, expression.dataType) + val ref = new BoundReference(0, expression.dataType, nullable = true) + actual = GenerateSafeProjection.generate(ref :: Nil)(plan(inputRow)).get(0, expression.dataType) assert(checkResult(actual, expected, expression)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala index 2db1c3b98819..0d594eb10962 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -51,7 +51,7 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { val unsafeBuffer = UnsafeRow.createFromByteArray(numBytes, fixedLengthTypes.length) val proj = createMutableProjection(fixedLengthTypes) val projUnsafeRow = proj.target(unsafeBuffer)(inputRow) - assert(FromUnsafeProjection.apply(fixedLengthTypes)(projUnsafeRow) === inputRow) + assert(SafeProjection.create(fixedLengthTypes)(projUnsafeRow) === inputRow) } testBothCodegenAndInterpreted("variable-length types") { 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 268372b5d050..ecb8047459b0 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types.{IntegerType, LongType, _} import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestBase with ExpressionEvalHelper { @@ -535,4 +535,91 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB assert(unsafeRow.getSizeInBytes == 8 + 8 * 2 + roundedSize(field1.getSizeInBytes) + roundedSize(field2.getSizeInBytes)) } + + testBothCodegenAndInterpreted("SPARK-25374 converts back into safe representation") { + def convertBackToInternalRow(inputRow: InternalRow, fields: Array[DataType]): InternalRow = { + val unsafeProj = UnsafeProjection.create(fields) + val unsafeRow = unsafeProj(inputRow) + val safeProj = SafeProjection.create(fields) + safeProj(unsafeRow) + } + + // Simple tests + val inputRow = InternalRow.fromSeq(Seq( + false, 3.toByte, 15.toShort, -83, 129L, 1.0f, 8.0, UTF8String.fromString("test"), + Decimal(255), CalendarInterval.fromString("interval 1 day"), Array[Byte](1, 2) + )) + val fields1 = Array( + BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, + DoubleType, StringType, DecimalType.defaultConcreteType, CalendarIntervalType, + BinaryType) + + assert(convertBackToInternalRow(inputRow, fields1) === inputRow) + + // Array tests + val arrayRow = InternalRow.fromSeq(Seq( + createArray(1, 2, 3), + createArray( + createArray(Seq("a", "b", "c").map(UTF8String.fromString): _*), + createArray(Seq("d").map(UTF8String.fromString): _*)) + )) + val fields2 = Array[DataType]( + ArrayType(IntegerType), + ArrayType(ArrayType(StringType))) + + assert(convertBackToInternalRow(arrayRow, fields2) === arrayRow) + + // Struct tests + val structRow = InternalRow.fromSeq(Seq( + InternalRow.fromSeq(Seq[Any](1, 4.0)), + InternalRow.fromSeq(Seq( + UTF8String.fromString("test"), + InternalRow.fromSeq(Seq( + 1, + createArray(Seq("2", "3").map(UTF8String.fromString): _*) + )) + )) + )) + val fields3 = Array[DataType]( + StructType( + StructField("c0", IntegerType) :: + StructField("c1", DoubleType) :: + Nil), + StructType( + StructField("c2", StringType) :: + StructField("c3", StructType( + StructField("c4", IntegerType) :: + StructField("c5", ArrayType(StringType)) :: + Nil)) :: + Nil)) + + assert(convertBackToInternalRow(structRow, fields3) === structRow) + + // Map tests + val mapRow = InternalRow.fromSeq(Seq( + createMap(Seq("k1", "k2").map(UTF8String.fromString): _*)(1, 2), + createMap( + createMap(3, 5)(Seq("v1", "v2").map(UTF8String.fromString): _*), + createMap(7, 9)(Seq("v3", "v4").map(UTF8String.fromString): _*) + )( + createMap(Seq("k3", "k4").map(UTF8String.fromString): _*)(3.toShort, 4.toShort), + createMap(Seq("k5", "k6").map(UTF8String.fromString): _*)(5.toShort, 6.toShort) + ))) + val fields4 = Array[DataType]( + MapType(StringType, IntegerType), + MapType(MapType(IntegerType, StringType), MapType(StringType, ShortType))) + + val mapResultRow = convertBackToInternalRow(mapRow, fields4) + val mapExpectedRow = mapRow + checkResult(mapExpectedRow, mapResultRow, + exprDataType = StructType(fields4.zipWithIndex.map(f => StructField(s"c${f._2}", f._1))), + exprNullable = false) + + // UDT tests + val vector = new TestUDT.MyDenseVector(Array(1.0, 3.0, 5.0, 7.0, 9.0)) + val udt = new TestUDT.MyDenseVectorUDT() + val udtRow = InternalRow.fromSeq(Seq(udt.serialize(vector))) + val fields5 = Array[DataType](udt) + assert(convertBackToInternalRow(udtRow, fields5) === udtRow) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala index 614f24db0aaf..b0f55b3b5c44 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala @@ -17,25 +17,24 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection +import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow, SafeProjection} /** * Evaluator for a [[DeclarativeAggregate]]. */ case class DeclarativeAggregateEvaluator(function: DeclarativeAggregate, input: Seq[Attribute]) { - lazy val initializer = GenerateSafeProjection.generate(function.initialValues) + lazy val initializer = SafeProjection.create(function.initialValues) - lazy val updater = GenerateSafeProjection.generate( + lazy val updater = SafeProjection.create( function.updateExpressions, function.aggBufferAttributes ++ input) - lazy val merger = GenerateSafeProjection.generate( + lazy val merger = SafeProjection.create( function.mergeExpressions, function.aggBufferAttributes ++ function.inputAggBufferAttributes) - lazy val evaluator = GenerateSafeProjection.generate( + lazy val evaluator = SafeProjection.create( function.evaluateExpression :: Nil, function.aggBufferAttributes) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index 2c45b3b0c73d..4c9bcfe8f93a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -58,7 +58,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { } // test generated SafeProjection - val safeProj = FromUnsafeProjection(nestedSchema) + val safeProj = SafeProjection.create(nestedSchema) val result = safeProj(unsafe) // Can't compare GenericInternalRow with JoinedRow directly (0 until N).foreach { i => @@ -109,7 +109,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { } // test generated SafeProjection - val safeProj = FromUnsafeProjection(nestedSchema) + val safeProj = SafeProjection.create(nestedSchema) val result = safeProj(unsafe) // Can't compare GenericInternalRow with JoinedRow directly (0 until N).foreach { i => @@ -147,7 +147,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { assert(unsafeRow.getArray(1).getBinary(1) === null) assert(java.util.Arrays.equals(unsafeRow.getArray(1).getBinary(2), Array[Byte](3, 4))) - val safeProj = FromUnsafeProjection(fields) + val safeProj = SafeProjection.create(fields) val row2 = safeProj(unsafeRow) assert(row2 === row) } @@ -233,7 +233,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { val nestedSchema = StructType( Seq(StructField("", joinedSchema), StructField("", joinedSchema)) ++ joinedSchema) - val safeProj = FromUnsafeProjection(nestedSchema) + val safeProj = SafeProjection.create(nestedSchema) val result = safeProj(nested) // test generated MutableProjection diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala index 6400898343ae..da71e3a4d53e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala @@ -22,7 +22,7 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.sql.RandomDataGenerator import org.apache.spark.sql.catalyst.encoders.{ExamplePointUDT, RowEncoder} -import org.apache.spark.sql.catalyst.expressions.{FromUnsafeProjection, UnsafeArrayData, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{SafeProjection, UnsafeArrayData, UnsafeProjection} import org.apache.spark.sql.types._ class ArrayDataIndexedSeqSuite extends SparkFunSuite { @@ -77,7 +77,7 @@ class ArrayDataIndexedSeqSuite extends SparkFunSuite { val internalRow = rowConverter.toRow(row) val unsafeRowConverter = UnsafeProjection.create(schema) - val safeRowConverter = FromUnsafeProjection(schema) + val safeRowConverter = SafeProjection.create(schema) val unsafeRow = unsafeRowConverter(internalRow) val safeRow = safeRowConverter(unsafeRow) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/TestUDT.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/TestUDT.scala new file mode 100644 index 000000000000..1be8ee9dfa92 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/TestUDT.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.types + +import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} + + +// Wrapped in an object to check Scala compatibility. See SPARK-13929 +object TestUDT { + + @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) + private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { + override def hashCode(): Int = java.util.Arrays.hashCode(data) + + override def equals(other: Any): Boolean = other match { + case v: MyDenseVector => java.util.Arrays.equals(this.data, v.data) + case _ => false + } + + override def toString: String = data.mkString("(", ", ", ")") + } + + private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { + + override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) + + override def serialize(features: MyDenseVector): ArrayData = { + new GenericArrayData(features.data.map(_.asInstanceOf[Any])) + } + + override def deserialize(datum: Any): MyDenseVector = { + datum match { + case data: ArrayData => + new MyDenseVector(data.toDoubleArray()) + } + } + + override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] + + private[spark] override def asNullable: MyDenseVectorUDT = this + + override def hashCode(): Int = getClass.hashCode() + + override def equals(other: Any): Boolean = other.isInstanceOf[MyDenseVectorUDT] + } +} 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 64b42c32b8b1..54299e9808bf 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 @@ -312,13 +312,13 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo assert(msg.contains("CSV data source does not support array data type")) msg = intercept[AnalysisException] { - Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25)))).toDF("id", "vectors") + Seq((1, new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25)))).toDF("id", "vectors") .write.mode("overwrite").csv(csvDir) }.getMessage assert(msg.contains("CSV data source does not support array data type")) msg = intercept[AnalysisException] { - val schema = StructType(StructField("a", new UDT.MyDenseVectorUDT(), true) :: Nil) + val schema = StructType(StructField("a", new TestUDT.MyDenseVectorUDT(), true) :: Nil) spark.range(1).write.mode("overwrite").csv(csvDir) spark.read.schema(schema).csv(csvDir).collect() }.getMessage 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 cf956316057e..6628d36ffc70 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 @@ -20,56 +20,14 @@ package org.apache.spark.sql import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, GenericInternalRow, Literal} -import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ -private[sql] case class MyLabeledPoint(label: Double, features: UDT.MyDenseVector) { +private[sql] case class MyLabeledPoint(label: Double, features: TestUDT.MyDenseVector) { def getLabel: Double = label - def getFeatures: UDT.MyDenseVector = features -} - -// Wrapped in an object to check Scala compatibility. See SPARK-13929 -object UDT { - - @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) - private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { - override def hashCode(): Int = java.util.Arrays.hashCode(data) - - override def equals(other: Any): Boolean = other match { - case v: MyDenseVector => java.util.Arrays.equals(this.data, v.data) - case _ => false - } - - override def toString: String = data.mkString("(", ", ", ")") - } - - private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { - - override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) - - override def serialize(features: MyDenseVector): ArrayData = { - new GenericArrayData(features.data.map(_.asInstanceOf[Any])) - } - - override def deserialize(datum: Any): MyDenseVector = { - datum match { - case data: ArrayData => - new MyDenseVector(data.toDoubleArray()) - } - } - - override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] - - private[spark] override def asNullable: MyDenseVectorUDT = this - - override def hashCode(): Int = getClass.hashCode() - - override def equals(other: Any): Boolean = other.isInstanceOf[MyDenseVectorUDT] - } - + def getFeatures: TestUDT.MyDenseVector = features } // object and classes to test SPARK-19311 @@ -148,12 +106,12 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT import testImplicits._ private lazy val pointsRDD = Seq( - MyLabeledPoint(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), - MyLabeledPoint(0.0, new UDT.MyDenseVector(Array(0.2, 2.0)))).toDF() + MyLabeledPoint(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new TestUDT.MyDenseVector(Array(0.2, 2.0)))).toDF() private lazy val pointsRDD2 = Seq( - MyLabeledPoint(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), - MyLabeledPoint(0.0, new UDT.MyDenseVector(Array(0.3, 3.0)))).toDF() + MyLabeledPoint(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new TestUDT.MyDenseVector(Array(0.3, 3.0)))).toDF() test("register user type: MyDenseVector for MyLabeledPoint") { val labels: RDD[Double] = pointsRDD.select('label).rdd.map { case Row(v: Double) => v } @@ -162,16 +120,17 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT assert(labelsArrays.contains(1.0)) assert(labelsArrays.contains(0.0)) - val features: RDD[UDT.MyDenseVector] = - pointsRDD.select('features).rdd.map { case Row(v: UDT.MyDenseVector) => v } - val featuresArrays: Array[UDT.MyDenseVector] = features.collect() + val features: RDD[TestUDT.MyDenseVector] = + pointsRDD.select('features).rdd.map { case Row(v: TestUDT.MyDenseVector) => v } + val featuresArrays: Array[TestUDT.MyDenseVector] = features.collect() assert(featuresArrays.size === 2) - assert(featuresArrays.contains(new UDT.MyDenseVector(Array(0.1, 1.0)))) - assert(featuresArrays.contains(new UDT.MyDenseVector(Array(0.2, 2.0)))) + assert(featuresArrays.contains(new TestUDT.MyDenseVector(Array(0.1, 1.0)))) + assert(featuresArrays.contains(new TestUDT.MyDenseVector(Array(0.2, 2.0)))) } test("UDTs and UDFs") { - spark.udf.register("testType", (d: UDT.MyDenseVector) => d.isInstanceOf[UDT.MyDenseVector]) + spark.udf.register("testType", + (d: TestUDT.MyDenseVector) => d.isInstanceOf[TestUDT.MyDenseVector]) pointsRDD.createOrReplaceTempView("points") checkAnswer( sql("SELECT testType(features) from points"), @@ -185,8 +144,8 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT checkAnswer( spark.read.parquet(path), Seq( - Row(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), - Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) + Row(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), + Row(0.0, new TestUDT.MyDenseVector(Array(0.2, 2.0))))) } } @@ -197,17 +156,17 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT checkAnswer( spark.read.parquet(path), Seq( - Row(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), - Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) + Row(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), + Row(0.0, new TestUDT.MyDenseVector(Array(0.2, 2.0))))) } } // Tests to make sure that all operators correctly convert types on the way out. test("Local UDTs") { - val vec = new UDT.MyDenseVector(Array(0.1, 1.0)) + val vec = new TestUDT.MyDenseVector(Array(0.1, 1.0)) val df = Seq((1, vec)).toDF("int", "vec") - assert(vec === df.collect()(0).getAs[UDT.MyDenseVector](1)) - assert(vec === df.take(1)(0).getAs[UDT.MyDenseVector](1)) + assert(vec === df.collect()(0).getAs[TestUDT.MyDenseVector](1)) + assert(vec === df.take(1)(0).getAs[TestUDT.MyDenseVector](1)) checkAnswer(df.limit(1).groupBy('int).agg(first('vec)), Row(1, vec)) checkAnswer(df.orderBy('int).limit(1).groupBy('int).agg(first('vec)), Row(1, vec)) } @@ -219,14 +178,14 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT ) val schema = StructType(Seq( StructField("id", IntegerType, false), - StructField("vec", new UDT.MyDenseVectorUDT, false) + StructField("vec", new TestUDT.MyDenseVectorUDT, false) )) val jsonRDD = spark.read.schema(schema).json(data.toDS()) checkAnswer( jsonRDD, - Row(1, new UDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))) :: - Row(2, new UDT.MyDenseVector(Array(2.25, 4.5, 8.75))) :: + Row(1, new TestUDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))) :: + Row(2, new TestUDT.MyDenseVector(Array(2.25, 4.5, 8.75))) :: Nil ) } @@ -239,25 +198,25 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT val schema = StructType(Seq( StructField("id", IntegerType, false), - StructField("vec", new UDT.MyDenseVectorUDT, false) + StructField("vec", new TestUDT.MyDenseVectorUDT, false) )) val jsonDataset = spark.read.schema(schema).json(data.toDS()) - .as[(Int, UDT.MyDenseVector)] + .as[(Int, TestUDT.MyDenseVector)] checkDataset( jsonDataset, - (1, new UDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))), - (2, new UDT.MyDenseVector(Array(2.25, 4.5, 8.75))) + (1, new TestUDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))), + (2, new TestUDT.MyDenseVector(Array(2.25, 4.5, 8.75))) ) } test("SPARK-10472 UserDefinedType.typeName") { assert(IntegerType.typeName === "integer") - assert(new UDT.MyDenseVectorUDT().typeName === "mydensevector") + assert(new TestUDT.MyDenseVectorUDT().typeName === "mydensevector") } test("Catalyst type converter null handling for UDTs") { - val udt = new UDT.MyDenseVectorUDT() + val udt = new TestUDT.MyDenseVectorUDT() val toScalaConverter = CatalystTypeConverters.createToScalaConverter(udt) assert(toScalaConverter(null) === null) @@ -303,12 +262,12 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT test("except on UDT") { checkAnswer( pointsRDD.except(pointsRDD2), - Seq(Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) + Seq(Row(0.0, new TestUDT.MyDenseVector(Array(0.2, 2.0))))) } test("SPARK-23054 Cast UserDefinedType to string") { - val udt = new UDT.MyDenseVectorUDT() - val vector = new UDT.MyDenseVector(Array(1.0, 3.0, 5.0, 7.0, 9.0)) + val udt = new TestUDT.MyDenseVectorUDT() + val vector = new TestUDT.MyDenseVector(Array(1.0, 3.0, 5.0, 7.0, 9.0)) val data = udt.serialize(vector) val ret = Cast(Literal(data, udt), StringType, None) checkEvaluation(ret, "(1.0, 3.0, 5.0, 7.0, 9.0)") 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 9d23161c1f24..dff37ca2d40f 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 @@ -1463,7 +1463,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), DateType, TimestampType, ArrayType(IntegerType), MapType(StringType, LongType), struct, - new UDT.MyDenseVectorUDT()) + new TestUDT.MyDenseVectorUDT()) val fields = dataTypes.zipWithIndex.map { case (dataType, index) => StructField(s"col$index", dataType, nullable = true) } @@ -1487,7 +1487,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Seq(2, 3, 4), Map("a string" -> 2000L), Row(4.75.toFloat, Seq(false, true)), - new UDT.MyDenseVector(Array(0.25, 2.25, 4.25))) + new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25))) val data = Row.fromSeq(Seq("Spark " + spark.sparkContext.version) ++ constantValues) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 998b7b31dcd6..918dbcdfa1cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, RecordReaderIterator} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.{IntegerType, StructType, TestUDT} import org.apache.spark.util.Utils case class AllDataTypesWithNonPrimitiveType( @@ -103,7 +103,7 @@ abstract class OrcQueryTest extends OrcTest { test("Read/write UserDefinedType") { withTempPath { path => - val data = Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25)))) + val data = Seq((1, new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25)))) val udtDF = data.toDF("id", "vectors") udtDF.write.orc(path.getAbsolutePath) val readBack = spark.read.schema(udtDF.schema).orc(path.getAbsolutePath) 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 c65bf7c14c7a..cfae2d82e273 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 @@ -884,7 +884,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), DateType, TimestampType, ArrayType(IntegerType), MapType(StringType, LongType), struct, - new UDT.MyDenseVectorUDT()) + new TestUDT.MyDenseVectorUDT()) // Right now, we will use SortAggregate to handle UDAFs. // UnsafeRow.mutableFieldTypes.asScala.toSeq will trigger SortAggregate to use // UnsafeRow as the aggregation buffer. While, dataTypes will trigger 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 c9309197791b..2391106cfb25 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 @@ -124,7 +124,7 @@ class ObjectHashAggregateSuite .add("f2", ArrayType(BooleanType), nullable = true), // UDT - new UDT.MyDenseVectorUDT(), + new TestUDT.MyDenseVectorUDT(), // Others StringType, @@ -259,7 +259,7 @@ class ObjectHashAggregateSuite StringType, BinaryType, NullType, BooleanType ) - val udt = new UDT.MyDenseVectorUDT() + val udt = new TestUDT.MyDenseVectorUDT() val fixedLengthTypes = builtinNumericTypes ++ Seq(BooleanType, NullType) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 6bd59fde550d..6075f2c8877d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -115,7 +115,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes new StructType() .add("f1", FloatType, nullable = true) .add("f2", ArrayType(BooleanType, containsNull = true), nullable = true), - new UDT.MyDenseVectorUDT() + new TestUDT.MyDenseVectorUDT() ).filter(supportsDataType) test(s"test all data types") { From 93f5592aa8c1254a93524fda81cf0e418c22cb2f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E7=81=BF00244106?= <00244106@zte.intra> Date: Tue, 4 Dec 2018 22:08:16 +0900 Subject: [PATCH 0066/1072] [MINOR][SQL] Combine the same codes in test cases MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? In the DDLSuit, there are four test cases have the same codes , writing a function can combine the same code. ## How was this patch tested? existing tests. Closes #23194 from CarolinePeng/Update_temp. Authored-by: 彭灿00244106 <00244106@zte.intra> Signed-off-by: Takeshi Yamamuro --- .../sql/execution/command/DDLSuite.scala | 40 ++++++++----------- 1 file changed, 16 insertions(+), 24 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 9d32fb6d4696..052a5e757c44 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 @@ -377,41 +377,41 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("CTAS a managed table with the existing empty directory") { - val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier("tab1"))) + private def withEmptyDirInTablePath(dirName: String)(f : File => Unit): Unit = { + val tableLoc = + new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier(dirName))) try { tableLoc.mkdir() + f(tableLoc) + } finally { + waitForTasksToFinish() + Utils.deleteRecursively(tableLoc) + } + } + + + test("CTAS a managed table with the existing empty directory") { + withEmptyDirInTablePath("tab1") { tableLoc => withTable("tab1") { sql(s"CREATE TABLE tab1 USING ${dataSource} AS SELECT 1, 'a'") checkAnswer(spark.table("tab1"), Row(1, "a")) } - } finally { - waitForTasksToFinish() - Utils.deleteRecursively(tableLoc) } } test("create a managed table with the existing empty directory") { - val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier("tab1"))) - try { - tableLoc.mkdir() + withEmptyDirInTablePath("tab1") { tableLoc => withTable("tab1") { sql(s"CREATE TABLE tab1 (col1 int, col2 string) USING ${dataSource}") sql("INSERT INTO tab1 VALUES (1, 'a')") checkAnswer(spark.table("tab1"), Row(1, "a")) } - } finally { - waitForTasksToFinish() - Utils.deleteRecursively(tableLoc) } } test("create a managed table with the existing non-empty directory") { withTable("tab1") { - val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier("tab1"))) - try { - // create an empty hidden file - tableLoc.mkdir() + withEmptyDirInTablePath("tab1") { tableLoc => val hiddenGarbageFile = new File(tableLoc.getCanonicalPath, ".garbage") hiddenGarbageFile.createNewFile() val exMsg = "Can not create the managed table('`tab1`'). The associated location" @@ -439,28 +439,20 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { }.getMessage assert(ex.contains(exMsgWithDefaultDB)) } - } finally { - waitForTasksToFinish() - Utils.deleteRecursively(tableLoc) } } } test("rename a managed table with existing empty directory") { - val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier("tab2"))) - try { + withEmptyDirInTablePath("tab2") { tableLoc => withTable("tab1") { sql(s"CREATE TABLE tab1 USING $dataSource AS SELECT 1, 'a'") - tableLoc.mkdir() val ex = intercept[AnalysisException] { sql("ALTER TABLE tab1 RENAME TO tab2") }.getMessage val expectedMsg = "Can not rename the managed table('`tab1`'). The associated location" assert(ex.contains(expectedMsg)) } - } finally { - waitForTasksToFinish() - Utils.deleteRecursively(tableLoc) } } From 06a3b6aafa510ede2f1376b29a46f99447286c67 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 4 Dec 2018 07:57:58 -0600 Subject: [PATCH 0067/1072] [SPARK-24423][FOLLOW-UP][SQL] Fix error example ## What changes were proposed in this pull request? ![image](https://user-images.githubusercontent.com/5399861/49172173-42ad9800-f37b-11e8-8135-7adc323357ae.png) It will throw: ``` requirement failed: When reading JDBC data sources, users need to specify all or none for the following options: 'partitionColumn', 'lowerBound', 'upperBound', and 'numPartitions' ``` and ``` User-defined partition column subq.c1 not found in the JDBC relation ... ``` This PR fix this error example. ## How was this patch tested? manual tests Closes #23170 from wangyum/SPARK-24499. Authored-by: Yuming Wang Signed-off-by: Sean Owen --- docs/sql-data-sources-jdbc.md | 6 +++--- .../sql/execution/datasources/jdbc/JDBCOptions.scala | 10 +++++++--- .../scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 10 +++++++--- 3 files changed, 17 insertions(+), 9 deletions(-) diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index 9a5d0fc7d424..a2b14620be12 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -64,9 +64,9 @@ the following case-insensitive options: Example:
    spark.read.format("jdbc")
    -    .option("dbtable", "(select c1, c2 from t1) as subq")
    -    .option("partitionColumn", "subq.c1"
    -    .load() + .option("url", jdbcUrl)
    + .option("query", "select c1, c2 from t1")
    + .load()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 7dfbb9d8b5c0..b4469cb538fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -137,9 +137,13 @@ class JDBCOptions( |the partition columns using the supplied subquery alias to resolve any ambiguity. |Example : |spark.read.format("jdbc") - | .option("dbtable", "(select c1, c2 from t1) as subq") - | .option("partitionColumn", "subq.c1" - | .load() + | .option("url", jdbcUrl) + | .option("dbtable", "(select c1, c2 from t1) as subq") + | .option("partitionColumn", "c1") + | .option("lowerBound", "1") + | .option("upperBound", "100") + | .option("numPartitions", "3") + | .load() """.stripMargin ) 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 7fa0e7fc162c..71e83767964a 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 @@ -1348,9 +1348,13 @@ class JDBCSuite extends QueryTest |the partition columns using the supplied subquery alias to resolve any ambiguity. |Example : |spark.read.format("jdbc") - | .option("dbtable", "(select c1, c2 from t1) as subq") - | .option("partitionColumn", "subq.c1" - | .load() + | .option("url", jdbcUrl) + | .option("dbtable", "(select c1, c2 from t1) as subq") + | .option("partitionColumn", "c1") + | .option("lowerBound", "1") + | .option("upperBound", "100") + | .option("numPartitions", "3") + | .load() """.stripMargin val e5 = intercept[RuntimeException] { sql( From f982ca07e80074bdc1e3b742c5e21cf368e4ede2 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 4 Dec 2018 08:36:33 -0600 Subject: [PATCH 0068/1072] [SPARK-26178][SQL] Use java.time API for parsing timestamps and dates from CSV ## What changes were proposed in this pull request? In the PR, I propose to use **java.time API** for parsing timestamps and dates from CSV content with microseconds precision. The SQL config `spark.sql.legacy.timeParser.enabled` allow to switch back to previous behaviour with using `java.text.SimpleDateFormat`/`FastDateFormat` for parsing/generating timestamps/dates. ## How was this patch tested? It was tested by `UnivocityParserSuite`, `CsvExpressionsSuite`, `CsvFunctionsSuite` and `CsvSuite`. Closes #23150 from MaxGekk/time-parser. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Sean Owen --- docs/sql-migration-guide-upgrade.md | 2 + .../sql/catalyst/csv/CSVInferSchema.scala | 15 +- .../spark/sql/catalyst/csv/CSVOptions.scala | 10 +- .../sql/catalyst/csv/UnivocityGenerator.scala | 14 +- .../sql/catalyst/csv/UnivocityParser.scala | 38 ++-- .../sql/catalyst/util/DateTimeFormatter.scala | 179 ++++++++++++++++++ .../sql/catalyst/util/DateTimeUtils.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 9 + .../catalyst/csv/CSVInferSchemaSuite.scala | 7 +- .../catalyst/csv/UnivocityParserSuite.scala | 113 ++++++----- .../sql/catalyst/util/DateTimeTestUtils.scala | 5 +- .../sql/util/DateTimeFormatterSuite.scala | 103 ++++++++++ .../apache/spark/sql/CsvFunctionsSuite.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 66 ++++--- 14 files changed, 431 insertions(+), 134 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimeFormatterSuite.scala diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 787f4bcbbea8..fee0e6df7177 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -33,6 +33,8 @@ displayTitle: Spark SQL Upgrading Guide - Spark applications which are built with Spark version 2.4 and prior, and call methods of `UserDefinedFunction`, need to be re-compiled with Spark 3.0, as they are not binary compatible with Spark 3.0. + - Since Spark 3.0, CSV datasource uses java.time API for parsing and generating CSV content. New formatting implementation supports date/timestamp patterns conformed to ISO 8601. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. + ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. 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 94cb4b114e6b..345dc4d41993 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 @@ -22,10 +22,16 @@ import scala.util.control.Exception.allCatch import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.expressions.ExprUtils -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.DateTimeFormatter import org.apache.spark.sql.types._ -class CSVInferSchema(options: CSVOptions) extends Serializable { +class CSVInferSchema(val options: CSVOptions) extends Serializable { + + @transient + private lazy val timeParser = DateTimeFormatter( + options.timestampFormat, + options.timeZone, + options.locale) private val decimalParser = { ExprUtils.getDecimalParser(options.locale) @@ -154,10 +160,7 @@ class CSVInferSchema(options: CSVOptions) extends Serializable { private def tryParseTimestamp(field: String): DataType = { // This case infers a custom `dataFormat` is set. - if ((allCatch opt options.timestampFormat.parse(field)).isDefined) { - TimestampType - } else if ((allCatch opt DateTimeUtils.stringToTime(field)).isDefined) { - // We keep this for backwards compatibility. + if ((allCatch opt timeParser.parse(field)).isDefined) { TimestampType } else { tryParseBoolean(field) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 94bdb72d675d..90c96d1f55c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -21,7 +21,6 @@ import java.nio.charset.StandardCharsets import java.util.{Locale, TimeZone} import com.univocity.parsers.csv.{CsvParserSettings, CsvWriterSettings, UnescapedQuoteHandling} -import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ @@ -146,13 +145,10 @@ class CSVOptions( // A language tag in IETF BCP 47 format val locale: Locale = parameters.get("locale").map(Locale.forLanguageTag).getOrElse(Locale.US) - // Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe. - val dateFormat: FastDateFormat = - FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd"), locale) + val dateFormat: String = parameters.getOrElse("dateFormat", "yyyy-MM-dd") - val timestampFormat: FastDateFormat = - FastDateFormat.getInstance( - parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), timeZone, locale) + val timestampFormat: String = + parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX") val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala index 2ab376c0ac20..af09cd6c8449 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala @@ -22,7 +22,7 @@ import java.io.Writer import com.univocity.parsers.csv.CsvWriter import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeFormatter} import org.apache.spark.sql.types._ class UnivocityGenerator( @@ -41,14 +41,18 @@ class UnivocityGenerator( private val valueConverters: Array[ValueConverter] = schema.map(_.dataType).map(makeConverter).toArray + private val timeFormatter = DateTimeFormatter( + options.timestampFormat, + options.timeZone, + options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.timeZone, options.locale) + private def makeConverter(dataType: DataType): ValueConverter = dataType match { case DateType => - (row: InternalRow, ordinal: Int) => - options.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal))) + (row: InternalRow, ordinal: Int) => dateFormatter.format(row.getInt(ordinal)) case TimestampType => - (row: InternalRow, ordinal: Int) => - options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal))) + (row: InternalRow, ordinal: Int) => timeFormatter.format(row.getLong(ordinal)) case udt: UserDefinedType[_] => makeConverter(udt.sqlType) 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 8fff4b0781b1..0f375e036029 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 @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.csv import java.io.InputStream -import scala.util.Try import scala.util.control.NonFatal import com.univocity.parsers.csv.CsvParser @@ -27,7 +26,7 @@ import com.univocity.parsers.csv.CsvParser import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} -import org.apache.spark.sql.catalyst.util.{BadRecordException, DateTimeUtils, FailureSafeParser} +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -75,6 +74,12 @@ class UnivocityParser( private val row = new GenericInternalRow(requiredSchema.length) + private val timeFormatter = DateTimeFormatter( + options.timestampFormat, + options.timeZone, + options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.timeZone, options.locale) + // Retrieve the raw record string. private def getCurrentInput: UTF8String = { UTF8String.fromString(tokenizer.getContext.currentParsedContent().stripLineEnd) @@ -100,7 +105,7 @@ class UnivocityParser( // // output row - ["A", 2] private val valueConverters: Array[ValueConverter] = { - requiredSchema.map(f => makeConverter(f.name, f.dataType, f.nullable, options)).toArray + requiredSchema.map(f => makeConverter(f.name, f.dataType, f.nullable)).toArray } private val decimalParser = ExprUtils.getDecimalParser(options.locale) @@ -115,8 +120,7 @@ class UnivocityParser( def makeConverter( name: String, dataType: DataType, - nullable: Boolean = true, - options: CSVOptions): ValueConverter = dataType match { + nullable: Boolean = true): ValueConverter = dataType match { case _: ByteType => (d: String) => nullSafeDatum(d, name, nullable, options)(_.toByte) @@ -154,34 +158,16 @@ class UnivocityParser( } case _: TimestampType => (d: String) => - nullSafeDatum(d, name, nullable, options) { datum => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681. - Try(options.timestampFormat.parse(datum).getTime * 1000L) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.stringToTime(datum).getTime * 1000L - } - } + nullSafeDatum(d, name, nullable, options)(timeFormatter.parse) case _: DateType => (d: String) => - nullSafeDatum(d, name, nullable, options) { datum => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681.x - Try(DateTimeUtils.millisToDays(options.dateFormat.parse(datum).getTime)) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) - } - } + nullSafeDatum(d, name, nullable, options)(dateFormatter.parse) case _: StringType => (d: String) => nullSafeDatum(d, name, nullable, options)(UTF8String.fromString) case udt: UserDefinedType[_] => (datum: String) => - makeConverter(name, udt.sqlType, nullable, options) + makeConverter(name, udt.sqlType, nullable) // We don't actually hit this exception though, we keep it for understandability case _ => throw new RuntimeException(s"Unsupported type: ${dataType.typeName}") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala new file mode 100644 index 000000000000..ad1f4131de2f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.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.util + +import java.time._ +import java.time.format.DateTimeFormatterBuilder +import java.time.temporal.{ChronoField, TemporalQueries} +import java.util.{Locale, TimeZone} + +import scala.util.Try + +import org.apache.commons.lang3.time.FastDateFormat + +import org.apache.spark.sql.internal.SQLConf + +sealed trait DateTimeFormatter { + def parse(s: String): Long // returns microseconds since epoch + def format(us: Long): String +} + +class Iso8601DateTimeFormatter( + pattern: String, + timeZone: TimeZone, + locale: Locale) extends DateTimeFormatter { + val formatter = new DateTimeFormatterBuilder() + .appendPattern(pattern) + .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970) + .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) + .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) + .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) + .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) + .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) + .toFormatter(locale) + + def toInstant(s: String): Instant = { + val temporalAccessor = formatter.parse(s) + if (temporalAccessor.query(TemporalQueries.offset()) == null) { + val localDateTime = LocalDateTime.from(temporalAccessor) + val zonedDateTime = ZonedDateTime.of(localDateTime, timeZone.toZoneId) + Instant.from(zonedDateTime) + } else { + Instant.from(temporalAccessor) + } + } + + private def instantToMicros(instant: Instant): Long = { + val sec = Math.multiplyExact(instant.getEpochSecond, DateTimeUtils.MICROS_PER_SECOND) + val result = Math.addExact(sec, instant.getNano / DateTimeUtils.NANOS_PER_MICROS) + result + } + + def parse(s: String): Long = instantToMicros(toInstant(s)) + + def format(us: Long): String = { + val secs = Math.floorDiv(us, DateTimeUtils.MICROS_PER_SECOND) + val mos = Math.floorMod(us, DateTimeUtils.MICROS_PER_SECOND) + val instant = Instant.ofEpochSecond(secs, mos * DateTimeUtils.NANOS_PER_MICROS) + + formatter.withZone(timeZone.toZoneId).format(instant) + } +} + +class LegacyDateTimeFormatter( + pattern: String, + timeZone: TimeZone, + locale: Locale) extends DateTimeFormatter { + val format = FastDateFormat.getInstance(pattern, timeZone, locale) + + protected def toMillis(s: String): Long = format.parse(s).getTime + + def parse(s: String): Long = toMillis(s) * DateTimeUtils.MICROS_PER_MILLIS + + def format(us: Long): String = { + format.format(DateTimeUtils.toJavaTimestamp(us)) + } +} + +class LegacyFallbackDateTimeFormatter( + pattern: String, + timeZone: TimeZone, + locale: Locale) extends LegacyDateTimeFormatter(pattern, timeZone, locale) { + override def toMillis(s: String): Long = { + Try {super.toMillis(s)}.getOrElse(DateTimeUtils.stringToTime(s).getTime) + } +} + +object DateTimeFormatter { + def apply(format: String, timeZone: TimeZone, locale: Locale): DateTimeFormatter = { + if (SQLConf.get.legacyTimeParserEnabled) { + new LegacyFallbackDateTimeFormatter(format, timeZone, locale) + } else { + new Iso8601DateTimeFormatter(format, timeZone, locale) + } + } +} + +sealed trait DateFormatter { + def parse(s: String): Int // returns days since epoch + def format(days: Int): String +} + +class Iso8601DateFormatter( + pattern: String, + timeZone: TimeZone, + locale: Locale) extends DateFormatter { + + val dateTimeFormatter = new Iso8601DateTimeFormatter(pattern, timeZone, locale) + + override def parse(s: String): Int = { + val seconds = dateTimeFormatter.toInstant(s).getEpochSecond + val days = Math.floorDiv(seconds, DateTimeUtils.SECONDS_PER_DAY) + + days.toInt + } + + override def format(days: Int): String = { + val instant = Instant.ofEpochSecond(days * DateTimeUtils.SECONDS_PER_DAY) + dateTimeFormatter.formatter.withZone(timeZone.toZoneId).format(instant) + } +} + +class LegacyDateFormatter( + pattern: String, + timeZone: TimeZone, + locale: Locale) extends DateFormatter { + val format = FastDateFormat.getInstance(pattern, timeZone, locale) + + def parse(s: String): Int = { + val milliseconds = format.parse(s).getTime + DateTimeUtils.millisToDays(milliseconds) + } + + def format(days: Int): String = { + val date = DateTimeUtils.toJavaDate(days) + format.format(date) + } +} + +class LegacyFallbackDateFormatter( + pattern: String, + timeZone: TimeZone, + locale: Locale) extends LegacyDateFormatter(pattern, timeZone, locale) { + override def parse(s: String): Int = { + Try(super.parse(s)).orElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(s).getTime)) + }.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. + s.toInt + } + } +} + +object DateFormatter { + def apply(format: String, timeZone: TimeZone, locale: Locale): DateFormatter = { + if (SQLConf.get.legacyTimeParserEnabled) { + new LegacyFallbackDateFormatter(format, timeZone, locale) + } else { + new Iso8601DateFormatter(format, timeZone, locale) + } + } +} 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 5ae75dc93930..c6dfdbf2505b 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 @@ -50,7 +50,7 @@ object DateTimeUtils { final val MILLIS_PER_SECOND = 1000L final val NANOS_PER_SECOND = MICROS_PER_SECOND * 1000L final val MICROS_PER_DAY = MICROS_PER_SECOND * SECONDS_PER_DAY - + final val NANOS_PER_MICROS = 1000L final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L // number of days in 400 years 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 c4f00d723c25..451b051f8407 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 @@ -1618,6 +1618,13 @@ object SQLConf { "a SparkConf entry.") .booleanConf .createWithDefault(true) + + val LEGACY_TIME_PARSER_ENABLED = buildConf("spark.sql.legacy.timeParser.enabled") + .doc("When set to true, java.text.SimpleDateFormat is used for formatting and parsing " + + " dates/timestamps in a locale-sensitive manner. When set to false, classes from " + + "java.time.* packages are used for the same purpose.") + .booleanConf + .createWithDefault(false) } /** @@ -2040,6 +2047,8 @@ class SQLConf extends Serializable with Logging { def setCommandRejectsSparkConfs: Boolean = getConf(SQLConf.SET_COMMAND_REJECTS_SPARK_CONFS) + def legacyTimeParserEnabled: Boolean = getConf(SQLConf.LEGACY_TIME_PARSER_ENABLED) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 1a020e67a75b..c2b525ad1a9f 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 @@ -22,13 +22,12 @@ import java.util.Locale import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { +class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { test("String fields types are inferred correctly from null types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "GMT") val inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(NullType, "") == NullType) @@ -48,7 +47,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("String fields types are inferred correctly from other types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "GMT") val inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(LongType, "1.0") == DoubleType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index 7212402ef5cf..2d0b0d3033a9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -19,20 +19,17 @@ package org.apache.spark.sql.catalyst.csv import java.math.BigDecimal import java.text.{DecimalFormat, DecimalFormatSymbols} -import java.util.Locale +import java.util.{Locale, TimeZone} + +import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String class UnivocityParserSuite extends SparkFunSuite with SQLHelper { - private val parser = new UnivocityParser( - StructType(Seq.empty), - new CSVOptions(Map.empty[String, String], false, "GMT")) - private def assertNull(v: Any) = assert(v == null) test("Can parse decimal type values") { @@ -43,7 +40,8 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { stringValues.zip(decimalValues).foreach { case (strVal, decimalVal) => val decimalValue = new BigDecimal(decimalVal.toString) val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(parser.makeConverter("_1", decimalType, options = options).apply(strVal) === + val parser = new UnivocityParser(StructType(Seq.empty), options) + assert(parser.makeConverter("_1", decimalType).apply(strVal) === Decimal(decimalValue, decimalType.precision, decimalType.scale)) } } @@ -56,22 +54,23 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { types.foreach { t => // Tests that a custom nullValue. val nullValueOptions = new CSVOptions(Map("nullValue" -> "-"), false, "GMT") - val converter = - parser.makeConverter("_1", t, nullable = true, options = nullValueOptions) + var parser = new UnivocityParser(StructType(Seq.empty), nullValueOptions) + val converter = parser.makeConverter("_1", t, nullable = true) assertNull(converter.apply("-")) assertNull(converter.apply(null)) // Tests that the default nullValue is empty string. val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assertNull(parser.makeConverter("_1", t, nullable = true, options = options).apply("")) + parser = new UnivocityParser(StructType(Seq.empty), options) + assertNull(parser.makeConverter("_1", t, nullable = true).apply("")) } // Not nullable field with nullValue option. types.foreach { t => // Casts a null to not nullable field should throw an exception. val options = new CSVOptions(Map("nullValue" -> "-"), false, "GMT") - val converter = - parser.makeConverter("_1", t, nullable = false, options = options) + val parser = new UnivocityParser(StructType(Seq.empty), options) + val converter = parser.makeConverter("_1", t, nullable = false) var message = intercept[RuntimeException] { converter.apply("-") }.getMessage @@ -86,62 +85,74 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { // null. Seq(true, false).foreach { b => val options = new CSVOptions(Map("nullValue" -> "null"), false, "GMT") - val converter = - parser.makeConverter("_1", StringType, nullable = b, options = options) + val parser = new UnivocityParser(StructType(Seq.empty), options) + val converter = parser.makeConverter("_1", StringType, nullable = b) assert(converter.apply("") == UTF8String.fromString("")) } } test("Throws exception for empty string with non null type") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) val exception = intercept[RuntimeException]{ - parser.makeConverter("_1", IntegerType, nullable = false, options = options).apply("") + parser.makeConverter("_1", IntegerType, nullable = false).apply("") } assert(exception.getMessage.contains("null value found but field _1 is not nullable.")) } test("Types are cast correctly") { val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(parser.makeConverter("_1", ByteType, options = options).apply("10") == 10) - assert(parser.makeConverter("_1", ShortType, options = options).apply("10") == 10) - assert(parser.makeConverter("_1", IntegerType, options = options).apply("10") == 10) - assert(parser.makeConverter("_1", LongType, options = options).apply("10") == 10) - assert(parser.makeConverter("_1", FloatType, options = options).apply("1.00") == 1.0) - assert(parser.makeConverter("_1", DoubleType, options = options).apply("1.00") == 1.0) - assert(parser.makeConverter("_1", BooleanType, options = options).apply("true") == true) - - val timestampsOptions = + var parser = new UnivocityParser(StructType(Seq.empty), options) + assert(parser.makeConverter("_1", ByteType).apply("10") == 10) + assert(parser.makeConverter("_1", ShortType).apply("10") == 10) + assert(parser.makeConverter("_1", IntegerType).apply("10") == 10) + assert(parser.makeConverter("_1", LongType).apply("10") == 10) + assert(parser.makeConverter("_1", FloatType).apply("1.00") == 1.0) + assert(parser.makeConverter("_1", DoubleType).apply("1.00") == 1.0) + assert(parser.makeConverter("_1", BooleanType).apply("true") == true) + + var timestampsOptions = new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy hh:mm"), false, "GMT") + parser = new UnivocityParser(StructType(Seq.empty), timestampsOptions) val customTimestamp = "31/01/2015 00:00" - val expectedTime = timestampsOptions.timestampFormat.parse(customTimestamp).getTime - val castedTimestamp = - parser.makeConverter("_1", TimestampType, nullable = true, options = timestampsOptions) + var format = FastDateFormat.getInstance( + timestampsOptions.timestampFormat, timestampsOptions.timeZone, timestampsOptions.locale) + val expectedTime = format.parse(customTimestamp).getTime + val castedTimestamp = parser.makeConverter("_1", TimestampType, nullable = true) .apply(customTimestamp) assert(castedTimestamp == expectedTime * 1000L) val customDate = "31/01/2015" val dateOptions = new CSVOptions(Map("dateFormat" -> "dd/MM/yyyy"), false, "GMT") - val expectedDate = dateOptions.dateFormat.parse(customDate).getTime - val castedDate = - parser.makeConverter("_1", DateType, nullable = true, options = dateOptions) - .apply(customTimestamp) - assert(castedDate == DateTimeUtils.millisToDays(expectedDate)) + parser = new UnivocityParser(StructType(Seq.empty), dateOptions) + format = FastDateFormat.getInstance( + dateOptions.dateFormat, dateOptions.timeZone, dateOptions.locale) + val expectedDate = format.parse(customDate).getTime + val castedDate = parser.makeConverter("_1", DateType, nullable = true) + .apply(customDate) + assert(castedDate == DateTimeUtils.millisToDays(expectedDate, TimeZone.getTimeZone("GMT"))) val timestamp = "2015-01-01 00:00:00" - assert(parser.makeConverter("_1", TimestampType, options = options).apply(timestamp) == - DateTimeUtils.stringToTime(timestamp).getTime * 1000L) - assert(parser.makeConverter("_1", DateType, options = options).apply("2015-01-01") == - DateTimeUtils.millisToDays(DateTimeUtils.stringToTime("2015-01-01").getTime)) + timestampsOptions = new CSVOptions(Map( + "timestampFormat" -> "yyyy-MM-dd HH:mm:ss", + "dateFormat" -> "yyyy-MM-dd"), false, "UTC") + parser = new UnivocityParser(StructType(Seq.empty), timestampsOptions) + val expected = 1420070400 * DateTimeUtils.MICROS_PER_SECOND + assert(parser.makeConverter("_1", TimestampType).apply(timestamp) == + expected) + assert(parser.makeConverter("_1", DateType).apply("2015-01-01") == + expected / DateTimeUtils.MICROS_PER_DAY) } test("Throws exception for casting an invalid string to Float and Double Types") { val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) val types = Seq(DoubleType, FloatType) val input = Seq("10u000", "abc", "1 2/3") types.foreach { dt => input.foreach { v => val message = intercept[NumberFormatException] { - parser.makeConverter("_1", dt, options = options).apply(v) + parser.makeConverter("_1", dt).apply(v) }.getMessage assert(message.contains(v)) } @@ -150,9 +161,9 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { test("Float NaN values are parsed correctly") { val options = new CSVOptions(Map("nanValue" -> "nn"), false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) val floatVal: Float = parser.makeConverter( - "_1", FloatType, nullable = true, options = options - ).apply("nn").asInstanceOf[Float] + "_1", FloatType, nullable = true).apply("nn").asInstanceOf[Float] // Java implements the IEEE-754 floating point standard which guarantees that any comparison // against NaN will return false (except != which returns true) @@ -161,41 +172,41 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { test("Double NaN values are parsed correctly") { val options = new CSVOptions(Map("nanValue" -> "-"), false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) val doubleVal: Double = parser.makeConverter( - "_1", DoubleType, nullable = true, options = options - ).apply("-").asInstanceOf[Double] + "_1", DoubleType, nullable = true).apply("-").asInstanceOf[Double] assert(doubleVal.isNaN) } test("Float infinite values can be parsed") { val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT") + var parser = new UnivocityParser(StructType(Seq.empty), negativeInfOptions) val floatVal1 = parser.makeConverter( - "_1", FloatType, nullable = true, options = negativeInfOptions - ).apply("max").asInstanceOf[Float] + "_1", FloatType, nullable = true).apply("max").asInstanceOf[Float] assert(floatVal1 == Float.NegativeInfinity) val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "GMT") + parser = new UnivocityParser(StructType(Seq.empty), positiveInfOptions) val floatVal2 = parser.makeConverter( - "_1", FloatType, nullable = true, options = positiveInfOptions - ).apply("max").asInstanceOf[Float] + "_1", FloatType, nullable = true).apply("max").asInstanceOf[Float] assert(floatVal2 == Float.PositiveInfinity) } test("Double infinite values can be parsed") { val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT") + var parser = new UnivocityParser(StructType(Seq.empty), negativeInfOptions) val doubleVal1 = parser.makeConverter( - "_1", DoubleType, nullable = true, options = negativeInfOptions - ).apply("max").asInstanceOf[Double] + "_1", DoubleType, nullable = true).apply("max").asInstanceOf[Double] assert(doubleVal1 == Double.NegativeInfinity) val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "GMT") + parser = new UnivocityParser(StructType(Seq.empty), positiveInfOptions) val doubleVal2 = parser.makeConverter( - "_1", DoubleType, nullable = true, options = positiveInfOptions - ).apply("max").asInstanceOf[Double] + "_1", DoubleType, nullable = true).apply("max").asInstanceOf[Double] assert(doubleVal2 == Double.PositiveInfinity) } @@ -211,7 +222,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { val options = new CSVOptions(Map("locale" -> langTag), false, "GMT") val parser = new UnivocityParser(new StructType().add("d", decimalType), options) - assert(parser.makeConverter("_1", decimalType, options = options).apply(input) === expected) + assert(parser.makeConverter("_1", decimalType).apply(input) === expected) } Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach(checkDecimalParsing) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala index dfa0fe93a2f9..66d8d28988f8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala @@ -26,7 +26,7 @@ object DateTimeTestUtils { val ALL_TIMEZONES: Seq[TimeZone] = TimeZone.getAvailableIDs.toSeq.map(TimeZone.getTimeZone) - val outstandingTimezones: Seq[TimeZone] = Seq( + val outstandingTimezonesIds: Seq[String] = Seq( "UTC", "PST", "CET", @@ -34,7 +34,8 @@ object DateTimeTestUtils { "America/Los_Angeles", "Antarctica/Vostok", "Asia/Hong_Kong", - "Europe/Amsterdam").map(TimeZone.getTimeZone) + "Europe/Amsterdam") + val outstandingTimezones: Seq[TimeZone] = outstandingTimezonesIds.map(TimeZone.getTimeZone) def withDefaultTimeZone[T](newDefaultTimeZone: TimeZone)(block: => T): T = { val originalDefaultTimeZone = TimeZone.getDefault diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimeFormatterSuite.scala new file mode 100644 index 000000000000..02d4ee049060 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimeFormatterSuite.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.util + +import java.util.{Locale, TimeZone} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeFormatter, DateTimeTestUtils} + +class DateTimeFormatterSuite extends SparkFunSuite { + test("parsing dates using time zones") { + val localDate = "2018-12-02" + val expectedDays = Map( + "UTC" -> 17867, + "PST" -> 17867, + "CET" -> 17866, + "Africa/Dakar" -> 17867, + "America/Los_Angeles" -> 17867, + "Antarctica/Vostok" -> 17866, + "Asia/Hong_Kong" -> 17866, + "Europe/Amsterdam" -> 17866) + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + val formatter = DateFormatter("yyyy-MM-dd", TimeZone.getTimeZone(timeZone), Locale.US) + val daysSinceEpoch = formatter.parse(localDate) + assert(daysSinceEpoch === expectedDays(timeZone)) + } + } + + test("parsing timestamps using time zones") { + val localDate = "2018-12-02T10:11:12.001234" + val expectedMicros = Map( + "UTC" -> 1543745472001234L, + "PST" -> 1543774272001234L, + "CET" -> 1543741872001234L, + "Africa/Dakar" -> 1543745472001234L, + "America/Los_Angeles" -> 1543774272001234L, + "Antarctica/Vostok" -> 1543723872001234L, + "Asia/Hong_Kong" -> 1543716672001234L, + "Europe/Amsterdam" -> 1543741872001234L) + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + val formatter = DateTimeFormatter( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + TimeZone.getTimeZone(timeZone), + Locale.US) + val microsSinceEpoch = formatter.parse(localDate) + assert(microsSinceEpoch === expectedMicros(timeZone)) + } + } + + test("format dates using time zones") { + val daysSinceEpoch = 17867 + val expectedDate = Map( + "UTC" -> "2018-12-02", + "PST" -> "2018-12-01", + "CET" -> "2018-12-02", + "Africa/Dakar" -> "2018-12-02", + "America/Los_Angeles" -> "2018-12-01", + "Antarctica/Vostok" -> "2018-12-02", + "Asia/Hong_Kong" -> "2018-12-02", + "Europe/Amsterdam" -> "2018-12-02") + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + val formatter = DateFormatter("yyyy-MM-dd", TimeZone.getTimeZone(timeZone), Locale.US) + val date = formatter.format(daysSinceEpoch) + assert(date === expectedDate(timeZone)) + } + } + + test("format timestamps using time zones") { + val microsSinceEpoch = 1543745472001234L + val expectedTimestamp = Map( + "UTC" -> "2018-12-02T10:11:12.001234", + "PST" -> "2018-12-02T02:11:12.001234", + "CET" -> "2018-12-02T11:11:12.001234", + "Africa/Dakar" -> "2018-12-02T10:11:12.001234", + "America/Los_Angeles" -> "2018-12-02T02:11:12.001234", + "Antarctica/Vostok" -> "2018-12-02T16:11:12.001234", + "Asia/Hong_Kong" -> "2018-12-02T18:11:12.001234", + "Europe/Amsterdam" -> "2018-12-02T11:11:12.001234") + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + val formatter = DateTimeFormatter( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + TimeZone.getTimeZone(timeZone), + Locale.US) + val timestamp = formatter.format(microsSinceEpoch) + assert(timestamp === expectedTimestamp(timeZone)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index 537d13b1bc8d..6b67fccf86b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -53,7 +53,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSQLContext { test("checking the columnNameOfCorruptRecord option") { val columnNameOfCorruptRecord = "_unparsed" val df = Seq("0,2013-111-11 12:13:14", "1,1983-08-04").toDS() - val schema = new StructType().add("a", IntegerType).add("b", TimestampType) + val schema = new StructType().add("a", IntegerType).add("b", DateType) val schemaWithCorrField1 = schema.add(columnNameOfCorruptRecord, StringType) val df2 = df .select(from_csv($"value", schemaWithCorrField1, Map( 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 c9273193b642..3b977d74053e 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 @@ -586,6 +586,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val results = spark.read .format("csv") .options(Map("comment" -> "~", "header" -> "false", "inferSchema" -> "true")) + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") .load(testFile(commentsFile)) .collect() @@ -622,10 +623,11 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val options = Map( "header" -> "true", "inferSchema" -> "false", - "dateFormat" -> "dd/MM/yyyy hh:mm") + "dateFormat" -> "dd/MM/yyyy HH:mm") val results = spark.read .format("csv") .options(options) + .option("timeZone", "UTC") .schema(customSchema) .load(testFile(datesFile)) .select("date") @@ -893,36 +895,38 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te } test("Write dates correctly in ISO8601 format by default") { - withTempDir { dir => - val customSchema = new StructType(Array(StructField("date", DateType, true))) - val iso8601datesPath = s"${dir.getCanonicalPath}/iso8601dates.csv" - val dates = spark.read - .format("csv") - .schema(customSchema) - .option("header", "true") - .option("inferSchema", "false") - .option("dateFormat", "dd/MM/yyyy HH:mm") - .load(testFile(datesFile)) - dates.write - .format("csv") - .option("header", "true") - .save(iso8601datesPath) + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + withTempDir { dir => + val customSchema = new StructType(Array(StructField("date", DateType, true))) + val iso8601datesPath = s"${dir.getCanonicalPath}/iso8601dates.csv" + val dates = spark.read + .format("csv") + .schema(customSchema) + .option("header", "true") + .option("inferSchema", "false") + .option("dateFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + dates.write + .format("csv") + .option("header", "true") + .save(iso8601datesPath) - // This will load back the dates as string. - val stringSchema = StructType(StructField("date", StringType, true) :: Nil) - val iso8601dates = spark.read - .format("csv") - .schema(stringSchema) - .option("header", "true") - .load(iso8601datesPath) + // This will load back the dates as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val iso8601dates = spark.read + .format("csv") + .schema(stringSchema) + .option("header", "true") + .load(iso8601datesPath) + + val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd", Locale.US) + val expectedDates = dates.collect().map { r => + // This should be ISO8601 formatted string. + Row(iso8501.format(r.toSeq.head)) + } - val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd", Locale.US) - val expectedDates = dates.collect().map { r => - // This should be ISO8601 formatted string. - Row(iso8501.format(r.toSeq.head)) + checkAnswer(iso8601dates, expectedDates) } - - checkAnswer(iso8601dates, expectedDates) } } @@ -1107,7 +1111,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te test("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") { Seq(false, true).foreach { multiLine => - val schema = new StructType().add("a", IntegerType).add("b", TimestampType) + val schema = new StructType().add("a", IntegerType).add("b", DateType) // We use `PERMISSIVE` mode by default if invalid string is given. val df1 = spark .read @@ -1139,7 +1143,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val schemaWithCorrField2 = new StructType() .add("a", IntegerType) .add(columnNameOfCorruptRecord, StringType) - .add("b", TimestampType) + .add("b", DateType) val df3 = spark .read .option("mode", "permissive") @@ -1325,7 +1329,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val columnNameOfCorruptRecord = "_corrupt_record" val schema = new StructType() .add("a", IntegerType) - .add("b", TimestampType) + .add("b", DateType) .add(columnNameOfCorruptRecord, StringType) // negative cases val msg = intercept[AnalysisException] { From 556d83e0d87a8f899f29544eb5ca4999a84c96c1 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Tue, 4 Dec 2018 10:33:27 -0800 Subject: [PATCH 0069/1072] [SPARK-26233][SQL] CheckOverflow when encoding a decimal value ## What changes were proposed in this pull request? When we encode a Decimal from external source we don't check for overflow. That method is useful not only in order to enforce that we can represent the correct value in the specified range, but it also changes the underlying data to the right precision/scale. Since in our code generation we assume that a decimal has exactly the same precision and scale of its data type, missing to enforce it can lead to corrupted output/results when there are subsequent transformations. ## How was this patch tested? added UT Closes #23210 from mgaido91/SPARK-26233. Authored-by: Marco Gaido Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/encoders/RowEncoder.scala | 4 ++-- .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 9 +++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index d905f8f9858e..8ca3d356f3bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -106,12 +106,12 @@ object RowEncoder { returnNullable = false) case d: DecimalType => - StaticInvoke( + CheckOverflow(StaticInvoke( Decimal.getClass, d, "fromDecimal", inputObject :: Nil, - returnNullable = false) + returnNullable = false), d) case StringType => StaticInvoke( 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 0f900833d2cf..525c7cef3956 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 @@ -1647,6 +1647,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { checkDataset(ds, data: _*) checkAnswer(ds.select("x"), Seq(Row(1), Row(2))) } + + test("SPARK-26233: serializer should enforce decimal precision and scale") { + val s = StructType(Seq(StructField("a", StringType), StructField("b", DecimalType(38, 8)))) + val encoder = RowEncoder(s) + implicit val uEnc = encoder + val df = spark.range(2).map(l => Row(l.toString, BigDecimal.valueOf(l + 0.1111))) + checkAnswer(df.groupBy(col("a")).agg(first(col("b"))), + Seq(Row("0", BigDecimal.valueOf(0.1111)), Row("1", BigDecimal.valueOf(1.1111)))) + } } case class TestDataUnion(x: Int, y: Int, z: Int) From 35f9163adf5c067229afbe57ed60d5dd5f2422c8 Mon Sep 17 00:00:00 2001 From: Shahid Date: Tue, 4 Dec 2018 11:00:58 -0800 Subject: [PATCH 0070/1072] [SPARK-26119][CORE][WEBUI] Task summary table should contain only successful tasks' metrics ## What changes were proposed in this pull request? Task summary table in the stage page currently displays the summary of all the tasks. However, we should display the task summary of only successful tasks, to follow the behavior of previous versions of spark. ## How was this patch tested? Added UT. attached screenshot Before patch: ![screenshot from 2018-11-20 00-36-18](https://user-images.githubusercontent.com/23054875/48729339-62e3a580-ec5d-11e8-81f0-0d191a234ffe.png) ![screenshot from 2018-11-20 01-18-37](https://user-images.githubusercontent.com/23054875/48731112-41d18380-ec62-11e8-8c31-1ffbfa04e746.png) Closes #23088 from shahidki31/summaryMetrics. Authored-by: Shahid Signed-off-by: Marcelo Vanzin --- .../apache/spark/status/AppStatusStore.scala | 73 +++++++++++++------ .../spark/status/AppStatusStoreSuite.scala | 33 ++++++++- 2 files changed, 81 insertions(+), 25 deletions(-) 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 5c0ed4d5d8f4..b35781cb36e8 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -148,11 +148,20 @@ private[spark] class AppStatusStore( // cheaper for disk stores (avoids deserialization). val count = { Utils.tryWithResource( - store.view(classOf[TaskDataWrapper]) - .parent(stageKey) - .index(TaskIndexNames.EXEC_RUN_TIME) - .first(0L) - .closeableIterator() + if (store.isInstanceOf[InMemoryStore]) { + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(TaskIndexNames.STATUS) + .first("SUCCESS") + .last("SUCCESS") + .closeableIterator() + } else { + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(TaskIndexNames.EXEC_RUN_TIME) + .first(0L) + .closeableIterator() + } ) { it => var _count = 0L while (it.hasNext()) { @@ -221,30 +230,50 @@ private[spark] class AppStatusStore( // stabilize once the stage finishes. It's also slow, especially with disk stores. val indices = quantiles.map { q => math.min((q * count).toLong, count - 1) } + // TODO: Summary metrics needs to display all the successful tasks' metrics (SPARK-26119). + // For InMemory case, it is efficient to find using the following code. But for diskStore case + // we need an efficient solution to avoid deserialization time overhead. For that, we need to + // rework on the way indexing works, so that we can index by specific metrics for successful + // and failed tasks differently (would be tricky). Also would require changing the disk store + // version (to invalidate old stores). def scanTasks(index: String)(fn: TaskDataWrapper => Long): IndexedSeq[Double] = { - Utils.tryWithResource( - store.view(classOf[TaskDataWrapper]) + if (store.isInstanceOf[InMemoryStore]) { + val quantileTasks = store.view(classOf[TaskDataWrapper]) .parent(stageKey) .index(index) .first(0L) - .closeableIterator() - ) { it => - var last = Double.NaN - var currentIdx = -1L - indices.map { idx => - if (idx == currentIdx) { - last - } else { - val diff = idx - currentIdx - currentIdx = idx - if (it.skip(diff - 1)) { - last = fn(it.next()).toDouble + .asScala + .filter { _.status == "SUCCESS"} // Filter "SUCCESS" tasks + .toIndexedSeq + + indices.map { index => + fn(quantileTasks(index.toInt)).toDouble + }.toIndexedSeq + } else { + Utils.tryWithResource( + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(index) + .first(0L) + .closeableIterator() + ) { it => + var last = Double.NaN + var currentIdx = -1L + indices.map { idx => + if (idx == currentIdx) { last } else { - Double.NaN + val diff = idx - currentIdx + currentIdx = idx + if (it.skip(diff - 1)) { + last = fn(it.next()).toDouble + last + } else { + Double.NaN + } } - } - }.toIndexedSeq + }.toIndexedSeq + } } } diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala index 92f90f3d96dd..75a658161d3f 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala @@ -77,6 +77,34 @@ class AppStatusStoreSuite extends SparkFunSuite { assert(store.count(classOf[CachedQuantile]) === 2) } + test("only successfull task have taskSummary") { + val store = new InMemoryStore() + (0 until 5).foreach { i => store.write(newTaskData(i, status = "FAILED")) } + val appStore = new AppStatusStore(store).taskSummary(stageId, attemptId, uiQuantiles) + assert(appStore.size === 0) + } + + test("summary should contain task metrics of only successfull tasks") { + val store = new InMemoryStore() + + for (i <- 0 to 5) { + if (i % 2 == 1) { + store.write(newTaskData(i, status = "FAILED")) + } else { + store.write(newTaskData(i)) + } + } + + val summary = new AppStatusStore(store).taskSummary(stageId, attemptId, uiQuantiles).get + + val values = Array(0.0, 2.0, 4.0) + + val dist = new Distribution(values, 0, values.length).getQuantiles(uiQuantiles.sorted) + dist.zip(summary.executorRunTime).foreach { case (expected, actual) => + assert(expected === actual) + } + } + private def compareQuantiles(count: Int, quantiles: Array[Double]): Unit = { val store = new InMemoryStore() val values = (0 until count).map { i => @@ -93,12 +121,11 @@ class AppStatusStoreSuite extends SparkFunSuite { } } - private def newTaskData(i: Int): TaskDataWrapper = { + private def newTaskData(i: Int, status: String = "SUCCESS"): TaskDataWrapper = { new TaskDataWrapper( - i, i, i, i, i, i, i.toString, i.toString, i.toString, i.toString, false, Nil, None, + i, i, i, i, i, i, i.toString, i.toString, status, i.toString, false, Nil, None, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, stageId, attemptId) } - } From 180f969c97a66b4c265e5fad8272665a00572f1a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 4 Dec 2018 14:35:04 -0800 Subject: [PATCH 0071/1072] [SPARK-26094][CORE][STREAMING] createNonEcFile creates parent dirs. ## What changes were proposed in this pull request? We explicitly avoid files with hdfs erasure coding for the streaming WAL and for event logs, as hdfs EC does not support all relevant apis. However, the new builder api used has different semantics -- it does not create parent dirs, and it does not resolve relative paths. This updates createNonEcFile to have similar semantics to the old api. ## How was this patch tested? Ran tests with the WAL pointed at a non-existent dir, which failed before this change. Manually tested the new function with a relative path as well. Unit tests via jenkins. Closes #23092 from squito/SPARK-26094. Authored-by: Imran Rashid Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 7bb2a419107d..937199273dab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -466,7 +466,13 @@ private[spark] object SparkHadoopUtil { try { // Use reflection as this uses apis only avialable in hadoop 3 val builderMethod = fs.getClass().getMethod("createFile", classOf[Path]) - val builder = builderMethod.invoke(fs, path) + // the builder api does not resolve relative paths, nor does it create parent dirs, while + // the old api does. + if (!fs.mkdirs(path.getParent())) { + throw new IOException(s"Failed to create parents of $path") + } + val qualifiedPath = fs.makeQualified(path) + val builder = builderMethod.invoke(fs, qualifiedPath) val builderCls = builder.getClass() // this may throw a NoSuchMethodException if the path is not on hdfs val replicateMethod = builderCls.getMethod("replicate") From 7143e9d7220bd98ceb82c5c5f045108a8a664ec1 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Wed, 5 Dec 2018 09:12:24 +0800 Subject: [PATCH 0072/1072] [SPARK-25829][SQL][FOLLOWUP] Refactor MapConcat in order to check properly the limit size ## What changes were proposed in this pull request? The PR starts from the [comment](https://github.com/apache/spark/pull/23124#discussion_r236112390) in the main one and it aims at: - simplifying the code for `MapConcat`; - be more precise in checking the limit size. ## How was this patch tested? existing tests Closes #23217 from mgaido91/SPARK-25829_followup. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan --- .../expressions/collectionOperations.scala | 77 +------------------ .../catalyst/util/ArrayBasedMapBuilder.scala | 10 +++ 2 files changed, 12 insertions(+), 75 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 fa8e38acd522..67f6739b1e18 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 @@ -554,13 +554,6 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres return null } - val numElements = maps.foldLeft(0L)((sum, ad) => sum + ad.numElements()) - if (numElements > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw new RuntimeException(s"Unsuccessful attempt to concat maps with $numElements " + - s"elements due to exceeding the map size limit " + - s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") - } - for (map <- maps) { mapBuilder.putAll(map.keyArray(), map.valueArray()) } @@ -569,8 +562,6 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val mapCodes = children.map(_.genCode(ctx)) - val keyType = dataType.keyType - val valueType = dataType.valueType val argsName = ctx.freshName("args") val hasNullName = ctx.freshName("hasNull") val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) @@ -610,41 +601,12 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres ) val idxName = ctx.freshName("idx") - val numElementsName = ctx.freshName("numElems") - val finKeysName = ctx.freshName("finalKeys") - val finValsName = ctx.freshName("finalValues") - - val keyConcat = genCodeForArrays(ctx, keyType, false) - - val valueConcat = - if (valueType.sameType(keyType) && - !(CodeGenerator.isPrimitiveType(valueType) && dataType.valueContainsNull)) { - keyConcat - } else { - genCodeForArrays(ctx, valueType, dataType.valueContainsNull) - } - - val keyArgsName = ctx.freshName("keyArgs") - val valArgsName = ctx.freshName("valArgs") - val mapMerge = s""" - |ArrayData[] $keyArgsName = new ArrayData[${mapCodes.size}]; - |ArrayData[] $valArgsName = new ArrayData[${mapCodes.size}]; - |long $numElementsName = 0; |for (int $idxName = 0; $idxName < $argsName.length; $idxName++) { - | $keyArgsName[$idxName] = $argsName[$idxName].keyArray(); - | $valArgsName[$idxName] = $argsName[$idxName].valueArray(); - | $numElementsName += $argsName[$idxName].numElements(); + | $builderTerm.putAll($argsName[$idxName].keyArray(), $argsName[$idxName].valueArray()); |} - |if ($numElementsName > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | throw new RuntimeException("Unsuccessful attempt to concat maps with " + - | $numElementsName + " elements due to exceeding the map size limit " + - | "${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}."); - |} - |ArrayData $finKeysName = $keyConcat($keyArgsName, (int) $numElementsName); - |ArrayData $finValsName = $valueConcat($valArgsName, (int) $numElementsName); - |${ev.value} = $builderTerm.from($finKeysName, $finValsName); + |${ev.value} = $builderTerm.build(); """.stripMargin ev.copy( @@ -660,41 +622,6 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres """.stripMargin) } - private def genCodeForArrays( - ctx: CodegenContext, - elementType: DataType, - checkForNull: Boolean): String = { - val counter = ctx.freshName("counter") - val arrayData = ctx.freshName("arrayData") - val argsName = ctx.freshName("args") - val numElemName = ctx.freshName("numElements") - val y = ctx.freshName("y") - val z = ctx.freshName("z") - - val allocation = CodeGenerator.createArrayData( - arrayData, elementType, numElemName, s" $prettyName failed.") - val assignment = CodeGenerator.createArrayAssignment( - arrayData, elementType, s"$argsName[$y]", counter, z, checkForNull) - - val concat = ctx.freshName("concat") - val concatDef = - s""" - |private ArrayData $concat(ArrayData[] $argsName, int $numElemName) { - | $allocation - | int $counter = 0; - | for (int $y = 0; $y < ${children.length}; $y++) { - | for (int $z = 0; $z < $argsName[$y].numElements(); $z++) { - | $assignment - | $counter++; - | } - | } - | return $arrayData; - |} - """.stripMargin - - ctx.addNewFunction(concat, concatDef) - } - override def prettyName: String = "map_concat" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala index e7cd61655dc9..98934368205e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala @@ -21,6 +21,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.array.ByteArrayMethods /** * A builder of [[ArrayBasedMapData]], which fails if a null map key is detected, and removes @@ -54,6 +55,10 @@ class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Seria val index = keyToIndex.getOrDefault(key, -1) if (index == -1) { + if (size >= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw new RuntimeException(s"Unsuccessful attempt to build maps with $size elements " + + s"due to exceeding the map size limit ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") + } keyToIndex.put(key, values.length) keys.append(key) values.append(value) @@ -117,4 +122,9 @@ class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Seria build() } } + + /** + * Returns the current size of the map which is going to be produced by the current builder. + */ + def size: Int = keys.size } From 7e3eb3cd209d83394ca2b2cec79b26b1bbe9d7ea Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 5 Dec 2018 15:22:08 +0800 Subject: [PATCH 0073/1072] [SPARK-26252][PYTHON] Add support to run specific unittests and/or doctests in python/run-tests script ## What changes were proposed in this pull request? This PR proposes add a developer option, `--testnames`, to our testing script to allow run specific set of unittests and doctests. **1. Run unittests in the class** ```bash ./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests' ``` ``` Running PySpark tests. Output is in /.../spark/python/unit-tests.log Will test against the following Python executables: ['python2.7', 'pypy'] Will test the following Python tests: ['pyspark.sql.tests.test_arrow ArrowTests'] Starting test(python2.7): pyspark.sql.tests.test_arrow ArrowTests Starting test(pypy): pyspark.sql.tests.test_arrow ArrowTests Finished test(python2.7): pyspark.sql.tests.test_arrow ArrowTests (14s) Finished test(pypy): pyspark.sql.tests.test_arrow ArrowTests (14s) ... 22 tests were skipped Tests passed in 14 seconds Skipped tests in pyspark.sql.tests.test_arrow ArrowTests with pypy: test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.' test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.' test_createDataFrame_fallback_disabled (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.' test_createDataFrame_fallback_enabled (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped ... ``` **2. Run single unittest in the class.** ```bash ./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion' ``` ``` Running PySpark tests. Output is in /.../spark/python/unit-tests.log Will test against the following Python executables: ['python2.7', 'pypy'] Will test the following Python tests: ['pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion'] Starting test(pypy): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion Starting test(python2.7): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion Finished test(pypy): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion (0s) ... 1 tests were skipped Finished test(python2.7): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion (8s) Tests passed in 8 seconds Skipped tests in pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion with pypy: test_null_conversion (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.' ``` **3. Run doctests in single PySpark module.** ```bash ./run-tests --testnames pyspark.sql.dataframe ``` ``` Running PySpark tests. Output is in /.../spark/python/unit-tests.log Will test against the following Python executables: ['python2.7', 'pypy'] Will test the following Python tests: ['pyspark.sql.dataframe'] Starting test(pypy): pyspark.sql.dataframe Starting test(python2.7): pyspark.sql.dataframe Finished test(python2.7): pyspark.sql.dataframe (47s) Finished test(pypy): pyspark.sql.dataframe (48s) Tests passed in 48 seconds ``` Of course, you can mix them: ```bash ./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests,pyspark.sql.dataframe' ``` ``` Running PySpark tests. Output is in /.../spark/python/unit-tests.log Will test against the following Python executables: ['python2.7', 'pypy'] Will test the following Python tests: ['pyspark.sql.tests.test_arrow ArrowTests', 'pyspark.sql.dataframe'] Starting test(pypy): pyspark.sql.dataframe Starting test(pypy): pyspark.sql.tests.test_arrow ArrowTests Starting test(python2.7): pyspark.sql.dataframe Starting test(python2.7): pyspark.sql.tests.test_arrow ArrowTests Finished test(pypy): pyspark.sql.tests.test_arrow ArrowTests (0s) ... 22 tests were skipped Finished test(python2.7): pyspark.sql.tests.test_arrow ArrowTests (18s) Finished test(python2.7): pyspark.sql.dataframe (50s) Finished test(pypy): pyspark.sql.dataframe (52s) Tests passed in 52 seconds Skipped tests in pyspark.sql.tests.test_arrow ArrowTests with pypy: test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.' test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.' test_createDataFrame_fallback_disabled (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.' ``` and also you can use all other options (except `--modules`, which will be ignored) ```bash ./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion' --python-executables=python ``` ``` Running PySpark tests. Output is in /.../spark/python/unit-tests.log Will test against the following Python executables: ['python'] Will test the following Python tests: ['pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion'] Starting test(python): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion Finished test(python): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion (12s) Tests passed in 12 seconds ``` See help below: ```bash ./run-tests --help ``` ``` Usage: run-tests [options] Options: ... Developer Options: --testnames=TESTNAMES A comma-separated list of specific modules, classes and functions of doctest or unittest to test. For example, 'pyspark.sql.foo' to run the module as unittests or doctests, 'pyspark.sql.tests FooTests' to run the specific class of unittests, 'pyspark.sql.tests FooTests.test_foo' to run the specific unittest in the class. '--modules' option is ignored if they are given. ``` I intentionally grouped it as a developer option to be more conservative. ## How was this patch tested? Manually tested. Negative tests were also done. ```bash ./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion1' --python-executables=python ``` ``` ... AttributeError: type object 'ArrowTests' has no attribute 'test_null_conversion1' ... ``` ```bash ./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowT' --python-executables=python ``` ``` ... AttributeError: 'module' object has no attribute 'ArrowT' ... ``` ```bash ./run-tests --testnames 'pyspark.sql.tests.test_ar' --python-executables=python ``` ``` ... /.../python2.7: No module named pyspark.sql.tests.test_ar ``` Closes #23203 from HyukjinKwon/SPARK-26252. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/run-tests-with-coverage | 2 - python/run-tests.py | 68 +++++++++++++++++++++++----------- 2 files changed, 46 insertions(+), 24 deletions(-) diff --git a/python/run-tests-with-coverage b/python/run-tests-with-coverage index 6d74b563e914..457821037d43 100755 --- a/python/run-tests-with-coverage +++ b/python/run-tests-with-coverage @@ -50,8 +50,6 @@ export SPARK_CONF_DIR="$COVERAGE_DIR/conf" # This environment variable enables the coverage. export COVERAGE_PROCESS_START="$FWDIR/.coveragerc" -# If you'd like to run a specific unittest class, you could do such as -# SPARK_TESTING=1 ../bin/pyspark pyspark.sql.tests VectorizedUDFTests ./run-tests "$@" # Don't run coverage for the coverage command itself diff --git a/python/run-tests.py b/python/run-tests.py index 01a6e81264dd..e45268c13769 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -19,7 +19,7 @@ from __future__ import print_function import logging -from optparse import OptionParser +from optparse import OptionParser, OptionGroup import os import re import shutil @@ -99,7 +99,7 @@ def run_individual_python_test(target_dir, test_name, pyspark_python): try: per_test_output = tempfile.TemporaryFile() retcode = subprocess.Popen( - [os.path.join(SPARK_HOME, "bin/pyspark"), test_name], + [os.path.join(SPARK_HOME, "bin/pyspark")] + test_name.split(), stderr=per_test_output, stdout=per_test_output, env=env).wait() shutil.rmtree(tmp_dir, ignore_errors=True) except: @@ -190,6 +190,20 @@ def parse_opts(): help="Enable additional debug logging" ) + group = OptionGroup(parser, "Developer Options") + group.add_option( + "--testnames", type="string", + default=None, + help=( + "A comma-separated list of specific modules, classes and functions of doctest " + "or unittest to test. " + "For example, 'pyspark.sql.foo' to run the module as unittests or doctests, " + "'pyspark.sql.tests FooTests' to run the specific class of unittests, " + "'pyspark.sql.tests FooTests.test_foo' to run the specific unittest in the class. " + "'--modules' option is ignored if they are given.") + ) + parser.add_option_group(group) + (opts, args) = parser.parse_args() if args: parser.error("Unsupported arguments: %s" % ' '.join(args)) @@ -213,25 +227,31 @@ def _check_coverage(python_exec): def main(): opts = parse_opts() - if (opts.verbose): + if opts.verbose: log_level = logging.DEBUG else: log_level = logging.INFO + should_test_modules = opts.testnames is None logging.basicConfig(stream=sys.stdout, level=log_level, format="%(message)s") LOGGER.info("Running PySpark tests. Output is in %s", LOG_FILE) if os.path.exists(LOG_FILE): os.remove(LOG_FILE) python_execs = opts.python_executables.split(',') - modules_to_test = [] - for module_name in opts.modules.split(','): - if module_name in python_modules: - modules_to_test.append(python_modules[module_name]) - else: - print("Error: unrecognized module '%s'. Supported modules: %s" % - (module_name, ", ".join(python_modules))) - sys.exit(-1) LOGGER.info("Will test against the following Python executables: %s", python_execs) - LOGGER.info("Will test the following Python modules: %s", [x.name for x in modules_to_test]) + + if should_test_modules: + modules_to_test = [] + for module_name in opts.modules.split(','): + if module_name in python_modules: + modules_to_test.append(python_modules[module_name]) + else: + print("Error: unrecognized module '%s'. Supported modules: %s" % + (module_name, ", ".join(python_modules))) + sys.exit(-1) + LOGGER.info("Will test the following Python modules: %s", [x.name for x in modules_to_test]) + else: + testnames_to_test = opts.testnames.split(',') + LOGGER.info("Will test the following Python tests: %s", testnames_to_test) task_queue = Queue.PriorityQueue() for python_exec in python_execs: @@ -246,16 +266,20 @@ def main(): LOGGER.debug("%s python_implementation is %s", python_exec, python_implementation) LOGGER.debug("%s version is: %s", python_exec, subprocess_check_output( [python_exec, "--version"], stderr=subprocess.STDOUT, universal_newlines=True).strip()) - for module in modules_to_test: - if python_implementation not in module.blacklisted_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'] - if any(map(lambda prefix: test_goal.startswith(prefix), heavy_tests)): - priority = 0 - else: - priority = 100 - task_queue.put((priority, (python_exec, test_goal))) + if should_test_modules: + for module in modules_to_test: + if python_implementation not in module.blacklisted_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'] + if any(map(lambda prefix: test_goal.startswith(prefix), heavy_tests)): + priority = 0 + else: + priority = 100 + task_queue.put((priority, (python_exec, test_goal))) + else: + for test_goal in testnames_to_test: + task_queue.put((0, (python_exec, test_goal))) # Create the target directory before starting tasks to avoid races. target_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), 'target')) From 169d9ad8f1b6006c8db0edbdfffc20dc73c78610 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 5 Dec 2018 19:30:25 +0800 Subject: [PATCH 0074/1072] [SPARK-26133][ML][FOLLOWUP] Fix doc for OneHotEncoder ## What changes were proposed in this pull request? This fixes doc of renamed OneHotEncoder in PySpark. ## How was this patch tested? N/A Closes #23230 from viirya/remove_one_hot_encoder_followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Hyukjin Kwon --- python/pyspark/ml/feature.py | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 6cc80e181e5e..c9507c20918e 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1648,22 +1648,22 @@ class OneHotEncoder(JavaEstimator, HasInputCols, HasOutputCols, HasHandleInvalid at most a single one-value per row that indicates the input category index. For example with 5 categories, an input value of 2.0 would map to an output vector of `[0.0, 0.0, 1.0, 0.0]`. - The last category is not included by default (configurable via `dropLast`), + The last category is not included by default (configurable via :py:attr:`dropLast`), because it makes the vector entries sum up to one, and hence linearly dependent. So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. - Note: This is different from scikit-learn's OneHotEncoder, which keeps all categories. - The output vectors are sparse. + .. note:: This is different from scikit-learn's OneHotEncoder, which keeps all categories. + The output vectors are sparse. - When `handleInvalid` is configured to 'keep', an extra "category" indicating invalid values is - added as last category. So when `dropLast` is true, invalid values are encoded as all-zeros - vector. + When :py:attr:`handleInvalid` is configured to 'keep', an extra "category" indicating invalid + values is added as last category. So when :py:attr:`dropLast` is true, invalid values are + encoded as all-zeros vector. - Note: When encoding multi-column by using `inputCols` and `outputCols` params, input/output - cols come in pairs, specified by the order in the arrays, and each pair is treated - independently. + .. note:: When encoding multi-column by using :py:attr:`inputCols` and + :py:attr:`outputCols` params, input/output cols come in pairs, specified by the order in + the arrays, and each pair is treated independently. - See `StringIndexer` for converting categorical values into category indices + .. seealso:: :py:class:`StringIndexer` for converting categorical values into category indices >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(0.0,), (1.0,), (2.0,)], ["input"]) @@ -1671,7 +1671,7 @@ class OneHotEncoder(JavaEstimator, HasInputCols, HasOutputCols, HasHandleInvalid >>> model = ohe.fit(df) >>> model.transform(df).head().output SparseVector(2, {0: 1.0}) - >>> ohePath = temp_path + "/oheEstimator" + >>> ohePath = temp_path + "/ohe" >>> ohe.save(ohePath) >>> loadedOHE = OneHotEncoder.load(ohePath) >>> loadedOHE.getInputCols() == ohe.getInputCols() From 7bb1dab8a006531d612e21d888c7fc6911990017 Mon Sep 17 00:00:00 2001 From: caoxuewen Date: Wed, 5 Dec 2018 23:10:48 +0800 Subject: [PATCH 0075/1072] [SPARK-26271][FOLLOW-UP][SQL] remove unuse object SparkPlan ## What changes were proposed in this pull request? this code come from PR: https://github.com/apache/spark/pull/11190, but this code has never been used, only since PR: https://github.com/apache/spark/pull/14548, Let's continue fix it. thanks. ## How was this patch tested? N / A Closes #23227 from heary-cao/unuseSparkPlan. Authored-by: caoxuewen Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/execution/SparkPlan.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 9d9b020309d9..a89ccca99d05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -423,11 +423,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } } -object SparkPlan { - private[execution] val subqueryExecutionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("subquery", 16)) -} - trait LeafExecNode extends SparkPlan { override final def children: Seq[SparkPlan] = Nil override def producedAttributes: AttributeSet = outputSet From dd518a196c2d40ae48034b8b0950d1c8045c02ed Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 5 Dec 2018 23:43:03 +0800 Subject: [PATCH 0076/1072] [SPARK-26151][SQL][FOLLOWUP] Return partial results for bad CSV records ## What changes were proposed in this pull request? Updated SQL migration guide according to changes in https://github.com/apache/spark/pull/23120 Closes #23235 from MaxGekk/failuresafe-partial-result-followup. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- docs/sql-migration-guide-upgrade.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index fee0e6df7177..ed2ff139bcc3 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -35,6 +35,8 @@ displayTitle: Spark SQL Upgrading Guide - Since Spark 3.0, CSV datasource uses java.time API for parsing and generating CSV content. New formatting implementation supports date/timestamp patterns conformed to ISO 8601. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. + - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. + ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. From ab76900fedc05df7080c9b6c81d65a3f260c1c26 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 6 Dec 2018 09:14:46 +0800 Subject: [PATCH 0077/1072] [SPARK-26275][PYTHON][ML] Increases timeout for StreamingLogisticRegressionWithSGDTests.test_training_and_prediction test ## What changes were proposed in this pull request? Looks this test is flaky https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99704/console https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99569/console https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99644/console https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99548/console https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99454/console https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99609/console ``` ====================================================================== FAIL: test_training_and_prediction (pyspark.mllib.tests.test_streaming_algorithms.StreamingLogisticRegressionWithSGDTests) Test that the model improves on toy data with no. of batches ---------------------------------------------------------------------- Traceback (most recent call last): File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 367, in test_training_and_prediction self._eventually(condition) File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 78, in _eventually % (timeout, lastValue)) AssertionError: Test failed due to timeout after 30 sec, with last condition returning: Latest errors: 0.67, 0.71, 0.78, 0.7, 0.75, 0.74, 0.73, 0.69, 0.62, 0.71, 0.69, 0.75, 0.72, 0.77, 0.71, 0.74 ---------------------------------------------------------------------- Ran 13 tests in 185.051s FAILED (failures=1, skipped=1) ``` This looks happening after increasing the parallelism in Jenkins to speed up at https://github.com/apache/spark/pull/23111. I am able to reproduce this manually when the resource usage is heavy (with manual decrease of timeout). ## How was this patch tested? Manually tested by ``` cd python ./run-tests --testnames 'pyspark.mllib.tests.test_streaming_algorithms StreamingLogisticRegressionWithSGDTests.test_training_and_prediction' --python-executables=python ``` Closes #23236 from HyukjinKwon/SPARK-26275. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/mllib/tests/test_streaming_algorithms.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/mllib/tests/test_streaming_algorithms.py b/python/pyspark/mllib/tests/test_streaming_algorithms.py index 4bc8904acd31..bf2ad2d267bb 100644 --- a/python/pyspark/mllib/tests/test_streaming_algorithms.py +++ b/python/pyspark/mllib/tests/test_streaming_algorithms.py @@ -364,7 +364,7 @@ def condition(): return True return "Latest errors: " + ", ".join(map(lambda x: str(x), errors)) - self._eventually(condition) + self._eventually(condition, timeout=60.0) class StreamingLinearRegressionWithTests(MLLibStreamingTestCase): From ecaa495b1fe532c36e952ccac42f4715809476af Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 6 Dec 2018 10:07:28 -0800 Subject: [PATCH 0078/1072] [SPARK-25274][PYTHON][SQL] In toPandas with Arrow send un-ordered record batches to improve performance ## What changes were proposed in this pull request? When executing `toPandas` with Arrow enabled, partitions that arrive in the JVM out-of-order must be buffered before they can be send to Python. This causes an excess of memory to be used in the driver JVM and increases the time it takes to complete because data must sit in the JVM waiting for preceding partitions to come in. This change sends un-ordered partitions to Python as soon as they arrive in the JVM, followed by a list of partition indices so that Python can assemble the data in the correct order. This way, data is not buffered at the JVM and there is no waiting on particular partitions so performance will be increased. Followup to #21546 ## How was this patch tested? Added new test with a large number of batches per partition, and test that forces a small delay in the first partition. These test that partitions are collected out-of-order and then are are put in the correct order in Python. ## Performance Tests - toPandas Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8 measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each. Test code ```python df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand()) for i in range(5): start = time.time() _ = df.toPandas() elapsed = time.time() - start ``` Spark config ``` spark.driver.memory 5g spark.executor.memory 5g spark.driver.maxResultSize 2g spark.sql.execution.arrow.enabled true ``` Current Master w/ Arrow stream | This PR ---------------------|------------ 5.16207 | 4.342533 5.133671 | 4.399408 5.147513 | 4.468471 5.105243 | 4.36524 5.018685 | 4.373791 Avg Master | Avg This PR ------------------|-------------- 5.1134364 | 4.3898886 Speedup of **1.164821449** Closes #22275 from BryanCutler/arrow-toPandas-oo-batches-SPARK-25274. Authored-by: Bryan Cutler Signed-off-by: Bryan Cutler --- python/pyspark/serializers.py | 33 ++++++++++++++ python/pyspark/sql/dataframe.py | 11 ++++- python/pyspark/sql/tests/test_arrow.py | 28 ++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 45 ++++++++++--------- 4 files changed, 95 insertions(+), 22 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index ff9a612b77f6..f3ebd3767a0a 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -185,6 +185,39 @@ def loads(self, obj): raise NotImplementedError +class ArrowCollectSerializer(Serializer): + """ + Deserialize a stream of batches followed by batch order information. Used in + DataFrame._collectAsArrow() after invoking Dataset.collectAsArrowToPython() in the JVM. + """ + + def __init__(self): + self.serializer = ArrowStreamSerializer() + + def dump_stream(self, iterator, stream): + return self.serializer.dump_stream(iterator, stream) + + def load_stream(self, stream): + """ + Load a stream of un-ordered Arrow RecordBatches, where the last iteration yields + a list of indices that can be used to put the RecordBatches in the correct order. + """ + # load the batches + for batch in self.serializer.load_stream(stream): + yield batch + + # load the batch order indices + num = read_int(stream) + batch_order = [] + for i in xrange(num): + index = read_int(stream) + batch_order.append(index) + yield batch_order + + def __repr__(self): + return "ArrowCollectSerializer(%s)" % self.serializer + + class ArrowStreamSerializer(Serializer): """ Serializes Arrow record batches as a stream. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 1b1092c409be..a1056d0b787e 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -29,7 +29,7 @@ from pyspark import copy_func, since, _NoValue from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix -from pyspark.serializers import ArrowStreamSerializer, BatchedSerializer, PickleSerializer, \ +from pyspark.serializers import ArrowCollectSerializer, BatchedSerializer, PickleSerializer, \ UTF8Deserializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -2168,7 +2168,14 @@ def _collectAsArrow(self): """ with SCCallSiteSync(self._sc) as css: sock_info = self._jdf.collectAsArrowToPython() - return list(_load_from_socket(sock_info, ArrowStreamSerializer())) + + # Collect list of un-ordered batches where last element is a list of correct order indices + results = list(_load_from_socket(sock_info, ArrowCollectSerializer())) + batches = results[:-1] + batch_order = results[-1] + + # Re-order the batch list using the correct order + return [batches[i] for i in batch_order] ########################################################################################## # Pandas compatibility diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index 6e75e82d5800..21fe5000df5d 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -381,6 +381,34 @@ def test_timestamp_dst(self): self.assertPandasEqual(pdf, df_from_python.toPandas()) self.assertPandasEqual(pdf, df_from_pandas.toPandas()) + def test_toPandas_batch_order(self): + + def delay_first_part(partition_index, iterator): + if partition_index == 0: + time.sleep(0.1) + return iterator + + # Collects Arrow RecordBatches out of order in driver JVM then re-orders in Python + def run_test(num_records, num_parts, max_records, use_delay=False): + df = self.spark.range(num_records, numPartitions=num_parts).toDF("a") + if use_delay: + df = df.rdd.mapPartitionsWithIndex(delay_first_part).toDF() + with self.sql_conf({"spark.sql.execution.arrow.maxRecordsPerBatch": max_records}): + pdf, pdf_arrow = self._toPandas_arrow_toggle(df) + self.assertPandasEqual(pdf, pdf_arrow) + + cases = [ + (1024, 512, 2), # Use large num partitions for more likely collecting out of order + (64, 8, 2, True), # Use delay in first partition to force collecting out of order + (64, 64, 1), # Test single batch per partition + (64, 1, 64), # Test single partition, single batch + (64, 1, 8), # Test single partition, multiple batches + (30, 7, 2), # Test different sized partitions + ] + + for case in cases: + run_test(*case) + class EncryptionArrowTests(ArrowTests): 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 b10d66dfb1ae..a664c7338bad 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 @@ -17,9 +17,10 @@ package org.apache.spark.sql -import java.io.CharArrayWriter +import java.io.{CharArrayWriter, DataOutputStream} import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import scala.util.control.NonFatal @@ -3200,34 +3201,38 @@ class Dataset[T] private[sql]( val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone withAction("collectAsArrowToPython", queryExecution) { plan => - PythonRDD.serveToStream("serve-Arrow") { out => + PythonRDD.serveToStream("serve-Arrow") { outputStream => + val out = new DataOutputStream(outputStream) val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId) val arrowBatchRdd = toArrowBatchRdd(plan) 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) - var lastIndex = -1 // index of last partition written + // Batches ordered by (index of partition, batch index in that partition) tuple + val batchOrder = new ArrayBuffer[(Int, Int)]() + var partitionCount = 0 - // Handler to eagerly write partitions to Python in order + // Handler to eagerly write batches to Python as they arrive, un-ordered def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = { - // If result is from next partition in order - if (index - 1 == lastIndex) { + if (arrowBatches.nonEmpty) { + // Write all batches (can be more than 1) in the partition, store the batch order tuple batchWriter.writeBatches(arrowBatches.iterator) - lastIndex += 1 - // Write stored partitions that come next in order - while (lastIndex < results.length && results(lastIndex) != null) { - batchWriter.writeBatches(results(lastIndex).iterator) - results(lastIndex) = null - lastIndex += 1 + arrowBatches.indices.foreach { + partition_batch_index => batchOrder.append((index, partition_batch_index)) } - // After last batch, end the stream - if (lastIndex == results.length) { - batchWriter.end() + } + partitionCount += 1 + + // After last batch, end the stream and write batch order indices + if (partitionCount == numPartitions) { + batchWriter.end() + out.writeInt(batchOrder.length) + // Sort by (index of partition, batch index in that partition) tuple to get the + // overall_batch_index from 0 to N-1 batches, which can be used to put the + // transferred batches in the correct order + batchOrder.zipWithIndex.sortBy(_._1).foreach { case (_, overall_batch_index) => + out.writeInt(overall_batch_index) } - } else { - // Store partitions received out of order - results(index - 1) = arrowBatches + out.flush() } } From b14a26ee5764aa98472bc69ab1dec408b89bc78a Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Thu, 6 Dec 2018 10:59:20 -0800 Subject: [PATCH 0079/1072] [SPARK-26236][SS] Add kafka delegation token support documentation. ## What changes were proposed in this pull request? Kafka delegation token support implemented in [PR#22598](https://github.com/apache/spark/pull/22598) but that didn't contain documentation because of rapid changes. Because it has been merged in this PR I've documented it. ## How was this patch tested? jekyll build + manual html check Closes #23195 from gaborgsomogyi/SPARK-26236. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../structured-streaming-kafka-integration.md | 216 +++++++++++++++++- 1 file changed, 206 insertions(+), 10 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index a549ce2a6a05..7040f8da2c61 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -66,8 +66,8 @@ Dataset df = spark .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("subscribe", "topic1") - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); // Subscribe to multiple topics Dataset df = spark @@ -75,8 +75,8 @@ Dataset df = spark .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("subscribe", "topic1,topic2") - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); // Subscribe to a pattern Dataset df = spark @@ -84,8 +84,8 @@ Dataset df = spark .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("subscribePattern", "topic.*") - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); {% endhighlight %} @@ -479,7 +479,7 @@ StreamingQuery ds = df .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("topic", "topic1") - .start() + .start(); // Write key-value data from a DataFrame to Kafka using a topic specified in the data StreamingQuery ds = df @@ -487,7 +487,7 @@ StreamingQuery ds = df .writeStream() .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .start() + .start(); {% endhighlight %} @@ -547,14 +547,14 @@ df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("topic", "topic1") - .save() + .save(); // Write key-value data from a DataFrame to Kafka using a topic specified in the data df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") .write() .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .save() + .save(); {% endhighlight %} @@ -624,3 +624,199 @@ For experimenting on `spark-shell`, you can also use `--packages` to add `spark- See [Application Submission Guide](submitting-applications.html) for more details about submitting applications with external dependencies. + +## Security + +Kafka 0.9.0.0 introduced several features that increases security in a cluster. For detailed +description about these possibilities, see [Kafka security docs](http://kafka.apache.org/documentation.html#security). + +It's worth noting that security is optional and turned off by default. + +Spark supports the following ways to authenticate against Kafka cluster: +- **Delegation token (introduced in Kafka broker 1.1.0)** +- **JAAS login configuration** + +### Delegation token + +This way the application can be configured via Spark parameters and may not need JAAS login +configuration (Spark can use Kafka's dynamic JAAS configuration feature). For further information +about delegation tokens, see [Kafka delegation token docs](http://kafka.apache.org/documentation/#security_delegation_token). + +The process is initiated by Spark's Kafka delegation token provider. When `spark.kafka.bootstrap.servers`, +Spark considers the following log in options, in order of preference: +- **JAAS login configuration** +- **Keytab file**, such as, + + ./bin/spark-submit \ + --keytab \ + --principal \ + --conf spark.kafka.bootstrap.servers= \ + ... + +- **Kerberos credential cache**, such as, + + ./bin/spark-submit \ + --conf spark.kafka.bootstrap.servers= \ + ... + +The Kafka delegation token provider can be turned off by setting `spark.security.credentials.kafka.enabled` to `false` (default: `true`). + +Spark can be configured to use the following authentication protocols to obtain token (it must match with +Kafka broker configuration): +- **SASL SSL (default)** +- **SSL** +- **SASL PLAINTEXT (for testing)** + +After obtaining delegation token successfully, Spark distributes it across nodes and renews it accordingly. +Delegation token uses `SCRAM` login module for authentication and because of that the appropriate +`sasl.mechanism` has to be configured on source/sink (it must match with Kafka broker configuration): + +
+
+{% highlight scala %} + +// Setting on Kafka Source for Streaming Queries +val df = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") + .option("subscribe", "topic1") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +// Setting on Kafka Source for Batch Queries +val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") + .option("subscribe", "topic1") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +// Setting on Kafka Sink for Streaming Queries +val ds = df + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .writeStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") + .option("topic", "topic1") + .start() + +// Setting on Kafka Sink for Batch Queries +val ds = df + .selectExpr("topic1", "CAST(key AS STRING)", "CAST(value AS STRING)") + .write + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") + .save() + +{% endhighlight %} +
+
+{% highlight java %} + +// Setting on Kafka Source for Streaming Queries +Dataset df = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") + .option("subscribe", "topic1") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); + +// Setting on Kafka Source for Batch Queries +Dataset df = spark + .read() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") + .option("subscribe", "topic1") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); + +// Setting on Kafka Sink for Streaming Queries +StreamingQuery ds = df + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .writeStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") + .option("topic", "topic1") + .start(); + +// Setting on Kafka Sink for Batch Queries +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .write() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") + .option("topic", "topic1") + .save(); + +{% endhighlight %} +
+
+{% highlight python %} + +// Setting on Kafka Source for Streaming Queries +df = spark \ + .readStream \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") \ + .option("subscribe", "topic1") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +// Setting on Kafka Source for Batch Queries +df = spark \ + .read \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") \ + .option("subscribe", "topic1") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +// Setting on Kafka Sink for Streaming Queries +ds = df \ + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") \ + .writeStream \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") \ + .option("topic", "topic1") \ + .start() + +// Setting on Kafka Sink for Batch Queries +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") \ + .write \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("kafka.sasl.mechanism", "SCRAM-SHA-512") \ + .option("topic", "topic1") \ + .save() + +{% endhighlight %} +
+
+ +When delegation token is available on an executor it can be overridden with JAAS login configuration. + +### JAAS login configuration + +JAAS login configuration must placed on all nodes where Spark tries to access Kafka cluster. +This provides the possibility to apply any custom authentication logic with a higher cost to maintain. +This can be done several ways. One possibility is to provide additional JVM parameters, such as, + + ./bin/spark-submit \ + --driver-java-options "-Djava.security.auth.login.config=/path/to/custom_jaas.conf" \ + --conf spark.executor.extraJavaOptions=-Djava.security.auth.login.config=/path/to/custom_jaas.conf \ + ... From dbd90e54408d593e02a3dd1e659fcf9a7b940535 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 6 Dec 2018 14:17:13 -0800 Subject: [PATCH 0080/1072] [SPARK-26194][K8S] Auto generate auth secret for k8s apps. This change modifies the logic in the SecurityManager to do two things: - generate unique app secrets also when k8s is being used - only store the secret in the user's UGI on YARN The latter is needed so that k8s won't unnecessarily create k8s secrets for the UGI credentials when only the auth token is stored there. On the k8s side, the secret is propagated to executors using an environment variable instead. This ensures it works in both client and cluster mode. Security doc was updated to mention the feature and clarify that proper access control in k8s should be enabled for it to be secure. Author: Marcelo Vanzin Closes #23174 from vanzin/SPARK-26194. --- .../org/apache/spark/SecurityManager.scala | 21 +++- .../apache/spark/SecurityManagerSuite.scala | 57 +++++++---- docs/security.md | 34 ++++--- .../features/BasicExecutorFeatureStep.scala | 96 +++++++++++-------- .../cluster/k8s/ExecutorPodsAllocator.scala | 5 +- .../k8s/KubernetesClusterManager.scala | 3 +- .../KubernetesClusterSchedulerBackend.scala | 5 +- .../k8s/KubernetesExecutorBuilder.scala | 13 ++- .../BasicExecutorFeatureStepSuite.scala | 46 ++++++--- .../k8s/ExecutorPodsAllocatorSuite.scala | 9 +- ...bernetesClusterSchedulerBackendSuite.scala | 9 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 18 ++-- .../k8s/integrationtest/KubernetesSuite.scala | 2 + 13 files changed, 205 insertions(+), 113 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 3cfafeb95110..96e4b53b2418 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -348,15 +348,23 @@ private[spark] class SecurityManager( */ def initializeAuth(): Unit = { import SparkMasterRegex._ + val k8sRegex = "k8s.*".r if (!sparkConf.get(NETWORK_AUTH_ENABLED)) { return } + // TODO: this really should be abstracted somewhere else. val master = sparkConf.get(SparkLauncher.SPARK_MASTER, "") - master match { + val storeInUgi = master match { case "yarn" | "local" | LOCAL_N_REGEX(_) | LOCAL_N_FAILURES_REGEX(_, _) => - // Secret generation allowed here + true + + case k8sRegex() => + // Don't propagate the secret through the user's credentials in kubernetes. That conflicts + // with the way k8s handles propagation of delegation tokens. + false + case _ => require(sparkConf.contains(SPARK_AUTH_SECRET_CONF), s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config.") @@ -364,9 +372,12 @@ private[spark] class SecurityManager( } secretKey = Utils.createSecret(sparkConf) - val creds = new Credentials() - creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8)) - UserGroupInformation.getCurrentUser().addCredentials(creds) + + if (storeInUgi) { + val creds = new Credentials() + creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8)) + UserGroupInformation.getCurrentUser().addCredentials(creds) + } } // Default SecurityManager only has a single secret key, so ignore appId. diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index e357299770a2..eec8004fc94f 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -395,15 +395,23 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(keyFromEnv === new SecurityManager(conf2).getSecretKey()) } - test("secret key generation") { - Seq( - ("yarn", true), - ("local", true), - ("local[*]", true), - ("local[1, 2]", true), - ("local-cluster[2, 1, 1024]", false), - ("invalid", false) - ).foreach { case (master, shouldGenerateSecret) => + // How is the secret expected to be generated and stored. + object SecretTestType extends Enumeration { + val MANUAL, AUTO, UGI = Value + } + + import SecretTestType._ + + Seq( + ("yarn", UGI), + ("local", UGI), + ("local[*]", UGI), + ("local[1, 2]", UGI), + ("k8s://127.0.0.1", AUTO), + ("local-cluster[2, 1, 1024]", MANUAL), + ("invalid", MANUAL) + ).foreach { case (master, secretType) => + test(s"secret key generation: master '$master'") { val conf = new SparkConf() .set(NETWORK_AUTH_ENABLED, true) .set(SparkLauncher.SPARK_MASTER, master) @@ -412,19 +420,26 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { UserGroupInformation.createUserForTesting("authTest", Array()).doAs( new PrivilegedExceptionAction[Unit]() { override def run(): Unit = { - if (shouldGenerateSecret) { - mgr.initializeAuth() - val creds = UserGroupInformation.getCurrentUser().getCredentials() - val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) - assert(secret != null) - assert(new String(secret, UTF_8) === mgr.getSecretKey()) - } else { - intercept[IllegalArgumentException] { + secretType match { + case UGI => + mgr.initializeAuth() + val creds = UserGroupInformation.getCurrentUser().getCredentials() + val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) + assert(secret != null) + assert(new String(secret, UTF_8) === mgr.getSecretKey()) + + case AUTO => mgr.initializeAuth() - } - intercept[IllegalArgumentException] { - mgr.getSecretKey() - } + val creds = UserGroupInformation.getCurrentUser().getCredentials() + assert(creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) === null) + + case MANUAL => + intercept[IllegalArgumentException] { + mgr.initializeAuth() + } + intercept[IllegalArgumentException] { + mgr.getSecretKey() + } } } } diff --git a/docs/security.md b/docs/security.md index be4834660fb7..2a4f3c074c1e 100644 --- a/docs/security.md +++ b/docs/security.md @@ -26,21 +26,29 @@ not documented, Spark does not support. Spark currently supports authentication for RPC channels using a shared secret. Authentication can be turned on by setting the `spark.authenticate` configuration parameter. -The exact mechanism used to generate and distribute the shared secret is deployment-specific. +The exact mechanism used to generate and distribute the shared secret is deployment-specific. Unless +specified below, the secret must be defined by setting the `spark.authenticate.secret` config +option. The same secret is shared by all Spark applications and daemons in that case, which limits +the security of these deployments, especially on multi-tenant clusters. -For Spark on [YARN](running-on-yarn.html) and local deployments, Spark will automatically handle -generating and distributing the shared secret. Each application will use a unique shared secret. In +The REST Submission Server and the MesosClusterDispatcher do not support authentication. You should +ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077 +respectively by default) are restricted to hosts that are trusted to submit jobs. + +### YARN + +For Spark on [YARN](running-on-yarn.html), Spark will automatically handle generating and +distributing the shared secret. Each application will use a unique shared secret. In the case of YARN, this feature relies on YARN RPC encryption being enabled for the distribution of secrets to be secure. -For other resource managers, `spark.authenticate.secret` must be configured on each of the nodes. -This secret will be shared by all the daemons and applications, so this deployment configuration is -not as secure as the above, especially when considering multi-tenant clusters. In this -configuration, a user with the secret can effectively impersonate any other user. +### Kubernetes -The Rest Submission Server and the MesosClusterDispatcher do not support authentication. You should -ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077 -respectively by default) are restricted to hosts that are trusted to submit jobs. +On Kubernetes, Spark will also automatically generate an authentication secret unique to each +application. The secret is propagated to executor pods using environment variables. This means +that any user that can list pods in the namespace where the Spark application is running can +also see their authentication secret. Access control rules should be properly set up by the +Kubernetes admin to ensure that Spark authentication is secure. @@ -738,10 +746,10 @@ tokens for supported will be created. ## Secure Interaction with Kubernetes When talking to Hadoop-based services behind Kerberos, it was noted that Spark needs to obtain delegation tokens -so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are -shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job: +so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are +shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job: -In all cases you must define the environment variable: `HADOOP_CONF_DIR` or +In all cases you must define the environment variable: `HADOOP_CONF_DIR` or `spark.kubernetes.hadoop.configMapName.` It also important to note that the KDC needs to be visible from inside the containers. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 8bf315248388..939aa88b0797 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -20,7 +20,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ -import org.apache.spark.SparkException +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -29,11 +29,12 @@ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils -private[spark] class BasicExecutorFeatureStep(kubernetesConf: KubernetesExecutorConf) +private[spark] class BasicExecutorFeatureStep( + kubernetesConf: KubernetesExecutorConf, + secMgr: SecurityManager) extends KubernetesFeatureConfigStep { // Consider moving some of these fields to KubernetesConf or KubernetesExecutorSpecificConf - private val executorExtraClasspath = kubernetesConf.get(EXECUTOR_CLASS_PATH) private val executorContainerImage = kubernetesConf .get(EXECUTOR_CONTAINER_IMAGE) .getOrElse(throw new SparkException("Must specify the executor container image")) @@ -87,44 +88,61 @@ private[spark] class BasicExecutorFeatureStep(kubernetesConf: KubernetesExecutor val executorCpuQuantity = new QuantityBuilder(false) .withAmount(executorCoresRequest) .build() - val executorExtraClasspathEnv = executorExtraClasspath.map { cp => - new EnvVarBuilder() - .withName(ENV_CLASSPATH) - .withValue(cp) - .build() - } - val executorExtraJavaOptionsEnv = kubernetesConf - .get(EXECUTOR_JAVA_OPTIONS) - .map { opts => - val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId, - kubernetesConf.executorId) - val delimitedOpts = Utils.splitCommandString(subsOpts) - delimitedOpts.zipWithIndex.map { - case (opt, index) => - new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() + + val executorEnv: Seq[EnvVar] = { + (Seq( + (ENV_DRIVER_URL, driverUrl), + (ENV_EXECUTOR_CORES, executorCores.toString), + (ENV_EXECUTOR_MEMORY, executorMemoryString), + (ENV_APPLICATION_ID, kubernetesConf.appId), + // This is to set the SPARK_CONF_DIR to be /opt/spark/conf + (ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL), + (ENV_EXECUTOR_ID, kubernetesConf.executorId) + ) ++ kubernetesConf.environment).map { case (k, v) => + new EnvVarBuilder() + .withName(k) + .withValue(v) + .build() } - }.getOrElse(Seq.empty[EnvVar]) - val executorEnv = (Seq( - (ENV_DRIVER_URL, driverUrl), - (ENV_EXECUTOR_CORES, executorCores.toString), - (ENV_EXECUTOR_MEMORY, executorMemoryString), - (ENV_APPLICATION_ID, kubernetesConf.appId), - // This is to set the SPARK_CONF_DIR to be /opt/spark/conf - (ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL), - (ENV_EXECUTOR_ID, kubernetesConf.executorId)) ++ - kubernetesConf.environment) - .map(env => new EnvVarBuilder() - .withName(env._1) - .withValue(env._2) - .build() - ) ++ Seq( - new EnvVarBuilder() - .withName(ENV_EXECUTOR_POD_IP) - .withValueFrom(new EnvVarSourceBuilder() - .withNewFieldRef("v1", "status.podIP") + } ++ { + Seq(new EnvVarBuilder() + .withName(ENV_EXECUTOR_POD_IP) + .withValueFrom(new EnvVarSourceBuilder() + .withNewFieldRef("v1", "status.podIP") + .build()) .build()) - .build() - ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq + } ++ { + Option(secMgr.getSecretKey()).map { authSecret => + new EnvVarBuilder() + .withName(SecurityManager.ENV_AUTH_SECRET) + .withValue(authSecret) + .build() + } + } ++ { + kubernetesConf.get(EXECUTOR_CLASS_PATH).map { cp => + new EnvVarBuilder() + .withName(ENV_CLASSPATH) + .withValue(cp) + .build() + } + } ++ { + val userOpts = kubernetesConf.get(EXECUTOR_JAVA_OPTIONS).toSeq.flatMap { opts => + val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId, + kubernetesConf.executorId) + Utils.splitCommandString(subsOpts) + } + + val sparkOpts = Utils.sparkJavaOpts(kubernetesConf.sparkConf, + SparkConf.isExecutorStartupConf) + + (userOpts ++ sparkOpts).zipWithIndex.map { case (opt, index) => + new EnvVarBuilder() + .withName(s"$ENV_JAVA_OPT_PREFIX$index") + .withValue(opt) + .build() + } + } + val requiredPorts = Seq( (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) .map { case (name, port) => diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 2f0f949566d6..ac42554b1334 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.PodBuilder import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.mutable -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesConf @@ -31,6 +31,7 @@ import org.apache.spark.util.{Clock, Utils} private[spark] class ExecutorPodsAllocator( conf: SparkConf, + secMgr: SecurityManager, executorBuilder: KubernetesExecutorBuilder, kubernetesClient: KubernetesClient, snapshotsStore: ExecutorPodsSnapshotsStore, @@ -135,7 +136,7 @@ private[spark] class ExecutorPodsAllocator( newExecutorId.toString, applicationId, driverPod) - val executorPod = executorBuilder.buildFromFeatures(executorConf) + val executorPod = executorBuilder.buildFromFeatures(executorConf, secMgr) val podWithAttachedContainer = new PodBuilder(executorPod.pod) .editOrNewSpec() .addToContainers(executorPod.container) 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 ce10f766334f..b31fbb420ed6 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 @@ -94,6 +94,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val executorPodsAllocator = new ExecutorPodsAllocator( sc.conf, + sc.env.securityManager, KubernetesExecutorBuilder(kubernetesClient, sc.conf), kubernetesClient, snapshotsStore, @@ -110,7 +111,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit new KubernetesClusterSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], - sc.env.rpcEnv, + sc, kubernetesClient, requestExecutorsService, snapshotsStore, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 6356b5864580..68f6f2e46e31 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -21,6 +21,7 @@ import java.util.concurrent.ExecutorService import io.fabric8.kubernetes.client.KubernetesClient import scala.concurrent.{ExecutionContext, Future} +import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.rpc.{RpcAddress, RpcEnv} @@ -30,7 +31,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, - rpcEnv: RpcEnv, + sc: SparkContext, kubernetesClient: KubernetesClient, requestExecutorsService: ExecutorService, snapshotsStore: ExecutorPodsSnapshotsStore, @@ -38,7 +39,7 @@ private[spark] class KubernetesClusterSchedulerBackend( lifecycleEventHandler: ExecutorPodsLifecycleManager, watchEvents: ExecutorPodsWatchSnapshotSource, pollEvents: ExecutorPodsPollingSnapshotSource) - extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( requestExecutorsService) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index d24ff0d1e660..ba273cad6a8e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -20,14 +20,14 @@ import java.io.File import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ private[spark] class KubernetesExecutorBuilder( - provideBasicStep: (KubernetesExecutorConf => BasicExecutorFeatureStep) = - new BasicExecutorFeatureStep(_), + provideBasicStep: (KubernetesExecutorConf, SecurityManager) => BasicExecutorFeatureStep = + new BasicExecutorFeatureStep(_, _), provideSecretsStep: (KubernetesConf => MountSecretsFeatureStep) = new MountSecretsFeatureStep(_), provideEnvSecretsStep: (KubernetesConf => EnvSecretsFeatureStep) = @@ -44,13 +44,16 @@ private[spark] class KubernetesExecutorBuilder( new HadoopSparkUserExecutorFeatureStep(_), provideInitialPod: () => SparkPod = () => SparkPod.initialPod()) { - def buildFromFeatures(kubernetesConf: KubernetesExecutorConf): SparkPod = { + def buildFromFeatures( + kubernetesConf: KubernetesExecutorConf, + secMgr: SecurityManager): SparkPod = { val sparkConf = kubernetesConf.sparkConf val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) val maybeDTSecretName = sparkConf.getOption(KERBEROS_DT_SECRET_NAME) val maybeDTDataItem = sparkConf.getOption(KERBEROS_DT_SECRET_KEY) - val baseFeatures = Seq(provideBasicStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) + val baseFeatures = Seq(provideBasicStep(kubernetesConf, secMgr), + provideLocalDirsStep(kubernetesConf)) val secretFeature = if (kubernetesConf.secretNamesToMountPaths.nonEmpty) { Seq(provideSecretsStep(kubernetesConf)) } else Nil diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index d6003c977937..6aa862643c78 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -21,13 +21,14 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.config._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { @@ -63,7 +64,7 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { private var baseConf: SparkConf = _ before { - baseConf = new SparkConf() + baseConf = new SparkConf(false) .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, RESOURCE_NAME_PREFIX) .set(CONTAINER_IMAGE, EXECUTOR_IMAGE) @@ -84,7 +85,7 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("basic executor pod has reasonable defaults") { - val step = new BasicExecutorFeatureStep(newExecutorConf()) + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) val executor = step.configurePod(SparkPod.initialPod()) // The executor pod name and default labels. @@ -106,7 +107,7 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { assert(executor.pod.getSpec.getNodeSelector.isEmpty) assert(executor.pod.getSpec.getVolumes.isEmpty) - checkEnv(executor, Map()) + checkEnv(executor, baseConf, Map()) checkOwnerReferences(executor.pod, DRIVER_POD_UID) } @@ -114,7 +115,7 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { val longPodNamePrefix = "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple" baseConf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, longPodNamePrefix) - val step = new BasicExecutorFeatureStep(newExecutorConf()) + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) assert(step.configurePod(SparkPod.initialPod()).pod.getSpec.getHostname.length === 63) } @@ -122,10 +123,10 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { baseConf.set(EXECUTOR_JAVA_OPTIONS, "foo=bar") baseConf.set(EXECUTOR_CLASS_PATH, "bar=baz") val kconf = newExecutorConf(environment = Map("qux" -> "quux")) - val step = new BasicExecutorFeatureStep(kconf) + val step = new BasicExecutorFeatureStep(kconf, new SecurityManager(baseConf)) val executor = step.configurePod(SparkPod.initialPod()) - checkEnv(executor, + checkEnv(executor, baseConf, Map("SPARK_JAVA_OPT_0" -> "foo=bar", ENV_CLASSPATH -> "bar=baz", "qux" -> "quux")) @@ -136,12 +137,27 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { baseConf.set("spark.kubernetes.resource.type", "python") baseConf.set(PYSPARK_EXECUTOR_MEMORY, 42L) - val step = new BasicExecutorFeatureStep(newExecutorConf()) + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) val executor = step.configurePod(SparkPod.initialPod()) // This is checking that basic executor + executorMemory = 1408 + 42 = 1450 assert(executor.container.getResources.getRequests.get("memory").getAmount === "1450Mi") } + test("auth secret propagation") { + val conf = baseConf.clone() + .set(NETWORK_AUTH_ENABLED, true) + .set("spark.master", "k8s://127.0.0.1") + + val secMgr = new SecurityManager(conf) + secMgr.initializeAuth() + + val step = new BasicExecutorFeatureStep(KubernetesTestConf.createExecutorConf(sparkConf = conf), + secMgr) + + val executor = step.configurePod(SparkPod.initialPod()) + checkEnv(executor, conf, Map(SecurityManager.ENV_AUTH_SECRET -> secMgr.getSecretKey())) + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) @@ -150,7 +166,10 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } // Check that the expected environment variables are present. - private def checkEnv(executorPod: SparkPod, additionalEnvVars: Map[String, String]): Unit = { + private def checkEnv( + executorPod: SparkPod, + conf: SparkConf, + additionalEnvVars: Map[String, String]): Unit = { val defaultEnvs = Map( ENV_EXECUTOR_ID -> "1", ENV_DRIVER_URL -> DRIVER_ADDRESS.toString, @@ -160,10 +179,15 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { ENV_SPARK_CONF_DIR -> SPARK_CONF_DIR_INTERNAL, ENV_EXECUTOR_POD_IP -> null) ++ additionalEnvVars - assert(executorPod.container.getEnv.size() === defaultEnvs.size) + val extraJavaOptsStart = additionalEnvVars.keys.count(_.startsWith(ENV_JAVA_OPT_PREFIX)) + val extraJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf) + val extraJavaOptsEnvs = extraJavaOpts.zipWithIndex.map { case (opt, ind) => + s"$ENV_JAVA_OPT_PREFIX${ind + extraJavaOptsStart}" -> opt + }.toMap + val mapEnvs = executorPod.container.getEnv.asScala.map { x => (x.getName, x.getValue) }.toMap - assert(defaultEnvs === mapEnvs) + assert((defaultEnvs ++ extraJavaOptsEnvs) === mapEnvs) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 303e24b8f497..d4fa31af3d5c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -20,13 +20,13 @@ import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{ArgumentMatcher, Matchers, Mock, MockitoAnnotations} -import org.mockito.Matchers.any +import org.mockito.Matchers.{any, eq => meq} import org.mockito.Mockito.{never, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -52,6 +52,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) private val podCreationTimeout = math.max(podAllocationDelay * 5, 60000L) + private val secMgr = new SecurityManager(conf) private var waitForExecutorPodsClock: ManualClock = _ @@ -79,12 +80,12 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) when(driverPodOperations.get).thenReturn(driverPod) - when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]))) + when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]), meq(secMgr))) .thenAnswer(executorPodAnswer()) snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() waitForExecutorPodsClock = new ManualClock(0L) podsAllocatorUnderTest = new ExecutorPodsAllocator( - conf, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock) + conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock) podsAllocatorUnderTest.start(TEST_SPARK_APP_ID) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 52e7a12dbaf0..75232f7b98b0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -23,7 +23,7 @@ import org.mockito.Matchers.{eq => mockitoEq} import org.mockito.Mockito.{never, verify, when} import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} @@ -41,6 +41,9 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn @Mock private var sc: SparkContext = _ + @Mock + private var env: SparkEnv = _ + @Mock private var rpcEnv: RpcEnv = _ @@ -81,6 +84,8 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn MockitoAnnotations.initMocks(this) when(taskScheduler.sc).thenReturn(sc) when(sc.conf).thenReturn(sparkConf) + when(sc.env).thenReturn(env) + when(env.rpcEnv).thenReturn(rpcEnv) driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) when(rpcEnv.setupEndpoint( mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) @@ -88,7 +93,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn when(kubernetesClient.pods()).thenReturn(podOperations) schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( taskScheduler, - rpcEnv, + sc, kubernetesClient, requestExecutorsService, eventQueue, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index b6a75b15af85..ef521fd801e9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.{Config => _, _} import io.fabric8.kubernetes.client.KubernetesClient import org.mockito.Mockito.{mock, never, verify} -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ @@ -39,6 +39,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { private val KERBEROS_CONF_STEP_TYPE = "kerberos-step" private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" + private val secMgr = new SecurityManager(new SparkConf(false)) + private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( BASIC_STEP_TYPE, classOf[BasicExecutorFeatureStep]) private val mountSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( @@ -57,7 +59,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) private val builderUnderTest = new KubernetesExecutorBuilder( - _ => basicFeatureStep, + (_, _) => basicFeatureStep, _ => mountSecretsStep, _ => envSecretsStep, _ => localDirsStep, @@ -69,7 +71,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { test("Basic steps are consistently applied.") { val conf = KubernetesTestConf.createExecutorConf() validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) + builderUnderTest.buildFromFeatures(conf, secMgr), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) } test("Apply secrets step if secrets are present.") { @@ -77,7 +79,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { secretEnvNamesToKeyRefs = Map("secret-name" -> "secret-key"), secretNamesToMountPaths = Map("secret" -> "secretMountPath")) validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), + builderUnderTest.buildFromFeatures(conf, secMgr), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, SECRETS_STEP_TYPE, @@ -94,7 +96,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { val conf = KubernetesTestConf.createExecutorConf( volumes = Seq(volumeSpec)) validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), + builderUnderTest.buildFromFeatures(conf, secMgr), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, MOUNT_VOLUMES_STEP_TYPE) @@ -107,7 +109,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name")) validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), + builderUnderTest.buildFromFeatures(conf, secMgr), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, HADOOP_CONF_STEP_TYPE, @@ -123,7 +125,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { .set(KERBEROS_DT_SECRET_NAME, "dt-secret") .set(KERBEROS_DT_SECRET_KEY, "dt-key" )) validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), + builderUnderTest.buildFromFeatures(conf, secMgr), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, HADOOP_CONF_STEP_TYPE, @@ -154,7 +156,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { .endMetadata() .build())) val sparkPod = KubernetesExecutorBuilder(kubernetesClient, sparkConf) - .buildFromFeatures(kubernetesConf) + .buildFromFeatures(kubernetesConf, secMgr) PodBuilderSuiteUtils.verifyPodWithSupportedFeatures(sparkPod) } } 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 b746a01eb529..f8f4b4177f3b 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 @@ -36,6 +36,7 @@ import org.apache.spark.{SPARK_VERSION, SparkFunSuite} import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite @@ -138,6 +139,7 @@ class KubernetesSuite extends SparkFunSuite .set("spark.kubernetes.driver.pod.name", driverPodName) .set("spark.kubernetes.driver.label.spark-app-locator", appLocator) .set("spark.kubernetes.executor.label.spark-app-locator", appLocator) + .set(NETWORK_AUTH_ENABLED.key, "true") if (!kubernetesTestComponents.hasUserSpecifiedNamespace) { kubernetesTestComponents.createNamespace() } From bfc5569a53510bc75c15384084ff89b418592875 Mon Sep 17 00:00:00 2001 From: caoxuewen Date: Fri, 7 Dec 2018 09:57:35 +0800 Subject: [PATCH 0081/1072] [SPARK-26289][CORE] cleanup enablePerfMetrics parameter from BytesToBytesMap ## What changes were proposed in this pull request? `enablePerfMetrics `was originally designed in `BytesToBytesMap `to control `getNumHashCollisions getTimeSpentResizingNs getAverageProbesPerLookup`. However, as the Spark version gradual progress. this parameter is only used for `getAverageProbesPerLookup ` and always given to true when using `BytesToBytesMap`. it is also dangerous to determine whether `getAverageProbesPerLookup `opens and throws an `IllegalStateException `exception. So this pr will be remove `enablePerfMetrics `parameter from `BytesToBytesMap`. thanks. ## How was this patch tested? the existed test cases. Closes #23244 from heary-cao/enablePerfMetrics. Authored-by: caoxuewen Signed-off-by: Wenchen Fan --- .../spark/unsafe/map/BytesToBytesMap.java | 33 ++++--------------- .../map/AbstractBytesToBytesMapSuite.java | 4 +-- .../UnsafeFixedWidthAggregationMap.java | 2 +- .../sql/execution/joins/HashedRelation.scala | 6 ++-- 4 files changed, 12 insertions(+), 33 deletions(-) diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index a4e88598f760..405e52946415 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -159,11 +159,9 @@ public final class BytesToBytesMap extends MemoryConsumer { */ private final Location loc; - private final boolean enablePerfMetrics; + private long numProbes = 0L; - private long numProbes = 0; - - private long numKeyLookups = 0; + private long numKeyLookups = 0L; private long peakMemoryUsedBytes = 0L; @@ -180,8 +178,7 @@ public BytesToBytesMap( SerializerManager serializerManager, int initialCapacity, double loadFactor, - long pageSizeBytes, - boolean enablePerfMetrics) { + long pageSizeBytes) { super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; @@ -189,7 +186,6 @@ public BytesToBytesMap( this.loadFactor = loadFactor; this.loc = new Location(); this.pageSizeBytes = pageSizeBytes; - this.enablePerfMetrics = enablePerfMetrics; if (initialCapacity <= 0) { throw new IllegalArgumentException("Initial capacity must be greater than 0"); } @@ -209,14 +205,6 @@ public BytesToBytesMap( TaskMemoryManager taskMemoryManager, int initialCapacity, long pageSizeBytes) { - this(taskMemoryManager, initialCapacity, pageSizeBytes, false); - } - - public BytesToBytesMap( - TaskMemoryManager taskMemoryManager, - int initialCapacity, - long pageSizeBytes, - boolean enablePerfMetrics) { this( taskMemoryManager, SparkEnv.get() != null ? SparkEnv.get().blockManager() : null, @@ -224,8 +212,7 @@ public BytesToBytesMap( initialCapacity, // In order to re-use the longArray for sorting, the load factor cannot be larger than 0.5. 0.5, - pageSizeBytes, - enablePerfMetrics); + pageSizeBytes); } /** @@ -462,15 +449,12 @@ public Location lookup(Object keyBase, long keyOffset, int keyLength, int hash) public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location loc, int hash) { assert(longArray != null); - if (enablePerfMetrics) { - numKeyLookups++; - } + numKeyLookups++; + int pos = hash & mask; int step = 1; while (true) { - if (enablePerfMetrics) { - numProbes++; - } + numProbes++; if (longArray.get(pos * 2) == 0) { // This is a new key. loc.with(pos, hash, false); @@ -860,9 +844,6 @@ public long getPeakMemoryUsedBytes() { * Returns the average number of probes per key lookup. */ public double getAverageProbesPerLookup() { - if (!enablePerfMetrics) { - throw new IllegalStateException(); - } return (1.0 * numProbes) / numKeyLookups; } diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index 53a233f698c7..aa29232e73e1 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -530,7 +530,7 @@ public void failureToGrow() { @Test public void spillInIterator() throws IOException { BytesToBytesMap map = new BytesToBytesMap( - taskMemoryManager, blockManager, serializerManager, 1, 0.75, 1024, false); + taskMemoryManager, blockManager, serializerManager, 1, 0.75, 1024); try { int i; for (i = 0; i < 1024; i++) { @@ -569,7 +569,7 @@ public void spillInIterator() throws IOException { @Test public void multipleValuesForSameKey() { BytesToBytesMap map = - new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024, false); + new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024); try { int i; for (i = 0; i < 1024; i++) { diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java index c8cf44b51df7..7e76a651ba2c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java @@ -98,7 +98,7 @@ public UnsafeFixedWidthAggregationMap( this.groupingKeyProjection = UnsafeProjection.create(groupingKeySchema); this.groupingKeySchema = groupingKeySchema; this.map = new BytesToBytesMap( - taskContext.taskMemoryManager(), initialCapacity, pageSizeBytes, true); + taskContext.taskMemoryManager(), initialCapacity, pageSizeBytes); // Initialize the buffer for aggregation value final UnsafeProjection valueProjection = UnsafeProjection.create(aggregationBufferSchema); 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 e8c01d46a84c..b1ff6e83acc2 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 @@ -248,8 +248,7 @@ private[joins] class UnsafeHashedRelation( binaryMap = new BytesToBytesMap( taskMemoryManager, (nKeys * 1.5 + 1).toInt, // reduce hash collision - pageSizeBytes, - true) + pageSizeBytes) var i = 0 var keyBuffer = new Array[Byte](1024) @@ -299,8 +298,7 @@ private[joins] object UnsafeHashedRelation { taskMemoryManager, // Only 70% of the slots can be used before growing, more capacity help to reduce collision (sizeEstimate * 1.5 + 1).toInt, - pageSizeBytes, - true) + pageSizeBytes) // Create a mapping of buildKeys -> rows val keyGenerator = UnsafeProjection.create(key) From 5a140b7844936cf2b65f08853b8cfd8c499d4f13 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 7 Dec 2018 11:13:14 +0800 Subject: [PATCH 0082/1072] [SPARK-26263][SQL] Validate partition values with user provided schema MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Currently if user provides data schema, partition column values are converted as per it. But if the conversion failed, e.g. converting string to int, the column value is null. This PR proposes to throw exception in such case, instead of converting into null value silently: 1. These null partition column values doesn't make sense to users in most cases. It is better to show the conversion failure, and then users can adjust the schema or ETL jobs to fix it. 2. There are always exceptions on such conversion failure for non-partition data columns. Partition columns should have the same behavior. We can reproduce the case above as following: ``` /tmp/testDir ├── p=bar └── p=foo ``` If we run: ``` val schema = StructType(Seq(StructField("p", IntegerType, false))) spark.read.schema(schema).csv("/tmp/testDir/").show() ``` We will get: ``` +----+ | p| +----+ |null| |null| +----+ ``` ## How was this patch tested? Unit test Closes #23215 from gengliangwang/SPARK-26263. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- docs/sql-migration-guide-upgrade.md | 2 ++ .../apache/spark/sql/internal/SQLConf.scala | 12 ++++++++ .../PartitioningAwareFileIndex.scala | 4 +-- .../datasources/PartitioningUtils.scala | 30 +++++++++++++------ .../datasources/FileIndexSuite.scala | 27 ++++++++++++++++- .../ParquetPartitionDiscoverySuite.scala | 18 ++++++++--- 6 files changed, 77 insertions(+), 16 deletions(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index ed2ff139bcc3..3638b0873aa4 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -29,6 +29,8 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be udefined. + - In Spark version 2.4 and earlier, partition column value is converted as null if it can't be casted to corresponding user provided schema. Since 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. + - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.execution.setCommandRejectsSparkConfs` to `false`. - Spark applications which are built with Spark version 2.4 and prior, and call methods of `UserDefinedFunction`, need to be re-compiled with Spark 3.0, as they are not binary compatible with Spark 3.0. 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 451b051f8407..6857b8de7975 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 @@ -1396,6 +1396,16 @@ object SQLConf { .booleanConf .createWithDefault(false) + val VALIDATE_PARTITION_COLUMNS = + buildConf("spark.sql.sources.validatePartitionColumns") + .internal() + .doc("When this option is set to true, partition column values will be validated with " + + "user-specified schema. If the validation fails, a runtime exception is thrown." + + "When this option is set to false, the partition column value will be converted to null " + + "if it can not be casted to corresponding user-specified schema.") + .booleanConf + .createWithDefault(true) + val CONTINUOUS_STREAMING_EXECUTOR_QUEUE_SIZE = buildConf("spark.sql.streaming.continuous.executorQueueSize") .internal() @@ -2014,6 +2024,8 @@ class SQLConf extends Serializable with Logging { def allowCreatingManagedTableUsingNonemptyLocation: Boolean = getConf(ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION) + def validatePartitionColumns: Boolean = getConf(VALIDATE_PARTITION_COLUMNS) + def partitionOverwriteMode: PartitionOverwriteMode.Value = PartitionOverwriteMode.withName(getConf(PARTITION_OVERWRITE_MODE)) 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 7b0e4dbcc25f..b2e4155e6f49 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 @@ -127,13 +127,13 @@ abstract class PartitioningAwareFileIndex( val timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone) - val caseSensitive = sparkSession.sqlContext.conf.caseSensitiveAnalysis PartitioningUtils.parsePartitions( leafDirs, typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, basePaths = basePaths, userSpecifiedSchema = userSpecifiedSchema, - caseSensitive = caseSensitive, + caseSensitive = sparkSession.sqlContext.conf.caseSensitiveAnalysis, + validatePartitionColumns = sparkSession.sqlContext.conf.validatePartitionColumns, timeZoneId = timeZoneId) } 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 d66cb09bda0c..6458b65466fb 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 @@ -26,12 +26,13 @@ import scala.util.Try import org.apache.hadoop.fs.Path -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -96,9 +97,10 @@ object PartitioningUtils { basePaths: Set[Path], userSpecifiedSchema: Option[StructType], caseSensitive: Boolean, + validatePartitionColumns: Boolean, timeZoneId: String): PartitionSpec = { - parsePartitions(paths, typeInference, basePaths, userSpecifiedSchema, - caseSensitive, DateTimeUtils.getTimeZone(timeZoneId)) + parsePartitions(paths, typeInference, basePaths, userSpecifiedSchema, caseSensitive, + validatePartitionColumns, DateTimeUtils.getTimeZone(timeZoneId)) } private[datasources] def parsePartitions( @@ -107,6 +109,7 @@ object PartitioningUtils { basePaths: Set[Path], userSpecifiedSchema: Option[StructType], caseSensitive: Boolean, + validatePartitionColumns: Boolean, timeZone: TimeZone): PartitionSpec = { val userSpecifiedDataTypes = if (userSpecifiedSchema.isDefined) { val nameToDataType = userSpecifiedSchema.get.fields.map(f => f.name -> f.dataType).toMap @@ -121,7 +124,8 @@ object PartitioningUtils { // First, we need to parse every partition's path and see if we can find partition values. val (partitionValues, optDiscoveredBasePaths) = paths.map { path => - parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, timeZone) + parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, + validatePartitionColumns, timeZone) }.unzip // We create pairs of (path -> path's partition value) here @@ -203,6 +207,7 @@ object PartitioningUtils { typeInference: Boolean, basePaths: Set[Path], userSpecifiedDataTypes: Map[String, DataType], + validatePartitionColumns: Boolean, timeZone: TimeZone): (Option[PartitionValues], Option[Path]) = { val columns = ArrayBuffer.empty[(String, Literal)] // Old Hadoop versions don't have `Path.isRoot` @@ -224,7 +229,8 @@ object PartitioningUtils { // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1. // Once we get the string, we try to parse it and find the partition column and value. val maybeColumn = - parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes, timeZone) + parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes, + validatePartitionColumns, timeZone) maybeColumn.foreach(columns += _) // Now, we determine if we should stop. @@ -258,6 +264,7 @@ object PartitioningUtils { columnSpec: String, typeInference: Boolean, userSpecifiedDataTypes: Map[String, DataType], + validatePartitionColumns: Boolean, timeZone: TimeZone): Option[(String, Literal)] = { val equalSignIndex = columnSpec.indexOf('=') if (equalSignIndex == -1) { @@ -272,10 +279,15 @@ object PartitioningUtils { val literal = if (userSpecifiedDataTypes.contains(columnName)) { // SPARK-26188: if user provides corresponding column schema, get the column value without // inference, and then cast it as user specified data type. - val columnValue = inferPartitionColumnValue(rawColumnValue, false, timeZone) - val castedValue = - Cast(columnValue, userSpecifiedDataTypes(columnName), Option(timeZone.getID)).eval() - Literal.create(castedValue, userSpecifiedDataTypes(columnName)) + val dataType = userSpecifiedDataTypes(columnName) + val columnValueLiteral = inferPartitionColumnValue(rawColumnValue, false, timeZone) + val columnValue = columnValueLiteral.eval() + val castedValue = Cast(columnValueLiteral, dataType, Option(timeZone.getID)).eval() + if (validatePartitionColumns && columnValue != null && castedValue == null) { + throw new RuntimeException(s"Failed to cast value `$columnValue` to `$dataType` " + + s"for partition column `$columnName`") + } + Literal.create(castedValue, dataType) } else { inferPartitionColumnValue(rawColumnValue, typeInference, timeZone) } 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 ec552f7ddf47..6bd0a2591fc1 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 @@ -30,7 +30,7 @@ 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.test.SharedSQLContext -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.{KnownSizeEstimation, SizeEstimator} class FileIndexSuite extends SharedSQLContext { @@ -95,6 +95,31 @@ class FileIndexSuite extends SharedSQLContext { } } + test("SPARK-26263: Throw exception when partition value can't be casted to user-specified type") { + withTempDir { dir => + val partitionDirectory = new File(dir, "a=foo") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val path = new Path(dir.getCanonicalPath) + val schema = StructType(Seq(StructField("a", IntegerType, false))) + withSQLConf(SQLConf.VALIDATE_PARTITION_COLUMNS.key -> "true") { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema)) + val msg = intercept[RuntimeException] { + fileIndex.partitionSpec() + }.getMessage + assert(msg == "Failed to cast value `foo` to `IntegerType` for partition column `a`") + } + + withSQLConf(SQLConf.VALIDATE_PARTITION_COLUMNS.key -> "false") { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema)) + val partitionValues = fileIndex.partitionSpec().partitions.map(_.values) + assert(partitionValues.length == 1 && partitionValues(0).numFields == 1 && + partitionValues(0).isNullAt(0)) + } + } + } + test("InMemoryFileIndex: input paths are converted to qualified paths") { withTempDir { dir => val file = new File(dir, "text.txt") 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 f808ca458aaa..88067358667c 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 @@ -101,7 +101,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/path/a=10.5/b=hello") var exception = intercept[AssertionError] { - parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], None, true, timeZoneId) + parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], None, true, true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -117,6 +117,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha Set(new Path("hdfs://host:9000/path/")), None, true, + true, timeZoneId) // Valid @@ -132,6 +133,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha Set(new Path("hdfs://host:9000/path/something=true/table")), None, true, + true, timeZoneId) // Valid @@ -147,6 +149,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha Set(new Path("hdfs://host:9000/path/table=true")), None, true, + true, timeZoneId) // Invalid @@ -162,6 +165,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha Set(new Path("hdfs://host:9000/path/")), None, true, + true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -184,6 +188,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha Set(new Path("hdfs://host:9000/tmp/tables/")), None, true, + true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -191,13 +196,14 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partition") { def check(path: String, expected: Option[PartitionValues]): Unit = { - val actual = parsePartition(new Path(path), true, Set.empty[Path], Map.empty, timeZone)._1 + val actual = parsePartition(new Path(path), true, Set.empty[Path], + Map.empty, true, timeZone)._1 assert(expected === actual) } def checkThrows[T <: Throwable: Manifest](path: String, expected: String): Unit = { val message = intercept[T] { - parsePartition(new Path(path), true, Set.empty[Path], Map.empty, timeZone) + parsePartition(new Path(path), true, Set.empty[Path], Map.empty, true, timeZone) }.getMessage assert(message.contains(expected)) @@ -242,6 +248,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha typeInference = true, basePaths = Set(new Path("file://path/a=10")), Map.empty, + true, timeZone = timeZone)._1 assert(partitionSpec1.isEmpty) @@ -252,6 +259,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha typeInference = true, basePaths = Set(new Path("file://path")), Map.empty, + true, timeZone = timeZone)._1 assert(partitionSpec2 == @@ -272,6 +280,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha rootPaths, None, true, + true, timeZoneId) assert(actualSpec.partitionColumns === spec.partitionColumns) assert(actualSpec.partitions.length === spec.partitions.length) @@ -384,7 +393,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partitions with type inference disabled") { def check(paths: Seq[String], spec: PartitionSpec): Unit = { val actualSpec = - parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], None, true, timeZoneId) + parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], None, + true, true, timeZoneId) assert(actualSpec === spec) } From 477226520358f0cc47d5ea255ad84d3c13f6d77d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 6 Dec 2018 20:50:57 -0800 Subject: [PATCH 0083/1072] [SPARK-26298][BUILD] Upgrade Janino to 3.0.11 ## What changes were proposed in this pull request? This PR aims to upgrade Janino compiler to the latest version 3.0.11. The followings are the changes from the [release note](http://janino-compiler.github.io/janino/changelog.html). - Script with many "helper" variables. - Java 9+ compatibility - Compilation Error Messages Generated by JDK. - Added experimental support for the "StackMapFrame" attribute; not active yet. - Make Unparser more flexible. - Fixed NPEs in various "toString()" methods. - Optimize static method invocation with rvalue target expression. - Added all missing "ClassFile.getConstant*Info()" methods, removing the necessity for many type casts. ## How was this patch tested? Pass the Jenkins with the existing tests. Closes #23250 from dongjoon-hyun/SPARK-26298. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 4 ++-- dev/deps/spark-deps-hadoop-3.1 | 4 ++-- pom.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index ec7c304c9e36..d250d5205586 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -34,7 +34,7 @@ commons-beanutils-core-1.8.0.jar commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar -commons-compiler-3.0.10.jar +commons-compiler-3.0.11.jar commons-compress-1.8.1.jar commons-configuration-1.6.jar commons-crypto-1.0.0.jar @@ -98,7 +98,7 @@ jackson-module-jaxb-annotations-2.9.6.jar jackson-module-paranamer-2.9.6.jar jackson-module-scala_2.12-2.9.6.jar jackson-xc-1.9.13.jar -janino-3.0.10.jar +janino-3.0.11.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 811febf22940..347503ace557 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -31,7 +31,7 @@ commons-beanutils-1.9.3.jar commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar -commons-compiler-3.0.10.jar +commons-compiler-3.0.11.jar commons-compress-1.8.1.jar commons-configuration2-2.1.1.jar commons-crypto-1.0.0.jar @@ -97,7 +97,7 @@ jackson-mapper-asl-1.9.13.jar jackson-module-jaxb-annotations-2.9.6.jar jackson-module-paranamer-2.9.6.jar jackson-module-scala_2.12-2.9.6.jar -janino-3.0.10.jar +janino-3.0.11.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar diff --git a/pom.xml b/pom.xml index 61321a145070..15e5c1829040 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 3.8.1 3.2.10 - 3.0.10 + 3.0.11 2.22.2 2.9.3 3.5.2 From 1ab3d3e474ce2e36d58aea8ad09fb61f0c73e5c5 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 7 Dec 2018 07:55:54 -0800 Subject: [PATCH 0084/1072] [SPARK-26060][SQL][FOLLOW-UP] Rename the config name. ## What changes were proposed in this pull request? This is a follow-up of #23031 to rename the config name to `spark.sql.legacy.setCommandRejectsSparkCoreConfs`. ## How was this patch tested? Existing tests. Closes #23245 from ueshin/issues/SPARK-26060/rename_config. Authored-by: Takuya UESHIN Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide-upgrade.md | 2 +- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 7 ++++--- .../main/scala/org/apache/spark/sql/RuntimeConfig.scala | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 3638b0873aa4..67c30fb941ec 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -31,7 +31,7 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, partition column value is converted as null if it can't be casted to corresponding user provided schema. Since 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. - - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.execution.setCommandRejectsSparkConfs` to `false`. + - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. - Spark applications which are built with Spark version 2.4 and prior, and call methods of `UserDefinedFunction`, need to be re-compiled with Spark 3.0, as they are not binary compatible with Spark 3.0. 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 6857b8de7975..86e068bf632b 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 @@ -1621,8 +1621,8 @@ object SQLConf { .intConf .createWithDefault(25) - val SET_COMMAND_REJECTS_SPARK_CONFS = - buildConf("spark.sql.legacy.execution.setCommandRejectsSparkConfs") + val SET_COMMAND_REJECTS_SPARK_CORE_CONFS = + buildConf("spark.sql.legacy.setCommandRejectsSparkCoreConfs") .internal() .doc("If it is set to true, SET command will fail when the key is registered as " + "a SparkConf entry.") @@ -2057,7 +2057,8 @@ class SQLConf extends Serializable with Logging { def maxToStringFields: Int = getConf(SQLConf.MAX_TO_STRING_FIELDS) - def setCommandRejectsSparkConfs: Boolean = getConf(SQLConf.SET_COMMAND_REJECTS_SPARK_CONFS) + def setCommandRejectsSparkCoreConfs: Boolean = + getConf(SQLConf.SET_COMMAND_REJECTS_SPARK_CORE_CONFS) def legacyTimeParserEnabled: Boolean = getConf(SQLConf.LEGACY_TIME_PARSER_ENABLED) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index d83a01ff9ea6..0f5aab7f47d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -153,7 +153,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { if (SQLConf.staticConfKeys.contains(key)) { throw new AnalysisException(s"Cannot modify the value of a static config: $key") } - if (sqlConf.setCommandRejectsSparkConfs && + if (sqlConf.setCommandRejectsSparkCoreConfs && ConfigEntry.findEntry(key) != null && !SQLConf.sqlConfEntries.containsKey(key)) { throw new AnalysisException(s"Cannot modify the value of a Spark config: $key") } From 543577a1e8c0904048b73008fa7c4cee33f69894 Mon Sep 17 00:00:00 2001 From: Sahil Takiar Date: Fri, 7 Dec 2018 10:33:42 -0800 Subject: [PATCH 0085/1072] [SPARK-24243][CORE] Expose exceptions from InProcessAppHandle Adds a new method to SparkAppHandle called getError which returns the exception (if present) that caused the underlying Spark app to fail. New tests added to SparkLauncherSuite for the new method. Closes #21849 Closes #23221 from vanzin/SPARK-24243. Signed-off-by: Marcelo Vanzin --- .../spark/launcher/SparkLauncherSuite.java | 102 ++++++++++++++++-- .../spark/launcher/ChildProcAppHandle.java | 20 +++- .../spark/launcher/InProcessAppHandle.java | 13 +++ .../spark/launcher/OutputRedirector.java | 25 +++++ .../apache/spark/launcher/SparkAppHandle.java | 8 ++ project/MimaExcludes.scala | 3 + 6 files changed, 159 insertions(+), 12 deletions(-) diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index 6a1a38c1a54f..773c390175b6 100644 --- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -41,6 +41,8 @@ public class SparkLauncherSuite extends BaseSuite { private static final NamedThreadFactory TF = new NamedThreadFactory("SparkLauncherSuite-%d"); + private static final String EXCEPTION_MESSAGE = "dummy-exception"; + private static final RuntimeException DUMMY_EXCEPTION = new RuntimeException(EXCEPTION_MESSAGE); private final SparkLauncher launcher = new SparkLauncher(); @@ -130,17 +132,8 @@ public void testInProcessLauncher() throws Exception { try { inProcessLauncherTestImpl(); } finally { - Properties p = new Properties(); - for (Map.Entry e : properties.entrySet()) { - p.put(e.getKey(), e.getValue()); - } - System.setProperties(p); - // Here DAGScheduler is stopped, while SparkContext.clearActiveContext may not be called yet. - // Wait for a reasonable amount of time to avoid creating two active SparkContext in JVM. - // See SPARK-23019 and SparkContext.stop() for details. - eventually(Duration.ofSeconds(5), Duration.ofMillis(10), () -> { - assertTrue("SparkContext is still alive.", SparkContext$.MODULE$.getActive().isEmpty()); - }); + restoreSystemProperties(properties); + waitForSparkContextShutdown(); } } @@ -227,6 +220,82 @@ public void testInProcessLauncherDoesNotKillJvm() throws Exception { assertEquals(SparkAppHandle.State.LOST, handle.getState()); } + @Test + public void testInProcessLauncherGetError() throws Exception { + // Because this test runs SparkLauncher in process and in client mode, it pollutes the system + // properties, and that can cause test failures down the test pipeline. So restore the original + // system properties after this test runs. + Map properties = new HashMap<>(System.getProperties()); + + SparkAppHandle handle = null; + try { + handle = new InProcessLauncher() + .setMaster("local") + .setAppResource(SparkLauncher.NO_RESOURCE) + .setMainClass(ErrorInProcessTestApp.class.getName()) + .addAppArgs("hello") + .startApplication(); + + final SparkAppHandle _handle = handle; + eventually(Duration.ofSeconds(60), Duration.ofMillis(1000), () -> { + assertEquals(SparkAppHandle.State.FAILED, _handle.getState()); + }); + + assertNotNull(handle.getError()); + assertTrue(handle.getError().isPresent()); + assertSame(handle.getError().get(), DUMMY_EXCEPTION); + } finally { + if (handle != null) { + handle.kill(); + } + restoreSystemProperties(properties); + waitForSparkContextShutdown(); + } + } + + @Test + public void testSparkLauncherGetError() throws Exception { + SparkAppHandle handle = null; + try { + handle = new SparkLauncher() + .setMaster("local") + .setAppResource(SparkLauncher.NO_RESOURCE) + .setMainClass(ErrorInProcessTestApp.class.getName()) + .addAppArgs("hello") + .startApplication(); + + final SparkAppHandle _handle = handle; + eventually(Duration.ofSeconds(60), Duration.ofMillis(1000), () -> { + assertEquals(SparkAppHandle.State.FAILED, _handle.getState()); + }); + + assertNotNull(handle.getError()); + assertTrue(handle.getError().isPresent()); + assertTrue(handle.getError().get().getMessage().contains(EXCEPTION_MESSAGE)); + } finally { + if (handle != null) { + handle.kill(); + } + } + } + + private void restoreSystemProperties(Map properties) { + Properties p = new Properties(); + for (Map.Entry e : properties.entrySet()) { + p.put(e.getKey(), e.getValue()); + } + System.setProperties(p); + } + + private void waitForSparkContextShutdown() throws Exception { + // Here DAGScheduler is stopped, while SparkContext.clearActiveContext may not be called yet. + // Wait for a reasonable amount of time to avoid creating two active SparkContext in JVM. + // See SPARK-23019 and SparkContext.stop() for details. + eventually(Duration.ofSeconds(5), Duration.ofMillis(10), () -> { + assertTrue("SparkContext is still alive.", SparkContext$.MODULE$.getActive().isEmpty()); + }); + } + public static class SparkLauncherTestApp { public static void main(String[] args) throws Exception { @@ -264,4 +333,15 @@ public static void main(String[] args) throws Exception { } + /** + * Similar to {@link InProcessTestApp} except it throws an exception + */ + public static class ErrorInProcessTestApp { + + public static void main(String[] args) { + assertNotEquals(0, args.length); + assertEquals(args[0], "hello"); + throw DUMMY_EXCEPTION; + } + } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java index 5609f8492f4f..7dfcf0e66734 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java @@ -18,6 +18,7 @@ package org.apache.spark.launcher; import java.io.InputStream; +import java.util.Optional; import java.util.logging.Level; import java.util.logging.Logger; @@ -29,7 +30,7 @@ class ChildProcAppHandle extends AbstractAppHandle { private static final Logger LOG = Logger.getLogger(ChildProcAppHandle.class.getName()); private volatile Process childProc; - private OutputRedirector redirector; + private volatile OutputRedirector redirector; ChildProcAppHandle(LauncherServer server) { super(server); @@ -46,6 +47,23 @@ public synchronized void disconnect() { } } + /** + * Parses the logs of {@code spark-submit} and returns the last exception thrown. + *

+ * Since {@link SparkLauncher} runs {@code spark-submit} in a sub-process, it's difficult to + * accurately retrieve the full {@link Throwable} from the {@code spark-submit} process. + * This method parses the logs of the sub-process and provides a best-effort attempt at + * returning the last exception thrown by the {@code spark-submit} process. Only the exception + * message is parsed, the associated stacktrace is meaningless. + * + * @return an {@link Optional} containing a {@link RuntimeException} with the parsed + * exception, otherwise returns a {@link Optional#EMPTY} + */ + @Override + public Optional getError() { + return redirector != null ? Optional.ofNullable(redirector.getError()) : Optional.empty(); + } + @Override public synchronized void kill() { if (!isDisposed()) { diff --git a/launcher/src/main/java/org/apache/spark/launcher/InProcessAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/InProcessAppHandle.java index 15fbca0facef..ba09050c756d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/InProcessAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/InProcessAppHandle.java @@ -17,7 +17,9 @@ package org.apache.spark.launcher; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.logging.Level; import java.util.logging.Logger; @@ -31,6 +33,8 @@ class InProcessAppHandle extends AbstractAppHandle { // Avoid really long thread names. private static final int MAX_APP_NAME_LEN = 16; + private volatile Throwable error; + private Thread app; InProcessAppHandle(LauncherServer server) { @@ -51,6 +55,11 @@ public synchronized void kill() { } } + @Override + public Optional getError() { + return Optional.ofNullable(error); + } + synchronized void start(String appName, Method main, String[] args) { CommandBuilderUtils.checkState(app == null, "Handle already started."); @@ -62,7 +71,11 @@ synchronized void start(String appName, Method main, String[] args) { try { main.invoke(null, (Object) args); } catch (Throwable t) { + if (t instanceof InvocationTargetException) { + t = t.getCause(); + } LOG.log(Level.WARNING, "Application failed with exception.", t); + error = t; setState(State.FAILED); } diff --git a/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java b/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java index 6f4b0bb38e03..0f097f831392 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java +++ b/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java @@ -37,6 +37,7 @@ class OutputRedirector { private final ChildProcAppHandle callback; private volatile boolean active; + private volatile Throwable error; OutputRedirector(InputStream in, String loggerName, ThreadFactory tf) { this(in, loggerName, tf, null); @@ -61,6 +62,10 @@ private void redirect() { while ((line = reader.readLine()) != null) { if (active) { sink.info(line.replaceFirst("\\s*$", "")); + if ((containsIgnoreCase(line, "Error") || containsIgnoreCase(line, "Exception")) && + !line.contains("at ")) { + error = new RuntimeException(line); + } } } } catch (IOException e) { @@ -85,4 +90,24 @@ boolean isAlive() { return thread.isAlive(); } + Throwable getError() { + return error; + } + + /** + * Copied from Apache Commons Lang {@code StringUtils#containsIgnoreCase(String, String)} + */ + private static boolean containsIgnoreCase(String str, String searchStr) { + if (str == null || searchStr == null) { + return false; + } + int len = searchStr.length(); + int max = str.length() - len; + for (int i = 0; i <= max; i++) { + if (str.regionMatches(true, i, searchStr, 0, len)) { + return true; + } + } + return false; + } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java index cefb4d1a95fb..afec270e2b11 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java @@ -17,6 +17,8 @@ package org.apache.spark.launcher; +import java.util.Optional; + /** * A handle to a running Spark application. *

@@ -100,6 +102,12 @@ public boolean isFinal() { */ void disconnect(); + /** + * If the application failed due to an error, return the underlying error. If the app + * succeeded, this method returns an empty {@link Optional}. + */ + Optional getError(); + /** * Listener for updates to a handle's state. The callbacks do not receive information about * what exactly has changed, just that an update has occurred. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1c83cf5860c5..4eeebb805070 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,9 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( + // [SPARK-24243][CORE] Expose exceptions from InProcessAppHandle + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.launcher.SparkAppHandle.getError"), + // [SPARK-25867] Remove KMeans computeCost ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.KMeansModel.computeCost"), From 9b7679a97ed2622081390301af8735e62055492d Mon Sep 17 00:00:00 2001 From: 10087686 Date: Fri, 7 Dec 2018 14:11:25 -0600 Subject: [PATCH 0086/1072] [SPARK-26294][CORE] Delete Unnecessary If statement ## What changes were proposed in this pull request? Delete unnecessary If statement, because it Impossible execution when records less than or equal to zero.it is only execution when records begin zero. ................... if (inMemSorter == null || inMemSorter.numRecords() <= 0) { return 0L; } .................... if (inMemSorter.numRecords() > 0) { ..................... } ## How was this patch tested? Existing tests (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23247 from wangjiaochun/inMemSorter. Authored-by: 10087686 Signed-off-by: Sean Owen --- .../unsafe/sort/UnsafeExternalSorter.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 5056652a2420..af5a934b7da6 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -213,14 +213,12 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { spillWriters.size() > 1 ? " times" : " time"); ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - // We only write out contents of the inMemSorter if it is not empty. - if (inMemSorter.numRecords() > 0) { - final UnsafeSorterSpillWriter spillWriter = - new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, - inMemSorter.numRecords()); - spillWriters.add(spillWriter); - spillIterator(inMemSorter.getSortedIterator(), spillWriter); - } + + final UnsafeSorterSpillWriter spillWriter = + new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, + inMemSorter.numRecords()); + spillWriters.add(spillWriter); + spillIterator(inMemSorter.getSortedIterator(), spillWriter); final long spillSize = freeMemory(); // Note that this is more-or-less going to be a multiple of the page size, so wasted space in From bd00f10773f3a08e50ecd06c7e70d9b38094a252 Mon Sep 17 00:00:00 2001 From: dima-asana <42555784+dima-asana@users.noreply.github.com> Date: Fri, 7 Dec 2018 14:14:43 -0600 Subject: [PATCH 0087/1072] [MINOR][SQL][DOC] Correct parquet nullability documentation ## What changes were proposed in this pull request? Parquet files appear to have nullability info when being written, not being read. ## How was this patch tested? Some test code: (running spark 2.3, but the relevant code in DataSource looks identical on master) case class NullTest(bo: Boolean, opbol: Option[Boolean]) val testDf = spark.createDataFrame(Seq(NullTest(true, Some(false)))) defined class NullTest testDf: org.apache.spark.sql.DataFrame = [bo: boolean, opbol: boolean] testDf.write.parquet("s3://asana-stats/tmp_dima/parquet_check_schema") spark.read.parquet("s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet4").printSchema() root |-- bo: boolean (nullable = true) |-- opbol: boolean (nullable = true) Meanwhile, the parquet file formed does have nullable info: []batchprod-report000:/tmp/dimakamalov-batch$ aws s3 ls s3://asana-stats/tmp_dima/parquet_check_schema/ 2018-10-17 21:03:52 0 _SUCCESS 2018-10-17 21:03:50 504 part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet []batchprod-report000:/tmp/dimakamalov-batch$ aws s3 cp s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet . download: s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet to ./part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet []batchprod-report000:/tmp/dimakamalov-batch$ java -jar parquet-tools-1.8.2.jar schema part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet message spark_schema { required boolean bo; optional boolean opbol; } Closes #22759 from dima-asana/dima-asana-nullable-parquet-doc. Authored-by: dima-asana <42555784+dima-asana@users.noreply.github.com> Signed-off-by: Sean Owen --- docs/sql-data-sources-parquet.md | 2 +- .../sql/test/DataFrameReaderWriterSuite.scala | 44 +++++++++++++++++-- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/docs/sql-data-sources-parquet.md b/docs/sql-data-sources-parquet.md index 4fed3eaf83e5..dcd293651846 100644 --- a/docs/sql-data-sources-parquet.md +++ b/docs/sql-data-sources-parquet.md @@ -9,7 +9,7 @@ displayTitle: Parquet Files [Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. When writing Parquet files, all columns are automatically converted to be nullable for +of the original data. When reading Parquet files, all columns are automatically converted to be nullable for compatibility reasons. ### Loading Data Programmatically 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 237872585e11..e45ab19aadbf 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 @@ -23,6 +23,13 @@ import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.JavaConverters._ +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.hadoop.util.HadoopInputFile +import org.apache.parquet.schema.PrimitiveType +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName +import org.apache.parquet.schema.Type.Repetition import org.scalatest.BeforeAndAfter import org.apache.spark.SparkContext @@ -31,6 +38,7 @@ import org.apache.spark.internal.io.HadoopMapReduceCommitProtocol import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -522,11 +530,12 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be Seq("json", "orc", "parquet", "csv").foreach { format => val schema = StructType( StructField("cl1", IntegerType, nullable = false).withComment("test") :: - StructField("cl2", IntegerType, nullable = true) :: - StructField("cl3", IntegerType, nullable = true) :: Nil) + StructField("cl2", IntegerType, nullable = true) :: + StructField("cl3", IntegerType, nullable = true) :: Nil) val row = Row(3, null, 4) val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + // if we write and then read, the read will enforce schema to be nullable val tableName = "tab" withTable(tableName) { df.write.format(format).mode("overwrite").saveAsTable(tableName) @@ -536,12 +545,41 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be Row("cl1", "test") :: Nil) // Verify the schema val expectedFields = schema.fields.map(f => f.copy(nullable = true)) - assert(spark.table(tableName).schema == schema.copy(fields = expectedFields)) + assert(spark.table(tableName).schema === schema.copy(fields = expectedFields)) } } } } + test("parquet - column nullability -- write only") { + val schema = StructType( + StructField("cl1", IntegerType, nullable = false) :: + StructField("cl2", IntegerType, nullable = true) :: Nil) + val row = Row(3, 4) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + withTempPath { dir => + val path = dir.getAbsolutePath + df.write.mode("overwrite").parquet(path) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0) + + val hadoopInputFile = HadoopInputFile.fromPath(new Path(file), new Configuration()) + val f = ParquetFileReader.open(hadoopInputFile) + val parquetSchema = f.getFileMetaData.getSchema.getColumns.asScala + .map(_.getPrimitiveType) + f.close() + + // the write keeps nullable info from the schema + val expectedParquetSchema = Seq( + new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.INT32, "cl1"), + new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.INT32, "cl2") + ) + + assert (expectedParquetSchema === parquetSchema) + } + + } + test("SPARK-17230: write out results of decimal calculation") { val df = spark.range(99, 101) .selectExpr("id", "cast(id as long) * cast('1.0' as decimal(38, 18)) as num") From 3b8ae23735f5b29db95516662190f606edc51fd7 Mon Sep 17 00:00:00 2001 From: Shahid Date: Fri, 7 Dec 2018 14:31:35 -0600 Subject: [PATCH 0088/1072] [SPARK-26196][SPARK-26281][WEBUI] Total tasks title in the stage page is incorrect when there are failed or killed tasks and update duration metrics ## What changes were proposed in this pull request? This PR fixes 3 issues 1) Total tasks message in the tasks table is incorrect, when there are failed or killed tasks 2) Sorting of the "Duration" column is not correct 3) Duration in the aggregated tasks summary table and the tasks table and not matching. Total tasks = numCompleteTasks + numActiveTasks + numKilledTasks + numFailedTasks; Corrected the duration metrics in the tasks table as executorRunTime based on the PR https://github.com/apache/spark/pull/23081 ## How was this patch tested? test step: 1) ``` bin/spark-shell scala > sc.parallelize(1 to 100, 10).map{ x => throw new RuntimeException("Bad executor")}.collect() ``` ![screenshot from 2018-11-28 07-26-00](https://user-images.githubusercontent.com/23054875/49123523-e2691880-f2de-11e8-9c16-60d1865e6e77.png) After patch: ![screenshot from 2018-11-28 07-24-31](https://user-images.githubusercontent.com/23054875/49123525-e432dc00-f2de-11e8-89ca-4a53e19c9c18.png) 2) Duration metrics: Before patch: ![screenshot from 2018-12-06 03-25-14](https://user-images.githubusercontent.com/23054875/49546591-9e8d9900-f906-11e8-8a0b-157742c47655.png) After patch: ![screenshot from 2018-12-06 03-23-14](https://user-images.githubusercontent.com/23054875/49546589-9cc3d580-f906-11e8-827f-52ef8ffdeaec.png) Closes #23160 from shahidki31/totalTasks. Authored-by: Shahid Signed-off-by: Sean Owen --- .../resources/org/apache/spark/ui/static/stagepage.js | 9 +++++---- .../org/apache/spark/status/api/v1/StagesResource.scala | 1 - 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 564467487e84..08de2b0fee03 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -616,7 +616,8 @@ $(document).ready(function () { $("#accumulator-table").DataTable(accumulatorConf); // building tasks table that uses server side functionality - var totalTasksToShow = responseBody.numCompleteTasks + responseBody.numActiveTasks; + var totalTasksToShow = responseBody.numCompleteTasks + responseBody.numActiveTasks + + responseBody.numKilledTasks + responseBody.numFailedTasks; var taskTable = "#active-tasks-table"; var taskConf = { "serverSide": true, @@ -667,8 +668,8 @@ $(document).ready(function () { {data : "launchTime", name: "Launch Time", render: formatDate}, { data : function (row, type) { - if (row.duration) { - return type === 'display' ? formatDuration(row.duration) : row.duration; + if (row.taskMetrics && row.taskMetrics.executorRunTime) { + return type === 'display' ? formatDuration(row.taskMetrics.executorRunTime) : row.taskMetrics.executorRunTime; } else { return ""; } @@ -927,7 +928,7 @@ $(document).ready(function () { // title number and toggle list $("#summaryMetricsTitle").html("Summary Metrics for " + "" + responseBody.numCompleteTasks + " Completed Tasks" + ""); - $("#tasksTitle").html("Task (" + totalTasksToShow + ")"); + $("#tasksTitle").html("Tasks (" + totalTasksToShow + ")"); // hide or show the accumulate update table if (accumulatorTable.length == 0) { 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 f81892734c2d..9d1d66a0e15a 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 @@ -210,7 +210,6 @@ private[v1] class StagesResource extends BaseAppResource { (containsValue(f.taskId) || containsValue(f.index) || containsValue(f.attempt) || containsValue(f.launchTime) || containsValue(f.resultFetchStart.getOrElse(defaultOptionString)) - || containsValue(f.duration.getOrElse(defaultOptionString)) || containsValue(f.executorId) || containsValue(f.host) || containsValue(f.status) || containsValue(f.taskLocality) || containsValue(f.speculative) || containsValue(f.errorMessage.getOrElse(defaultOptionString)) From 20278e719e28fc5d7a8069e0498a8df143ecee90 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 7 Dec 2018 13:53:35 -0800 Subject: [PATCH 0089/1072] [SPARK-24333][ML][PYTHON] Add fit with validation set to spark.ml GBT: Python API ## What changes were proposed in this pull request? Add validationIndicatorCol and validationTol to GBT Python. ## How was this patch tested? Add test in doctest to test the new API. Closes #21465 from huaxingao/spark-24333. Authored-by: Huaxin Gao Signed-off-by: Bryan Cutler --- python/pyspark/ml/classification.py | 81 ++++++++------ .../ml/param/_shared_params_code_gen.py | 5 +- python/pyspark/ml/param/shared.py | 71 ++++++++----- python/pyspark/ml/regression.py | 100 +++++++++++++----- 4 files changed, 169 insertions(+), 88 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index ce028512357f..6ddfce95a3d4 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -23,7 +23,7 @@ from pyspark.ml import Estimator, Model from pyspark.ml.param.shared import * from pyspark.ml.regression import DecisionTreeModel, DecisionTreeRegressionModel, \ - RandomForestParams, TreeEnsembleModel, TreeEnsembleParams + GBTParams, HasVarianceImpurity, RandomForestParams, TreeEnsembleModel, TreeEnsembleParams from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams from pyspark.ml.wrapper import JavaWrapper @@ -895,15 +895,6 @@ def getImpurity(self): return self.getOrDefault(self.impurity) -class GBTParams(TreeEnsembleParams): - """ - Private class to track supported GBT params. - - .. versionadded:: 1.4.0 - """ - supportedLossTypes = ["logistic"] - - @inherit_doc class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, HasRawPredictionCol, DecisionTreeParams, @@ -1174,9 +1165,31 @@ def trees(self): return [DecisionTreeClassificationModel(m) for m in list(self._call_java("trees"))] +class GBTClassifierParams(GBTParams, HasVarianceImpurity): + """ + Private class to track supported GBTClassifier params. + + .. versionadded:: 3.0.0 + """ + + supportedLossTypes = ["logistic"] + + lossType = Param(Params._dummy(), "lossType", + "Loss function which GBT tries to minimize (case-insensitive). " + + "Supported options: " + ", ".join(supportedLossTypes), + typeConverter=TypeConverters.toString) + + @since("1.4.0") + def getLossType(self): + """ + Gets the value of lossType or its default value. + """ + return self.getOrDefault(self.lossType) + + @inherit_doc -class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, - GBTParams, HasCheckpointInterval, HasStepSize, HasSeed, JavaMLWritable, +class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, + GBTClassifierParams, HasCheckpointInterval, HasSeed, JavaMLWritable, JavaMLReadable): """ `Gradient-Boosted Trees (GBTs) `_ @@ -1242,32 +1255,28 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol [0.25..., 0.23..., 0.21..., 0.19..., 0.18...] >>> model.numClasses 2 + >>> gbt = gbt.setValidationIndicatorCol("validationIndicator") + >>> gbt.getValidationIndicatorCol() + 'validationIndicator' + >>> gbt.getValidationTol() + 0.01 .. versionadded:: 1.4.0 """ - lossType = Param(Params._dummy(), "lossType", - "Loss function which GBT tries to minimize (case-insensitive). " + - "Supported options: " + ", ".join(GBTParams.supportedLossTypes), - typeConverter=TypeConverters.toString) - - stepSize = Param(Params._dummy(), "stepSize", - "Step size (a.k.a. learning rate) in interval (0, 1] for shrinking " + - "the contribution of each estimator.", - typeConverter=TypeConverters.toFloat) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, lossType="logistic", - maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, - featureSubsetStrategy="all"): + maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, impurity="variance", + featureSubsetStrategy="all", validationTol=0.01, validationIndicatorCol=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ lossType="logistic", maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, \ - featureSubsetStrategy="all") + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, \ + validationIndicatorCol=None) """ super(GBTClassifier, self).__init__() self._java_obj = self._new_java_obj( @@ -1275,7 +1284,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred 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, - featureSubsetStrategy="all") + impurity="variance", featureSubsetStrategy="all", validationTol=0.01) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1285,13 +1294,15 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, lossType="logistic", maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, - featureSubsetStrategy="all"): + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, + validationIndicatorCol=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ lossType="logistic", maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, \ - featureSubsetStrategy="all") + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, \ + validationIndicatorCol=None) Sets params for Gradient Boosted Tree Classification. """ kwargs = self._input_kwargs @@ -1307,13 +1318,6 @@ def setLossType(self, value): """ return self._set(lossType=value) - @since("1.4.0") - def getLossType(self): - """ - Gets the value of lossType or its default value. - """ - return self.getOrDefault(self.lossType) - @since("2.4.0") def setFeatureSubsetStrategy(self, value): """ @@ -1321,6 +1325,13 @@ def setFeatureSubsetStrategy(self, value): """ return self._set(featureSubsetStrategy=value) + @since("3.0.0") + def setValidationIndicatorCol(self, value): + """ + Sets the value of :py:attr:`validationIndicatorCol`. + """ + return self._set(validationIndicatorCol=value) + class GBTClassificationModel(TreeEnsembleModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable): diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index e45ba840b412..1b0c8c5d28b7 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -164,7 +164,10 @@ def get$Name(self): "False", "TypeConverters.toBoolean"), ("loss", "the loss function to be optimized.", None, "TypeConverters.toString"), ("distanceMeasure", "the distance measure. Supported options: 'euclidean' and 'cosine'.", - "'euclidean'", "TypeConverters.toString")] + "'euclidean'", "TypeConverters.toString"), + ("validationIndicatorCol", "name of the column that indicates whether each row is for " + + "training or for validation. False indicates training; true indicates validation.", + None, "TypeConverters.toString")] code = [] for name, doc, defaultValueStr, typeConverter in shared: diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 618f5bf0a810..6405b9fce7ef 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -702,6 +702,53 @@ def getLoss(self): return self.getOrDefault(self.loss) +class HasDistanceMeasure(Params): + """ + Mixin for param distanceMeasure: the distance measure. Supported options: 'euclidean' and 'cosine'. + """ + + distanceMeasure = Param(Params._dummy(), "distanceMeasure", "the distance measure. Supported options: 'euclidean' and 'cosine'.", typeConverter=TypeConverters.toString) + + def __init__(self): + super(HasDistanceMeasure, self).__init__() + self._setDefault(distanceMeasure='euclidean') + + def setDistanceMeasure(self, value): + """ + Sets the value of :py:attr:`distanceMeasure`. + """ + return self._set(distanceMeasure=value) + + def getDistanceMeasure(self): + """ + Gets the value of distanceMeasure or its default value. + """ + return self.getOrDefault(self.distanceMeasure) + + +class HasValidationIndicatorCol(Params): + """ + Mixin for param validationIndicatorCol: name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation. + """ + + validationIndicatorCol = Param(Params._dummy(), "validationIndicatorCol", "name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation.", typeConverter=TypeConverters.toString) + + def __init__(self): + super(HasValidationIndicatorCol, self).__init__() + + def setValidationIndicatorCol(self, value): + """ + Sets the value of :py:attr:`validationIndicatorCol`. + """ + return self._set(validationIndicatorCol=value) + + def getValidationIndicatorCol(self): + """ + Gets the value of validationIndicatorCol or its default value. + """ + return self.getOrDefault(self.validationIndicatorCol) + + class DecisionTreeParams(Params): """ Mixin for Decision Tree parameters. @@ -790,27 +837,3 @@ def getCacheNodeIds(self): """ return self.getOrDefault(self.cacheNodeIds) - -class HasDistanceMeasure(Params): - """ - Mixin for param distanceMeasure: the distance measure. Supported options: 'euclidean' and 'cosine'. - """ - - distanceMeasure = Param(Params._dummy(), "distanceMeasure", "the distance measure. Supported options: 'euclidean' and 'cosine'.", typeConverter=TypeConverters.toString) - - def __init__(self): - super(HasDistanceMeasure, self).__init__() - self._setDefault(distanceMeasure='euclidean') - - def setDistanceMeasure(self, value): - """ - Sets the value of :py:attr:`distanceMeasure`. - """ - return self._set(distanceMeasure=value) - - def getDistanceMeasure(self): - """ - Gets the value of distanceMeasure or its default value. - """ - return self.getOrDefault(self.distanceMeasure) - diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 98f436135184..78cb4a670355 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -650,19 +650,20 @@ def getFeatureSubsetStrategy(self): return self.getOrDefault(self.featureSubsetStrategy) -class TreeRegressorParams(Params): +class HasVarianceImpurity(Params): """ Private class to track supported impurity measures. """ supportedImpurities = ["variance"] + impurity = Param(Params._dummy(), "impurity", "Criterion used for information gain calculation (case-insensitive). " + "Supported options: " + ", ".join(supportedImpurities), typeConverter=TypeConverters.toString) def __init__(self): - super(TreeRegressorParams, self).__init__() + super(HasVarianceImpurity, self).__init__() @since("1.4.0") def setImpurity(self, value): @@ -679,6 +680,10 @@ def getImpurity(self): return self.getOrDefault(self.impurity) +class TreeRegressorParams(HasVarianceImpurity): + pass + + class RandomForestParams(TreeEnsembleParams): """ Private class to track supported random forest parameters. @@ -705,12 +710,52 @@ def getNumTrees(self): return self.getOrDefault(self.numTrees) -class GBTParams(TreeEnsembleParams): +class GBTParams(TreeEnsembleParams, HasMaxIter, HasStepSize, HasValidationIndicatorCol): """ Private class to track supported GBT params. """ + + stepSize = Param(Params._dummy(), "stepSize", + "Step size (a.k.a. learning rate) in interval (0, 1] for shrinking " + + "the contribution of each estimator.", + typeConverter=TypeConverters.toFloat) + + validationTol = Param(Params._dummy(), "validationTol", + "Threshold for stopping early when fit with validation is used. " + + "If the error rate on the validation input changes by less than the " + + "validationTol, then learning will stop early (before `maxIter`). " + + "This parameter is ignored when fit without validation is used.", + typeConverter=TypeConverters.toFloat) + + @since("3.0.0") + def getValidationTol(self): + """ + Gets the value of validationTol or its default value. + """ + return self.getOrDefault(self.validationTol) + + +class GBTRegressorParams(GBTParams, TreeRegressorParams): + """ + Private class to track supported GBTRegressor params. + + .. versionadded:: 3.0.0 + """ + supportedLossTypes = ["squared", "absolute"] + lossType = Param(Params._dummy(), "lossType", + "Loss function which GBT tries to minimize (case-insensitive). " + + "Supported options: " + ", ".join(supportedLossTypes), + typeConverter=TypeConverters.toString) + + @since("1.4.0") + def getLossType(self): + """ + Gets the value of lossType or its default value. + """ + return self.getOrDefault(self.lossType) + @inherit_doc class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, @@ -1030,9 +1075,9 @@ def featureImportances(self): @inherit_doc -class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, - GBTParams, HasCheckpointInterval, HasStepSize, HasSeed, JavaMLWritable, - JavaMLReadable, TreeRegressorParams): +class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, + GBTRegressorParams, HasCheckpointInterval, HasSeed, JavaMLWritable, + JavaMLReadable): """ `Gradient-Boosted Trees (GBTs) `_ learning algorithm for regression. @@ -1079,39 +1124,36 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, ... ["label", "features"]) >>> model.evaluateEachIteration(validation, "squared") [0.0, 0.0, 0.0, 0.0, 0.0] + >>> gbt = gbt.setValidationIndicatorCol("validationIndicator") + >>> gbt.getValidationIndicatorCol() + 'validationIndicator' + >>> gbt.getValidationTol() + 0.01 .. versionadded:: 1.4.0 """ - lossType = Param(Params._dummy(), "lossType", - "Loss function which GBT tries to minimize (case-insensitive). " + - "Supported options: " + ", ".join(GBTParams.supportedLossTypes), - typeConverter=TypeConverters.toString) - - stepSize = Param(Params._dummy(), "stepSize", - "Step size (a.k.a. learning rate) in interval (0, 1] for shrinking " + - "the contribution of each estimator.", - typeConverter=TypeConverters.toFloat) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", 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, seed=None, - impurity="variance", featureSubsetStrategy="all"): + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, + validationIndicatorCol=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ 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, seed=None, \ - impurity="variance", featureSubsetStrategy="all") + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, \ + validationIndicatorCol=None) """ 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") + impurity="variance", featureSubsetStrategy="all", validationTol=0.01) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1121,13 +1163,15 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre 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, seed=None, - impuriy="variance", featureSubsetStrategy="all"): + impuriy="variance", featureSubsetStrategy="all", validationTol=0.01, + validationIndicatorCol=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ 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, seed=None, \ - impurity="variance", featureSubsetStrategy="all") + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, \ + validationIndicatorCol=None) Sets params for Gradient Boosted Tree Regression. """ kwargs = self._input_kwargs @@ -1143,13 +1187,6 @@ def setLossType(self, value): """ return self._set(lossType=value) - @since("1.4.0") - def getLossType(self): - """ - Gets the value of lossType or its default value. - """ - return self.getOrDefault(self.lossType) - @since("2.4.0") def setFeatureSubsetStrategy(self, value): """ @@ -1157,6 +1194,13 @@ def setFeatureSubsetStrategy(self, value): """ return self._set(featureSubsetStrategy=value) + @since("3.0.0") + def setValidationIndicatorCol(self, value): + """ + Sets the value of :py:attr:`validationIndicatorCol`. + """ + return self._set(validationIndicatorCol=value) + class GBTRegressionModel(TreeEnsembleModel, JavaPredictionModel, JavaMLWritable, JavaMLReadable): """ From 9b1f6c8bab5401258c653d4e2efb50e97c6d282f Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Fri, 7 Dec 2018 13:58:02 -0800 Subject: [PATCH 0090/1072] [SPARK-26304][SS] Add default value to spark.kafka.sasl.kerberos.service.name parameter ## What changes were proposed in this pull request? spark.kafka.sasl.kerberos.service.name is an optional parameter but most of the time value `kafka` has to be set. As I've written in the jira the following reasoning is behind: * Kafka's configuration guide suggest the same value: https://kafka.apache.org/documentation/#security_sasl_kerberos_brokerconfig * It would be easier for spark users by providing less configuration * Other streaming engines are doing the same In this PR I've changed the parameter from optional to `WithDefault` and set `kafka` as default value. ## How was this patch tested? Available unit tests + on cluster. Closes #23254 from gaborgsomogyi/SPARK-26304. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../deploy/security/KafkaTokenUtil.scala | 7 ++---- .../apache/spark/internal/config/Kafka.scala | 2 +- .../deploy/security/KafkaTokenUtilSuite.scala | 24 ------------------- .../sql/kafka010/KafkaSecurityHelper.scala | 5 +--- .../kafka010/KafkaSecurityHelperSuite.scala | 15 ------------ 5 files changed, 4 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala b/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala index c890cee59ffe..aec0f72feb3c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala @@ -143,14 +143,11 @@ private[spark] object KafkaTokenUtil extends Logging { } private[security] def getKeytabJaasParams(sparkConf: SparkConf): String = { - val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) - require(serviceName.nonEmpty, "Kerberos service name must be defined") - val params = s""" |${getKrb5LoginModuleName} required | useKeyTab=true - | serviceName="${serviceName.get}" + | serviceName="${sparkConf.get(Kafka.KERBEROS_SERVICE_NAME)}" | keyTab="${sparkConf.get(KEYTAB).get}" | principal="${sparkConf.get(PRINCIPAL).get}"; """.stripMargin.replace("\n", "") @@ -166,7 +163,7 @@ private[spark] object KafkaTokenUtil extends Logging { s""" |${getKrb5LoginModuleName} required | useTicketCache=true - | serviceName="${serviceName.get}"; + | serviceName="${sparkConf.get(Kafka.KERBEROS_SERVICE_NAME)}"; """.stripMargin.replace("\n", "") logDebug(s"Krb ticket cache JAAS params: $params") params diff --git a/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala b/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala index 85d74c27142a..064fc93cb8ed 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala @@ -40,7 +40,7 @@ private[spark] object Kafka { "Kafka's JAAS config or in Kafka's config. For further details please see kafka " + "documentation. Only used to obtain delegation token.") .stringConf - .createOptional + .createWithDefault("kafka") val TRUSTSTORE_LOCATION = ConfigBuilder("spark.kafka.ssl.truststore.location") diff --git a/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala index 682bebde916f..18aa537b3a51 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala @@ -36,7 +36,6 @@ class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { private val keyStorePassword = "keyStoreSecret" private val keyPassword = "keySecret" private val keytab = "/path/to/keytab" - private val kerberosServiceName = "kafka" private val principal = "user@domain.com" private var sparkConf: SparkConf = null @@ -96,7 +95,6 @@ class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { sparkConf.set(Kafka.KEYSTORE_LOCATION, keyStoreLocation) sparkConf.set(Kafka.KEYSTORE_PASSWORD, keyStorePassword) sparkConf.set(Kafka.KEY_PASSWORD, keyPassword) - sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) @@ -119,7 +117,6 @@ class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { sparkConf.set(Kafka.KEYSTORE_LOCATION, keyStoreLocation) sparkConf.set(Kafka.KEYSTORE_PASSWORD, keyStorePassword) sparkConf.set(Kafka.KEY_PASSWORD, keyPassword) - sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) @@ -143,7 +140,6 @@ class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { sparkConf.set(Kafka.KEYSTORE_LOCATION, keyStoreLocation) sparkConf.set(Kafka.KEYSTORE_PASSWORD, keyStorePassword) sparkConf.set(Kafka.KEY_PASSWORD, keyPassword) - sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) @@ -177,7 +173,6 @@ class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { sparkConf.set(Kafka.BOOTSTRAP_SERVERS, bootStrapServers) sparkConf.set(Kafka.SECURITY_PROTOCOL, SASL_SSL.name) sparkConf.set(KEYTAB, keytab) - sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) sparkConf.set(PRINCIPAL, principal) val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) @@ -195,7 +190,6 @@ class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { test("createAdminClientProperties without keytab should set ticket cache dynamic jaas config") { sparkConf.set(Kafka.BOOTSTRAP_SERVERS, bootStrapServers) sparkConf.set(Kafka.SECURITY_PROTOCOL, SASL_SSL.name) - sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf) @@ -218,22 +212,4 @@ class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { assert(KafkaTokenUtil.isGlobalJaasConfigurationProvided) } - - test("getKeytabJaasParams with keytab no service should throw exception") { - sparkConf.set(KEYTAB, keytab) - - val thrown = intercept[IllegalArgumentException] { - KafkaTokenUtil.getKeytabJaasParams(sparkConf) - } - - assert(thrown.getMessage contains "Kerberos service name must be defined") - } - - test("getTicketCacheJaasParams without service should throw exception") { - val thrown = intercept[IllegalArgumentException] { - KafkaTokenUtil.getTicketCacheJaasParams(sparkConf) - } - - assert(thrown.getMessage contains "Kerberos service name must be defined") - } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala index 74d5ef9c05f1..7215295b1009 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.kafka010 import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.kafka.common.security.scram.ScramLoginModule import org.apache.spark.SparkConf @@ -35,8 +34,6 @@ private[kafka010] object KafkaSecurityHelper extends Logging { def getTokenJaasParams(sparkConf: SparkConf): String = { val token = UserGroupInformation.getCurrentUser().getCredentials.getToken( KafkaTokenUtil.TOKEN_SERVICE) - val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) - require(serviceName.isDefined, "Kerberos service name must be defined") val username = new String(token.getIdentifier) val password = new String(token.getPassword) @@ -45,7 +42,7 @@ private[kafka010] object KafkaSecurityHelper extends Logging { s""" |$loginModuleName required | tokenauth=true - | serviceName="${serviceName.get}" + | serviceName="${sparkConf.get(Kafka.KERBEROS_SERVICE_NAME)}" | username="$username" | password="$password"; """.stripMargin.replace("\n", "") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala index 772fe4614bad..fd9dee390d18 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala @@ -26,12 +26,8 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.security.KafkaTokenUtil import org.apache.spark.deploy.security.KafkaTokenUtil.KafkaDelegationTokenIdentifier -import org.apache.spark.internal.config._ class KafkaSecurityHelperSuite extends SparkFunSuite with BeforeAndAfterEach { - private val keytab = "/path/to/keytab" - private val kerberosServiceName = "kafka" - private val principal = "user@domain.com" private val tokenId = "tokenId" + UUID.randomUUID().toString private val tokenPassword = "tokenPassword" + UUID.randomUUID().toString @@ -76,19 +72,8 @@ class KafkaSecurityHelperSuite extends SparkFunSuite with BeforeAndAfterEach { assert(KafkaSecurityHelper.isTokenAvailable()) } - test("getTokenJaasParams with token no service should throw exception") { - addTokenToUGI() - - val thrown = intercept[IllegalArgumentException] { - KafkaSecurityHelper.getTokenJaasParams(sparkConf) - } - - assert(thrown.getMessage contains "Kerberos service name must be defined") - } - test("getTokenJaasParams with token should return scram module") { addTokenToUGI() - sparkConf.set(Kafka.KERBEROS_SERVICE_NAME, kerberosServiceName) val jaasParams = KafkaSecurityHelper.getTokenJaasParams(sparkConf) From 2ea9792fdeb07be19d63e7625cfc483e062a1d9c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 8 Dec 2018 05:59:53 -0600 Subject: [PATCH 0091/1072] [SPARK-26266][BUILD] Update to Scala 2.12.8 ## What changes were proposed in this pull request? Update to Scala 2.12.8 ## How was this patch tested? Existing tests. Closes #23218 from srowen/SPARK-26266. Authored-by: Sean Owen Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.7 | 6 +++--- dev/deps/spark-deps-hadoop-3.1 | 6 +++--- docs/_config.yml | 2 +- pom.xml | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index d250d5205586..71423af0789c 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -171,10 +171,10 @@ parquet-jackson-1.10.0.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.13.jar -scala-compiler-2.12.7.jar -scala-library-2.12.7.jar +scala-compiler-2.12.8.jar +scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar -scala-reflect-2.12.7.jar +scala-reflect-2.12.8.jar scala-xml_2.12-1.0.5.jar shapeless_2.12-2.3.2.jar slf4j-api-1.7.16.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 347503ace557..93eafef04533 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -189,10 +189,10 @@ protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.13.jar re2j-1.1.jar -scala-compiler-2.12.7.jar -scala-library-2.12.7.jar +scala-compiler-2.12.8.jar +scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar -scala-reflect-2.12.7.jar +scala-reflect-2.12.8.jar scala-xml_2.12-1.0.5.jar shapeless_2.12-2.3.2.jar slf4j-api-1.7.16.jar diff --git a/docs/_config.yml b/docs/_config.yml index 649d18bf72b5..146c90fcff6e 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -17,7 +17,7 @@ include: SPARK_VERSION: 3.0.0-SNAPSHOT SPARK_VERSION_SHORT: 3.0.0 SCALA_BINARY_VERSION: "2.12" -SCALA_VERSION: "2.12.7" +SCALA_VERSION: "2.12.8" MESOS_VERSION: 1.0.0 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/pom.xml b/pom.xml index 15e5c1829040..310d7de95512 100644 --- a/pom.xml +++ b/pom.xml @@ -156,7 +156,7 @@ 3.4.1 3.2.2 - 2.12.7 + 2.12.8 2.12 --diff --test From 678e1aca6901944c119d2ec56169d4e69fce66de Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 8 Dec 2018 22:23:50 +0800 Subject: [PATCH 0092/1072] [SPARK-24207][R] follow-up PR for SPARK-24207 to fix code style problems ## What changes were proposed in this pull request? follow-up PR for SPARK-24207 to fix code style problems Closes #23256 from huaxingao/spark-24207-cnt. Authored-by: Huaxin Gao Signed-off-by: Hyukjin Kwon --- R/pkg/R/mllib_fpm.R | 7 +++-- R/pkg/tests/fulltests/test_mllib_fpm.R | 29 ++++++++++--------- R/pkg/vignettes/sparkr-vignettes.Rmd | 7 +++-- examples/src/main/r/ml/prefixSpan.R | 9 +++--- .../spark/examples/ml/FPGrowthExample.scala | 3 -- .../spark/examples/ml/PrefixSpanExample.scala | 3 -- 6 files changed, 28 insertions(+), 30 deletions(-) diff --git a/R/pkg/R/mllib_fpm.R b/R/pkg/R/mllib_fpm.R index ac37580c6b37..c248e9ec9be9 100644 --- a/R/pkg/R/mllib_fpm.R +++ b/R/pkg/R/mllib_fpm.R @@ -190,9 +190,10 @@ setMethod("write.ml", signature(object = "FPGrowthModel", path = "character"), #' @examples #' \dontrun{ #' df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), -#' list(list(list(1L), list(3L, 2L), list(1L, 2L))), -#' list(list(list(1L, 2L), list(5L))), -#' list(list(list(6L)))), schema = c("sequence")) +#' list(list(list(1L), list(3L, 2L), list(1L, 2L))), +#' list(list(list(1L, 2L), list(5L))), +#' list(list(list(6L)))), +#' schema = c("sequence")) #' frequency <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, #' maxLocalProjDBSize = 32000000L) #' showDF(frequency) diff --git a/R/pkg/tests/fulltests/test_mllib_fpm.R b/R/pkg/tests/fulltests/test_mllib_fpm.R index daf9ff97a821..bc1e17538d41 100644 --- a/R/pkg/tests/fulltests/test_mllib_fpm.R +++ b/R/pkg/tests/fulltests/test_mllib_fpm.R @@ -84,19 +84,20 @@ test_that("spark.fpGrowth", { }) test_that("spark.prefixSpan", { - df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), - list(list(list(1L), list(3L, 2L), list(1L, 2L))), - list(list(list(1L, 2L), list(5L))), - list(list(list(6L)))), schema = c("sequence")) - result1 <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, - maxLocalProjDBSize = 32000000L) - - expected_result <- createDataFrame(list(list(list(list(1L)), 3L), - list(list(list(3L)), 2L), - list(list(list(2L)), 3L), - list(list(list(1L, 2L)), 3L), - list(list(list(1L), list(3L)), 2L)), - schema = c("sequence", "freq")) - }) + df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), + list(list(list(1L), list(3L, 2L), list(1L, 2L))), + list(list(list(1L, 2L), list(5L))), + list(list(list(6L)))), + schema = c("sequence")) + result <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, + maxLocalProjDBSize = 32000000L) + + expected_result <- createDataFrame(list(list(list(list(1L)), 3L), list(list(list(3L)), 2L), + list(list(list(2L)), 3L), list(list(list(1L, 2L)), 3L), + list(list(list(1L), list(3L)), 2L)), + schema = c("sequence", "freq")) + + expect_equivalent(expected_result, result) +}) sparkR.session.stop() diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index f80b45b4f36a..1c6a03c4b9bc 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -1019,9 +1019,10 @@ head(predict(fpm, df)) ```{r} df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), - list(list(list(1L), list(3L, 2L), list(1L, 2L))), - list(list(list(1L, 2L), list(5L))), - list(list(list(6L)))), schema = c("sequence")) + list(list(list(1L), list(3L, 2L), list(1L, 2L))), + list(list(list(1L, 2L), list(5L))), + list(list(list(6L)))), + schema = c("sequence")) head(spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L)) ``` diff --git a/examples/src/main/r/ml/prefixSpan.R b/examples/src/main/r/ml/prefixSpan.R index 9b70573ffb78..02908aeb0296 100644 --- a/examples/src/main/r/ml/prefixSpan.R +++ b/examples/src/main/r/ml/prefixSpan.R @@ -28,9 +28,10 @@ sparkR.session(appName = "SparkR-ML-prefixSpan-example") # Load training data df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), - list(list(list(1L), list(3L, 2L), list(1L, 2L))), - list(list(list(1L, 2L), list(5L))), - list(list(list(6L)))), schema = c("sequence")) + list(list(list(1L), list(3L, 2L), list(1L, 2L))), + list(list(list(1L, 2L), list(5L))), + list(list(list(6L)))), + schema = c("sequence")) # Finding frequent sequential patterns frequency <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, @@ -39,4 +40,4 @@ showDF(frequency) # $example off$ -sparkR.session.stop() \ No newline at end of file +sparkR.session.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala index 59110d70de55..bece0d96c030 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml -// scalastyle:off println - // $example on$ import org.apache.spark.ml.fpm.FPGrowth // $example off$ @@ -64,4 +62,3 @@ object FPGrowthExample { spark.stop() } } -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala index 0a2d31097a02..b4e0811c506b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml -// scalastyle:off println - // $example on$ import org.apache.spark.ml.fpm.PrefixSpan // $example off$ @@ -59,4 +57,3 @@ object PrefixSpanExample { spark.stop() } } -// scalastyle:on println From bdf32847b1ffcb3aa4d0bef058f86e65656e99fb Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 8 Dec 2018 11:18:09 -0800 Subject: [PATCH 0093/1072] [SPARK-26021][SQL][FOLLOWUP] only deal with NaN and -0.0 in UnsafeWriter ## What changes were proposed in this pull request? A followup of https://github.com/apache/spark/pull/23043 There are 4 places we need to deal with NaN and -0.0: 1. comparison expressions. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same. 2. Join keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same. 3. grouping keys. `-0.0` and `0.0` should be assigned to the same group. Different NaNs should be assigned to the same group. 4. window partition keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same. The case 1 is OK. Our comparison already handles NaN and -0.0, and for struct/array/map, we will recursively compare the fields/elements. Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary directly, and different NaNs have different binary representation, and the same thing happens for -0.0 and 0.0. To fix it, a simple solution is: normalize float/double when building unsafe data (`UnsafeRow`, `UnsafeArrayData`, `UnsafeMapData`). Then we don't need to worry about it anymore. Following this direction, this PR moves the handling of NaN and -0.0 from `Platform` to `UnsafeWriter`, so that places like `UnsafeRow.setFloat` will not handle them, which reduces the perf overhead. It's also easier to add comments explaining why we do it in `UnsafeWriter`. ## How was this patch tested? existing tests Closes #23239 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/unsafe/Platform.java | 10 ------ .../spark/unsafe/PlatformUtilSuite.java | 18 ---------- .../expressions/codegen/UnsafeWriter.java | 35 +++++++++++++++++++ .../codegen/UnsafeRowWriterSuite.scala | 20 +++++++++++ .../apache/spark/sql/DataFrameJoinSuite.scala | 12 +++++++ .../sql/DataFrameWindowFunctionsSuite.scala | 14 ++++++++ 6 files changed, 81 insertions(+), 28 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index 4563efcfcf47..076b693f81c8 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -174,11 +174,6 @@ public static float getFloat(Object object, long offset) { } public static void putFloat(Object object, long offset, float value) { - if (Float.isNaN(value)) { - value = Float.NaN; - } else if (value == -0.0f) { - value = 0.0f; - } _UNSAFE.putFloat(object, offset, value); } @@ -187,11 +182,6 @@ public static double getDouble(Object object, long offset) { } public static void putDouble(Object object, long offset, double value) { - if (Double.isNaN(value)) { - value = Double.NaN; - } else if (value == -0.0d) { - value = 0.0d; - } _UNSAFE.putDouble(object, offset, value); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java index 2474081dad5c..3ad9ac7b4de9 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -157,22 +157,4 @@ public void heapMemoryReuse() { Assert.assertEquals(onheap4.size(), 1024 * 1024 + 7); Assert.assertEquals(obj3, onheap4.getBaseObject()); } - - @Test - // SPARK-26021 - public void writeMinusZeroIsReplacedWithZero() { - byte[] doubleBytes = new byte[Double.BYTES]; - byte[] floatBytes = new byte[Float.BYTES]; - Platform.putDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET, -0.0d); - Platform.putFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET, -0.0f); - - byte[] doubleBytes2 = new byte[Double.BYTES]; - byte[] floatBytes2 = new byte[Float.BYTES]; - Platform.putDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET, 0.0d); - Platform.putFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET, 0.0f); - - // Make sure the bytes we write from 0.0 and -0.0 are same. - Assert.assertArrayEquals(doubleBytes, doubleBytes2); - Assert.assertArrayEquals(floatBytes, floatBytes2); - } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java index 95263a0da95a..7553ab8cf700 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java @@ -198,11 +198,46 @@ protected final void writeLong(long offset, long value) { Platform.putLong(getBuffer(), offset, value); } + // We need to take care of NaN and -0.0 in several places: + // 1. When compare values, different NaNs should be treated as same, `-0.0` and `0.0` should be + // treated as same. + // 2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 0.0 should belong + // to the same group. + // 3. As join keys, different NaNs should be treated as same, `-0.0` and `0.0` should be + // treated as same. + // 4. As window partition keys, different NaNs should be treated as same, `-0.0` and `0.0` + // should be treated as same. + // + // Case 1 is fine, as we handle NaN and -0.0 well during comparison. For complex types, we + // recursively compare the fields/elements, so it's also fine. + // + // Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary directly, and different + // NaNs have different binary representation, and the same thing happens for -0.0 and 0.0. + // + // Here we normalize NaN and -0.0, so that `UnsafeProjection` will normalize them when writing + // float/double columns and nested fields to `UnsafeRow`. + // + // Note that, we must do this for all the `UnsafeProjection`s, not only the ones that extract + // join/grouping/window partition keys. `UnsafeProjection` copies unsafe data directly for complex + // types, so nested float/double may not be normalized. We need to make sure that all the unsafe + // data(`UnsafeRow`, `UnsafeArrayData`, `UnsafeMapData`) will have flat/double normalized during + // creation. protected final void writeFloat(long offset, float value) { + if (Float.isNaN(value)) { + value = Float.NaN; + } else if (value == -0.0f) { + value = 0.0f; + } Platform.putFloat(getBuffer(), offset, value); } + // See comments for `writeFloat`. protected final void writeDouble(long offset, double value) { + if (Double.isNaN(value)) { + value = Double.NaN; + } else if (value == -0.0d) { + value = 0.0d; + } Platform.putDouble(getBuffer(), offset, value); } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala index fb651b76fc16..22e1fa6dfed4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala @@ -50,4 +50,24 @@ class UnsafeRowWriterSuite extends SparkFunSuite { assert(res1 == res2) } + test("SPARK-26021: normalize float/double NaN and -0.0") { + val unsafeRowWriter1 = new UnsafeRowWriter(4) + unsafeRowWriter1.resetRowWriter() + unsafeRowWriter1.write(0, Float.NaN) + unsafeRowWriter1.write(1, Double.NaN) + unsafeRowWriter1.write(2, 0.0f) + unsafeRowWriter1.write(3, 0.0) + val res1 = unsafeRowWriter1.getRow + + val unsafeRowWriter2 = new UnsafeRowWriter(4) + unsafeRowWriter2.resetRowWriter() + unsafeRowWriter2.write(0, 0.0f/0.0f) + unsafeRowWriter2.write(1, 0.0/0.0) + unsafeRowWriter2.write(2, -0.0f) + unsafeRowWriter2.write(3, -0.0) + val res2 = unsafeRowWriter2.getRow + + // The two rows should be the equal + assert(res1 == res2) + } } 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 e6b30f9956da..c9f41ab1c017 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 @@ -295,4 +295,16 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { df.join(df, df("id") <=> df("id")).queryExecution.optimizedPlan } } + + test("NaN and -0.0 in join keys") { + val df1 = Seq(Float.NaN -> Double.NaN, 0.0f -> 0.0, -0.0f -> -0.0).toDF("f", "d") + val df2 = Seq(Float.NaN -> Double.NaN, 0.0f -> 0.0, -0.0f -> -0.0).toDF("f", "d") + val joined = df1.join(df2, Seq("f", "d")) + checkAnswer(joined, Seq( + Row(Float.NaN, Double.NaN), + Row(0.0f, 0.0), + Row(0.0f, 0.0), + Row(0.0f, 0.0), + Row(0.0f, 0.0))) + } } 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 78277d7dcf75..9a5d5a9966ab 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 @@ -681,4 +681,18 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { Row("S2", "P2", 300, 300, 500))) } + + test("NaN and -0.0 in window partition keys") { + val df = Seq( + (Float.NaN, Double.NaN, 1), + (0.0f/0.0f, 0.0/0.0, 1), + (0.0f, 0.0, 1), + (-0.0f, -0.0, 1)).toDF("f", "d", "i") + val result = df.select($"f", count("i").over(Window.partitionBy("f", "d"))) + checkAnswer(result, Seq( + Row(Float.NaN, 2), + Row(Float.NaN, 2), + Row(0.0f, 2), + Row(0.0f, 2))) + } } From 55276d3a26474e7479941db3e9c065d86344885f Mon Sep 17 00:00:00 2001 From: seancxmao Date: Sat, 8 Dec 2018 17:53:12 -0800 Subject: [PATCH 0094/1072] [SPARK-25132][SQL][FOLLOWUP][DOC] Add migration doc for case-insensitive field resolution when reading from Parquet ## What changes were proposed in this pull request? #22148 introduces a behavior change. According to discussion at #22184, this PR updates migration guide when upgrade from Spark 2.3 to 2.4. ## How was this patch tested? N/A Closes #23238 from seancxmao/SPARK-25132-doc-2.4. Authored-by: seancxmao Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide-upgrade.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 67c30fb941ec..f6458a9b2730 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -145,6 +145,8 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. + - In version 2.3 and earlier, when reading from a Parquet data source table, Spark always returns null for any column whose column names in Hive metastore schema and Parquet schema are in different letter cases, no matter whether `spark.sql.caseSensitive` is set to `true` or `false`. Since 2.4, when `spark.sql.caseSensitive` is set to `false`, Spark does case insensitive column name resolution between Hive metastore schema and Parquet schema, so even column names are in different letter cases, Spark returns corresponding column values. An exception is thrown if there is ambiguity, i.e. more than one Parquet column is matched. This change also applies to Parquet Hive tables when `spark.sql.hive.convertMetastoreParquet` is set to `true`. + ## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. From 877f82cb30bc4edef770b36e1e394a887ab535c6 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sun, 9 Dec 2018 10:49:15 +0800 Subject: [PATCH 0095/1072] [SPARK-26193][SQL] Implement shuffle write metrics in SQL ## What changes were proposed in this pull request? 1. Implement `SQLShuffleWriteMetricsReporter` on the SQL side as the customized `ShuffleWriteMetricsReporter`. 2. Add shuffle write metrics to `ShuffleExchangeExec`, and use these metrics to create corresponding `SQLShuffleWriteMetricsReporter` in shuffle dependency. 3. Rework on `ShuffleMapTask` to add new class named `ShuffleWriteProcessor` which control shuffle write process, we use sql shuffle write metrics by customizing a ShuffleWriteProcessor on SQL side. ## How was this patch tested? Add UT in SQLMetricsSuite. Manually test locally, update screen shot to document attached in JIRA. Closes #23207 from xuanyuanking/SPARK-26193. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/Dependency.scala | 6 +- .../spark/scheduler/ShuffleMapTask.scala | 20 +---- .../spark/shuffle/ShuffleWriteProcessor.scala | 74 +++++++++++++++++++ project/MimaExcludes.scala | 3 + .../exchange/ShuffleExchangeExec.scala | 38 ++++++++-- .../apache/spark/sql/execution/limit.scala | 30 ++++++-- .../sql/execution/metric/SQLMetrics.scala | 12 +++ .../metric/SQLShuffleMetricsReporter.scala | 55 ++++++++++++++ .../execution/metric/SQLMetricsSuite.scala | 36 +++++++-- 9 files changed, 234 insertions(+), 40 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 9ea6d2fa2fd9..fb051a8c0db8 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.ShuffleHandle +import org.apache.spark.shuffle.{ShuffleHandle, ShuffleWriteProcessor} /** * :: DeveloperApi :: @@ -65,6 +65,7 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { * @param keyOrdering key ordering for RDD's shuffles * @param aggregator map/reduce-side aggregator for RDD's shuffle * @param mapSideCombine whether to perform partial aggregation (also known as map-side combine) + * @param shuffleWriterProcessor the processor to control the write behavior in ShuffleMapTask */ @DeveloperApi class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( @@ -73,7 +74,8 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val serializer: Serializer = SparkEnv.get.serializer, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, - val mapSideCombine: Boolean = false) + val mapSideCombine: Boolean = false, + val shuffleWriterProcessor: ShuffleWriteProcessor = new ShuffleWriteProcessor) extends Dependency[Product2[K, V]] { if (mapSideCombine) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 5412717d6198..2a8d1dd995e2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -92,25 +92,7 @@ private[spark] class ShuffleMapTask( threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime } else 0L - var writer: ShuffleWriter[Any, Any] = null - try { - val manager = SparkEnv.get.shuffleManager - writer = manager.getWriter[Any, Any]( - dep.shuffleHandle, partitionId, context, context.taskMetrics().shuffleWriteMetrics) - writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) - writer.stop(success = true).get - } catch { - case e: Exception => - try { - if (writer != null) { - writer.stop(success = false) - } - } catch { - case e: Exception => - log.debug("Could not stop writer", e) - } - throw e - } + dep.shuffleWriterProcessor.writeProcess(rdd, dep, partitionId, context, partition) } override def preferredLocations: Seq[TaskLocation] = preferredLocs diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala new file mode 100644 index 000000000000..f5213157a9a8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import org.apache.spark.{Partition, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.MapStatus + +/** + * The interface for customizing shuffle write process. The driver create a ShuffleWriteProcessor + * and put it into [[ShuffleDependency]], and executors use it in each ShuffleMapTask. + */ +private[spark] class ShuffleWriteProcessor extends Serializable with Logging { + + /** + * Create a [[ShuffleWriteMetricsReporter]] from the task context. As the reporter is a + * per-row operator, here need a careful consideration on performance. + */ + protected def createMetricsReporter(context: TaskContext): ShuffleWriteMetricsReporter = { + context.taskMetrics().shuffleWriteMetrics + } + + /** + * The write process for particular partition, it controls the life circle of [[ShuffleWriter]] + * get from [[ShuffleManager]] and triggers rdd compute, finally return the [[MapStatus]] for + * this task. + */ + def writeProcess( + rdd: RDD[_], + dep: ShuffleDependency[_, _, _], + partitionId: Int, + context: TaskContext, + partition: Partition): MapStatus = { + var writer: ShuffleWriter[Any, Any] = null + try { + val manager = SparkEnv.get.shuffleManager + writer = manager.getWriter[Any, Any]( + dep.shuffleHandle, + partitionId, + context, + createMetricsReporter(context)) + writer.write( + rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) + writer.stop(success = true).get + } catch { + case e: Exception => + try { + if (writer != null) { + writer.stop(success = false) + } + } catch { + case e: Exception => + log.debug("Could not stop writer", e) + } + throw e + } + } +} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4eeebb805070..b3252d70a80c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -217,6 +217,9 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.sort.UnsafeShuffleWriter.this"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.TimeTrackingOutputStream.this"), + // [SPARK-26139] Implement shuffle write metrics in SQL + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ShuffleDependency.this"), + // Data Source V2 API changes (problem: Problem) => problem match { case MissingClassProblem(cls) => 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 c9ca395bceaa..0c2020572e72 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 @@ -23,6 +23,7 @@ import java.util.function.Supplier import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{ShuffleWriteMetricsReporter, ShuffleWriteProcessor} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ @@ -30,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Uns import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.metric.{SQLMetrics, SQLShuffleMetricsReporter} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.MutablePair @@ -46,10 +47,13 @@ case class ShuffleExchangeExec( // NOTE: coordinator can be null after serialization/deserialization, // e.g. it can be null on the Executor side - + private lazy val writeMetrics = + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) + private lazy val readMetrics = + SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size") - ) ++ SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) + ) ++ readMetrics ++ writeMetrics override def nodeName: String = { val extraInfo = coordinator match { @@ -90,7 +94,11 @@ case class ShuffleExchangeExec( private[exchange] def prepareShuffleDependency() : ShuffleDependency[Int, InternalRow, InternalRow] = { ShuffleExchangeExec.prepareShuffleDependency( - child.execute(), child.output, newPartitioning, serializer) + child.execute(), + child.output, + newPartitioning, + serializer, + writeMetrics) } /** @@ -109,7 +117,7 @@ case class ShuffleExchangeExec( assert(newPartitioning.isInstanceOf[HashPartitioning]) newPartitioning = UnknownPartitioning(indices.length) } - new ShuffledRowRDD(shuffleDependency, metrics, specifiedPartitionStartIndices) + new ShuffledRowRDD(shuffleDependency, readMetrics, specifiedPartitionStartIndices) } /** @@ -204,7 +212,9 @@ object ShuffleExchangeExec { rdd: RDD[InternalRow], outputAttributes: Seq[Attribute], newPartitioning: Partitioning, - serializer: Serializer): ShuffleDependency[Int, InternalRow, InternalRow] = { + serializer: Serializer, + writeMetrics: Map[String, SQLMetric]) + : ShuffleDependency[Int, InternalRow, InternalRow] = { val part: Partitioner = newPartitioning match { case RoundRobinPartitioning(numPartitions) => new HashPartitioner(numPartitions) case HashPartitioning(_, n) => @@ -333,8 +343,22 @@ object ShuffleExchangeExec { new ShuffleDependency[Int, InternalRow, InternalRow]( rddWithPartitionIds, new PartitionIdPassthrough(part.numPartitions), - serializer) + serializer, + shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics)) dependency } + + /** + * Create a customized [[ShuffleWriteProcessor]] for SQL which wrap the default metrics reporter + * with [[SQLShuffleWriteMetricsReporter]] as new reporter for [[ShuffleWriteProcessor]]. + */ + def createShuffleWriteProcessor(metrics: Map[String, SQLMetric]): ShuffleWriteProcessor = { + new ShuffleWriteProcessor { + override protected def createMetricsReporter( + context: TaskContext): ShuffleWriteMetricsReporter = { + new SQLShuffleWriteMetricsReporter(context.taskMetrics().shuffleWriteMetrics, metrics) + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index e9ab7cd138d9..1f2fdde53864 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGe import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.sql.execution.metric.SQLShuffleMetricsReporter +import org.apache.spark.sql.execution.metric.{SQLShuffleMetricsReporter, SQLShuffleWriteMetricsReporter} /** * Take the first `limit` elements and collect them to a single partition. @@ -38,13 +38,21 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode override def outputPartitioning: Partitioning = SinglePartition override def executeCollect(): Array[InternalRow] = child.executeTake(limit) private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) - override lazy val metrics = SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) + private lazy val writeMetrics = + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) + private lazy val readMetrics = + SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) + override lazy val metrics = readMetrics ++ writeMetrics protected override def doExecute(): RDD[InternalRow] = { val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit)) val shuffled = new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( - locallyLimited, child.output, SinglePartition, serializer), - metrics) + locallyLimited, + child.output, + SinglePartition, + serializer, + writeMetrics), + readMetrics) shuffled.mapPartitionsInternal(_.take(limit)) } } @@ -154,7 +162,11 @@ case class TakeOrderedAndProjectExec( private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) - override lazy val metrics = SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) + private lazy val writeMetrics = + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) + private lazy val readMetrics = + SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) + override lazy val metrics = readMetrics ++ writeMetrics protected override def doExecute(): RDD[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) @@ -165,8 +177,12 @@ case class TakeOrderedAndProjectExec( } val shuffled = new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( - localTopK, child.output, SinglePartition, serializer), - metrics) + localTopK, + child.output, + SinglePartition, + serializer, + writeMetrics), + readMetrics) shuffled.mapPartitions { iter => val topK = org.apache.spark.util.collection.Utils.takeOrdered(iter.map(_.copy()), limit)(ord) if (projectList != child.output) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index cbf707f4a9cf..19809b07508d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.metric import java.text.NumberFormat import java.util.Locale +import scala.concurrent.duration._ + import org.apache.spark.SparkContext import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates @@ -78,6 +80,7 @@ object SQLMetrics { private val SUM_METRIC = "sum" private val SIZE_METRIC = "size" private val TIMING_METRIC = "timing" + private val NS_TIMING_METRIC = "nsTiming" private val AVERAGE_METRIC = "average" private val baseForAvgMetric: Int = 10 @@ -121,6 +124,13 @@ object SQLMetrics { acc } + def createNanoTimingMetric(sc: SparkContext, name: String): SQLMetric = { + // Same with createTimingMetric, just normalize the unit of time to millisecond. + val acc = new SQLMetric(NS_TIMING_METRIC, -1) + acc.register(sc, name = Some(s"$name total (min, med, max)"), countFailedValues = false) + acc + } + /** * Create a metric to report the average information (including min, med, max) like * avg hash probe. As average metrics are double values, this kind of metrics should be @@ -163,6 +173,8 @@ object SQLMetrics { Utils.bytesToString } else if (metricsType == TIMING_METRIC) { Utils.msDurationToString + } else if (metricsType == NS_TIMING_METRIC) { + duration => Utils.msDurationToString(duration.nanos.toMillis) } else { throw new IllegalStateException("unexpected metrics type: " + metricsType) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala index 780f0d762229..ff7941e3b3e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.metric import org.apache.spark.SparkContext import org.apache.spark.executor.TempShuffleReadMetrics +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter /** * A shuffle metrics reporter for SQL exchange operators. @@ -95,3 +96,57 @@ private[spark] object SQLShuffleMetricsReporter { FETCH_WAIT_TIME -> SQLMetrics.createTimingMetric(sc, "fetch wait time"), RECORDS_READ -> SQLMetrics.createMetric(sc, "records read")) } + +/** + * A shuffle write metrics reporter for SQL exchange operators. + * @param metricsReporter Other reporter need to be updated in this SQLShuffleWriteMetricsReporter. + * @param metrics Shuffle write metrics in current SparkPlan. + */ +private[spark] class SQLShuffleWriteMetricsReporter( + metricsReporter: ShuffleWriteMetricsReporter, + metrics: Map[String, SQLMetric]) extends ShuffleWriteMetricsReporter { + private[this] val _bytesWritten = + metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_BYTES_WRITTEN) + private[this] val _recordsWritten = + metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN) + private[this] val _writeTime = + metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_WRITE_TIME) + + override private[spark] def incBytesWritten(v: Long): Unit = { + metricsReporter.incBytesWritten(v) + _bytesWritten.add(v) + } + override private[spark] def decRecordsWritten(v: Long): Unit = { + metricsReporter.decBytesWritten(v) + _recordsWritten.set(_recordsWritten.value - v) + } + override private[spark] def incRecordsWritten(v: Long): Unit = { + metricsReporter.incRecordsWritten(v) + _recordsWritten.add(v) + } + override private[spark] def incWriteTime(v: Long): Unit = { + metricsReporter.incWriteTime(v) + _writeTime.add(v) + } + override private[spark] def decBytesWritten(v: Long): Unit = { + metricsReporter.decBytesWritten(v) + _bytesWritten.set(_bytesWritten.value - v) + } +} + +private[spark] object SQLShuffleWriteMetricsReporter { + val SHUFFLE_BYTES_WRITTEN = "shuffleBytesWritten" + val SHUFFLE_RECORDS_WRITTEN = "shuffleRecordsWritten" + val SHUFFLE_WRITE_TIME = "shuffleWriteTime" + + /** + * Create all shuffle write relative metrics and return the Map. + */ + def createShuffleWriteMetrics(sc: SparkContext): Map[String, SQLMetric] = Map( + SHUFFLE_BYTES_WRITTEN -> + SQLMetrics.createSizeMetric(sc, "shuffle bytes written"), + SHUFFLE_RECORDS_WRITTEN -> + SQLMetrics.createMetric(sc, "shuffle records written"), + SHUFFLE_WRITE_TIME -> + SQLMetrics.createNanoTimingMetric(sc, "shuffle write time")) +} 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 0f1d08b6af5d..2251607e76af 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 @@ -97,7 +97,8 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val shuffleExpected1 = Map( "records read" -> 2L, "local blocks fetched" -> 2L, - "remote blocks fetched" -> 0L) + "remote blocks fetched" -> 0L, + "shuffle records written" -> 2L) testSparkPlanMetrics(df, 1, Map( 2L -> (("HashAggregate", expected1(0))), 1L -> (("Exchange", shuffleExpected1)), @@ -114,7 +115,8 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val shuffleExpected2 = Map( "records read" -> 4L, "local blocks fetched" -> 4L, - "remote blocks fetched" -> 0L) + "remote blocks fetched" -> 0L, + "shuffle records written" -> 4L) testSparkPlanMetrics(df2, 1, Map( 2L -> (("HashAggregate", expected2(0))), 1L -> (("Exchange", shuffleExpected2)), @@ -170,6 +172,11 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df = testData2.groupBy().agg(collect_set('a)) // 2 partitions testSparkPlanMetrics(df, 1, Map( 2L -> (("ObjectHashAggregate", Map("number of output rows" -> 2L))), + 1L -> (("Exchange", Map( + "shuffle records written" -> 2L, + "records read" -> 2L, + "local blocks fetched" -> 2L, + "remote blocks fetched" -> 0L))), 0L -> (("ObjectHashAggregate", Map("number of output rows" -> 1L)))) ) @@ -177,6 +184,11 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df2 = testData2.groupBy('a).agg(collect_set('a)) testSparkPlanMetrics(df2, 1, Map( 2L -> (("ObjectHashAggregate", Map("number of output rows" -> 4L))), + 1L -> (("Exchange", Map( + "shuffle records written" -> 4L, + "records read" -> 4L, + "local blocks fetched" -> 4L, + "remote blocks fetched" -> 0L))), 0L -> (("ObjectHashAggregate", Map("number of output rows" -> 3L)))) ) } @@ -205,7 +217,8 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared 2L -> (("Exchange", Map( "records read" -> 4L, "local blocks fetched" -> 2L, - "remote blocks fetched" -> 0L)))) + "remote blocks fetched" -> 0L, + "shuffle records written" -> 2L)))) ) } } @@ -299,12 +312,25 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") val df2 = (1 to 10).map(i => (i, i.toString)).toSeq.toDF("key", "value") // Assume the execution plan is - // ... -> ShuffledHashJoin(nodeId = 1) -> Project(nodeId = 0) + // Project(nodeId = 0) + // +- ShuffledHashJoin(nodeId = 1) + // :- Exchange(nodeId = 2) + // : +- Project(nodeId = 3) + // : +- LocalTableScan(nodeId = 4) + // +- Exchange(nodeId = 5) + // +- Project(nodeId = 6) + // +- LocalTableScan(nodeId = 7) val df = df1.join(df2, "key") testSparkPlanMetrics(df, 1, Map( 1L -> (("ShuffledHashJoin", Map( "number of output rows" -> 2L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")))) + "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"))), + 2L -> (("Exchange", Map( + "shuffle records written" -> 2L, + "records read" -> 2L))), + 5L -> (("Exchange", Map( + "shuffle records written" -> 10L, + "records read" -> 10L)))) ) } } From ec506bd30c2ca324c12c9ec811764081c2eb8c42 Mon Sep 17 00:00:00 2001 From: Shahid Date: Sun, 9 Dec 2018 11:44:16 -0600 Subject: [PATCH 0096/1072] [SPARK-26283][CORE] Enable reading from open frames of zstd, when reading zstd compressed eventLog ## What changes were proposed in this pull request? Root cause: Prior to Spark2.4, When we enable zst for eventLog compression, for inprogress application, It always throws exception in the Application UI, when we open from the history server. But after 2.4 it will display the UI information based on the completed frames in the zstd compressed eventLog. But doesn't read incomplete frames for inprogress application. In this PR, we have added 'setContinous(true)' for reading input stream from eventLog, so that it can read from open frames also. (By default 'isContinous=false' for zstd inputStream and when we try to read an open frame, it throws truncated error) ## How was this patch tested? Test steps: 1) Add the configurations in the spark-defaults.conf (i) spark.eventLog.compress true (ii) spark.io.compression.codec zstd 2) Restart history server 3) bin/spark-shell 4) sc.parallelize(1 to 1000, 1000).count 5) Open app UI from the history server UI **Before fix** ![screenshot from 2018-12-06 00-01-38](https://user-images.githubusercontent.com/23054875/49537340-bfe28b00-f8ee-11e8-9fca-6d42fdc89e1a.png) **After fix:** ![screenshot from 2018-12-06 00-34-39](https://user-images.githubusercontent.com/23054875/49537353-ca9d2000-f8ee-11e8-803d-645897b9153b.png) Closes #23241 from shahidki31/zstdEventLog. Authored-by: Shahid Signed-off-by: Sean Owen --- .../scala/org/apache/spark/io/CompressionCodec.scala | 12 ++++++++++++ .../spark/scheduler/EventLoggingListener.scala | 2 +- .../apache/spark/scheduler/ReplayListenerBus.scala | 2 -- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 0664c5ac752c..c4f4b18769d2 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -43,6 +43,10 @@ trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream def compressedInputStream(s: InputStream): InputStream + + private[spark] def compressedContinuousInputStream(s: InputStream): InputStream = { + compressedInputStream(s) + } } private[spark] object CompressionCodec { @@ -197,4 +201,12 @@ class ZStdCompressionCodec(conf: SparkConf) extends CompressionCodec { // avoid overhead excessive of JNI call while trying to uncompress small amount of data. new BufferedInputStream(new ZstdInputStream(s), bufferSize) } + + override def compressedContinuousInputStream(s: InputStream): InputStream = { + // SPARK-26283: Enable reading from open frames of zstd (for eg: zstd compressed eventLog + // Reading). By default `isContinuous` is false, and when we try to read from open frames, + // `compressedInputStream` method above throws truncated error exception. This method set + // `isContinuous` true to allow reading from open frames. + new BufferedInputStream(new ZstdInputStream(s).setContinuous(true), bufferSize) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 5f697fe99258..069a91f1a8fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -402,7 +402,7 @@ private[spark] object EventLoggingListener extends Logging { val codec = codecName(log).map { c => codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) } - codec.map(_.compressedInputStream(in)).getOrElse(in) + codec.map(_.compressedContinuousInputStream(in)).getOrElse(in) } catch { case e: Throwable => in.close() diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 4c6b0c1227b1..226c23733c87 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -118,8 +118,6 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { case e: HaltReplayException => // Just stop replay. case _: EOFException if maybeTruncated => - case _: IOException if maybeTruncated => - logWarning(s"Failed to read Spark event log: $sourceName") case ioe: IOException => throw ioe case e: Exception => From 403c8d5a60b2712561044e320d6f9233ed3172bf Mon Sep 17 00:00:00 2001 From: 10087686 Date: Sun, 9 Dec 2018 22:44:41 -0800 Subject: [PATCH 0097/1072] [SPARK-26287][CORE] Don't need to create an empty spill file when memory has no records ## What changes were proposed in this pull request? If there are no records in memory, then we don't need to create an empty temp spill file. ## How was this patch tested? Existing tests (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23225 from wangjiaochun/ShufflSorter. Authored-by: 10087686 Signed-off-by: Dongjoon Hyun --- .../spark/shuffle/sort/ShuffleExternalSorter.java | 13 +++++++++---- .../shuffle/sort/UnsafeShuffleWriterSuite.java | 1 + 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 6ee9d5f0eec3..dc43215373e1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -145,6 +145,15 @@ final class ShuffleExternalSorter extends MemoryConsumer { */ private void writeSortedFile(boolean isLastFile) { + // This call performs the actual sort. + final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = + inMemSorter.getSortedIterator(); + + // If there are no sorted records, so we don't need to create an empty spill file. + if (!sortedRecords.hasNext()) { + return; + } + final ShuffleWriteMetricsReporter writeMetricsToUse; if (isLastFile) { @@ -157,10 +166,6 @@ private void writeSortedFile(boolean isLastFile) { writeMetricsToUse = new ShuffleWriteMetrics(); } - // This call performs the actual sort. - final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = - inMemSorter.getSortedIterator(); - // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to // be an API to directly transfer bytes from managed memory to the disk writer, we buffer // data through a byte array. This array does not need to be large enough to hold a single diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 30ad3f557554..aa5082f1ac7f 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -235,6 +235,7 @@ public void writeEmptyIterator() throws Exception { final Option mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); + assertEquals(0, spillFilesCreated.size()); assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile); assertEquals(0, taskMetrics.shuffleWriteMetrics().recordsWritten()); assertEquals(0, taskMetrics.shuffleWriteMetrics().bytesWritten()); From 3bc83de3cce86a06c275c86b547a99afd781761f Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 10 Dec 2018 14:57:20 +0800 Subject: [PATCH 0098/1072] [SPARK-26307][SQL] Fix CTAS when INSERT a partitioned table using Hive serde ## What changes were proposed in this pull request? This is a Spark 2.3 regression introduced in https://github.com/apache/spark/pull/20521. We should add the partition info for InsertIntoHiveTable in CreateHiveTableAsSelectCommand. Otherwise, we will hit the following error by running the newly added test case: ``` [info] - CTAS: INSERT a partitioned table using Hive serde *** FAILED *** (829 milliseconds) [info] org.apache.spark.SparkException: Requested partitioning does not match the tab1 table: [info] Requested partitions: [info] Table partitions: part [info] at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.processInsert(InsertIntoHiveTable.scala:179) [info] at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.run(InsertIntoHiveTable.scala:107) ``` ## How was this patch tested? Added a test case. Closes #23255 from gatorsmile/fixCTAS. Authored-by: gatorsmile Signed-off-by: Wenchen Fan --- .../execution/CreateHiveTableAsSelectCommand.scala | 4 +++- .../scala/org/apache/spark/sql/hive/InsertSuite.scala | 11 +++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 630bea5161f1..fd1e931ee0c7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -57,9 +57,11 @@ case class CreateHiveTableAsSelectCommand( return Seq.empty } + // For CTAS, there is no static partition values to insert. + val partition = tableDesc.partitionColumnNames.map(_ -> None).toMap InsertIntoHiveTable( tableDesc, - Map.empty, + partition, query, overwrite = false, ifPartitionNotExists = false, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 5879748d05b2..510de3a7eab5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -752,6 +752,17 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } + test("SPARK-26307: CTAS - INSERT a partitioned table using Hive serde") { + withTable("tab1") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + val df = Seq(("a", 100)).toDF("part", "id") + df.write.format("hive").partitionBy("part").mode("overwrite").saveAsTable("tab1") + df.write.format("hive").partitionBy("part").mode("append").saveAsTable("tab1") + } + } + } + + Seq("LOCAL", "").foreach { local => Seq(true, false).foreach { caseSensitivity => Seq("orc", "parquet").foreach { format => From c8ac6ae84c8a3cc7ed787155a84cbeb56c78a048 Mon Sep 17 00:00:00 2001 From: Darcy Shen Date: Mon, 10 Dec 2018 22:26:28 +0800 Subject: [PATCH 0099/1072] [SPARK-26319][SQL][TEST] Add appendReadColumns Unit Test for HiveShimSuite ## What changes were proposed in this pull request? Add appendReadColumns Unit Test for HiveShimSuite. ## How was this patch tested? ``` $ build/sbt > project hive > testOnly *HiveShimSuite ``` Closes #23268 from sadhen/refactor/hiveshim. Authored-by: Darcy Shen Signed-off-by: Hyukjin Kwon --- .../apache/spark/sql/hive/HiveShimSuite.scala | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala new file mode 100644 index 000000000000..a716f739b5c2 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive + +import scala.collection.JavaConverters._ +import scala.language.implicitConversions + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils + +import org.apache.spark.SparkFunSuite + +class HiveShimSuite extends SparkFunSuite { + + test("appendReadColumns") { + val conf = new Configuration + val ids = Seq(1, 2, 3).map(Int.box) + val names = Seq("a", "b", "c") + val moreIds = Seq(4, 5).map(Int.box) + val moreNames = Seq("d", "e") + + // test when READ_COLUMN_NAMES_CONF_STR is empty + HiveShim.appendReadColumns(conf, ids, names) + assert(names.asJava === ColumnProjectionUtils.getReadColumnNames(conf)) + + // test when READ_COLUMN_NAMES_CONF_STR is non-empty + HiveShim.appendReadColumns(conf, moreIds, moreNames) + assert((names ++ moreNames).asJava === ColumnProjectionUtils.getReadColumnNames(conf)) + } +} From 42e8c381b15dd48c2f00c088c897ebdd25405aef Mon Sep 17 00:00:00 2001 From: 10087686 Date: Mon, 10 Dec 2018 22:28:26 +0800 Subject: [PATCH 0100/1072] [SPARK-26286][TEST] Add MAXIMUM_PAGE_SIZE_BYTES exception bound unit test ## What changes were proposed in this pull request? Add MAXIMUM_PAGE_SIZE_BYTES Exception test ## How was this patch tested? Existing tests (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23226 from wangjiaochun/BytesToBytesMapSuite. Authored-by: 10087686 Signed-off-by: Hyukjin Kwon --- .../unsafe/map/AbstractBytesToBytesMapSuite.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index aa29232e73e1..a11cd535b547 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -622,6 +622,17 @@ public void initialCapacityBoundsChecking() { } catch (IllegalArgumentException e) { // expected exception } + + try { + new BytesToBytesMap( + taskMemoryManager, + 1, + TaskMemoryManager.MAXIMUM_PAGE_SIZE_BYTES + 1); + Assert.fail("Expected IllegalArgumentException to be thrown"); + } catch (IllegalArgumentException e) { + // expected exception + } + } @Test From 9794923272c26ee5ba760a57718a368c33d09f04 Mon Sep 17 00:00:00 2001 From: liuxian Date: Mon, 10 Dec 2018 22:37:17 +0800 Subject: [PATCH 0101/1072] [MINOR][DOC] Update the condition description of serialized shuffle ## What changes were proposed in this pull request? `1. The shuffle dependency specifies no aggregation or output ordering.` If the shuffle dependency specifies aggregation, but it only aggregates at the reduce-side, serialized shuffle can still be used. `3. The shuffle produces fewer than 16777216 output partitions.` If the number of output partitions is 16777216 , we can use serialized shuffle. We can see this mothod: `canUseSerializedShuffle` ## How was this patch tested? N/A Closes #23228 from 10110346/SerializedShuffle_doc. Authored-by: liuxian Signed-off-by: Wenchen Fan --- .../org/apache/spark/shuffle/sort/SortShuffleManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 b51a843a31c3..b59fa8e8a3cc 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 @@ -33,10 +33,10 @@ import org.apache.spark.shuffle._ * Sort-based shuffle has two different write paths for producing its map output files: * * - Serialized sorting: used when all three of the following conditions hold: - * 1. The shuffle dependency specifies no aggregation or output ordering. + * 1. The shuffle dependency specifies no map-side combine. * 2. The shuffle serializer supports relocation of serialized values (this is currently * supported by KryoSerializer and Spark SQL's custom serializers). - * 3. The shuffle produces fewer than 16777216 output partitions. + * 3. The shuffle produces fewer than or equal to 16777216 output partitions. * - Deserialized sorting: used to handle all other cases. * * ----------------------- From 0bf6c77141e40cc636351c5e77194bb75144bb12 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 10 Dec 2018 10:27:04 -0600 Subject: [PATCH 0102/1072] This tests pushing to gitbox From b1a724b468d5c1c4aee2a22ffc6d8edac537c3d6 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 10 Dec 2018 11:11:06 -0600 Subject: [PATCH 0103/1072] This tests pushing directly to github From 90c77ea3132d0b7a12c316bd42fb8d0f59bee253 Mon Sep 17 00:00:00 2001 From: Reza Safi Date: Mon, 10 Dec 2018 11:14:11 -0600 Subject: [PATCH 0104/1072] [SPARK-24958][CORE] Add memory from procfs to executor metrics. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This adds the entire memory used by spark’s executor (as measured by procfs) to the executor metrics. The memory usage is collected from the entire process tree under the executor. The metrics are subdivided into memory used by java, by python, and by other processes, to aid users in diagnosing the source of high memory usage. The additional metrics are sent to the driver in heartbeats, using the mechanism introduced by SPARK-23429. This also slightly extends that approach to allow one ExecutorMetricType to collect multiple metrics. Added unit tests and also tested on a live cluster. Closes #22612 from rezasafi/ptreememory2. Authored-by: Reza Safi Signed-off-by: Imran Rashid --- .../scala/org/apache/spark/Heartbeater.scala | 11 +- .../spark/executor/ExecutorMetrics.scala | 23 +- .../spark/executor/ProcfsMetricsGetter.scala | 228 ++++++++++++++++++ .../spark/internal/config/package.scala | 5 + .../spark/metrics/ExecutorMetricType.scala | 74 +++++- .../org/apache/spark/status/api/v1/api.scala | 6 +- .../org/apache/spark/util/JsonProtocol.scala | 16 +- .../application_list_json_expectation.json | 15 ++ .../completed_app_list_json_expectation.json | 15 ++ ...ith_executor_metrics_json_expectation.json | 40 ++- ...process_tree_metrics_json_expectation.json | 98 ++++++++ .../limit_app_list_json_expectation.json | 30 +-- .../minDate_app_list_json_expectation.json | 15 ++ .../minEndDate_app_list_json_expectation.json | 15 ++ .../test/resources/ProcfsMetrics/22763/stat | 1 + .../test/resources/ProcfsMetrics/26109/stat | 1 + .../application_1538416563558_0014 | 190 +++++++++++++++ .../deploy/history/HistoryServerSuite.scala | 3 + .../executor/ProcfsMetricsGetterSuite.scala | 41 ++++ .../scheduler/EventLoggingListenerSuite.scala | 85 ++++--- .../spark/status/AppStatusListenerSuite.scala | 74 ++++-- .../apache/spark/util/JsonProtocolSuite.scala | 46 ++-- dev/.rat-excludes | 2 + 23 files changed, 901 insertions(+), 133 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala create mode 100644 core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json create mode 100644 core/src/test/resources/ProcfsMetrics/22763/stat create mode 100644 core/src/test/resources/ProcfsMetrics/26109/stat create mode 100644 core/src/test/resources/spark-events/application_1538416563558_0014 create mode 100644 core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 84091eef0430..1012755e068d 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -61,10 +61,17 @@ private[spark] class Heartbeater( /** * Get the current executor level metrics. These are returned as an array, with the index - * determined by ExecutorMetricType.values + * determined by ExecutorMetricType.metricToOffset */ def getCurrentMetrics(): ExecutorMetrics = { - val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray + + val metrics = new Array[Long](ExecutorMetricType.numMetrics) + var offset = 0 + ExecutorMetricType.metricGetters.foreach { metric => + val newMetrics = metric.getMetricValues(memoryManager) + Array.copy(newMetrics, 0, metrics, offset, newMetrics.size) + offset += newMetrics.length + } new ExecutorMetrics(metrics) } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 1befd27de1cb..f19ac813fde3 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -27,17 +27,15 @@ import org.apache.spark.metrics.ExecutorMetricType */ @DeveloperApi class ExecutorMetrics private[spark] extends Serializable { - - // Metrics are indexed by ExecutorMetricType.values - private val metrics = new Array[Long](ExecutorMetricType.values.length) - + // Metrics are indexed by ExecutorMetricType.metricToOffset + private val metrics = new Array[Long](ExecutorMetricType.numMetrics) // the first element is initialized to -1, indicating that the values for the array // haven't been set yet. metrics(0) = -1 - /** Returns the value for the specified metricType. */ - def getMetricValue(metricType: ExecutorMetricType): Long = { - metrics(ExecutorMetricType.metricIdxMap(metricType)) + /** Returns the value for the specified metric. */ + def getMetricValue(metricName: String): Long = { + metrics(ExecutorMetricType.metricToOffset(metricName)) } /** Returns true if the values for the metrics have been set, false otherwise. */ @@ -49,14 +47,14 @@ class ExecutorMetrics private[spark] extends Serializable { } /** - * Constructor: create the ExecutorMetrics with the values specified. + * Constructor: create the ExecutorMetrics with using a given map. * * @param executorMetrics map of executor metric name to value */ private[spark] def this(executorMetrics: Map[String, Long]) { this() - (0 until ExecutorMetricType.values.length).foreach { idx => - metrics(idx) = executorMetrics.getOrElse(ExecutorMetricType.values(idx).name, 0L) + ExecutorMetricType.metricToOffset.foreach { case(name, idx) => + metrics(idx) = executorMetrics.getOrElse(name, 0L) } } @@ -69,9 +67,8 @@ class ExecutorMetrics private[spark] extends Serializable { */ private[spark] def compareAndUpdatePeakValues(executorMetrics: ExecutorMetrics): Boolean = { var updated = false - - (0 until ExecutorMetricType.values.length).foreach { idx => - if (executorMetrics.metrics(idx) > metrics(idx)) { + (0 until ExecutorMetricType.numMetrics).foreach { idx => + if (executorMetrics.metrics(idx) > metrics(idx)) { updated = true metrics(idx) = executorMetrics.metrics(idx) } diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala new file mode 100644 index 000000000000..af67f41e94af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import java.io._ +import java.nio.charset.Charset +import java.nio.file.{Files, Paths} +import java.util.Locale + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.Try + +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.util.Utils + + +private[spark] case class ProcfsMetrics( + jvmVmemTotal: Long, + jvmRSSTotal: Long, + pythonVmemTotal: Long, + pythonRSSTotal: Long, + otherVmemTotal: Long, + otherRSSTotal: Long) + +// Some of the ideas here are taken from the ProcfsBasedProcessTree class in hadoop +// project. +private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends Logging { + private val procfsStatFile = "stat" + private val testing = sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + private val pageSize = computePageSize() + private var isAvailable: Boolean = isProcfsAvailable + private val pid = computePid() + + private lazy val isProcfsAvailable: Boolean = { + if (testing) { + true + } + else { + val procDirExists = Try(Files.exists(Paths.get(procfsDir))).recover { + case ioe: IOException => + logWarning("Exception checking for procfs dir", ioe) + false + } + val shouldLogStageExecutorMetrics = + SparkEnv.get.conf.get(config.EVENT_LOG_STAGE_EXECUTOR_METRICS) + val shouldLogStageExecutorProcessTreeMetrics = + SparkEnv.get.conf.get(config.EVENT_LOG_PROCESS_TREE_METRICS) + procDirExists.get && shouldLogStageExecutorProcessTreeMetrics && shouldLogStageExecutorMetrics + } + } + + private def computePid(): Int = { + if (!isAvailable || testing) { + return -1; + } + try { + // This can be simplified in java9: + // https://docs.oracle.com/javase/9/docs/api/java/lang/ProcessHandle.html + val cmd = Array("bash", "-c", "echo $PPID") + val out = Utils.executeAndGetOutput(cmd) + Integer.parseInt(out.split("\n")(0)) + } + catch { + case e: SparkException => + logWarning("Exception when trying to compute process tree." + + " As a result reporting of ProcessTree metrics is stopped", e) + isAvailable = false + -1 + } + } + + private def computePageSize(): Long = { + if (testing) { + return 4096; + } + try { + val cmd = Array("getconf", "PAGESIZE") + val out = Utils.executeAndGetOutput(cmd) + Integer.parseInt(out.split("\n")(0)) + } catch { + case e: Exception => + logWarning("Exception when trying to compute pagesize, as a" + + " result reporting of ProcessTree metrics is stopped") + isAvailable = false + 0 + } + } + + private def computeProcessTree(): Set[Int] = { + if (!isAvailable || testing) { + return Set() + } + var ptree: Set[Int] = Set() + ptree += pid + val queue = mutable.Queue.empty[Int] + queue += pid + while ( !queue.isEmpty ) { + val p = queue.dequeue() + val c = getChildPids(p) + if (!c.isEmpty) { + queue ++= c + ptree ++= c.toSet + } + } + ptree + } + + private def getChildPids(pid: Int): ArrayBuffer[Int] = { + try { + val builder = new ProcessBuilder("pgrep", "-P", pid.toString) + val process = builder.start() + val childPidsInInt = mutable.ArrayBuffer.empty[Int] + def appendChildPid(s: String): Unit = { + if (s != "") { + logTrace("Found a child pid:" + s) + childPidsInInt += Integer.parseInt(s) + } + } + val stdoutThread = Utils.processStreamByLine("read stdout for pgrep", + process.getInputStream, appendChildPid) + val errorStringBuilder = new StringBuilder() + val stdErrThread = Utils.processStreamByLine( + "stderr for pgrep", + process.getErrorStream, + line => errorStringBuilder.append(line)) + val exitCode = process.waitFor() + stdoutThread.join() + stdErrThread.join() + val errorString = errorStringBuilder.toString() + // pgrep will have exit code of 1 if there are more than one child process + // and it will have a exit code of 2 if there is no child process + if (exitCode != 0 && exitCode > 2) { + val cmd = builder.command().toArray.mkString(" ") + logWarning(s"Process $cmd exited with code $exitCode and stderr: $errorString") + throw new SparkException(s"Process $cmd exited with code $exitCode") + } + childPidsInInt + } catch { + case e: Exception => + logWarning("Exception when trying to compute process tree." + + " As a result reporting of ProcessTree metrics is stopped.", e) + isAvailable = false + mutable.ArrayBuffer.empty[Int] + } + } + + def addProcfsMetricsFromOneProcess( + allMetrics: ProcfsMetrics, + pid: Int): ProcfsMetrics = { + + // The computation of RSS and Vmem are based on proc(5): + // http://man7.org/linux/man-pages/man5/proc.5.html + try { + val pidDir = new File(procfsDir, pid.toString) + def openReader(): BufferedReader = { + val f = new File(new File(procfsDir, pid.toString), procfsStatFile) + new BufferedReader(new InputStreamReader(new FileInputStream(f), Charset.forName("UTF-8"))) + } + Utils.tryWithResource(openReader) { in => + val procInfo = in.readLine + val procInfoSplit = procInfo.split(" ") + val vmem = procInfoSplit(22).toLong + val rssMem = procInfoSplit(23).toLong * pageSize + if (procInfoSplit(1).toLowerCase(Locale.US).contains("java")) { + allMetrics.copy( + jvmVmemTotal = allMetrics.jvmVmemTotal + vmem, + jvmRSSTotal = allMetrics.jvmRSSTotal + (rssMem) + ) + } + else if (procInfoSplit(1).toLowerCase(Locale.US).contains("python")) { + allMetrics.copy( + pythonVmemTotal = allMetrics.pythonVmemTotal + vmem, + pythonRSSTotal = allMetrics.pythonRSSTotal + (rssMem) + ) + } + else { + allMetrics.copy( + otherVmemTotal = allMetrics.otherVmemTotal + vmem, + otherRSSTotal = allMetrics.otherRSSTotal + (rssMem) + ) + } + } + } catch { + case f: IOException => + logWarning("There was a problem with reading" + + " the stat file of the process. ", f) + ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + } + + private[spark] def computeAllMetrics(): ProcfsMetrics = { + if (!isAvailable) { + return ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + val pids = computeProcessTree + var allMetrics = ProcfsMetrics(0, 0, 0, 0, 0, 0) + for (p <- pids) { + allMetrics = addProcfsMetricsFromOneProcess(allMetrics, p) + // if we had an error getting any of the metrics, we don't want to report partial metrics, as + // that would be misleading. + if (!isAvailable) { + return ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + } + allMetrics + } +} + +private[spark] object ProcfsMetricsGetter { + final val pTreeInfo = new ProcfsMetricsGetter +} 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 646b3881a79b..85bb557abef5 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 @@ -93,6 +93,11 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val EVENT_LOG_PROCESS_TREE_METRICS = + ConfigBuilder("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled") + .booleanConf + .createWithDefault(false) + private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index cd10dad25e87..704b36d3118b 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -19,25 +19,43 @@ package org.apache.spark.metrics import java.lang.management.{BufferPoolMXBean, ManagementFactory} import javax.management.ObjectName +import scala.collection.mutable + +import org.apache.spark.executor.ProcfsMetricsGetter import org.apache.spark.memory.MemoryManager /** * Executor metric types for executor-level metrics stored in ExecutorMetrics. */ sealed trait ExecutorMetricType { + private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] + private[spark] def names: Seq[String] +} + +sealed trait SingleValueExecutorMetricType extends ExecutorMetricType { + override private[spark] def names = { + Seq(getClass().getName(). + stripSuffix("$").split("""\.""").last) + } + + override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = { + val metrics = new Array[Long](1) + metrics(0) = getMetricValue(memoryManager) + metrics + } + private[spark] def getMetricValue(memoryManager: MemoryManager): Long - private[spark] val name = getClass().getName().stripSuffix("$").split("""\.""").last } private[spark] abstract class MemoryManagerExecutorMetricType( - f: MemoryManager => Long) extends ExecutorMetricType { + f: MemoryManager => Long) extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { f(memoryManager) } } private[spark] abstract class MBeanExecutorMetricType(mBeanName: String) - extends ExecutorMetricType { + extends SingleValueExecutorMetricType { private val bean = ManagementFactory.newPlatformMXBeanProxy( ManagementFactory.getPlatformMBeanServer, new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean]) @@ -47,18 +65,40 @@ private[spark] abstract class MBeanExecutorMetricType(mBeanName: String) } } -case object JVMHeapMemory extends ExecutorMetricType { +case object JVMHeapMemory extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed() } } -case object JVMOffHeapMemory extends ExecutorMetricType { +case object JVMOffHeapMemory extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed() } } +case object ProcessTreeMetrics extends ExecutorMetricType { + override val names = Seq( + "ProcessTreeJVMVMemory", + "ProcessTreeJVMRSSMemory", + "ProcessTreePythonVMemory", + "ProcessTreePythonRSSMemory", + "ProcessTreeOtherVMemory", + "ProcessTreeOtherRSSMemory") + + override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = { + val allMetrics = ProcfsMetricsGetter.pTreeInfo.computeAllMetrics() + val processTreeMetrics = new Array[Long](names.length) + processTreeMetrics(0) = allMetrics.jvmVmemTotal + processTreeMetrics(1) = allMetrics.jvmRSSTotal + processTreeMetrics(2) = allMetrics.pythonVmemTotal + processTreeMetrics(3) = allMetrics.pythonRSSTotal + processTreeMetrics(4) = allMetrics.otherVmemTotal + processTreeMetrics(5) = allMetrics.otherRSSTotal + processTreeMetrics + } +} + case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType( _.onHeapExecutionMemoryUsed) @@ -84,8 +124,9 @@ case object MappedPoolMemory extends MBeanExecutorMetricType( "java.nio:type=BufferPool,name=mapped") private[spark] object ExecutorMetricType { - // List of all executor metric types - val values = IndexedSeq( + + // List of all executor metric getters + val metricGetters = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, OnHeapExecutionMemory, @@ -95,10 +136,21 @@ private[spark] object ExecutorMetricType { OnHeapUnifiedMemory, OffHeapUnifiedMemory, DirectPoolMemory, - MappedPoolMemory + MappedPoolMemory, + ProcessTreeMetrics ) - // Map of executor metric type to its index in values. - val metricIdxMap = - Map[ExecutorMetricType, Int](ExecutorMetricType.values.zipWithIndex: _*) + + val (metricToOffset, numMetrics) = { + var numberOfMetrics = 0 + val definedMetricsAndOffset = mutable.LinkedHashMap.empty[String, Int] + metricGetters.foreach { m => + var metricInSet = 0 + (0 until m.names.length).foreach { idx => + definedMetricsAndOffset += (m.names(idx) -> (idx + numberOfMetrics)) + } + numberOfMetrics += m.names.length + } + (definedMetricsAndOffset, numberOfMetrics) + } } 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 aa21da2b66ab..c7d3cd37db6f 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 @@ -133,9 +133,9 @@ private[spark] class ExecutorMetricsJsonSerializer jsonGenerator: JsonGenerator, serializerProvider: SerializerProvider): Unit = { metrics.foreach { m: ExecutorMetrics => - val metricsMap = ExecutorMetricType.values.map { metricType => - metricType.name -> m.getMetricValue(metricType) - }.toMap + val metricsMap = ExecutorMetricType.metricToOffset.map { case (metric, _) => + metric -> m.getMetricValue(metric) + } jsonGenerator.writeObject(metricsMap) } } 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 0cd8612b8fd1..348291fe5e7a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -394,10 +394,10 @@ private[spark] object JsonProtocol { /** Convert executor metrics to JSON. */ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { - val metrics = ExecutorMetricType.values.map{ metricType => - JField(metricType.name, executorMetrics.getMetricValue(metricType)) - } - JObject(metrics: _*) + val metrics = ExecutorMetricType.metricToOffset.map { case (m, _) => + JField(m, executorMetrics.getMetricValue(m)) + } + JObject(metrics.toSeq: _*) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -611,10 +611,10 @@ private[spark] object JsonProtocol { /** Extract the executor metrics from JSON. */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val metrics = - ExecutorMetricType.values.map { metric => - metric.name -> jsonOption(json \ metric.name).map(_.extract[Long]).getOrElse(0L) - }.toMap - new ExecutorMetrics(metrics) + ExecutorMetricType.metricToOffset.map { case (metric, _) => + metric -> jsonOption(json \ metric).map(_.extract[Long]).getOrElse(0L) + } + new ExecutorMetrics(metrics.toMap) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { 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 eea6f595efd2..0f0ccf9858a3 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" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "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 7bc7f31be097..e136a35a1e3a 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" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json index 9bf2086cc8e7..75674778dd1f 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json @@ -37,7 +37,13 @@ "DirectPoolMemory" : 397602, "MappedPoolMemory" : 0, "JVMHeapMemory" : 629553808, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } }, { "id" : "7", @@ -177,7 +183,13 @@ "DirectPoolMemory" : 126261, "MappedPoolMemory" : 0, "JVMHeapMemory" : 518613056, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } }, { "id" : "3", @@ -221,7 +233,13 @@ "DirectPoolMemory" : 87796, "MappedPoolMemory" : 0, "JVMHeapMemory" : 726805712, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } }, { "id" : "2", @@ -265,7 +283,13 @@ "DirectPoolMemory" : 87796, "MappedPoolMemory" : 0, "JVMHeapMemory" : 595946552, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } }, { "id" : "1", @@ -309,6 +333,12 @@ "DirectPoolMemory" : 98230, "MappedPoolMemory" : 0, "JVMHeapMemory" : 755008624, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json new file mode 100644 index 000000000000..69efefe736dd --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json @@ -0,0 +1,98 @@ +[ { + "id" : "driver", + "hostPort" : "rezamemory-1.gce.something.com:43959", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2018-10-02T00:42:47.690GMT", + "executorLogs" : { }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "OnHeapStorageMemory" : 554933, + "JVMOffHeapMemory" : 104976128, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 554933, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 228407, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 350990264, + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory" : 5067235328, + "ProcessTreeJVMRSSMemory" : 710475776, + "ProcessTreePythonVMemory" : 408375296, + "ProcessTreePythonRSSMemory" : 40284160, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0 + } +}, { + "id" : "9", + "hostPort" : "rezamemory-2.gce.something.com:40797", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 2, + "totalTasks" : 2, + "totalDuration" : 6191, + "totalGCTime" : 288, + "totalInputBytes" : 108, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2018-10-02T00:43:56.142GMT", + "executorLogs" : { + "stdout" : "http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stdout?start=-4096", + "stderr" : "http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "OnHeapStorageMemory" : 1088805, + "JVMOffHeapMemory" : 59006656, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 1088805, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 20181, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 193766856, + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory" : 3016261632, + "ProcessTreeJVMRSSMemory" : 405860352, + "ProcessTreePythonVMemory" : 625926144, + "ProcessTreePythonRSSMemory" : 69013504, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0 + } +} ] 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 9e1e65a35881..0ef9377dcb08 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" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { @@ -28,19 +43,4 @@ "startTimeEpoch" : 1516300235119, "endTimeEpoch" : 1516300707938 } ] -}, { - "id" : "app-20180109111548-0000", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-01-09T10:15:42.372GMT", - "endTime" : "2018-01-09T10:24:37.606GMT", - "lastUpdated" : "", - "duration" : 535234, - "sparkUser" : "attilapiros", - "completed" : true, - "appSparkVersion" : "2.3.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1515492942372, - "endTimeEpoch" : 1515493477606 - } ] } ] 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 28c6bf1b3e01..ea9dc1b97afc 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" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { 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 f547b79f47e1..2a77071a9ffd 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" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/ProcfsMetrics/22763/stat b/core/src/test/resources/ProcfsMetrics/22763/stat new file mode 100644 index 000000000000..cea4b713d0ee --- /dev/null +++ b/core/src/test/resources/ProcfsMetrics/22763/stat @@ -0,0 +1 @@ +22763 (python2.7) S 22756 22756 7051 0 -1 1077944384 449 0 0 0 4 3 0 0 20 0 3 0 117445 360595456 1912 18446744073709551615 4194304 4196756 140726192435536 140726192432528 140707465485051 0 0 16781312 2 18446744073709551615 0 0 17 1 0 0 0 0 0 6294976 6295604 38744064 140726192440006 140726192440119 140726192440119 140726192443369 0 \ No newline at end of file diff --git a/core/src/test/resources/ProcfsMetrics/26109/stat b/core/src/test/resources/ProcfsMetrics/26109/stat new file mode 100644 index 000000000000..ae46bfabd047 --- /dev/null +++ b/core/src/test/resources/ProcfsMetrics/26109/stat @@ -0,0 +1 @@ +26109 (java) S 1 26107 5788 0 -1 1077944320 75354 0 0 0 572 52 0 0 20 0 34 0 4355257 4769947648 64114 18446744073709551615 4194304 4196468 140737190381776 140737190364320 139976994791319 0 0 0 16800975 18446744073709551615 0 0 17 2 0 0 0 0 0 6293624 6294260 11276288 140737190385424 140737190414250 140737190414250 140737190416335 0 diff --git a/core/src/test/resources/spark-events/application_1538416563558_0014 b/core/src/test/resources/spark-events/application_1538416563558_0014 new file mode 100644 index 000000000000..000288dbc454 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1538416563558_0014 @@ -0,0 +1,190 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.5.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"rezamemory-1.gce.something.com","Port":43959},"Maximum Memory":384093388,"Timestamp":1538440967690,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_121/jre","Java Version":"1.8.0_121 (Oracle Corporation)","Scala Version":"version 2.11.12"},"Spark Properties":{"spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.yarn.jars":"local:/opt/some/path/lib/spark2/jars/*","spark.driver.host":"rezamemory-1.gce.something.com","spark.serializer.objectStreamReset":"100","spark.eventLog.enabled":"true","spark.executor.heartbeatInterval":"100ms","spark.hadoop.mapreduce.application.classpath":"","spark.driver.port":"35918","spark.shuffle.service.enabled":"true","spark.rdd.compress":"True","spark.driver.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.executorEnv.PYTHONPATH":"/opt/some/path/lib/spark2/python/lib/py4j-0.10.7-src.zip/opt/some/path/lib/spark2/python/lib/pyspark.zip","spark.yarn.historyServer.address":"http://rezamemory-1.gce.something.com:18089","spark.app.name":"PythonBisectingKMeansExample","spark.ui.killEnabled":"true","spark.sql.hive.metastore.jars":"${env:HADOOP_COMMON_HOME}/../hive/lib/*:${env:HADOOP_COMMON_HOME}/client/*","spark.dynamicAllocation.schedulerBacklogTimeout":"1","spark.yarn.am.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.scheduler.mode":"FIFO","spark.eventLog.logStageExecutorMetrics.enabled":"true","spark.yarn.config.gatewayPath":"/opt/cloudera/parcels","spark.executor.id":"driver","spark.yarn.config.replacementPath":"{{HADOOP_COMMON_HOME}}/../../..","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.submit.deployMode":"client","spark.shuffle.service.port":"7337","spark.master":"yarn","spark.authenticate":"false","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.eventLog.dir":"hdfs://rezamemory-1.gce.something.com:8020/user/spark/spark2ApplicationHistory","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.hadoop.yarn.application.classpath":"","spark.driver.appUIAddress":"http://rezamemory-1.gce.something.com:4040","spark.yarn.isPython":"true","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"rezamemory-1.gce.something.com","spark.dynamicAllocation.minExecutors":"0","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://rezamemory-1.gce.something.com:8088/proxy/application_1538416563558_0014","spark.dynamicAllocation.executorIdleTimeout":"60","spark.app.id":"application_1538416563558_0014","spark.sql.hive.metastore.version":"1.1.0"},"System Properties":{"java.io.tmpdir":"/tmp","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":"/root","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_121/jre/lib/amd64","user.dir":"/","java.library.path":":/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native:/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.121-b13","jetty.git.hash":"unknown","java.endorsed.dirs":"/usr/java/jdk1.8.0_121/jre/lib/endorsed","java.runtime.version":"1.8.0_121-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_121/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_121/jre/lib/resources.jar:/usr/java/jdk1.8.0_121/jre/lib/rt.jar:/usr/java/jdk1.8.0_121/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_121/jre/lib/jsse.jar:/usr/java/jdk1.8.0_121/jre/lib/jce.jar:/usr/java/jdk1.8.0_121/jre/lib/charsets.jar:/usr/java/jdk1.8.0_121/jre/lib/jfr.jar:/usr/java/jdk1.8.0_121/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-693.5.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"root","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.executor.heartbeatInterval=100ms --conf spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true --conf spark.eventLog.logStageExecutorMetrics.enabled=true ./opt/some/path/lib/spark2/examples/src/main/python/mllib/bisecting_k_means_example.py","java.home":"/usr/java/jdk1.8.0_121/jre","java.version":"1.8.0_121","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/opt/some/path/lib/spark2/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/netty-3.10.5.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/validation-api-1.1.0.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-annotations-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-azure-datalake-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jaxb-impl-2.2.3-1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jasper-compiler-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/logredactor-1.0.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-streaming_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-common-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guice-servlet-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-hadoop-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-hadoop-1.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-jackson-1.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-server-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jtransforms-2.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aircompressor-0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-el-1.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/minlog-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-daemon-1.0.13.jar":"System Classpath","/opt/some/path/lib/spark2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-crypto-1.0.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-mllib-local_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-openstack-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-lang3-3.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/univocity-parsers-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javassist-3.18.1-GA.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-api-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-registry-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/activation-1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/objenesis-2.5.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-xc-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-applications-unmanaged-am-launcher-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-repl_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hue-plugins-3.9.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-digester-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-resourcemanager-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-math3-3.1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/activation-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-tests-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/snappy-java-1.0.4.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-json-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-kvstore_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-common-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-guice-1.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-archives-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-archive-logs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jetty-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-examples-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/snappy-java-1.1.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-web-proxy-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-mapreduce-1.5.2-nohive.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsch-0.1.42.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-jvm-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.annotation-api-1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/pyrolite-4.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-jackson-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-client-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jline-2.11.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-scala_2.10-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/spark-streaming-kafka-0-8_2.11-2.2.0.cloudera1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsp-api-2.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-nodemanager-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-logging-1.1.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-compiler-3.0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-generator-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-format-2.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-server-1.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-common-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jetty-6.1.26.cloudera.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/mockito-all-1.8.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-core-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jsp-api-2.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-unsafe_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/oro-2.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-hs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-codec-1.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xmlenc-0.52.jar":"System Classpath","/opt/some/path/lib/spark2/jars/opencsv-2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xbean-asm6-shaded-4.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/javax.inject-1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-encoding-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-library-2.11.12.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/log4j-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/LICENSE.txt":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/netty-3.9.9.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-api-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/httpcore-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jettison-1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/zookeeper-3.4.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-core-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-auth-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-core-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-network-shuffle_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-beanutils-1.9.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/chill_2.11-0.9.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-core-2.6.7.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/paranamer-2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/janino-3.0.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jetty-util-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/py4j-0.10.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/ivy-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-lang-2.6.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-client-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/stream-2.7.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xml-apis-1.3.04.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/kafka_2.11-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/metrics-core-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-applicationhistoryservice-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/conf/":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guice-servlet-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/slf4j-api-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-configuration-1.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xz-1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-tools-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-format-0.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/okio-1.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/compress-lzf-1.0.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-jobclient-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hppc-0.7.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/stax-api-1.0-2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-yarn_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-applications-distributedshell-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/joda-time-2.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-sls-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jets3t-0.9.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guice-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-streaming-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guava-14.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hamcrest-core-1.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/aws-java-sdk-bundle-1.11.134.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-client-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-hs-plugins-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-gridmix-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xz-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-pig-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-guava-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-compiler-2.11.12.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-sql_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-app-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/java-xmlbuilder-0.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/slf4j-api-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-hadoop-bundle-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-shuffle-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-pig-bundle-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-digester-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-json-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-codec-1.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-beanutils-1.7.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-catalyst_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-common-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-parser-combinators_2.11-1.1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jetty-util-6.1.26.cloudera.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/httpclient-4.5.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/antlr4-runtime-4.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-lang-2.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-mllib_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-app-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/machinist_2.11-0.6.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-core_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spire_2.11-0.13.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-xc-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-thrift-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/opt/some/path/lib/spark2/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-annotations-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-io-2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-annotations-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-core-1.5.2-nohive.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-net-3.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-memory-0.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-graphx_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-core-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/okhttp-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0-sources.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-datajoin-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-aws-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-reflect-2.11.12.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-net-3.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-databind-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-column-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xmlenc-0.52.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/kafka-clients-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-io-2.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/lz4-java-1.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/core-1.1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-vector-0.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-azure-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0-javadoc.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-nfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-client-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/breeze_2.11-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/httpcore-4.4.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-core-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/servlet-api-2.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-math3-3.4.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.inject-1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-jaxrs-1.8.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/spark-1.6.0-cdh5.12.0-yarn-shuffle.jar":"System Classpath","/opt/some/path/lib/spark2/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/zookeeper-3.4.5-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/httpclient-4.2.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-graphite-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-compress-1.4.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-sketch_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-network-common_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/gson-2.2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-cascading-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-auth-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-shims-1.5.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/log4j-1.2.17.jar":"System Classpath","/opt/some/path/lib/spark2/jars/stax-api-1.0-2.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/zkclient-0.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/paranamer-2.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/some/path/lib/spark2/jars/gson-2.2.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-tags_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-configuration-1.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-hdfs-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guice-3.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jsr305-1.3.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-client-2.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/conf/yarn-conf/":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/azure-data-lake-store-sdk-2.1.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-distcp-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/junit-4.11.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-extras-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jasper-runtime-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-client-2.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-1.8.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-compress-1.8.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsr305-3.0.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guava-11.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/asm-3.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/avro-1.7.6-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-client-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-cli-1.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-scrooge_2.10-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-cli-1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-ant-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-launcher_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-nativetask-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-nfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-encoding-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-shuffle-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-avro-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/opt/some/path/lib/spark2/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-test-hadoop2-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-column-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/slf4j-log4j12-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-protobuf-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-ipc-1.8.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arpack_combined_all-0.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/netty-all-4.1.17.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/chill-java-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/NOTICE.txt":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-common-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-mapper-asl-1.8.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-annotations-2.6.7.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-rumen-2.6.0-cdh5.12.0.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"PythonBisectingKMeansExample","App ID":"application_1538416563558_0014","Timestamp":1538440959580,"User":"root"} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1538440969009,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"3\",\"name\":\"first\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440969044,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"3\",\"name\":\"first\"}"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538440973727,"Executor ID":"1","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000002/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000002/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1538440973735,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Port":46411},"Maximum Memory":384093388,"Timestamp":1538440973890,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"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":1538440973735,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440977628,"Failed":false,"Killed":false,"Accumulables":[{"ID":23,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.jvmGCTime","Update":208,"Value":208,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.resultSize","Update":1448,"Value":1448,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.executorCpuTime","Update":1105071149,"Value":1105071149,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorRunTime","Update":2307,"Value":2307,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorDeserializeCpuTime","Update":651096062,"Value":651096062,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeTime","Update":1322,"Value":1322,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1322,"Executor Deserialize CPU Time":651096062,"Executor Run Time":2307,"Executor CPU Time":1105071149,"Result Size":1448,"JVM GC Time":208,"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":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":256071440,"JVMOffHeapMemory":92211424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":333371,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":333371,"OffHeapUnifiedMemory":0,"DirectPoolMemory":134726,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4926242816,"ProcessTreeJVMRSSMemory":525656064,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":182536928,"JVMOffHeapMemory":58263224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1086483,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1086483,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20304,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3009855488,"ProcessTreeJVMRSSMemory":404488192,"ProcessTreePythonVMemory":626200576,"ProcessTreePythonRSSMemory":69218304,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440969044,"Completion Time":1538440977644,"Accumulables":[{"ID":23,"Name":"internal.metrics.input.recordsRead","Value":4,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorDeserializeCpuTime","Value":651096062,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.resultSize","Value":1448,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"internal.metrics.input.bytesRead","Value":72,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.executorCpuTime","Value":1105071149,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeTime","Value":1322,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorRunTime","Value":2307,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.jvmGCTime","Value":208,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1538440977650,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1538440977784,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[1,2],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440977793,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1538440977816,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_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":2,"Index":1,"Attempt":0,"Launch Time":1538440978659,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1538440977816,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440978683,"Failed":false,"Killed":false,"Accumulables":[{"ID":48,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Update":13535058,"Value":13535058,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":1088,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Update":202227536,"Value":202227536,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Update":705,"Value":705,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Update":65694833,"Value":65694833,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Update":119,"Value":119,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":119,"Executor Deserialize CPU Time":65694833,"Executor Run Time":705,"Executor CPU Time":202227536,"Result Size":1662,"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":178,"Shuffle Write Time":13535058,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":1,"Attempt":0,"Launch Time":1538440978659,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440978820,"Failed":false,"Killed":false,"Accumulables":[{"ID":48,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Update":289555,"Value":13824613,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2176,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Update":36560031,"Value":238787567,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Update":120,"Value":825,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7042587,"Value":72737420,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":127,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7042587,"Executor Run Time":120,"Executor CPU Time":36560031,"Result Size":1662,"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":178,"Shuffle Write Time":289555,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":292935952,"JVMOffHeapMemory":95141200,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":351534,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":351534,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4929392640,"ProcessTreeJVMRSSMemory":539996160,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":215586960,"JVMOffHeapMemory":60718904,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1492038,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1492038,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20637,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3014057984,"ProcessTreeJVMRSSMemory":422723584,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440977793,"Completion Time":1538440978821,"Accumulables":[{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Value":127,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Value":2176,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":356,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Value":238787567,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Value":13824613,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Value":825,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Value":72737420,"Internal":true,"Count Failed Values":true},{"ID":48,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440978830,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Launch Time":1538440978844,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_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":4,"Index":0,"Attempt":0,"Launch Time":1538440979033,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","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":3,"Index":1,"Attempt":0,"Launch Time":1538440978844,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979050,"Failed":false,"Killed":false,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":356,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":992,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Update":88389028,"Value":88389028,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Update":122,"Value":122,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Update":27126551,"Value":27126551,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Update":45,"Value":45,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":45,"Executor Deserialize CPU Time":27126551,"Executor Run Time":122,"Executor CPU Time":88389028,"Result Size":1828,"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":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":356,"Total Records Read":2},"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":4,"Index":0,"Attempt":0,"Launch Time":1538440979033,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979084,"Failed":false,"Killed":false,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Update":0,"Value":992,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Update":1706,"Value":3534,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Update":15055355,"Value":103444383,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Update":26,"Value":148,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4722422,"Value":31848973,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":50,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4722422,"Executor Run Time":26,"Executor CPU Time":15055355,"Result Size":1706,"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":"driver","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":303792496,"JVMOffHeapMemory":95545824,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":371127,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":371127,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4931497984,"ProcessTreeJVMRSSMemory":549777408,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":227393200,"JVMOffHeapMemory":61799392,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":463135,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":463135,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20637,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016163328,"ProcessTreeJVMRSSMemory":436539392,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440978830,"Completion Time":1538440979086,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Value":148,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Value":3534,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Value":31848973,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Value":992,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Value":103444383,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":2,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":356,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Value":50,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1538440979087,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1538440979161,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[3,4],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979163,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1538440979184,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":1,"Attempt":0,"Launch Time":1538440979344,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1538440979184,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979348,"Failed":false,"Killed":false,"Accumulables":[{"ID":98,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Update":259310,"Value":259310,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Update":40081727,"Value":40081727,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Update":98,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Update":24271689,"Value":24271689,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Update":39,"Value":39,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":39,"Executor Deserialize CPU Time":24271689,"Executor Run Time":98,"Executor CPU Time":40081727,"Result Size":1662,"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":355,"Shuffle Write Time":259310,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":1,"Attempt":0,"Launch Time":1538440979344,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979441,"Failed":false,"Killed":false,"Accumulables":[{"ID":98,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Update":221381,"Value":480691,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Update":23089017,"Value":63170744,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Update":74,"Value":172,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3471167,"Value":27742856,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":43,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3471167,"Executor Run Time":74,"Executor CPU Time":23089017,"Result Size":1662,"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":178,"Shuffle Write Time":221381,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":3,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":318926040,"JVMOffHeapMemory":96521592,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":391718,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":391718,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4932550656,"ProcessTreeJVMRSSMemory":569753600,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":3,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":236711480,"JVMOffHeapMemory":62683008,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":483726,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":483726,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20922,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3019313152,"ProcessTreeJVMRSSMemory":445640704,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979163,"Completion Time":1538440979444,"Accumulables":[{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Value":27742856,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Value":43,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Value":63170744,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Value":480691,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Value":172,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979446,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1538440979462,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1538440979527,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1538440979462,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979528,"Failed":false,"Killed":false,"Accumulables":[{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Update":17714408,"Value":17714408,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Update":30,"Value":30,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12579502,"Value":12579502,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":22,"Executor Deserialize CPU Time":12579502,"Executor Run Time":30,"Executor CPU Time":17714408,"Result Size":1828,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"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":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1538440979527,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979572,"Failed":false,"Killed":false,"Accumulables":[{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Update":1828,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Update":16462125,"Value":34176533,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Update":16,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3190663,"Value":15770165,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":26,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3190663,"Executor Run Time":16,"Executor CPU Time":16462125,"Result Size":1828,"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":2,"Fetch Wait Time":1,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"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":4,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":329919832,"JVMOffHeapMemory":96756344,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":413740,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":413740,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4935208960,"ProcessTreeJVMRSSMemory":585252864,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":4,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":242876648,"JVMOffHeapMemory":62975784,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":505748,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":505748,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20922,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3019313152,"ProcessTreeJVMRSSMemory":451244032,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979446,"Completion Time":1538440979573,"Accumulables":[{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Value":26,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Value":34176533,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Value":46,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Value":3656,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Value":15770165,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1538440979573,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1538440979609,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[5,6],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979619,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1538440979638,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":1,"Attempt":0,"Launch Time":1538440979754,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1538440979638,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979756,"Failed":false,"Killed":false,"Accumulables":[{"ID":148,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Update":272852,"Value":272852,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Update":23042622,"Value":23042622,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Update":76,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13112180,"Value":13112180,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":13112180,"Executor Run Time":76,"Executor CPU Time":23042622,"Result Size":1662,"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":355,"Shuffle Write Time":272852,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":1,"Attempt":0,"Launch Time":1538440979754,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979851,"Failed":false,"Killed":false,"Accumulables":[{"ID":148,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Update":229882,"Value":502734,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Update":22093052,"Value":45135674,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Update":81,"Value":157,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3862579,"Value":16974759,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":32,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3862579,"Executor Run Time":81,"Executor CPU Time":22093052,"Result Size":1662,"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":178,"Shuffle Write Time":229882,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":5,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":341682304,"JVMOffHeapMemory":97514672,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":434309,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":434309,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4935254016,"ProcessTreeJVMRSSMemory":597999616,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":5,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":252029672,"JVMOffHeapMemory":63463032,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":526317,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":526317,"OffHeapUnifiedMemory":0,"DirectPoolMemory":21041,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3020365824,"ProcessTreeJVMRSSMemory":458960896,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979619,"Completion Time":1538440979852,"Accumulables":[{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Value":502734,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Value":157,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Value":16974759,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":148,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Value":45135674,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Value":32,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979854,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1538440979869,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1538440979920,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1538440979869,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979921,"Failed":false,"Killed":false,"Accumulables":[{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Update":15546330,"Value":15546330,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Update":19,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11263754,"Value":11263754,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":22,"Executor Deserialize CPU Time":11263754,"Executor Run Time":19,"Executor CPU Time":15546330,"Result Size":1828,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"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":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1538440979920,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979972,"Failed":false,"Killed":false,"Accumulables":[{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":157,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Update":1871,"Value":3699,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Update":15089701,"Value":30636031,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Update":27,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3045280,"Value":14309034,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":25,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3045280,"Executor Run Time":27,"Executor CPU Time":15089701,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"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":6,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":350990264,"JVMOffHeapMemory":97710440,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":456312,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":456312,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4932550656,"ProcessTreeJVMRSSMemory":604299264,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979854,"Completion Time":1538440979973,"Accumulables":[{"ID":155,"Name":"internal.metrics.resultSize","Value":3699,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Value":25,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Value":30636031,"Internal":true,"Count Failed Values":true},{"ID":157,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Value":46,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Value":14309034,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1538440979974,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1538440980008,"Stage Infos":[{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[7,8],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440980015,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1538440980049,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Port":46411},"Timestamp":1538440980522} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440980759,"Executor ID":"1","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000002 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000002\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"1","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000002 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000002\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1538440980049,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440980757,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538440986317,"Executor ID":"2","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000003/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000003/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":0,"Attempt":1,"Launch Time":1538440986317,"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Port":39119},"Maximum Memory":384093388,"Timestamp":1538440986696,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538440988793,"Executor ID":"3","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000004/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000004/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":1,"Attempt":0,"Launch Time":1538440988793,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Port":40911},"Maximum Memory":384093388,"Timestamp":1538440989162,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Port":39119},"Timestamp":1538440993798} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"2","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000003 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000003\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":14,"Index":0,"Attempt":1,"Launch Time":1538440986317,"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440994010,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440994012,"Executor ID":"2","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000003 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000003\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":2,"Launch Time":1538440995449,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":1,"Attempt":0,"Launch Time":1538440988793,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440995450,"Failed":false,"Killed":false,"Accumulables":[{"ID":198,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Update":10065137,"Value":10065137,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":1088,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.jvmGCTime","Update":360,"Value":360,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Update":1705,"Value":1705,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Update":1406669099,"Value":1406669099,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Update":4128,"Value":4128,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Update":726605764,"Value":726605764,"Internal":true,"Count Failed Values":true},{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Update":1995,"Value":1995,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1995,"Executor Deserialize CPU Time":726605764,"Executor Run Time":4128,"Executor CPU Time":1406669099,"Result Size":1705,"JVM GC Time":360,"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":178,"Shuffle Write Time":10065137,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":2,"Launch Time":1538440995449,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440995696,"Failed":false,"Killed":false,"Accumulables":[{"ID":198,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Update":293846,"Value":10358983,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Update":1662,"Value":3367,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Update":91844758,"Value":1498513857,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Update":220,"Value":4348,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8316162,"Value":734921926,"Internal":true,"Count Failed Values":true},{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":2004,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8316162,"Executor Run Time":220,"Executor CPU Time":91844758,"Result Size":1662,"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":355,"Shuffle Write Time":293846,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":201931120,"JVMOffHeapMemory":58230320,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1094710,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1094710,"OffHeapUnifiedMemory":0,"DirectPoolMemory":45633,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3023769600,"ProcessTreeJVMRSSMemory":410324992,"ProcessTreePythonVMemory":285470720,"ProcessTreePythonRSSMemory":30171136,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":195471784,"JVMOffHeapMemory":100867584,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":476885,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":476885,"OffHeapUnifiedMemory":0,"DirectPoolMemory":171571,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4971368448,"ProcessTreeJVMRSSMemory":663375872,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":258718872,"JVMOffHeapMemory":63737056,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":548320,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":548320,"OffHeapUnifiedMemory":0,"DirectPoolMemory":21084,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3021418496,"ProcessTreeJVMRSSMemory":466001920,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":223684056,"JVMOffHeapMemory":60665000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1482102,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1482102,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20318,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3015626752,"ProcessTreeJVMRSSMemory":404672512,"ProcessTreePythonVMemory":958963712,"ProcessTreePythonRSSMemory":106639360,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440980015,"Completion Time":1538440995697,"Accumulables":[{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Value":2004,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Value":4348,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Value":10358983,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.jvmGCTime","Value":360,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Value":3367,"Internal":true,"Count Failed Values":true},{"ID":198,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Value":734921926,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Value":1498513857,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440995698,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1538440995710,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Port":40911},"Timestamp":1538440996257} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"3","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000004 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000004\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1538440995710,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440996467,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440996468,"Executor ID":"3","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000004 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000004\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441002826,"Executor ID":"4","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000005/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000005/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":0,"Attempt":1,"Launch Time":1538441002828,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441003031,"Executor ID":"5","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000006/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000006/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":1,"Attempt":0,"Launch Time":1538441003032,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Port":39248},"Maximum Memory":384093388,"Timestamp":1538441003132,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Port":43165},"Maximum Memory":384093388,"Timestamp":1538441003383,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":0,"Attempt":1,"Launch Time":1538441002828,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441006147,"Failed":false,"Killed":false,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Update":350,"Value":350,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Update":1914,"Value":1914,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Update":219243972,"Value":219243972,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Update":893,"Value":893,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Update":717217987,"Value":717217987,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Update":1972,"Value":1972,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1972,"Executor Deserialize CPU Time":717217987,"Executor Run Time":893,"Executor CPU Time":219243972,"Result Size":1914,"JVM GC Time":350,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":1,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":178,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":1},"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":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":1,"Attempt":0,"Launch Time":1538441003032,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441006584,"Failed":false,"Killed":false,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Update":10,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Update":270,"Value":620,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Update":1914,"Value":3828,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Update":210863492,"Value":430107464,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Update":412,"Value":1305,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Update":727356712,"Value":1444574699,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Update":2604,"Value":4576,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2604,"Executor Deserialize CPU Time":727356712,"Executor Run Time":412,"Executor CPU Time":210863492,"Result Size":1914,"JVM GC Time":270,"Result Serialization Time":10,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":2,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":355,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":2},"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":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":263995432,"JVMOffHeapMemory":101978136,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":498888,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":498888,"OffHeapUnifiedMemory":0,"DirectPoolMemory":191656,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5008089088,"ProcessTreeJVMRSSMemory":663732224,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"5","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":150497592,"JVMOffHeapMemory":45958576,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":22003,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":22003,"OffHeapUnifiedMemory":0,"DirectPoolMemory":3446,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2984218624,"ProcessTreeJVMRSSMemory":325042176,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":181352744,"JVMOffHeapMemory":47061200,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":22003,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":22003,"OffHeapUnifiedMemory":0,"DirectPoolMemory":11272,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3013332992,"ProcessTreeJVMRSSMemory":416645120,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":226223752,"JVMOffHeapMemory":60840424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":433558,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":433558,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20318,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016937472,"ProcessTreeJVMRSSMemory":406044672,"ProcessTreePythonVMemory":958963712,"ProcessTreePythonRSSMemory":106639360,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440995698,"Completion Time":1538441006585,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1444574699,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Value":3828,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Value":12,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Value":4576,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Value":430107464,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Value":1305,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Value":620,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1538441006585,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1538441006610,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[9,10],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441006612,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":0,"Attempt":0,"Launch Time":1538441006622,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":1,"Attempt":0,"Launch Time":1538441006623,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Port":39248},"Timestamp":1538441010070} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Port":43165},"Timestamp":1538441010233} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"4","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000005 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000005\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":20,"Index":0,"Attempt":0,"Launch Time":1538441006622,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441010280,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441010281,"Executor ID":"4","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000005 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000005\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"5","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000006 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000006\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":21,"Index":1,"Attempt":0,"Launch Time":1538441006623,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441010484,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441010485,"Executor ID":"5","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000006 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000006\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441015443,"Executor ID":"6","Executor Info":{"Host":"rezamemory-3.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000007/root/stdout?start=-4096","stderr":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000007/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":1,"Attempt":1,"Launch Time":1538441015444,"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Port":45593},"Maximum Memory":384093388,"Timestamp":1538441015852,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441020314,"Executor ID":"7","Executor Info":{"Host":"rezamemory-3.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000008/root/stdout?start=-4096","stderr":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000008/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":0,"Attempt":1,"Launch Time":1538441020315,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Port":40992},"Maximum Memory":384093388,"Timestamp":1538441020602,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Port":45593},"Timestamp":1538441022942} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"6","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000007 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000007\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":22,"Index":1,"Attempt":1,"Launch Time":1538441015444,"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441023152,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441023153,"Executor ID":"6","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000007 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000007\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":1,"Attempt":2,"Launch Time":1538441025899,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":0,"Attempt":1,"Launch Time":1538441020315,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441025900,"Failed":false,"Killed":false,"Accumulables":[{"ID":248,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Update":3971129,"Value":3971129,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":231,"Name":"internal.metrics.jvmGCTime","Update":244,"Value":244,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Update":1705,"Value":1705,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Update":1268816374,"Value":1268816374,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Update":2978,"Value":2978,"Internal":true,"Count Failed Values":true},{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Update":714859741,"Value":714859741,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Update":2106,"Value":2106,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2106,"Executor Deserialize CPU Time":714859741,"Executor Run Time":2978,"Executor CPU Time":1268816374,"Result Size":1705,"JVM GC Time":244,"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":355,"Shuffle Write Time":3971129,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":1,"Attempt":2,"Launch Time":1538441025899,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026136,"Failed":false,"Killed":false,"Accumulables":[{"ID":248,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Update":265841,"Value":4236970,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":232,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Update":1705,"Value":3410,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Update":88980290,"Value":1357796664,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Update":201,"Value":3179,"Internal":true,"Count Failed Values":true},{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8550572,"Value":723410313,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":2119,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":8550572,"Executor Run Time":201,"Executor CPU Time":88980290,"Result Size":1705,"JVM GC Time":0,"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":178,"Shuffle Write Time":265841,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":332727504,"JVMOffHeapMemory":103237664,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":519462,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":519462,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228406,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5011247104,"ProcessTreeJVMRSSMemory":658915328,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"5","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":184519808,"JVMOffHeapMemory":58341088,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1116714,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1116714,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20420,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2998673408,"ProcessTreeJVMRSSMemory":378527744,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":218694008,"JVMOffHeapMemory":60757008,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1482103,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1482103,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20668,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3020120064,"ProcessTreeJVMRSSMemory":423698432,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":220189424,"JVMOffHeapMemory":59534504,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1116714,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1116714,"OffHeapUnifiedMemory":0,"DirectPoolMemory":27895,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3024392192,"ProcessTreeJVMRSSMemory":431939584,"ProcessTreePythonVMemory":283738112,"ProcessTreePythonRSSMemory":27226112,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"6","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":208356192,"JVMOffHeapMemory":58297728,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1094711,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1094711,"OffHeapUnifiedMemory":0,"DirectPoolMemory":27296,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3027820544,"ProcessTreeJVMRSSMemory":439750656,"ProcessTreePythonVMemory":286220288,"ProcessTreePythonRSSMemory":30846976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441006612,"Completion Time":1538441026137,"Accumulables":[{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Value":723410313,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Value":2119,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Value":1357796664,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true},{"ID":232,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Value":3179,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Value":4236970,"Internal":true,"Count Failed Values":true},{"ID":231,"Name":"internal.metrics.jvmGCTime","Value":244,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Value":3410,"Internal":true,"Count Failed Values":true},{"ID":248,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026138,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1538441026147,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1538441026309,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1538441026147,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026311,"Failed":false,"Killed":false,"Accumulables":[{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":254,"Name":"internal.metrics.executorCpuTime","Update":80311930,"Value":80311930,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Update":89,"Value":89,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29610969,"Value":29610969,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":62,"Executor Deserialize CPU Time":29610969,"Executor Run Time":89,"Executor CPU Time":80311930,"Result Size":1828,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"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":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1538441026309,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026375,"Failed":false,"Killed":false,"Accumulables":[{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Update":1828,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":254,"Name":"internal.metrics.executorCpuTime","Update":18625831,"Value":98937761,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Update":38,"Value":127,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6238101,"Value":35849070,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":68,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6238101,"Executor Run Time":38,"Executor CPU Time":18625831,"Result Size":1828,"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":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"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":10,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":341644736,"JVMOffHeapMemory":103378144,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":541469,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":541469,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228406,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5011247104,"ProcessTreeJVMRSSMemory":658989056,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":10,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":228132872,"JVMOffHeapMemory":61634808,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":455614,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":455614,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20669,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3021172736,"ProcessTreeJVMRSSMemory":436867072,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026138,"Completion Time":1538441026376,"Accumulables":[{"ID":254,"Name":"internal.metrics.executorCpuTime","Value":98937761,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Value":127,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Value":35849070,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Value":3656,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Value":68,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1538441026376,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1538441026404,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[12,11],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026408,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":0,"Attempt":0,"Launch Time":1538441026450,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":1,"Attempt":0,"Launch Time":1538441026585,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":0,"Attempt":0,"Launch Time":1538441026450,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026586,"Failed":false,"Killed":false,"Accumulables":[{"ID":298,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Update":278446,"Value":278446,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Update":23317154,"Value":23317154,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Update":69,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Update":17832528,"Value":17832528,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Update":53,"Value":53,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":53,"Executor Deserialize CPU Time":17832528,"Executor Run Time":69,"Executor CPU Time":23317154,"Result Size":1662,"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":355,"Shuffle Write Time":278446,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":1,"Attempt":0,"Launch Time":1538441026585,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026700,"Failed":false,"Killed":false,"Accumulables":[{"ID":298,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Update":215244,"Value":493690,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Update":23292541,"Value":46609695,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Update":94,"Value":163,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4400590,"Value":22233118,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":57,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4400590,"Executor Run Time":94,"Executor CPU Time":23292541,"Result Size":1662,"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":178,"Shuffle Write Time":215244,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":11,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":198912952,"JVMOffHeapMemory":104016864,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":554933,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":554933,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5040721920,"ProcessTreeJVMRSSMemory":705302528,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":11,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":249428840,"JVMOffHeapMemory":62917480,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":455614,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":455614,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20911,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3035901952,"ProcessTreeJVMRSSMemory":447041536,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026408,"Completion Time":1538441026701,"Accumulables":[{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Value":22233118,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Value":57,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Value":46609695,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Value":163,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Value":493690,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026702,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":0,"Attempt":0,"Launch Time":1538441026714,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":1,"Attempt":0,"Launch Time":1538441026794,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":0,"Attempt":0,"Launch Time":1538441026714,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026795,"Failed":false,"Killed":false,"Accumulables":[{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Update":1871,"Value":1871,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.executorCpuTime","Update":16951615,"Value":16951615,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12613041,"Value":12613041,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Update":31,"Value":31,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":31,"Executor Deserialize CPU Time":12613041,"Executor Run Time":28,"Executor CPU Time":16951615,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"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":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":1,"Attempt":0,"Launch Time":1538441026794,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026839,"Failed":false,"Killed":false,"Accumulables":[{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Update":1871,"Value":3742,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.executorCpuTime","Update":17828037,"Value":34779652,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Update":24,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3879530,"Value":16492571,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":36,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3879530,"Executor Run Time":24,"Executor CPU Time":17828037,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"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":12,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":204287872,"JVMOffHeapMemory":104055736,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":519458,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":519458,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5047037952,"ProcessTreeJVMRSSMemory":708661248,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":12,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":252161344,"JVMOffHeapMemory":63019944,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":441078,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":441078,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20911,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3038007296,"ProcessTreeJVMRSSMemory":451837952,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026702,"Completion Time":1538441026840,"Accumulables":[{"ID":304,"Name":"internal.metrics.executorCpuTime","Value":34779652,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Value":36,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Value":52,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Value":16492571,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Value":3742,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1538441026840,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerUnpersistRDD","RDD ID":32} +{"Event":"SparkListenerUnpersistRDD","RDD ID":5} +{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1538441026935,"Stage Infos":[{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[13],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"79\",\"name\":\"sum\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026936,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"79\",\"name\":\"sum\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":0,"Attempt":0,"Launch Time":1538441026947,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Port":40992},"Timestamp":1538441027285} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"7","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000008 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000008\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":31,"Index":0,"Attempt":0,"Launch Time":1538441026947,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441027494,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441027495,"Executor ID":"7","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000008 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000008\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441032740,"Executor ID":"8","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000009/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000009/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":1,"Launch Time":1538441032741,"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Port":41485},"Maximum Memory":384093388,"Timestamp":1538441033142,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441036142,"Executor ID":"9","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1538441036144,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Port":40797},"Maximum Memory":384093388,"Timestamp":1538441036560,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Port":41485},"Timestamp":1538441040323} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"8","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000009 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000009\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":32,"Index":0,"Attempt":1,"Launch Time":1538441032741,"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441040533,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441040534,"Executor ID":"8","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000009 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000009\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\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\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":0,"Attempt":2,"Launch Time":1538441042184,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1538441036144,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441042185,"Failed":false,"Killed":false,"Accumulables":[{"ID":348,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":331,"Name":"internal.metrics.jvmGCTime","Update":288,"Value":288,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Update":1539,"Value":1539,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Update":1278640624,"Value":1278640624,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Update":2796,"Value":2796,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Update":720112530,"Value":720112530,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Update":2587,"Value":2587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2587,"Executor Deserialize CPU Time":720112530,"Executor Run Time":2796,"Executor CPU Time":1278640624,"Result Size":1539,"JVM GC Time":288,"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":36,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":0,"Attempt":2,"Launch Time":1538441042184,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441042334,"Failed":false,"Killed":false,"Accumulables":[{"ID":348,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Update":1453,"Value":2992,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Update":69678739,"Value":1348319363,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Update":118,"Value":2914,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6252896,"Value":726365426,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":2593,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6252896,"Executor Run Time":118,"Executor CPU Time":69678739,"Result Size":1453,"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":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":266240264,"JVMOffHeapMemory":104976128,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":534126,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":534126,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5067235328,"ProcessTreeJVMRSSMemory":710475776,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"8","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":197860072,"JVMOffHeapMemory":57762424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1088805,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1088805,"OffHeapUnifiedMemory":0,"DirectPoolMemory":25453,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3028791296,"ProcessTreeJVMRSSMemory":430297088,"ProcessTreePythonVMemory":286212096,"ProcessTreePythonRSSMemory":30441472,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"9","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":193766856,"JVMOffHeapMemory":59006656,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1088805,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1088805,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20181,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016261632,"ProcessTreeJVMRSSMemory":405860352,"ProcessTreePythonVMemory":625926144,"ProcessTreePythonRSSMemory":69013504,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\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)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026936,"Completion Time":1538441042335,"Accumulables":[{"ID":331,"Name":"internal.metrics.jvmGCTime","Value":288,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Value":2914,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Value":726365426,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.input.recordsRead","Value":6,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Value":2992,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Value":2593,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Value":1348319363,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1538441042335,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1538441042338} 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 7c9f8aba17f3..2a2d013bacbd 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 @@ -83,6 +83,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers .set("spark.testing", "true") .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) .set("spark.eventLog.logStageExecutorMetrics.enabled", "true") + .set("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled", "true") conf.setAll(extraConf) provider = new FsHistoryProvider(conf) provider.checkForLogs() @@ -131,6 +132,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "executor list json" -> "applications/local-1422981780767/executors", "executor list with executor metrics json" -> "applications/application_1506645932520_24630151/executors", + "executor list with executor process tree metrics json" -> + "applications/application_1538416563558_0014/executors", "stage list json" -> "applications/local-1422981780767/stages", "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", diff --git a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala new file mode 100644 index 000000000000..9ed1497db5e1 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.SparkFunSuite + + +class ProcfsMetricsGetterSuite extends SparkFunSuite { + + val p = new ProcfsMetricsGetter(getTestResourcePath("ProcfsMetrics")) + + test("testGetProcessInfo") { + var r = ProcfsMetrics(0, 0, 0, 0, 0, 0) + r = p.addProcfsMetricsFromOneProcess(r, 26109) + assert(r.jvmVmemTotal == 4769947648L) + assert(r.jvmRSSTotal == 262610944) + assert(r.pythonVmemTotal == 0) + assert(r.pythonRSSTotal == 0) + + r = p.addProcfsMetricsFromOneProcess(r, 22763) + assert(r.pythonVmemTotal == 360595456) + assert(r.pythonRSSTotal == 7831552) + assert(r.jvmVmemTotal == 4769947648L) + assert(r.jvmRSSTotal == 262610944) + } +} 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 cecd6996df7b..0c04a93646d7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -282,53 +282,67 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), + new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), - // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 + new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))), + // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), + new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), + new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L))), // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), + new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), + new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9; + new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, + 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 3, 6, 7, 9; // initialize stage 1 peaks createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, + 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L))), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), - // enew ExecutorMetrics(xec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 + new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))), + // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), + new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, + 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 5000L, 2000L))), // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), + new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, + 2000L, 1500L, 1000L, 500L))), // exec 2: new stage 1 peak for metrics at index: 7 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), + new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, + 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L))), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), + new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, + 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L))), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), + new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, + 4000L, 6000L, 3000L, 5000L, 2000L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -342,20 +356,23 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // expected StageExecutorMetrics, for the given stage id and executor id val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] = - Map( - ((0, "1"), - new SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), - ((0, "2"), - new SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), - ((1, "1"), - new SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), - ((1, "2"), - new SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) - + Map( + ((0, "1"), + new SparkListenerStageExecutorMetrics("1", 0, 0, + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)))), + ((0, "2"), + new SparkListenerStageExecutorMetrics("2", 0, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), + ((1, "1"), + new SparkListenerStageExecutorMetrics("1", 1, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, + 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)))), + ((1, "2"), + new SparkListenerStageExecutorMetrics("2", 1, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, + 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) // Verify the log file contains the expected events. // Posted events should be logged, except for ExecutorMetricsUpdate events -- these // are consolidated, and the peak values for each stage are logged at stage end. @@ -456,9 +473,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(executorMetrics.execId === expectedMetrics.execId) assert(executorMetrics.stageId === expectedMetrics.stageId) assert(executorMetrics.stageAttemptId === expectedMetrics.stageAttemptId) - ExecutorMetricType.values.foreach { metricType => - assert(executorMetrics.executorMetrics.getMetricValue(metricType) === - expectedMetrics.executorMetrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(executorMetrics.executorMetrics.getMetricValue(metric._1) === + expectedMetrics.executorMetrics.getMetricValue(metric._1)) } case None => assert(false) 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 61fec8c1d0e4..71eeb0480245 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1367,58 +1367,74 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))) // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L))) // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))) // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks listener.onStageSubmitted(createStageSubmittedEvent(1)) // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))) // complete stage 0, and 3 more updates for each executor with just // stage 1 running listener.onStageCompleted(createStageCompletedEvent(0)) // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))) // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L, 8000L, 5000L, + 7000L, 4000L, 6000L, 3000L))) // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, 2000L, + 1500L, 1000L, 500L))) // exec 2: new stage 1 peak for metrics at index: 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L, 7000L, 3000L, + 6000L, 2000L, 5000L, 1000L))) // exec 1: no new stage 1 peaks listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L, 4000L, 2500L, + 3000L, 1500, 2000L, 500L))) listener.onExecutorRemoved(createExecutorRemovedEvent(1)) // exec 2: new stage 1 peak for metrics at index: 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, 4000L, 6000L, + 3000L, 5000L, 2000L))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L, 9000L, 5000L, 8000L, 4000L, 7000L, 3000L))) // check that the stored peak values match the expected values expectedValues.foreach { case (id, metrics) => @@ -1426,8 +1442,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.values.foreach { metricType => - assert(actual.getMetricValue(metricType) === metrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) } case _ => assert(false) @@ -1446,23 +1462,29 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onStageSubmitted(createStageSubmittedEvent(0)) listener.onStageSubmitted(createStageSubmittedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))) + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L, 9000L, + 4000L, 8000L, 3000L, 7000L, 2000L)))) listener.onStageCompleted(createStageCompletedEvent(0)) // executor 1 is removed before stage 1 has finished, the stage executor metrics // are logged afterwards and should still be used to update the executor metrics. listener.onExecutorRemoved(createExecutorRemovedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L, 8000L, 5000L, + 7000L, 4000L, 6000L, 3000L)))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L, 9000L, 5000L, 8000L, 4000L, 7000L, 3000L))) // check that the stored peak values match the expected values for ((id, metrics) <- expectedValues) { @@ -1470,8 +1492,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.values.foreach { metricType => - assert(actual.getMetricValue(metricType) === metrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) } case _ => assert(false) 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 1e0d2af9a471..303ca7cb8801 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -96,7 +96,8 @@ class JsonProtocolSuite extends SparkFunSuite { .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } val executorUpdates = new ExecutorMetrics( - Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) + Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), Some(executorUpdates)) } @@ -105,8 +106,8 @@ class JsonProtocolSuite extends SparkFunSuite { "In your multitude...", 300), RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100L, 0L)) val stageExecutorMetrics = SparkListenerStageExecutorMetrics("1", 2, 3, - new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L))) - + new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) testEvent(taskStart, taskStartJsonString) @@ -440,14 +441,14 @@ class JsonProtocolSuite extends SparkFunSuite { test("executorMetricsFromJson backward compatibility: handle missing metrics") { // any missing metrics should be set to 0 - val executorMetrics = new ExecutorMetrics( - Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 789L)) + val executorMetrics = new ExecutorMetrics(Array(12L, 23L, 45L, 67L, 78L, 89L, + 90L, 123L, 456L, 789L, 40L, 20L, 20L, 10L, 20L, 10L)) val oldExecutorMetricsJson = JsonProtocol.executorMetricsToJson(executorMetrics) .removeField( _._1 == "MappedPoolMemory") - val expectedExecutorMetrics = new ExecutorMetrics( - Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 0L)) - assertEquals(expectedExecutorMetrics, + val exepectedExecutorMetrics = new ExecutorMetrics(Array(12L, 23L, 45L, 67L, + 78L, 89L, 90L, 123L, 456L, 0L, 40L, 20L, 20L, 10L, 20L, 10L)) + assertEquals(exepectedExecutorMetrics, JsonProtocol.executorMetricsFromJson(oldExecutorMetricsJson)) } @@ -753,9 +754,9 @@ private[spark] object JsonProtocolSuite extends Assertions { assertStackTraceElementEquals) } - private def assertEquals(metrics1: ExecutorMetrics, metrics2: ExecutorMetrics) { - ExecutorMetricType.values.foreach { metricType => - assert(metrics1.getMetricValue(metricType) === metrics2.getMetricValue(metricType)) + private def assertEquals(metrics1: ExecutorMetrics, metrics2: ExecutorMetrics): Unit = { + ExecutorMetricType.metricToOffset.foreach { metric => + assert(metrics1.getMetricValue(metric._1) === metrics2.getMetricValue(metric._1)) } } @@ -872,13 +873,14 @@ private[spark] object JsonProtocolSuite extends Assertions { if (includeTaskMetrics) { Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), makeAccumulableInfo(2, false, false, None)))) - } else { + } else { Seq() } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L))) - } else { + Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, + 0L, 0L, 0L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) + } else { None } SparkListenerExecutorMetricsUpdate(execId, taskMetrics, executorMetricsUpdate) @@ -2082,7 +2084,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "OnHeapUnifiedMemory" : 432, | "OffHeapUnifiedMemory" : 321, | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765 + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 | } | |} @@ -2105,7 +2113,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "OnHeapUnifiedMemory" : 432, | "OffHeapUnifiedMemory" : 321, | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765 + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 | } |} """.stripMargin diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 777950016801..8239cbc3a381 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -82,6 +82,8 @@ app-20161115172038-0000 app-20161116163331-0000 application_1516285256255_0012 application_1506645932520_24630151 +application_1538416563558_0014 +stat local-1422981759269 local-1422981780767 local-1425081759269 From 0a37da68e1cbca0e0120beab916309898022ba85 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 10 Dec 2018 12:04:44 -0800 Subject: [PATCH 0105/1072] [SPARK-26317][BUILD] Upgrade SBT to 0.13.18 ## What changes were proposed in this pull request? SBT 0.13.14 ~ 1.1.1 has a bug on accessing `java.util.Base64.getDecoder` with JDK9+. It's fixed at 1.1.2 and backported to [0.13.18 (released on Nov 28th)](https://github.com/sbt/sbt/releases/tag/v0.13.18). This PR aims to update SBT. ## How was this patch tested? Pass the Jenkins with the building and existing tests. Closes #23270 from dongjoon-hyun/SPARK-26317. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- project/build.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/build.properties b/project/build.properties index d03985d980ec..23aa187fb35a 100644 --- a/project/build.properties +++ b/project/build.properties @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. # -sbt.version=0.13.17 +sbt.version=0.13.18 From 82c1ac48a37bcc929db86515bffd602c381415be Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=9F=A9=E7=94=B0=E7=94=B000222924?= Date: Mon, 10 Dec 2018 18:27:01 -0600 Subject: [PATCH 0106/1072] =?UTF-8?q?[SPARK-25696]=20The=20storage=20memor?= =?UTF-8?q?y=20displayed=20on=20spark=20Application=20UI=20is=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … incorrect. ## What changes were proposed in this pull request? In the reported heartbeat information, the unit of the memory data is bytes, which is converted by the formatBytes() function in the utils.js file before being displayed in the interface. The cardinality of the unit conversion in the formatBytes function is 1000, which should be 1024. Change the cardinality of the unit conversion in the formatBytes function to 1024. ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #22683 from httfighter/SPARK-25696. Lead-authored-by: 韩田田00222924 Co-authored-by: han.tiantian@zte.com.cn Signed-off-by: Sean Owen --- R/pkg/R/context.R | 2 +- R/pkg/R/mllib_tree.R | 6 +-- .../org/apache/spark/ui/static/utils.js | 4 +- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../spark/serializer/KryoSerializer.scala | 4 +- .../scala/org/apache/spark/util/Utils.scala | 46 +++++++++---------- .../apache/spark/MapOutputTrackerSuite.scala | 2 +- .../serializer/KryoSerializerSuite.scala | 2 +- .../apache/spark/storage/DiskStoreSuite.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 18 ++++---- docs/configuration.md | 6 +-- docs/hardware-provisioning.md | 4 +- docs/mllib-decision-tree.md | 2 +- docs/running-on-mesos.md | 2 +- docs/spark-standalone.md | 4 +- docs/streaming-kinesis-integration.md | 2 +- docs/tuning.md | 10 ++-- .../linalg/distributed/BlockMatrix.scala | 2 +- .../optimization/GradientDescentSuite.scala | 2 +- python/pyspark/rdd.py | 2 +- python/pyspark/shuffle.py | 4 +- .../spark/deploy/yarn/YarnAllocator.scala | 2 +- .../expressions/NullExpressionsSuite.scala | 4 +- .../catalyst/expressions/OrderingSuite.scala | 2 +- .../exchange/ExchangeCoordinator.scala | 14 +++--- .../execution/python/WindowInPandasExec.scala | 2 +- .../sql/execution/window/WindowExec.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../spark/sql/StatisticsCollectionSuite.scala | 12 ++--- 29 files changed, 85 insertions(+), 85 deletions(-) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index e99136723f65..0207f249f9aa 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -87,7 +87,7 @@ objectFile <- function(sc, path, minPartitions = NULL) { #' in the list are split into \code{numSlices} slices and distributed to nodes #' in the cluster. #' -#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MB), the function +#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MiB), the function #' will write it to disk and send the file name to JVM. Also to make sure each slice is not #' larger than that limit, number of slices may be increased. #' diff --git a/R/pkg/R/mllib_tree.R b/R/pkg/R/mllib_tree.R index 0e60842dd44c..9844061cfd07 100644 --- a/R/pkg/R/mllib_tree.R +++ b/R/pkg/R/mllib_tree.R @@ -157,7 +157,7 @@ print.summary.decisionTree <- function(x) { #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the @@ -382,7 +382,7 @@ setMethod("write.ml", signature(object = "GBTClassificationModel", path = "chara #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the @@ -588,7 +588,7 @@ setMethod("write.ml", signature(object = "RandomForestClassificationModel", path #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the 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 deeafad4eb5f..22985e31a780 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 @@ -40,9 +40,9 @@ function formatDuration(milliseconds) { function formatBytes(bytes, type) { if (type !== 'display') return bytes; if (bytes == 0) return '0.0 B'; - var k = 1000; + var k = 1024; var dm = 1; - var sizes = ['B', 'KB', 'MB', 'GB', 'TB', 'PB', 'EB', 'ZB', 'YB']; + var sizes = ['B', 'KiB', 'MiB', 'GiB', 'TiB', 'PiB', 'EiB', 'ZiB', 'YiB']; var i = Math.floor(Math.log(bytes) / Math.log(k)); return parseFloat((bytes / Math.pow(k, i)).toFixed(dm)) + ' ' + sizes[i]; } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 845a3d5f6d6f..696dafda6d1e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1043,7 +1043,7 @@ class SparkContext(config: SparkConf) extends Logging { // See SPARK-11227 for details. FileSystem.getLocal(hadoopConfiguration) - // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it. + // A Hadoop configuration can be about 10 KiB, which is pretty big, so broadcast it. val confBroadcast = broadcast(new SerializableConfiguration(hadoopConfiguration)) val setInputPathsFunc = (jobConf: JobConf) => FileInputFormat.setInputPaths(jobConf, path) new HadoopRDD( @@ -2723,7 +2723,7 @@ object SparkContext extends Logging { val memoryPerSlaveInt = memoryPerSlave.toInt if (sc.executorMemory > memoryPerSlaveInt) { throw new SparkException( - "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( + "Asked to launch cluster with %d MiB RAM / worker but requested %d MiB/worker".format( memoryPerSlaveInt, sc.executorMemory)) } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 1e1c27c47787..72ca0fbe667e 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -62,14 +62,14 @@ class KryoSerializer(conf: SparkConf) if (bufferSizeKb >= ByteUnit.GiB.toKiB(2)) { throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " + - s"2048 mb, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} mb.") + s"2048 MiB, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} MiB.") } private val bufferSize = ByteUnit.KiB.toBytes(bufferSizeKb).toInt val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt if (maxBufferSizeMb >= ByteUnit.GiB.toMiB(2)) { throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " + - s"2048 mb, got: + $maxBufferSizeMb mb.") + s"2048 MiB, got: + $maxBufferSizeMb MiB.") } private val maxBufferSize = ByteUnit.MiB.toBytes(maxBufferSizeMb).toInt 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 227c9e734f0a..b4ea1ee95021 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1092,41 +1092,41 @@ private[spark] object Utils extends Logging { * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of mebibytes. */ def memoryStringToMb(str: String): Int = { - // Convert to bytes, rather than directly to MB, because when no units are specified the unit + // Convert to bytes, rather than directly to MiB, because when no units are specified the unit // is assumed to be bytes (JavaUtils.byteStringAsBytes(str) / 1024 / 1024).toInt } /** - * Convert a quantity in bytes to a human-readable string such as "4.0 MB". + * Convert a quantity in bytes to a human-readable string such as "4.0 MiB". */ def bytesToString(size: Long): String = bytesToString(BigInt(size)) def bytesToString(size: BigInt): String = { - val EB = 1L << 60 - val PB = 1L << 50 - val TB = 1L << 40 - val GB = 1L << 30 - val MB = 1L << 20 - val KB = 1L << 10 - - if (size >= BigInt(1L << 11) * EB) { + val EiB = 1L << 60 + val PiB = 1L << 50 + val TiB = 1L << 40 + val GiB = 1L << 30 + val MiB = 1L << 20 + val KiB = 1L << 10 + + if (size >= BigInt(1L << 11) * EiB) { // The number is too large, show it in scientific notation. BigDecimal(size, new MathContext(3, RoundingMode.HALF_UP)).toString() + " B" } else { val (value, unit) = { - if (size >= 2 * EB) { - (BigDecimal(size) / EB, "EB") - } else if (size >= 2 * PB) { - (BigDecimal(size) / PB, "PB") - } else if (size >= 2 * TB) { - (BigDecimal(size) / TB, "TB") - } else if (size >= 2 * GB) { - (BigDecimal(size) / GB, "GB") - } else if (size >= 2 * MB) { - (BigDecimal(size) / MB, "MB") - } else if (size >= 2 * KB) { - (BigDecimal(size) / KB, "KB") + if (size >= 2 * EiB) { + (BigDecimal(size) / EiB, "EiB") + } else if (size >= 2 * PiB) { + (BigDecimal(size) / PiB, "PiB") + } else if (size >= 2 * TiB) { + (BigDecimal(size) / TiB, "TiB") + } else if (size >= 2 * GiB) { + (BigDecimal(size) / GiB, "GiB") + } else if (size >= 2 * MiB) { + (BigDecimal(size) / MiB, "MiB") + } else if (size >= 2 * KiB) { + (BigDecimal(size) / KiB, "KiB") } else { (BigDecimal(size), "B") } @@ -1157,7 +1157,7 @@ private[spark] object Utils extends Logging { } /** - * Convert a quantity in megabytes to a human-readable string such as "4.0 MB". + * Convert a quantity in megabytes to a human-readable string such as "4.0 MiB". */ def megabytesToString(megabytes: Long): String = { bytesToString(megabytes * 1024L * 1024L) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 21f481d47724..3e1a3d4f7306 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -244,7 +244,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { val newConf = new SparkConf newConf.set("spark.rpc.message.maxSize", "1") newConf.set("spark.rpc.askTimeout", "1") // Fail fast - newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "10240") // 10 KB << 1MB framesize + newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "10240") // 10 KiB << 1MiB framesize // needs TorrentBroadcast so need a SparkContext withSpark(new SparkContext("local", "MapOutputTrackerSuite", newConf)) { sc => diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 467e49026a02..8af53274d9b2 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -75,7 +75,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val thrown3 = intercept[IllegalArgumentException](newKryoInstance(conf, "2g", "3g")) assert(thrown3.getMessage.contains(kryoBufferProperty)) assert(!thrown3.getMessage.contains(kryoBufferMaxProperty)) - // test configuration with mb is supported properly + // test configuration with MiB is supported properly newKryoInstance(conf, "8m", "9m") } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala index 959cf58fa053..6f60b08088cd 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala @@ -128,7 +128,7 @@ class DiskStoreSuite extends SparkFunSuite { assert(e.getMessage === s"requirement failed: can't create a byte buffer of size ${blockData.size}" + - " since it exceeds 10.0 KB.") + " since it exceeds 10.0 KiB.") } test("block data encryption") { 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 901a724da8a1..b2ff1cce3eb0 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -133,7 +133,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.byteStringAsBytes("1p") === ByteUnit.PiB.toBytes(1)) // Overflow handling, 1073741824p exceeds Long.MAX_VALUE if converted straight to Bytes - // This demonstrates that we can have e.g 1024^3 PB without overflowing. + // This demonstrates that we can have e.g 1024^3 PiB without overflowing. assert(Utils.byteStringAsGb("1073741824p") === ByteUnit.PiB.toGiB(1073741824)) assert(Utils.byteStringAsMb("1073741824p") === ByteUnit.PiB.toMiB(1073741824)) @@ -149,7 +149,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { // Test overflow exception intercept[IllegalArgumentException] { - // This value exceeds Long.MAX when converted to TB + // This value exceeds Long.MAX when converted to TiB ByteUnit.PiB.toTiB(9223372036854775807L) } @@ -189,13 +189,13 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") assert(Utils.bytesToString(1500) === "1500.0 B") - assert(Utils.bytesToString(2000000) === "1953.1 KB") - assert(Utils.bytesToString(2097152) === "2.0 MB") - assert(Utils.bytesToString(2306867) === "2.2 MB") - assert(Utils.bytesToString(5368709120L) === "5.0 GB") - assert(Utils.bytesToString(5L * (1L << 40)) === "5.0 TB") - assert(Utils.bytesToString(5L * (1L << 50)) === "5.0 PB") - assert(Utils.bytesToString(5L * (1L << 60)) === "5.0 EB") + assert(Utils.bytesToString(2000000) === "1953.1 KiB") + assert(Utils.bytesToString(2097152) === "2.0 MiB") + assert(Utils.bytesToString(2306867) === "2.2 MiB") + assert(Utils.bytesToString(5368709120L) === "5.0 GiB") + assert(Utils.bytesToString(5L * (1L << 40)) === "5.0 TiB") + assert(Utils.bytesToString(5L * (1L << 50)) === "5.0 PiB") + assert(Utils.bytesToString(5L * (1L << 60)) === "5.0 EiB") assert(Utils.bytesToString(BigInt(1L << 11) * (1L << 60)) === "2.36E+21 B") } diff --git a/docs/configuration.md b/docs/configuration.md index 9abbb3f63490..ff9b802617f0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1384,14 +1384,14 @@ Apart from these, the following properties are also available, and may be useful

- + - + diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md index 896f9302ef30..29876a51b280 100644 --- a/docs/hardware-provisioning.md +++ b/docs/hardware-provisioning.md @@ -37,7 +37,7 @@ use the same disks as HDFS. # Memory -In general, Spark can run well with anywhere from **8 GB to hundreds of gigabytes** of memory per +In general, Spark can run well with anywhere from **8 GiB to hundreds of gigabytes** of memory per machine. In all cases, we recommend allocating only at most 75% of the memory for Spark; leave the rest for the operating system and buffer cache. @@ -47,7 +47,7 @@ Storage tab of Spark's monitoring UI (`http://:4040`) to see its si Note that memory usage is greatly affected by storage level and serialization format -- see the [tuning guide](tuning.html) for tips on how to reduce it. -Finally, note that the Java VM does not always behave well with more than 200 GB of RAM. If you +Finally, note that the Java VM does not always behave well with more than 200 GiB of RAM. If you purchase machines with more RAM than this, you can run _multiple worker JVMs per node_. In Spark's [standalone mode](spark-standalone.html), you can set the number of workers per node with the `SPARK_WORKER_INSTANCES` variable in `conf/spark-env.sh`, and the number of cores diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index ec13b81f8555..281755f4cea8 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -149,7 +149,7 @@ These parameters may be tuned. Be careful to validate on held-out test data whe * Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for any categorical feature. * **`maxMemoryInMB`**: Amount of memory to be used for collecting sufficient statistics. - * The default value is conservatively chosen to be 256 MB to allow the decision algorithm to work in most scenarios. Increasing `maxMemoryInMB` can lead to faster training (if the memory is available) by allowing fewer passes over the data. However, there may be decreasing returns as `maxMemoryInMB` grows since the amount of communication on each iteration can be proportional to `maxMemoryInMB`. + * The default value is conservatively chosen to be 256 MiB to allow the decision algorithm to work in most scenarios. Increasing `maxMemoryInMB` can lead to faster training (if the memory is available) by allowing fewer passes over the data. However, there may be decreasing returns as `maxMemoryInMB` grows since the amount of communication on each iteration can be proportional to `maxMemoryInMB`. * *Implementation details*: For faster processing, the decision tree algorithm collects statistics about groups of nodes to split (rather than 1 node at a time). The number of nodes which can be handled in one group is determined by the memory requirements (which vary per features). The `maxMemoryInMB` parameter specifies the memory limit in terms of megabytes which each worker can use for these statistics. * **`subsamplingRate`**: Fraction of the training data used for learning the decision tree. This parameter is most relevant for training ensembles of trees (using [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) and [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees)), where it can be useful to subsample the original data. For training a single decision tree, this parameter is less useful since the number of training instances is generally not the main constraint. diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index b3ba4b255b71..968d668e2c93 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -437,7 +437,7 @@ See the [configuration page](configuration.html) for information on Spark config diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 49ef2e1ce2a1..672a4d0f3199 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -60,7 +60,7 @@ Finally, the following configuration options can be passed to the master and wor - + @@ -128,7 +128,7 @@ You can optionally configure the cluster further by setting environment variable - + diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 6a52e8a7b0eb..4a1812bbb40a 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -248,5 +248,5 @@ de-aggregate records during consumption. - `InitialPositionInStream.TRIM_HORIZON` may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency. #### Kinesis retry configuration - - `spark.streaming.kinesis.retry.waitTime` : Wait time between Kinesis retries as a duration string. When reading from Amazon Kinesis, users may hit `ProvisionedThroughputExceededException`'s, when consuming faster than 5 transactions/second or, exceeding the maximum read rate of 2 MB/second. This configuration can be tweaked to increase the sleep between fetches when a fetch fails to reduce these exceptions. Default is "100ms". + - `spark.streaming.kinesis.retry.waitTime` : Wait time between Kinesis retries as a duration string. When reading from Amazon Kinesis, users may hit `ProvisionedThroughputExceededException`'s, when consuming faster than 5 transactions/second or, exceeding the maximum read rate of 2 MiB/second. This configuration can be tweaked to increase the sleep between fetches when a fetch fails to reduce these exceptions. Default is "100ms". - `spark.streaming.kinesis.retry.maxAttempts` : Max number of retries for Kinesis fetches. This config can also be used to tackle the Kinesis `ProvisionedThroughputExceededException`'s in scenarios mentioned above. It can be increased to have more number of retries for Kinesis reads. Default is 3. diff --git a/docs/tuning.md b/docs/tuning.md index cd0f9cd08136..43acacb98cbf 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -115,7 +115,7 @@ variety of workloads without requiring user expertise of how memory is divided i Although there are two relevant configurations, the typical user should not need to adjust them as the default values are applicable to most workloads: -* `spark.memory.fraction` expresses the size of `M` as a fraction of the (JVM heap space - 300MB) +* `spark.memory.fraction` expresses the size of `M` as a fraction of the (JVM heap space - 300MiB) (default 0.6). The rest of the space (40%) is reserved for user data structures, internal metadata in Spark, and safeguarding against OOM errors in the case of sparse and unusually large records. @@ -147,7 +147,7 @@ pointer-based data structures and wrapper objects. There are several ways to do Java standard library. 2. Avoid nested structures with a lot of small objects and pointers when possible. 3. Consider using numeric IDs or enumeration objects instead of strings for keys. -4. If you have less than 32 GB of RAM, set the JVM flag `-XX:+UseCompressedOops` to make pointers be +4. If you have less than 32 GiB of RAM, set the JVM flag `-XX:+UseCompressedOops` to make pointers be four bytes instead of eight. You can add these options in [`spark-env.sh`](configuration.html#environment-variables). @@ -224,8 +224,8 @@ temporary objects created during task execution. Some steps which may be useful * As an example, if your task is reading data from HDFS, the amount of memory used by the task can be estimated using the size of the data block read from HDFS. Note that the size of a decompressed block is often 2 or 3 times the - size of the block. So if we wish to have 3 or 4 tasks' worth of working space, and the HDFS block size is 128 MB, - we can estimate size of Eden to be `4*3*128MB`. + size of the block. So if we wish to have 3 or 4 tasks' worth of working space, and the HDFS block size is 128 MiB, + we can estimate size of Eden to be `4*3*128MiB`. * Monitor how the frequency and time taken by garbage collection changes with the new settings. @@ -267,7 +267,7 @@ available in `SparkContext` can greatly reduce the size of each serialized task, of launching a job over a cluster. If your tasks use any large object from the driver program inside of them (e.g. a static lookup table), consider turning it into a broadcast variable. Spark prints the serialized size of each task on the master, so you can look at that to -decide whether your tasks are too large; in general tasks larger than about 20 KB are probably +decide whether your tasks are too large; in general tasks larger than about 20 KiB are probably worth optimizing. ## Data Locality diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index e58860fea97d..e32d615af2a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -322,7 +322,7 @@ class BlockMatrix @Since("1.3.0") ( val m = numRows().toInt val n = numCols().toInt val mem = m * n / 125000 - if (mem > 500) logWarning(s"Storing this matrix will require $mem MB of memory!") + if (mem > 500) logWarning(s"Storing this matrix will require $mem MiB of memory!") val localBlocks = blocks.collect() val values = new Array[Double](m * n) localBlocks.foreach { case ((blockRowIndex, blockColIndex), submat) => 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 37eb794b0c5c..6250b0363ee3 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 @@ -190,7 +190,7 @@ class GradientDescentClusterSuite extends SparkFunSuite with LocalClusterSparkCo iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) }.cache() // If we serialize data directly in the task closure, the size of the serialized task would be - // greater than 1MB and hence Spark would throw an error. + // greater than 1MiB and hence Spark would throw an error. val (weights, loss) = GradientDescent.runMiniBatchSGD( points, new LogisticGradient, diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8bd6897df925..b6e17cab44e9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -127,7 +127,7 @@ def __new__(cls, mean, confidence, low, high): def _parse_memory(s): """ Parse a memory string in the format supported by Java (e.g. 1g, 200m) and - return the value in MB + return the value in MiB >>> _parse_memory("256m") 256 diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index bd0ac0039ffe..5d2d63850e9b 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -37,7 +37,7 @@ process = None def get_used_memory(): - """ Return the used memory in MB """ + """ Return the used memory in MiB """ global process if process is None or process._pid != os.getpid(): process = psutil.Process(os.getpid()) @@ -50,7 +50,7 @@ def get_used_memory(): except ImportError: def get_used_memory(): - """ Return the used memory in MB """ + """ Return the used memory in MiB """ if platform.system() == 'Linux': for line in open('/proc/self/status'): if line.startswith('VmRSS:'): 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 9497530805c1..d37d0d66d8ae 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 @@ -127,7 +127,7 @@ private[yarn] class YarnAllocator( private var numUnexpectedContainerRelease = 0L private val containerIdToExecutorId = new HashMap[ContainerId, String] - // Executor memory in MB. + // Executor memory in MiB. protected val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt // Additional memory overhead. protected val memoryOverhead: Int = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala index 8818d0135b29..b7ce36723081 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala @@ -160,7 +160,7 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(AtLeastNNonNulls(4, nullOnly), false, EmptyRow) } - test("Coalesce should not throw 64kb exception") { + test("Coalesce should not throw 64KiB exception") { val inputs = (1 to 2500).map(x => Literal(s"x_$x")) checkEvaluation(Coalesce(inputs), "x_1") } @@ -171,7 +171,7 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(ctx.inlinedMutableStates.size == 1) } - test("AtLeastNNonNulls should not throw 64kb exception") { + test("AtLeastNNonNulls should not throw 64KiB exception") { val inputs = (1 to 4000).map(x => Literal(s"x_$x")) checkEvaluation(AtLeastNNonNulls(1, inputs), true) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala index d0604b8eb767..94e251d90bcf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala @@ -128,7 +128,7 @@ class OrderingSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("SPARK-16845: GeneratedClass$SpecificOrdering grows beyond 64 KB") { + test("SPARK-16845: GeneratedClass$SpecificOrdering grows beyond 64 KiB") { val sortOrder = Literal("abc").asc // this is passing prior to SPARK-16845, and it should also be passing after SPARK-16845 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala index f5d93ee5fa91..e4ec76f0b9a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala @@ -73,14 +73,14 @@ import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} * greater than the target size. * * For example, we have two stages with the following pre-shuffle partition size statistics: - * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB] - * stage 2: [10 MB, 10 MB, 70 MB, 5 MB, 5 MB] - * assuming the target input size is 128 MB, we will have four post-shuffle partitions, + * stage 1: [100 MiB, 20 MiB, 100 MiB, 10MiB, 30 MiB] + * stage 2: [10 MiB, 10 MiB, 70 MiB, 5 MiB, 5 MiB] + * assuming the target input size is 128 MiB, we will have four post-shuffle partitions, * which are: - * - post-shuffle partition 0: pre-shuffle partition 0 (size 110 MB) - * - post-shuffle partition 1: pre-shuffle partition 1 (size 30 MB) - * - post-shuffle partition 2: pre-shuffle partition 2 (size 170 MB) - * - post-shuffle partition 3: pre-shuffle partition 3 and 4 (size 50 MB) + * - post-shuffle partition 0: pre-shuffle partition 0 (size 110 MiB) + * - post-shuffle partition 1: pre-shuffle partition 1 (size 30 MiB) + * - post-shuffle partition 2: pre-shuffle partition 2 (size 170 MiB) + * - post-shuffle partition 3: pre-shuffle partition 3 and 4 (size 50 MiB) */ class ExchangeCoordinator( advisoryTargetPostShuffleInputSize: Long, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala index 27bed1137e5b..82973307feef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala @@ -44,7 +44,7 @@ case class WindowInPandasExec( override def requiredChildDistribution: Seq[Distribution] = { if (partitionSpec.isEmpty) { - // Only show warning when the number of bytes is larger than 100 MB? + // Only show warning when the number of bytes is larger than 100 MiB? logWarning("No Partition Defined for Window operation! Moving all data to a single " + "partition, this can cause serious performance degradation.") AllTuples :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala index fede0f3e92d6..729b8bdb3dae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala @@ -90,7 +90,7 @@ case class WindowExec( override def requiredChildDistribution: Seq[Distribution] = { if (partitionSpec.isEmpty) { - // Only show warning when the number of bytes is larger than 100 MB? + // Only show warning when the number of bytes is larger than 100 MiB? logWarning("No Partition Defined for Window operation! Moving all data to a single " + "partition, this can cause serious performance degradation.") AllTuples :: Nil 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 fc3faa08d55f..b51c51e66350 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 @@ -1904,7 +1904,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val e = intercept[SparkException] { df.filter(filter).count() }.getMessage - assert(e.contains("grows beyond 64 KB")) + assert(e.contains("grows beyond 64 KiB")) } } 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 cb562d65b614..02dc32d5f90b 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 @@ -227,12 +227,12 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared BigInt(0) -> (("0.0 B", "0")), BigInt(100) -> (("100.0 B", "100")), BigInt(2047) -> (("2047.0 B", "2.05E+3")), - BigInt(2048) -> (("2.0 KB", "2.05E+3")), - BigInt(3333333) -> (("3.2 MB", "3.33E+6")), - BigInt(4444444444L) -> (("4.1 GB", "4.44E+9")), - BigInt(5555555555555L) -> (("5.1 TB", "5.56E+12")), - BigInt(6666666666666666L) -> (("5.9 PB", "6.67E+15")), - BigInt(1L << 10 ) * (1L << 60) -> (("1024.0 EB", "1.18E+21")), + BigInt(2048) -> (("2.0 KiB", "2.05E+3")), + BigInt(3333333) -> (("3.2 MiB", "3.33E+6")), + BigInt(4444444444L) -> (("4.1 GiB", "4.44E+9")), + BigInt(5555555555555L) -> (("5.1 TiB", "5.56E+12")), + BigInt(6666666666666666L) -> (("5.9 PiB", "6.67E+15")), + BigInt(1L << 10 ) * (1L << 60) -> (("1024.0 EiB", "1.18E+21")), BigInt(1L << 11) * (1L << 60) -> (("2.36E+21 B", "2.36E+21")) ) numbers.foreach { case (input, (expectedSize, expectedRows)) => From 05cf81e6de3d61ddb0af81cd179665693f23351f Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 10 Dec 2018 18:28:13 -0600 Subject: [PATCH 0107/1072] [SPARK-19827][R] spark.ml R API for PIC ## What changes were proposed in this pull request? Add PowerIterationCluster (PIC) in R ## How was this patch tested? Add test case Closes #23072 from huaxingao/spark-19827. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- R/pkg/NAMESPACE | 3 +- R/pkg/R/generics.R | 4 ++ R/pkg/R/mllib_clustering.R | 62 +++++++++++++++++++ R/pkg/tests/fulltests/test_mllib_clustering.R | 13 ++++ R/pkg/vignettes/sparkr-vignettes.Rmd | 14 +++++ docs/ml-clustering.md | 41 ++++++++++++ docs/sparkr.md | 1 + .../src/main/r/ml/powerIterationClustering.R | 38 ++++++++++++ .../clustering/PowerIterationClustering.scala | 4 +- .../r/PowerIterationClusteringWrapper.scala | 39 ++++++++++++ python/pyspark/ml/clustering.py | 4 +- 11 files changed, 218 insertions(+), 5 deletions(-) create mode 100644 examples/src/main/r/ml/powerIterationClustering.R create mode 100644 mllib/src/main/scala/org/apache/spark/ml/r/PowerIterationClusteringWrapper.scala diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 1f8ba0bcf1cf..cfad20db16c7 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -67,7 +67,8 @@ exportMethods("glm", "spark.fpGrowth", "spark.freqItemsets", "spark.associationRules", - "spark.findFrequentSequentialPatterns") + "spark.findFrequentSequentialPatterns", + "spark.assignClusters") # Job group lifecycle management methods export("setJobGroup", diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index eed76465221c..09d817127edd 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1479,6 +1479,10 @@ setGeneric("spark.associationRules", function(object) { standardGeneric("spark.a setGeneric("spark.findFrequentSequentialPatterns", function(data, ...) { standardGeneric("spark.findFrequentSequentialPatterns") }) +#' @rdname spark.powerIterationClustering +setGeneric("spark.assignClusters", + function(data, ...) { standardGeneric("spark.assignClusters") }) + #' @param object a fitted ML model object. #' @param path the directory where the model is saved. #' @param ... additional argument(s) passed to the method. diff --git a/R/pkg/R/mllib_clustering.R b/R/pkg/R/mllib_clustering.R index 900be685824d..7d9dcebfe70d 100644 --- a/R/pkg/R/mllib_clustering.R +++ b/R/pkg/R/mllib_clustering.R @@ -41,6 +41,12 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' @note LDAModel since 2.1.0 setClass("LDAModel", representation(jobj = "jobj")) +#' S4 class that represents a PowerIterationClustering +#' +#' @param jobj a Java object reference to the backing Scala PowerIterationClustering +#' @note PowerIterationClustering since 3.0.0 +setClass("PowerIterationClustering", slots = list(jobj = "jobj")) + #' Bisecting K-Means Clustering Model #' #' Fits a bisecting k-means clustering model against a SparkDataFrame. @@ -610,3 +616,59 @@ setMethod("write.ml", signature(object = "LDAModel", path = "character"), function(object, path, overwrite = FALSE) { write_internal(object, path, overwrite) }) + +#' PowerIterationClustering +#' +#' A scalable graph clustering algorithm. Users can call \code{spark.assignClusters} to +#' return a cluster assignment for each input vertex. +#' +# Run the PIC algorithm and returns a cluster assignment for each input vertex. +#' @param data a SparkDataFrame. +#' @param k the number of clusters to create. +#' @param initMode the initialization algorithm. +#' @param maxIter the maximum number of iterations. +#' @param sourceCol the name of the input column for source vertex IDs. +#' @param destinationCol the name of the input column for destination vertex IDs +#' @param weightCol weight column name. If this is not set or \code{NULL}, +#' we treat all instance weights as 1.0. +#' @param ... additional argument(s) passed to the method. +#' @return A dataset that contains columns of vertex id and the corresponding cluster for the id. +#' The schema of it will be: +#' \code{id: Long} +#' \code{cluster: Int} +#' @rdname spark.powerIterationClustering +#' @aliases assignClusters,PowerIterationClustering-method,SparkDataFrame-method +#' @examples +#' \dontrun{ +#' df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), +#' list(1L, 2L, 1.0), list(3L, 4L, 1.0), +#' list(4L, 0L, 0.1)), +#' schema = c("src", "dst", "weight")) +#' clusters <- spark.assignClusters(df, initMode="degree", weightCol="weight") +#' showDF(clusters) +#' } +#' @note spark.assignClusters(SparkDataFrame) since 3.0.0 +setMethod("spark.assignClusters", + signature(data = "SparkDataFrame"), + function(data, k = 2L, initMode = c("random", "degree"), maxIter = 20L, + sourceCol = "src", destinationCol = "dst", weightCol = NULL) { + if (!is.numeric(k) || k < 1) { + stop("k should be a number with value >= 1.") + } + if (!is.integer(maxIter) || maxIter <= 0) { + stop("maxIter should be a number with value > 0.") + } + initMode <- match.arg(initMode) + if (!is.null(weightCol) && weightCol == "") { + weightCol <- NULL + } else if (!is.null(weightCol)) { + weightCol <- as.character(weightCol) + } + jobj <- callJStatic("org.apache.spark.ml.r.PowerIterationClusteringWrapper", + "getPowerIterationClustering", + as.integer(k), initMode, + as.integer(maxIter), as.character(sourceCol), + as.character(destinationCol), weightCol) + object <- new("PowerIterationClustering", jobj = jobj) + dataFrame(callJMethod(object@jobj, "assignClusters", data@sdf)) + }) diff --git a/R/pkg/tests/fulltests/test_mllib_clustering.R b/R/pkg/tests/fulltests/test_mllib_clustering.R index 4110e13da494..b78a476f1d05 100644 --- a/R/pkg/tests/fulltests/test_mllib_clustering.R +++ b/R/pkg/tests/fulltests/test_mllib_clustering.R @@ -319,4 +319,17 @@ test_that("spark.posterior and spark.perplexity", { expect_equal(length(local.posterior), sum(unlist(local.posterior))) }) +test_that("spark.assignClusters", { + df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), + list(1L, 2L, 1.0), list(3L, 4L, 1.0), + list(4L, 0L, 0.1)), + schema = c("src", "dst", "weight")) + clusters <- spark.assignClusters(df, initMode = "degree", weightCol = "weight") + expected_result <- createDataFrame(list(list(4L, 1L), list(0L, 0L), + list(1L, 0L), list(3L, 1L), + list(2L, 0L)), + schema = c("id", "cluster")) + expect_equivalent(expected_result, clusters) +}) + sparkR.session.stop() diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index 1c6a03c4b9bc..cbe8c61725c8 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -549,6 +549,8 @@ SparkR supports the following machine learning models and algorithms. * Latent Dirichlet Allocation (LDA) +* Power Iteration Clustering (PIC) + #### Collaborative Filtering * Alternating Least Squares (ALS) @@ -982,6 +984,18 @@ predicted <- predict(model, df) head(predicted) ``` +#### Power Iteration Clustering + +Power Iteration Clustering (PIC) is a scalable graph clustering algorithm. `spark.assignClusters` method runs the PIC algorithm and returns a cluster assignment for each input vertex. + +```{r} +df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), + list(1L, 2L, 1.0), list(3L, 4L, 1.0), + list(4L, 0L, 0.1)), + schema = c("src", "dst", "weight")) +head(spark.assignClusters(df, initMode = "degree", weightCol = "weight")) +``` + #### FP-growth `spark.fpGrowth` executes FP-growth algorithm to mine frequent itemsets on a `SparkDataFrame`. `itemsCol` should be an array of values. diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 1186fb73d0fa..65f265256200 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -265,3 +265,44 @@ Refer to the [R API docs](api/R/spark.gaussianMixture.html) for more details. + +## Power Iteration Clustering (PIC) + +Power Iteration Clustering (PIC) is a scalable graph clustering algorithm +developed by [Lin and Cohen](http://www.cs.cmu.edu/~frank/papers/icml2010-pic-final.pdf). +From the abstract: PIC finds a very low-dimensional embedding of a dataset +using truncated power iteration on a normalized pair-wise similarity matrix of the data. + +`spark.ml`'s PowerIterationClustering implementation takes the following parameters: + +* `k`: the number of clusters to create +* `initMode`: param for the initialization algorithm +* `maxIter`: param for maximum number of iterations +* `srcCol`: param for the name of the input column for source vertex IDs +* `dstCol`: name of the input column for destination vertex IDs +* `weightCol`: Param for weight column name + +**Examples** + +
+ +
+Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.PowerIterationClustering) for more details. + +{% include_example scala/org/apache/spark/examples/ml/PowerIterationClusteringExample.scala %} +
+ +
+Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/PowerIterationClustering.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaPowerIterationClusteringExample.java %} +
+ +
+ +Refer to the [R API docs](api/R/spark.powerIterationClustering.html) for more details. + +{% include_example r/ml/powerIterationClustering.R %} +
+ +
diff --git a/docs/sparkr.md b/docs/sparkr.md index 0057f05de0ff..dbb61241007f 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -544,6 +544,7 @@ SparkR supports the following machine learning algorithms currently: * [`spark.gaussianMixture`](api/R/spark.gaussianMixture.html): [`Gaussian Mixture Model (GMM)`](ml-clustering.html#gaussian-mixture-model-gmm) * [`spark.kmeans`](api/R/spark.kmeans.html): [`K-Means`](ml-clustering.html#k-means) * [`spark.lda`](api/R/spark.lda.html): [`Latent Dirichlet Allocation (LDA)`](ml-clustering.html#latent-dirichlet-allocation-lda) +* [`spark.powerIterationClustering (PIC)`](api/R/spark.powerIterationClustering.html): [`Power Iteration Clustering (PIC)`](ml-clustering.html#power-iteration-clustering-pic) #### Collaborative Filtering diff --git a/examples/src/main/r/ml/powerIterationClustering.R b/examples/src/main/r/ml/powerIterationClustering.R new file mode 100644 index 000000000000..ba43037106d1 --- /dev/null +++ b/examples/src/main/r/ml/powerIterationClustering.R @@ -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. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/powerIterationClustering.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-powerIterationCLustering-example") + +# $example on$ +df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), + list(1L, 2L, 1.0), list(3L, 4L, 1.0), + list(4L, 0L, 0.1)), + schema = c("src", "dst", "weight")) +# assign clusters +clusters <- spark.assignClusters(df, k=2L, maxIter=20L, initMode="degree", weightCol="weight") + +showDF(arrange(clusters, clusters$id)) +# $example off$ + +sparkR.session.stop() 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 1b9a3499947d..d9a330f67e8d 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 @@ -97,8 +97,8 @@ private[clustering] trait PowerIterationClusteringParams extends Params with Has /** * :: Experimental :: * Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by - * Lin and Cohen. From the abstract: - * PIC finds a very low-dimensional embedding of a dataset using truncated power + * Lin and Cohen. From + * the abstract: PIC finds a very low-dimensional embedding of a dataset using truncated power * iteration on a normalized pair-wise similarity matrix of the data. * * This class is not yet an Estimator/Transformer, use `assignClusters` method to run the diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/PowerIterationClusteringWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/PowerIterationClusteringWrapper.scala new file mode 100644 index 000000000000..b5dfad0224ed --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/PowerIterationClusteringWrapper.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.spark.ml.clustering.PowerIterationClustering + +private[r] object PowerIterationClusteringWrapper { + def getPowerIterationClustering( + k: Int, + initMode: String, + maxIter: Int, + srcCol: String, + dstCol: String, + weightCol: String): PowerIterationClustering = { + val pic = new PowerIterationClustering() + .setK(k) + .setInitMode(initMode) + .setMaxIter(maxIter) + .setSrcCol(srcCol) + .setDstCol(dstCol) + if (weightCol != null) pic.setWeightCol(weightCol) + pic + } +} diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index d0b507ec5dad..d8a6dfb7d3a7 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -1193,8 +1193,8 @@ class PowerIterationClustering(HasMaxIter, HasWeightCol, JavaParams, JavaMLReada .. note:: Experimental Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by - `Lin and Cohen `_. From the abstract: - PIC finds a very low-dimensional embedding of a dataset using truncated power + `Lin and Cohen `_. From the + abstract: PIC finds a very low-dimensional embedding of a dataset using truncated power iteration on a normalized pair-wise similarity matrix of the data. This class is not yet an Estimator/Transformer, use :py:func:`assignClusters` method From cbe92305cd4f80725a251cf74353e8985d28306c Mon Sep 17 00:00:00 2001 From: 10129659 Date: Tue, 11 Dec 2018 09:50:21 +0800 Subject: [PATCH 0108/1072] [SPARK-26312][SQL] Replace RDDConversions.rowToRowRdd with RowEncoder to improve its conversion performance ## What changes were proposed in this pull request? `RDDConversions` would get disproportionately slower as the number of columns in the query increased, for the type of `converters` before is `scala.collection.immutable.::` which is a subtype of list. This PR removing `RDDConversions` and using `RowEncoder` to convert the Row to InternalRow. The test of `PrunedScanSuite` for 2000 columns and 20k rows takes 409 seconds before this PR, and 361 seconds after. ## How was this patch tested? Test case of `PrunedScanSuite` Closes #23262 from eatoncys/toarray. Authored-by: 10129659 Signed-off-by: Hyukjin Kwon --- .../spark/sql/execution/ExistingRDD.scala | 44 +------------------ .../datasources/DataSourceStrategy.scala | 7 ++- 2 files changed, 7 insertions(+), 44 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index e214bfd05041..49fb288fdea6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -18,54 +18,14 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Encoder, Row, SparkSession} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.{Encoder, SparkSession} +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.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.DataType - -object RDDConversions { - def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = { - data.mapPartitions { iterator => - val numColumns = outputTypes.length - val mutableRow = new GenericInternalRow(numColumns) - val converters = outputTypes.map(CatalystTypeConverters.createToCatalystConverter) - iterator.map { r => - var i = 0 - while (i < numColumns) { - mutableRow(i) = converters(i)(r.productElement(i)) - i += 1 - } - - mutableRow - } - } - } - - /** - * Convert the objects inside Row into the types Catalyst expected. - */ - def rowToRowRdd(data: RDD[Row], outputTypes: Seq[DataType]): RDD[InternalRow] = { - data.mapPartitions { iterator => - val numColumns = outputTypes.length - val mutableRow = new GenericInternalRow(numColumns) - val converters = outputTypes.map(CatalystTypeConverters.createToCatalystConverter) - iterator.map { r => - var i = 0 - while (i < numColumns) { - mutableRow(i) = converters(i)(r(i)) - i += 1 - } - - mutableRow - } - } - } -} object ExternalRDD { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index c6000442fae7..b304e2da6e1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -29,11 +29,11 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, Quali import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ @@ -416,7 +416,10 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with output: Seq[Attribute], rdd: RDD[Row]): RDD[InternalRow] = { if (relation.relation.needConversion) { - execution.RDDConversions.rowToRowRdd(rdd, output.map(_.dataType)) + val converters = RowEncoder(StructType.fromAttributes(output)) + rdd.mapPartitions { iterator => + iterator.map(converters.toRow) + } } else { rdd.asInstanceOf[RDD[InternalRow]] } From 7d5f6e8c493b96898ba01edede1522121fe945fc Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 11 Dec 2018 14:16:51 +0800 Subject: [PATCH 0109/1072] [SPARK-26293][SQL] Cast exception when having python udf in subquery ## What changes were proposed in this pull request? This is a regression introduced by https://github.com/apache/spark/pull/22104 at Spark 2.4.0. When we have Python UDF in subquery, we will hit an exception ``` Caused by: java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.AttributeReference cannot be cast to org.apache.spark.sql.catalyst.expressions.PythonUDF at scala.collection.immutable.Stream.map(Stream.scala:414) at org.apache.spark.sql.execution.python.EvalPythonExec.$anonfun$doExecute$2(EvalPythonExec.scala:98) at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:815) ... ``` https://github.com/apache/spark/pull/22104 turned `ExtractPythonUDFs` from a physical rule to optimizer rule. However, there is a difference between a physical rule and optimizer rule. A physical rule always runs once, an optimizer rule may be applied twice on a query tree even the rule is located in a batch that only runs once. For a subquery, the `OptimizeSubqueries` rule will execute the entire optimizer on the query plan inside subquery. Later on subquery will be turned to joins, and the optimizer rules will be applied to it again. Unfortunately, the `ExtractPythonUDFs` rule is not idempotent. When it's applied twice on a query plan inside subquery, it will produce a malformed plan. It extracts Python UDF from Python exec plans. This PR proposes 2 changes to be double safe: 1. `ExtractPythonUDFs` should skip python exec plans, to make the rule idempotent 2. `ExtractPythonUDFs` should skip subquery ## How was this patch tested? a new test. Closes #23248 from cloud-fan/python. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- python/pyspark/sql/tests/test_udf.py | 52 +++++++------------ .../python/ArrowEvalPythonExec.scala | 8 ++- .../python/BatchEvalPythonExec.scala | 8 ++- .../execution/python/ExtractPythonUDFs.scala | 18 +++++-- 4 files changed, 46 insertions(+), 40 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index ed298f724d55..12cf8c7de1da 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -23,7 +23,7 @@ from pyspark import SparkContext from pyspark.sql import SparkSession, Column, Row -from pyspark.sql.functions import UserDefinedFunction +from pyspark.sql.functions import UserDefinedFunction, udf from pyspark.sql.types import * from pyspark.sql.utils import AnalysisException from pyspark.testing.sqlutils import ReusedSQLTestCase, test_compiled, test_not_compiled_message @@ -102,7 +102,6 @@ def test_udf_registration_return_type_not_none(self): def test_nondeterministic_udf(self): # Test that nondeterministic UDFs are evaluated only once in chained UDF evaluations - from pyspark.sql.functions import udf import random udf_random_col = udf(lambda: int(100 * random.random()), IntegerType()).asNondeterministic() self.assertEqual(udf_random_col.deterministic, False) @@ -113,7 +112,6 @@ def test_nondeterministic_udf(self): def test_nondeterministic_udf2(self): import random - from pyspark.sql.functions import udf random_udf = udf(lambda: random.randint(6, 6), IntegerType()).asNondeterministic() self.assertEqual(random_udf.deterministic, False) random_udf1 = self.spark.catalog.registerFunction("randInt", random_udf) @@ -132,7 +130,6 @@ def test_nondeterministic_udf2(self): def test_nondeterministic_udf3(self): # regression test for SPARK-23233 - from pyspark.sql.functions import udf f = udf(lambda x: x) # Here we cache the JVM UDF instance. self.spark.range(1).select(f("id")) @@ -144,7 +141,7 @@ def test_nondeterministic_udf3(self): self.assertFalse(deterministic) def test_nondeterministic_udf_in_aggregate(self): - from pyspark.sql.functions import udf, sum + from pyspark.sql.functions import sum import random udf_random_col = udf(lambda: int(100 * random.random()), 'int').asNondeterministic() df = self.spark.range(10) @@ -181,7 +178,6 @@ def test_multiple_udfs(self): self.assertEqual(tuple(row), (6, 5)) def test_udf_in_filter_on_top_of_outer_join(self): - from pyspark.sql.functions import udf left = self.spark.createDataFrame([Row(a=1)]) right = self.spark.createDataFrame([Row(a=1)]) df = left.join(right, on='a', how='left_outer') @@ -190,7 +186,6 @@ def test_udf_in_filter_on_top_of_outer_join(self): def test_udf_in_filter_on_top_of_join(self): # regression test for SPARK-18589 - from pyspark.sql.functions import udf left = self.spark.createDataFrame([Row(a=1)]) right = self.spark.createDataFrame([Row(b=1)]) f = udf(lambda a, b: a == b, BooleanType()) @@ -199,7 +194,6 @@ def test_udf_in_filter_on_top_of_join(self): def test_udf_in_join_condition(self): # regression test for SPARK-25314 - from pyspark.sql.functions import udf left = self.spark.createDataFrame([Row(a=1)]) right = self.spark.createDataFrame([Row(b=1)]) f = udf(lambda a, b: a == b, BooleanType()) @@ -211,7 +205,7 @@ def test_udf_in_join_condition(self): def test_udf_in_left_outer_join_condition(self): # regression test for SPARK-26147 - from pyspark.sql.functions import udf, col + from pyspark.sql.functions import col left = self.spark.createDataFrame([Row(a=1)]) right = self.spark.createDataFrame([Row(b=1)]) f = udf(lambda a: str(a), StringType()) @@ -223,7 +217,6 @@ def test_udf_in_left_outer_join_condition(self): def test_udf_in_left_semi_join_condition(self): # regression test for SPARK-25314 - from pyspark.sql.functions import udf left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1)]) f = udf(lambda a, b: a == b, BooleanType()) @@ -236,7 +229,6 @@ def test_udf_in_left_semi_join_condition(self): def test_udf_and_common_filter_in_join_condition(self): # regression test for SPARK-25314 # test the complex scenario with both udf and common filter - from pyspark.sql.functions import udf left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) f = udf(lambda a, b: a == b, BooleanType()) @@ -247,7 +239,6 @@ def test_udf_and_common_filter_in_join_condition(self): def test_udf_and_common_filter_in_left_semi_join_condition(self): # regression test for SPARK-25314 # test the complex scenario with both udf and common filter - from pyspark.sql.functions import udf left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) f = udf(lambda a, b: a == b, BooleanType()) @@ -258,7 +249,6 @@ def test_udf_and_common_filter_in_left_semi_join_condition(self): def test_udf_not_supported_in_join_condition(self): # regression test for SPARK-25314 # test python udf is not supported in join type besides left_semi and inner join. - from pyspark.sql.functions import udf left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) f = udf(lambda a, b: a == b, BooleanType()) @@ -301,7 +291,7 @@ def test_broadcast_in_udf(self): def test_udf_with_filter_function(self): df = self.spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"]) - from pyspark.sql.functions import udf, col + from pyspark.sql.functions import col from pyspark.sql.types import BooleanType my_filter = udf(lambda a: a < 2, BooleanType()) @@ -310,7 +300,7 @@ def test_udf_with_filter_function(self): def test_udf_with_aggregate_function(self): df = self.spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"]) - from pyspark.sql.functions import udf, col, sum + from pyspark.sql.functions import col, sum from pyspark.sql.types import BooleanType my_filter = udf(lambda a: a == 1, BooleanType()) @@ -326,7 +316,7 @@ def test_udf_with_aggregate_function(self): self.assertEqual(sel.collect(), [Row(t=4), Row(t=3)]) def test_udf_in_generate(self): - from pyspark.sql.functions import udf, explode + from pyspark.sql.functions import explode df = self.spark.range(5) f = udf(lambda x: list(range(x)), ArrayType(LongType())) row = df.select(explode(f(*df))).groupBy().sum().first() @@ -353,7 +343,6 @@ def test_udf_in_generate(self): self.assertEqual(res[3][1], 1) def test_udf_with_order_by_and_limit(self): - from pyspark.sql.functions import udf my_copy = udf(lambda x: x, IntegerType()) df = self.spark.range(10).orderBy("id") res = df.select(df.id, my_copy(df.id).alias("copy")).limit(1) @@ -394,14 +383,14 @@ def test_non_existed_udaf(self): lambda: spark.udf.registerJavaUDAF("udaf1", "non_existed_udaf")) def test_udf_with_input_file_name(self): - from pyspark.sql.functions import udf, input_file_name + from pyspark.sql.functions import input_file_name sourceFile = udf(lambda path: path, StringType()) filePath = "python/test_support/sql/people1.json" row = self.spark.read.json(filePath).select(sourceFile(input_file_name())).first() self.assertTrue(row[0].find("people1.json") != -1) def test_udf_with_input_file_name_for_hadooprdd(self): - from pyspark.sql.functions import udf, input_file_name + from pyspark.sql.functions import input_file_name def filename(path): return path @@ -427,9 +416,6 @@ def test_udf_defers_judf_initialization(self): # This is separate of UDFInitializationTests # to avoid context initialization # when udf is called - - from pyspark.sql.functions import UserDefinedFunction - f = UserDefinedFunction(lambda x: x, StringType()) self.assertIsNone( @@ -445,8 +431,6 @@ def test_udf_defers_judf_initialization(self): ) def test_udf_with_string_return_type(self): - from pyspark.sql.functions import UserDefinedFunction - add_one = UserDefinedFunction(lambda x: x + 1, "integer") make_pair = UserDefinedFunction(lambda x: (-x, x), "struct") make_array = UserDefinedFunction( @@ -460,13 +444,11 @@ def test_udf_with_string_return_type(self): self.assertTupleEqual(expected, actual) def test_udf_shouldnt_accept_noncallable_object(self): - from pyspark.sql.functions import UserDefinedFunction - non_callable = None self.assertRaises(TypeError, UserDefinedFunction, non_callable, StringType()) def test_udf_with_decorator(self): - from pyspark.sql.functions import lit, udf + from pyspark.sql.functions import lit from pyspark.sql.types import IntegerType, DoubleType @udf(IntegerType()) @@ -523,7 +505,6 @@ def as_double(x): ) def test_udf_wrapper(self): - from pyspark.sql.functions import udf from pyspark.sql.types import IntegerType def f(x): @@ -569,7 +550,7 @@ def test_nonparam_udf_with_aggregate(self): # SPARK-24721 @unittest.skipIf(not test_compiled, test_not_compiled_message) def test_datasource_with_udf(self): - from pyspark.sql.functions import udf, lit, col + from pyspark.sql.functions import lit, col path = tempfile.mkdtemp() shutil.rmtree(path) @@ -609,8 +590,6 @@ def test_datasource_with_udf(self): # SPARK-25591 def test_same_accumulator_in_udfs(self): - from pyspark.sql.functions import udf - data_schema = StructType([StructField("a", IntegerType(), True), StructField("b", IntegerType(), True)]) data = self.spark.createDataFrame([[1, 2]], schema=data_schema) @@ -632,6 +611,15 @@ def second_udf(x): data.collect() self.assertEqual(test_accum.value, 101) + # SPARK-26293 + def test_udf_in_subquery(self): + f = udf(lambda x: x, "long") + with self.tempView("v"): + self.spark.range(1).filter(f("id") >= 0).createTempView("v") + sql = self.spark.sql + result = sql("select i from values(0L) as data(i) where i in (select id from v)") + self.assertEqual(result.collect(), [Row(i=0)]) + class UDFInitializationTests(unittest.TestCase): def tearDown(self): @@ -642,8 +630,6 @@ def tearDown(self): SparkContext._active_spark_context.stop() def test_udf_init_shouldnt_initialize_context(self): - from pyspark.sql.functions import UserDefinedFunction - UserDefinedFunction(lambda x: x, StringType()) self.assertIsNone( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala index 2b87796dc683..a5203daea9cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala @@ -60,8 +60,12 @@ private class BatchIterator[T](iter: Iterator[T], batchSize: Int) /** * A logical plan that evaluates a [[PythonUDF]]. */ -case class ArrowEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan) - extends UnaryNode +case class ArrowEvalPython( + udfs: Seq[PythonUDF], + output: Seq[Attribute], + child: LogicalPlan) extends UnaryNode { + override def producedAttributes: AttributeSet = AttributeSet(output.drop(child.output.length)) +} /** * A physical plan that evaluates a [[PythonUDF]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index b08b7e60e130..d3736d24e501 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -32,8 +32,12 @@ import org.apache.spark.sql.types.{StructField, StructType} /** * A logical plan that evaluates a [[PythonUDF]] */ -case class BatchEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan) - extends UnaryNode +case class BatchEvalPython( + udfs: Seq[PythonUDF], + output: Seq[Attribute], + child: LogicalPlan) extends UnaryNode { + override def producedAttributes: AttributeSet = AttributeSet(output.drop(child.output.length)) +} /** * A physical plan that evaluates a [[PythonUDF]] 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 90b5325919e9..380c31baa621 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 @@ -24,7 +24,7 @@ import org.apache.spark.api.python.PythonEvalType 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -131,8 +131,20 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { expressions.flatMap(collectEvaluableUDFs) } - def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case plan: LogicalPlan => extract(plan) + def apply(plan: LogicalPlan): LogicalPlan = plan match { + // SPARK-26293: A subquery will be rewritten into join later, and will go through this rule + // eventually. Here we skip subquery, as Python UDF only needs to be extracted once. + case _: Subquery => plan + + case _ => plan transformUp { + // A safe guard. `ExtractPythonUDFs` only runs once, so we will not hit `BatchEvalPython` and + // `ArrowEvalPython` in the input plan. However if we hit them, we must skip them, as we can't + // extract Python UDFs from them. + case p: BatchEvalPython => p + case p: ArrowEvalPython => p + + case plan: LogicalPlan => extract(plan) + } } /** From 4e1d859c19d3bfdfcb8acf915a97c68633b9ca95 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 11 Dec 2018 16:06:57 +0800 Subject: [PATCH 0110/1072] [SPARK-26303][SQL] Return partial results for bad JSON records ## What changes were proposed in this pull request? In the PR, I propose to return partial results from JSON datasource and JSON functions in the PERMISSIVE mode if some of JSON fields are parsed and converted to desired types successfully. The changes are made only for `StructType`. Whole bad JSON records are placed into the corrupt column specified by the `columnNameOfCorruptRecord` option or SQL config. Partial results are not returned for malformed JSON input. ## How was this patch tested? Added new UT which checks converting JSON strings with one invalid and one valid field at the end of the string. Closes #23253 from MaxGekk/json-bad-record. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Hyukjin Kwon --- docs/sql-migration-guide-upgrade.md | 4 +++- python/pyspark/sql/readwriter.py | 4 ++-- python/pyspark/sql/streaming.py | 4 ++-- .../sql/catalyst/json/JacksonParser.scala | 24 +++++++++++++++---- .../catalyst/util/BadRecordException.scala | 10 ++++++++ .../apache/spark/sql/DataFrameReader.scala | 16 ++++++------- .../sql/streaming/DataStreamReader.scala | 16 ++++++------- .../datasources/json/JsonSuite.scala | 15 +++++++++++- 8 files changed, 67 insertions(+), 26 deletions(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index f6458a9b2730..8834e8991d8c 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -37,7 +37,9 @@ displayTitle: Spark SQL Upgrading Guide - Since Spark 3.0, CSV datasource uses java.time API for parsing and generating CSV content. New formatting implementation supports date/timestamp patterns conformed to ISO 8601. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. - - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. + - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. + + - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. ## Upgrading From Spark SQL 2.3 to 2.4 diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 1d2dd4d80893..7b10512a4329 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -211,7 +211,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, set, it uses the default value, ``PERMISSIVE``. * ``PERMISSIVE`` : when it meets a corrupted record, puts the malformed string \ - into a field configured by ``columnNameOfCorruptRecord``, and sets other \ + into a field configured by ``columnNameOfCorruptRecord``, and sets malformed \ fields to ``null``. To keep corrupt records, an user can set a string type \ field named ``columnNameOfCorruptRecord`` in an user-defined schema. If a \ schema does not have the field, it drops corrupt records during parsing. \ @@ -424,7 +424,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non set, it uses the default value, ``PERMISSIVE``. * ``PERMISSIVE`` : when it meets a corrupted record, puts the malformed string \ - into a field configured by ``columnNameOfCorruptRecord``, and sets other \ + into a field configured by ``columnNameOfCorruptRecord``, and sets malformed \ fields to ``null``. To keep corrupt records, an user can set a string type \ field named ``columnNameOfCorruptRecord`` in an user-defined schema. If a \ schema does not have the field, it drops corrupt records during parsing. \ diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index d92b0d5677e2..fc23b9d99c34 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -441,7 +441,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, set, it uses the default value, ``PERMISSIVE``. * ``PERMISSIVE`` : when it meets a corrupted record, puts the malformed string \ - into a field configured by ``columnNameOfCorruptRecord``, and sets other \ + into a field configured by ``columnNameOfCorruptRecord``, and sets malformed \ fields to ``null``. To keep corrupt records, an user can set a string type \ field named ``columnNameOfCorruptRecord`` in an user-defined schema. If a \ schema does not have the field, it drops corrupt records during parsing. \ @@ -648,7 +648,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non set, it uses the default value, ``PERMISSIVE``. * ``PERMISSIVE`` : when it meets a corrupted record, puts the malformed string \ - into a field configured by ``columnNameOfCorruptRecord``, and sets other \ + into a field configured by ``columnNameOfCorruptRecord``, and sets malformed \ fields to ``null``. To keep corrupt records, an user can set a string type \ field named ``columnNameOfCorruptRecord`` in an user-defined schema. If a \ schema does not have the field, it drops corrupt records during parsing. \ 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 2357595906b1..7e3bd4df51bb 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 @@ -22,6 +22,7 @@ import java.nio.charset.MalformedInputException import scala.collection.mutable.ArrayBuffer import scala.util.Try +import scala.util.control.NonFatal import com.fasterxml.jackson.core._ @@ -29,7 +30,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._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -347,17 +347,28 @@ class JacksonParser( schema: StructType, fieldConverters: Array[ValueConverter]): InternalRow = { val row = new GenericInternalRow(schema.length) + var badRecordException: Option[Throwable] = None + while (nextUntil(parser, JsonToken.END_OBJECT)) { schema.getFieldIndex(parser.getCurrentName) match { case Some(index) => - row.update(index, fieldConverters(index).apply(parser)) - + try { + row.update(index, fieldConverters(index).apply(parser)) + } catch { + case NonFatal(e) => + badRecordException = badRecordException.orElse(Some(e)) + parser.skipChildren() + } case None => parser.skipChildren() } } - row + if (badRecordException.isEmpty) { + row + } else { + throw PartialResultException(row, badRecordException.get) + } } /** @@ -428,6 +439,11 @@ class JacksonParser( val wrappedCharException = new CharConversionException(msg) wrappedCharException.initCause(e) throw BadRecordException(() => recordLiteral(record), () => None, wrappedCharException) + case PartialResultException(row, cause) => + throw BadRecordException( + record = () => recordLiteral(record), + partialResult = () => Some(row), + cause) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BadRecordException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BadRecordException.scala index 985f0dc1cd60..d719a33929fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BadRecordException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BadRecordException.scala @@ -20,6 +20,16 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.unsafe.types.UTF8String +/** + * Exception thrown when the underlying parser returns a partial result of parsing. + * @param partialResult the partial result of parsing a bad record. + * @param cause the actual exception about why the parser cannot return full result. + */ +case class PartialResultException( + partialResult: InternalRow, + cause: Throwable) + extends Exception(cause) + /** * Exception thrown when the underlying parser meet a bad record and can't parse it. * @param record a function to return the record that cause the parser to fail 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 661fe98d8c90..9751528654ff 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 @@ -362,7 +362,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * during parsing. *
    *
  • `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a - * field configured by `columnNameOfCorruptRecord`, and sets other fields to `null`. To + * field configured by `columnNameOfCorruptRecord`, and sets malformed fields to `null`. To * keep corrupt records, an user can set a string type field named * `columnNameOfCorruptRecord` in an user-defined schema. If a schema does not have the * field, it drops corrupt records during parsing. When inferring a schema, it implicitly @@ -598,13 +598,13 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * during parsing. It supports the following case-insensitive modes. *
      *
    • `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a - * field configured by `columnNameOfCorruptRecord`, and sets other fields to `null`. To keep - * corrupt records, an user can set a string type field named `columnNameOfCorruptRecord` - * in an user-defined schema. If a schema does not have the field, it drops corrupt records - * during parsing. A record with less/more tokens than schema is not a corrupted record to - * CSV. When it meets a record having fewer tokens than the length of the schema, sets - * `null` to extra fields. When the record has more tokens than the length of the schema, - * it drops extra tokens.
    • + * field configured by `columnNameOfCorruptRecord`, and sets malformed fields to `null`. + * To keep corrupt records, an user can set a string type field named + * `columnNameOfCorruptRecord` in an user-defined schema. If a schema does not have + * the field, it drops corrupt records during parsing. A record with less/more tokens + * than schema is not a corrupted record to CSV. When it meets a record having fewer + * tokens than the length of the schema, sets `null` to extra fields. When the record + * has more tokens than the length of the schema, it drops extra tokens. *
    • `DROPMALFORMED` : ignores the whole corrupted records.
    • *
    • `FAILFAST` : throws an exception when it meets corrupted records.
    • *
    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 c8e3e1c19104..914fa90ae7e1 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 @@ -273,7 +273,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * during parsing. *
      *
    • `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a - * field configured by `columnNameOfCorruptRecord`, and sets other fields to `null`. To + * field configured by `columnNameOfCorruptRecord`, and sets malformed fields to `null`. To * keep corrupt records, an user can set a string type field named * `columnNameOfCorruptRecord` in an user-defined schema. If a schema does not have the * field, it drops corrupt records during parsing. When inferring a schema, it implicitly @@ -360,13 +360,13 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * during parsing. It supports the following case-insensitive modes. *
        *
      • `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a - * field configured by `columnNameOfCorruptRecord`, and sets other fields to `null`. To keep - * corrupt records, an user can set a string type field named `columnNameOfCorruptRecord` - * in an user-defined schema. If a schema does not have the field, it drops corrupt records - * during parsing. A record with less/more tokens than schema is not a corrupted record to - * CSV. When it meets a record having fewer tokens than the length of the schema, sets - * `null` to extra fields. When the record has more tokens than the length of the schema, - * it drops extra tokens.
      • + * field configured by `columnNameOfCorruptRecord`, and sets malformed fields to `null`. + * To keep corrupt records, an user can set a string type field named + * `columnNameOfCorruptRecord` in an user-defined schema. If a schema does not have + * the field, it drops corrupt records during parsing. A record with less/more tokens + * than schema is not a corrupted record to CSV. When it meets a record having fewer + * tokens than the length of the schema, sets `null` to extra fields. When the record + * has more tokens than the length of the schema, it drops extra tokens. *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • *
      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 dff37ca2d40f..3330de3584eb 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 @@ -248,7 +248,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer( sql("select nullstr, headers.Host from jsonTable"), - Seq(Row("", "1.abc.com"), Row("", null), Row(null, null), Row(null, null)) + Seq(Row("", "1.abc.com"), Row("", null), Row("", null), Row(null, null)) ) } @@ -2563,4 +2563,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(!files.exists(_.getName.endsWith("json"))) } } + + test("return partial result for bad records") { + val schema = "a double, b array, c string, _corrupt_record string" + val badRecords = Seq( + """{"a":"-","b":[0, 1, 2],"c":"abc"}""", + """{"a":0.1,"b":{},"c":"def"}""").toDS() + val df = spark.read.schema(schema).json(badRecords) + + checkAnswer( + df, + Row(null, Array(0, 1, 2), "abc", """{"a":"-","b":[0, 1, 2],"c":"abc"}""") :: + Row(0.1, null, "def", """{"a":0.1,"b":{},"c":"def"}""") :: Nil) + } } From bd7df6b1e129741136d09a3d29f9ffcc32ce1de3 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 11 Dec 2018 18:47:21 +0800 Subject: [PATCH 0111/1072] [SPARK-26327][SQL] Bug fix for `FileSourceScanExec` metrics update and name changing ## What changes were proposed in this pull request? As the description in [SPARK-26327](https://issues.apache.org/jira/browse/SPARK-26327), `postDriverMetricUpdates` was called on wrong place cause this bug, fix this by split the initializing of `selectedPartitions` and metrics updating logic. Add the updating logic in `inputRDD` initializing which can take effect in both code generation node and normal node. Also rename `metadataTime` to `fileListingTime` for clearer meaning. ## How was this patch tested? New test case in `SQLMetricsSuite`. Manual test: | | Before | After | |---------|:--------:|:-------:| | CodeGen |![image](https://user-images.githubusercontent.com/4833765/49741753-13c7e800-fcd2-11e8-97a8-8057b657aa3c.png)|![image](https://user-images.githubusercontent.com/4833765/49741774-1f1b1380-fcd2-11e8-98d9-78b950f4e43a.png)| | Normal |![image](https://user-images.githubusercontent.com/4833765/49741836-378b2e00-fcd2-11e8-80c3-ab462a6a3184.png)|![image](https://user-images.githubusercontent.com/4833765/49741860-4a056780-fcd2-11e8-9ef1-863de217f183.png)| Closes #23277 from xuanyuanking/SPARK-26327. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../sql/execution/DataSourceScanExec.scala | 28 +++++++++++++------ .../execution/metric/SQLMetricsSuite.scala | 15 ++++++++++ 2 files changed, 34 insertions(+), 9 deletions(-) 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 b29d5c76c5f3..c0fa4e777b49 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 @@ -167,19 +167,14 @@ case class FileSourceScanExec( partitionSchema = relation.partitionSchema, relation.sparkSession.sessionState.conf) + private var fileListingTime = 0L + @transient private lazy val selectedPartitions: Seq[PartitionDirectory] = { val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) val startTime = System.nanoTime() val ret = relation.location.listFiles(partitionFilters, dataFilters) val timeTakenMs = ((System.nanoTime() - startTime) + optimizerMetadataTimeNs) / 1000 / 1000 - - metrics("numFiles").add(ret.map(_.files.size.toLong).sum) - metrics("metadataTime").add(timeTakenMs) - - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, - metrics("numFiles") :: metrics("metadataTime") :: Nil) - + fileListingTime = timeTakenMs ret } @@ -291,6 +286,8 @@ case class FileSourceScanExec( } private lazy val inputRDD: RDD[InternalRow] = { + // Update metrics for taking effect in both code generation node and normal node. + updateDriverMetrics() val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( sparkSession = relation.sparkSession, @@ -316,7 +313,7 @@ case class FileSourceScanExec( override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of files"), - "metadataTime" -> SQLMetrics.createMetric(sparkContext, "metadata time (ms)"), + "fileListingTime" -> SQLMetrics.createMetric(sparkContext, "file listing time (ms)"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) protected override def doExecute(): RDD[InternalRow] = { @@ -507,6 +504,19 @@ case class FileSourceScanExec( } } + /** + * Send the updated metrics to driver, while this function calling, selectedPartitions has + * been initialized. See SPARK-26327 for more detail. + */ + private def updateDriverMetrics() = { + metrics("numFiles").add(selectedPartitions.map(_.files.size.toLong).sum) + metrics("fileListingTime").add(fileListingTime) + + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, + metrics("numFiles") :: metrics("fileListingTime") :: Nil) + } + override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, 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 2251607e76af..4a80638f6885 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 @@ -636,4 +636,19 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared assert(filters.head.metrics("numOutputRows").value == 1) } } + + test("SPARK-26327: FileSourceScanExec metrics") { + withTable("testDataForScan") { + spark.range(10).selectExpr("id", "id % 3 as p") + .write.partitionBy("p").saveAsTable("testDataForScan") + // The execution plan only has 1 FileScan node. + val df = spark.sql( + "SELECT * FROM testDataForScan WHERE p = 1") + testSparkPlanMetrics(df, 1, Map( + 0L -> (("Scan parquet default.testdataforscan", Map( + "number of output rows" -> 3L, + "number of files" -> 2L)))) + ) + } + } } From a3bbca98d7d120f22727a55cdc448608e6bb9fad Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 11 Dec 2018 21:08:39 +0800 Subject: [PATCH 0112/1072] [SPARK-26265][CORE] Fix deadlock in BytesToBytesMap.MapIterator when locking both BytesToBytesMap.MapIterator and TaskMemoryManager ## What changes were proposed in this pull request? In `BytesToBytesMap.MapIterator.advanceToNextPage`, We will first lock this `MapIterator` and then `TaskMemoryManager` when going to free a memory page by calling `freePage`. At the same time, it is possibly that another memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it acquires memory and causes spilling on this `MapIterator`. So it ends with the `MapIterator` object holds lock to the `MapIterator` object and waits for lock on `TaskMemoryManager`, and the other consumer holds lock to `TaskMemoryManager` and waits for lock on the `MapIterator` object. To avoid deadlock here, this patch proposes to keep reference to the page to free and free it after releasing the lock of `MapIterator`. ## How was this patch tested? Added test and manually test by running the test 100 times to make sure there is no deadlock. Closes #23272 from viirya/SPARK-26265. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../spark/unsafe/map/BytesToBytesMap.java | 86 ++++++++++--------- .../spark/memory/TestMemoryConsumer.java | 4 +- .../map/AbstractBytesToBytesMapSuite.java | 47 ++++++++++ 3 files changed, 96 insertions(+), 41 deletions(-) diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 405e52946415..fbba002f1f80 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -255,11 +255,18 @@ private MapIterator(int numRecords, Location loc, boolean destructive) { } private void advanceToNextPage() { + // SPARK-26265: We will first lock this `MapIterator` and then `TaskMemoryManager` when going + // to free a memory page by calling `freePage`. At the same time, it is possibly that another + // memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it + // acquires memory and causes spilling on this `MapIterator`. To avoid deadlock here, we keep + // reference to the page to free and free it after releasing the lock of `MapIterator`. + MemoryBlock pageToFree = null; + synchronized (this) { int nextIdx = dataPages.indexOf(currentPage) + 1; if (destructive && currentPage != null) { dataPages.remove(currentPage); - freePage(currentPage); + pageToFree = currentPage; nextIdx --; } if (dataPages.size() > nextIdx) { @@ -283,6 +290,9 @@ private void advanceToNextPage() { } } } + if (pageToFree != null) { + freePage(pageToFree); + } } @Override @@ -329,52 +339,50 @@ public Location next() { } } - public long spill(long numBytes) throws IOException { - synchronized (this) { - if (!destructive || dataPages.size() == 1) { - return 0L; - } + public synchronized long spill(long numBytes) throws IOException { + if (!destructive || dataPages.size() == 1) { + return 0L; + } - updatePeakMemoryUsed(); + updatePeakMemoryUsed(); - // TODO: use existing ShuffleWriteMetrics - ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); + // TODO: use existing ShuffleWriteMetrics + ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - long released = 0L; - while (dataPages.size() > 0) { - MemoryBlock block = dataPages.getLast(); - // The currentPage is used, cannot be released - if (block == currentPage) { - break; - } + long released = 0L; + while (dataPages.size() > 0) { + MemoryBlock block = dataPages.getLast(); + // The currentPage is used, cannot be released + if (block == currentPage) { + break; + } - Object base = block.getBaseObject(); - long offset = block.getBaseOffset(); - int numRecords = UnsafeAlignedOffset.getSize(base, offset); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - offset += uaoSize; - final UnsafeSorterSpillWriter writer = - new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords); - while (numRecords > 0) { - int length = UnsafeAlignedOffset.getSize(base, offset); - writer.write(base, offset + uaoSize, length, 0); - offset += uaoSize + length + 8; - numRecords--; - } - writer.close(); - spillWriters.add(writer); + Object base = block.getBaseObject(); + long offset = block.getBaseOffset(); + int numRecords = UnsafeAlignedOffset.getSize(base, offset); + int uaoSize = UnsafeAlignedOffset.getUaoSize(); + offset += uaoSize; + final UnsafeSorterSpillWriter writer = + new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords); + while (numRecords > 0) { + int length = UnsafeAlignedOffset.getSize(base, offset); + writer.write(base, offset + uaoSize, length, 0); + offset += uaoSize + length + 8; + numRecords--; + } + writer.close(); + spillWriters.add(writer); - dataPages.removeLast(); - released += block.size(); - freePage(block); + dataPages.removeLast(); + released += block.size(); + freePage(block); - if (released >= numBytes) { - break; - } + if (released >= numBytes) { + break; } - - return released; } + + return released; } @Override diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java index 0bbaea6b834b..6aa577d1bf79 100644 --- a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java +++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java @@ -38,12 +38,12 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { return used; } - void use(long size) { + public void use(long size) { long got = taskMemoryManager.acquireExecutionMemory(size, this); used += got; } - void free(long size) { + public void free(long size) { used -= size; taskMemoryManager.releaseExecutionMemory(size, this); } diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index a11cd535b547..e5fbafc23d95 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -33,6 +33,8 @@ import org.apache.spark.SparkConf; import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.network.util.JavaUtils; @@ -678,4 +680,49 @@ public void testPeakMemoryUsed() { } } + @Test + public void avoidDeadlock() throws InterruptedException { + memoryManager.limit(PAGE_SIZE_BYTES); + MemoryMode mode = useOffHeapMemoryAllocator() ? MemoryMode.OFF_HEAP: MemoryMode.ON_HEAP; + TestMemoryConsumer c1 = new TestMemoryConsumer(taskMemoryManager, mode); + BytesToBytesMap map = + new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024); + + Thread thread = new Thread(() -> { + int i = 0; + long used = 0; + while (i < 10) { + c1.use(10000000); + used += 10000000; + i++; + } + c1.free(used); + }); + + try { + int i; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8); + loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + } + + // Starts to require memory at another memory consumer. + thread.start(); + + BytesToBytesMap.MapIterator iter = map.destructiveIterator(); + for (i = 0; i < 1024; i++) { + iter.next(); + } + assertFalse(iter.hasNext()); + } finally { + map.free(); + thread.join(); + for (File spillFile : spillFilesCreated) { + assertFalse("Spill file " + spillFile.getPath() + " was not cleaned up", + spillFile.exists()); + } + } + } + } From 5c67a9a7fa29836fc825504bbcc3c3fc820009c6 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 11 Dec 2018 21:23:27 +0800 Subject: [PATCH 0113/1072] [SPARK-26316][SPARK-21052] Revert hash join metrics in that causes performance degradation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? The wrong implementation in the hash join metrics in [spark 21052](https://issues.apache.org/jira/browse/SPARK-21052) caused significant performance degradation in TPC-DS. And the result is [here](https://docs.google.com/spreadsheets/d/18a5BdOlmm8euTaRodyeWum9yu92mbWWu6JbhGXtr7yE/edit#gid=0) in TPC-DS 1TB scale. So we currently partial revert 21052. **Cluster info:**   | Master Node | Worker Nodes -- | -- | -- Node | 1x | 4x Processor | Intel(R) Xeon(R) Platinum 8170 CPU 2.10GHz | Intel(R) Xeon(R) Platinum 8180 CPU 2.50GHz Memory | 192 GB | 384 GB Storage Main | 8 x 960G SSD | 8 x 960G SSD Network | 10Gbe |   Role | CM Management NameNodeSecondary NameNodeResource ManagerHive Metastore Server | DataNodeNodeManager OS Version | CentOS 7.2 | CentOS 7.2 Hadoop | Apache Hadoop 2.7.5 | Apache Hadoop 2.7.5 Hive | Apache Hive 2.2.0 |   Spark | Apache Spark 2.1.0  & Apache Spark2.3.0 |   JDK  version | 1.8.0_112 | 1.8.0_112 **Related parameters setting:** Component | Parameter | Value -- | -- | -- Yarn Resource Manager | yarn.scheduler.maximum-allocation-mb | 120GB   | yarn.scheduler.minimum-allocation-mb | 1GB   | yarn.scheduler.maximum-allocation-vcores | 121   | Yarn.resourcemanager.scheduler.class | Fair Scheduler Yarn Node Manager | yarn.nodemanager.resource.memory-mb | 120GB   | yarn.nodemanager.resource.cpu-vcores | 121 Spark | spark.executor.memory | 110GB   | spark.executor.cores | 50 ## How was this patch tested? N/A Closes #23269 from JkSelf/partial-revert-21052. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../joins/BroadcastHashJoinExec.scala | 28 +----- .../spark/sql/execution/joins/HashJoin.scala | 8 +- .../sql/execution/joins/HashedRelation.scala | 35 ------- .../joins/ShuffledHashJoinExec.scala | 6 +- .../execution/metric/SQLMetricsSuite.scala | 94 +------------------ 5 files changed, 6 insertions(+), 165 deletions(-) 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 a6f3ea47c849..fd4a7897c7ad 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 @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Dist import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.{BooleanType, LongType} -import org.apache.spark.util.TaskCompletionListener /** * Performs an inner hash join of two child relations. When the output RDD of this operator is @@ -48,8 +47,7 @@ case class BroadcastHashJoinExec( extends BinaryExecNode with HashJoin with CodegenSupport { override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe")) + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) @@ -63,13 +61,12 @@ case class BroadcastHashJoinExec( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val avgHashProbe = longMetric("avgHashProbe") val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]() streamedPlan.execute().mapPartitions { streamedIter => val hashed = broadcastRelation.value.asReadOnlyCopy() TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) - join(streamedIter, hashed, numOutputRows, avgHashProbe) + join(streamedIter, hashed, numOutputRows) } } @@ -111,23 +108,6 @@ case class BroadcastHashJoinExec( } } - /** - * Returns the codes used to add a task completion listener to update avg hash probe - * at the end of the task. - */ - private def genTaskListener(avgHashProbe: String, relationTerm: String): String = { - val listenerClass = classOf[TaskCompletionListener].getName - val taskContextClass = classOf[TaskContext].getName - s""" - | $taskContextClass$$.MODULE$$.get().addTaskCompletionListener(new $listenerClass() { - | @Override - | public void onTaskCompletion($taskContextClass context) { - | $avgHashProbe.set($relationTerm.getAverageProbesPerLookup()); - | } - | }); - """.stripMargin - } - /** * Returns a tuple of Broadcast of HashedRelation and the variable name for it. */ @@ -137,15 +117,11 @@ case class BroadcastHashJoinExec( val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation) val clsName = broadcastRelation.value.getClass.getName - // At the end of the task, we update the avg hash probe. - val avgHashProbe = metricTerm(ctx, "avgHashProbe") - // Inline mutable state since not many join operations in a task val relationTerm = ctx.addMutableState(clsName, "relation", v => s""" | $v = (($clsName) $broadcast.value()).asReadOnlyCopy(); | incPeakExecutionMemory($v.estimatedSize()); - | ${genTaskListener(avgHashProbe, v)} """.stripMargin, forceInline = true) (broadcastRelation, relationTerm) } 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 dab873bf9b9a..1aef5f686426 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ @@ -194,8 +193,7 @@ trait HashJoin { protected def join( streamedIter: Iterator[InternalRow], hashed: HashedRelation, - numOutputRows: SQLMetric, - avgHashProbe: SQLMetric): Iterator[InternalRow] = { + numOutputRows: SQLMetric): Iterator[InternalRow] = { val joinedIter = joinType match { case _: InnerLike => @@ -213,10 +211,6 @@ trait HashJoin { s"BroadcastHashJoin should not take $x as the JoinType") } - // At the end of the task, we update the avg hash probe. - TaskContext.get().addTaskCompletionListener[Unit](_ => - avgHashProbe.set(hashed.getAverageProbesPerLookup)) - val resultProj = createResultProjection joinedIter.map { r => numOutputRows += 1 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 b1ff6e83acc2..7c21062c4cec 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 @@ -80,11 +80,6 @@ private[execution] sealed trait HashedRelation extends KnownSizeEstimation { * Release any used resources. */ def close(): Unit - - /** - * Returns the average number of probes per key lookup. - */ - def getAverageProbesPerLookup: Double } private[execution] object HashedRelation { @@ -279,8 +274,6 @@ private[joins] class UnsafeHashedRelation( override def read(kryo: Kryo, in: Input): Unit = Utils.tryOrIOException { read(() => in.readInt(), () => in.readLong(), in.readBytes) } - - override def getAverageProbesPerLookup: Double = binaryMap.getAverageProbesPerLookup } private[joins] object UnsafeHashedRelation { @@ -395,10 +388,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap // The number of unique keys. private var numKeys = 0L - // Tracking average number of probes per key lookup. - private var numKeyLookups = 0L - private var numProbes = 0L - // needed by serializer def this() = { this( @@ -483,8 +472,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def getValue(key: Long, resultRow: UnsafeRow): UnsafeRow = { if (isDense) { - numKeyLookups += 1 - numProbes += 1 if (key >= minKey && key <= maxKey) { val value = array((key - minKey).toInt) if (value > 0) { @@ -493,14 +480,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } else { var pos = firstSlot(key) - numKeyLookups += 1 - numProbes += 1 while (array(pos + 1) != 0) { if (array(pos) == key) { return getRow(array(pos + 1), resultRow) } pos = nextSlot(pos) - numProbes += 1 } } null @@ -528,8 +512,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def get(key: Long, resultRow: UnsafeRow): Iterator[UnsafeRow] = { if (isDense) { - numKeyLookups += 1 - numProbes += 1 if (key >= minKey && key <= maxKey) { val value = array((key - minKey).toInt) if (value > 0) { @@ -538,14 +520,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } else { var pos = firstSlot(key) - numKeyLookups += 1 - numProbes += 1 while (array(pos + 1) != 0) { if (array(pos) == key) { return valueIter(array(pos + 1), resultRow) } pos = nextSlot(pos) - numProbes += 1 } } null @@ -585,11 +564,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap private def updateIndex(key: Long, address: Long): Unit = { var pos = firstSlot(key) assert(numKeys < array.length / 2) - numKeyLookups += 1 - numProbes += 1 while (array(pos) != key && array(pos + 1) != 0) { pos = nextSlot(pos) - numProbes += 1 } if (array(pos + 1) == 0) { // this is the first value for this key, put the address in array. @@ -721,8 +697,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap writeLong(maxKey) writeLong(numKeys) writeLong(numValues) - writeLong(numKeyLookups) - writeLong(numProbes) writeLong(array.length) writeLongArray(writeBuffer, array, array.length) @@ -764,8 +738,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap maxKey = readLong() numKeys = readLong() numValues = readLong() - numKeyLookups = readLong() - numProbes = readLong() val length = readLong().toInt mask = length - 2 @@ -783,11 +755,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap override def read(kryo: Kryo, in: Input): Unit = { read(() => in.readBoolean(), () => in.readLong(), in.readBytes) } - - /** - * Returns the average number of probes per key lookup. - */ - def getAverageProbesPerLookup: Double = numProbes.toDouble / numKeyLookups } private[joins] class LongHashedRelation( @@ -839,8 +806,6 @@ private[joins] class LongHashedRelation( resultRow = new UnsafeRow(nFields) map = in.readObject().asInstanceOf[LongToUnsafeRowMap] } - - override def getAverageProbesPerLookup: Double = map.getAverageProbesPerLookup } /** 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 2b59ed6e4d16..524804d61e59 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 @@ -42,8 +42,7 @@ case class ShuffledHashJoinExec( 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"), - "avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe")) + "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) override def requiredChildDistribution: Seq[Distribution] = HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil @@ -63,10 +62,9 @@ case class ShuffledHashJoinExec( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val avgHashProbe = longMetric("avgHashProbe") streamedPlan.execute().zipPartitions(buildPlan.execute()) { (streamIter, buildIter) => val hashed = buildHashedRelation(buildIter) - join(streamIter, hashed, numOutputRows, avgHashProbe) + join(streamIter, hashed, numOutputRows) } } } 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 4a80638f6885..f6495496a58e 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 @@ -261,50 +261,6 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared ) } - test("BroadcastHashJoin metrics: track avg probe") { - // The executed plan looks like: - // Project [a#210, b#211, b#221] - // +- BroadcastHashJoin [a#210], [a#220], Inner, BuildRight - // :- Project [_1#207 AS a#210, _2#208 AS b#211] - // : +- Filter isnotnull(_1#207) - // : +- LocalTableScan [_1#207, _2#208] - // +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, binary, true])) - // +- Project [_1#217 AS a#220, _2#218 AS b#221] - // +- Filter isnotnull(_1#217) - // +- LocalTableScan [_1#217, _2#218] - // - // Assume the execution plan with node id is - // WholeStageCodegen disabled: - // Project(nodeId = 0) - // BroadcastHashJoin(nodeId = 1) - // ...(ignored) - // - // WholeStageCodegen enabled: - // WholeStageCodegen(nodeId = 0) - // Project(nodeId = 1) - // BroadcastHashJoin(nodeId = 2) - // Project(nodeId = 3) - // Filter(nodeId = 4) - // ...(ignored) - Seq(true, false).foreach { enableWholeStage => - val df1 = generateRandomBytesDF() - val df2 = generateRandomBytesDF() - val df = df1.join(broadcast(df2), "a") - val nodeIds = if (enableWholeStage) { - Set(2L) - } else { - Set(1L) - } - val metrics = getSparkPlanMetrics(df, 2, nodeIds, enableWholeStage).get - nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") - probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => - assert(probe.toDouble > 1.0) - } - } - } - } - test("ShuffledHashJoin metrics") { withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "40", "spark.sql.shuffle.partitions" -> "2", @@ -323,8 +279,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df = df1.join(df2, "key") testSparkPlanMetrics(df, 1, Map( 1L -> (("ShuffledHashJoin", Map( - "number of output rows" -> 2L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"))), + "number of output rows" -> 2L))), 2L -> (("Exchange", Map( "shuffle records written" -> 2L, "records read" -> 2L))), @@ -335,53 +290,6 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } } - test("ShuffledHashJoin metrics: track avg probe") { - // The executed plan looks like: - // Project [a#308, b#309, b#319] - // +- ShuffledHashJoin [a#308], [a#318], Inner, BuildRight - // :- Exchange hashpartitioning(a#308, 2) - // : +- Project [_1#305 AS a#308, _2#306 AS b#309] - // : +- Filter isnotnull(_1#305) - // : +- LocalTableScan [_1#305, _2#306] - // +- Exchange hashpartitioning(a#318, 2) - // +- Project [_1#315 AS a#318, _2#316 AS b#319] - // +- Filter isnotnull(_1#315) - // +- LocalTableScan [_1#315, _2#316] - // - // Assume the execution plan with node id is - // WholeStageCodegen disabled: - // Project(nodeId = 0) - // ShuffledHashJoin(nodeId = 1) - // ...(ignored) - // - // WholeStageCodegen enabled: - // WholeStageCodegen(nodeId = 0) - // Project(nodeId = 1) - // ShuffledHashJoin(nodeId = 2) - // ...(ignored) - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "5000000", - "spark.sql.shuffle.partitions" -> "2", - "spark.sql.join.preferSortMergeJoin" -> "false") { - Seq(true, false).foreach { enableWholeStage => - val df1 = generateRandomBytesDF(65535 * 5) - val df2 = generateRandomBytesDF(65535) - val df = df1.join(df2, "a") - val nodeIds = if (enableWholeStage) { - Set(2L) - } else { - Set(1L) - } - val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get - nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") - probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => - assert(probe.toDouble > 1.0) - } - } - } - } - } - test("BroadcastHashJoin(outer) metrics") { val df1 = Seq((1, "a"), (1, "b"), (4, "c")).toDF("key", "value") val df2 = Seq((1, "a"), (1, "b"), (2, "c"), (3, "d")).toDF("key2", "value") From d811369ce23186cbb3208ad665e15408e13fea87 Mon Sep 17 00:00:00 2001 From: liuxian Date: Tue, 11 Dec 2018 09:12:17 -0800 Subject: [PATCH 0114/1072] [SPARK-26300][SS] Remove a redundant `checkForStreaming` call ## What changes were proposed in this pull request? If `checkForContinuous` is called ( `checkForStreaming` is called in `checkForContinuous` ), the `checkForStreaming` mothod will be called twice in `createQuery` , this is not necessary, and the `checkForStreaming` method has a lot of statements, so it's better to remove one of them. ## How was this patch tested? Existing unit tests in `StreamingQueryManagerSuite` and `ContinuousAggregationSuite` Closes #23251 from 10110346/isUnsupportedOperationCheckEnabled. Authored-by: liuxian Signed-off-by: Dongjoon Hyun --- .../spark/sql/streaming/StreamingQueryManager.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index d9fe1a992a09..881cd96cc9dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -246,9 +246,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo val analyzedPlan = df.queryExecution.analyzed df.queryExecution.assertAnalyzed() - if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) { - UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode) - } + val operationCheckEnabled = sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " + @@ -257,7 +255,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo (sink, trigger) match { case (v2Sink: StreamingWriteSupportProvider, trigger: ContinuousTrigger) => - if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) { + if (operationCheckEnabled) { UnsupportedOperationChecker.checkForContinuous(analyzedPlan, outputMode) } new StreamingQueryWrapper(new ContinuousExecution( @@ -272,6 +270,9 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo extraOptions, deleteCheckpointOnStop)) case _ => + if (operationCheckEnabled) { + UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode) + } new StreamingQueryWrapper(new MicroBatchExecution( sparkSession, userSpecifiedName.orNull, From 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 11 Dec 2018 13:49:52 -0800 Subject: [PATCH 0115/1072] [SPARK-26239] File-based secret key loading for SASL. This proposes an alternative way to load secret keys into a Spark application that is running on Kubernetes. Instead of automatically generating the secret, the secret key can reside in a file that is shared between both the driver and executor containers. Unit tests. Closes #23252 from mccheah/auth-secret-with-file. Authored-by: mcheah Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/SecurityManager.scala | 34 +++++++++- .../scala/org/apache/spark/SparkEnv.scala | 4 +- .../spark/internal/config/package.scala | 31 +++++++++ .../apache/spark/SecurityManagerSuite.scala | 66 ++++++++++++++++++- docs/security.md | 44 +++++++++++++ .../features/BasicExecutorFeatureStep.scala | 16 +++-- .../BasicExecutorFeatureStepSuite.scala | 23 +++++++ 7 files changed, 205 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 96e4b53b2418..15783c952c23 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -17,8 +17,11 @@ package org.apache.spark +import java.io.File import java.net.{Authenticator, PasswordAuthentication} import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.Files +import java.util.Base64 import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} @@ -43,7 +46,8 @@ import org.apache.spark.util.Utils */ private[spark] class SecurityManager( sparkConf: SparkConf, - val ioEncryptionKey: Option[Array[Byte]] = None) + val ioEncryptionKey: Option[Array[Byte]] = None, + authSecretFileConf: ConfigEntry[Option[String]] = AUTH_SECRET_FILE) extends Logging with SecretKeyHolder { import SecurityManager._ @@ -328,6 +332,7 @@ private[spark] class SecurityManager( .orElse(Option(secretKey)) .orElse(Option(sparkConf.getenv(ENV_AUTH_SECRET))) .orElse(sparkConf.getOption(SPARK_AUTH_SECRET_CONF)) + .orElse(secretKeyFromFile()) .getOrElse { throw new IllegalArgumentException( s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config") @@ -348,7 +353,6 @@ private[spark] class SecurityManager( */ def initializeAuth(): Unit = { import SparkMasterRegex._ - val k8sRegex = "k8s.*".r if (!sparkConf.get(NETWORK_AUTH_ENABLED)) { return @@ -371,7 +375,14 @@ private[spark] class SecurityManager( return } - secretKey = Utils.createSecret(sparkConf) + if (sparkConf.get(AUTH_SECRET_FILE_DRIVER).isDefined != + sparkConf.get(AUTH_SECRET_FILE_EXECUTOR).isDefined) { + throw new IllegalArgumentException( + "Invalid secret configuration: Secret files must be specified for both the driver and the" + + " executors, not only one or the other.") + } + + secretKey = secretKeyFromFile().getOrElse(Utils.createSecret(sparkConf)) if (storeInUgi) { val creds = new Credentials() @@ -380,6 +391,22 @@ private[spark] class SecurityManager( } } + private def secretKeyFromFile(): Option[String] = { + sparkConf.get(authSecretFileConf).flatMap { secretFilePath => + sparkConf.getOption(SparkLauncher.SPARK_MASTER).map { + case k8sRegex() => + val secretFile = new File(secretFilePath) + require(secretFile.isFile, s"No file found containing the secret key at $secretFilePath.") + val base64Key = Base64.getEncoder.encodeToString(Files.readAllBytes(secretFile.toPath)) + require(!base64Key.isEmpty, s"Secret key from file located at $secretFilePath is empty.") + base64Key + case _ => + throw new IllegalArgumentException( + "Secret keys provided via files is only allowed in Kubernetes mode.") + } + } + } + // Default SecurityManager only has a single secret key, so ignore appId. override def getSaslUser(appId: String): String = getSaslUser() override def getSecretKey(appId: String): String = getSecretKey() @@ -387,6 +414,7 @@ private[spark] class SecurityManager( private[spark] object SecurityManager { + val k8sRegex = "k8s.*".r val SPARK_AUTH_CONF = NETWORK_AUTH_ENABLED.key val SPARK_AUTH_SECRET_CONF = "spark.authenticate.secret" // This is used to set auth secret to an executor's env variable. It should have the same diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 66038eeaea54..de0c8579d9ac 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -232,8 +232,8 @@ object SparkEnv extends Logging { if (isDriver) { assert(listenerBus != null, "Attempted to create driver SparkEnv with null listener bus!") } - - val securityManager = new SecurityManager(conf, ioEncryptionKey) + val authSecretFileConf = if (isDriver) AUTH_SECRET_FILE_DRIVER else AUTH_SECRET_FILE_EXECUTOR + val securityManager = new SecurityManager(conf, ioEncryptionKey, authSecretFileConf) if (isDriver) { securityManager.initializeAuth() } 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 85bb557abef5..f1c1c034df49 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 @@ -424,6 +424,37 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val AUTH_SECRET_FILE = + ConfigBuilder("spark.authenticate.secret.file") + .doc("Path to a file that contains the authentication secret to use. The secret key is " + + "loaded from this path on both the driver and the executors if overrides are not set for " + + "either entity (see below). File-based secret keys are only allowed when using " + + "Kubernetes.") + .stringConf + .createOptional + + private[spark] val AUTH_SECRET_FILE_DRIVER = + ConfigBuilder("spark.authenticate.secret.driver.file") + .doc("Path to a file that contains the authentication secret to use. Loaded by the " + + "driver. In Kubernetes client mode it is often useful to set a different secret " + + "path for the driver vs. the executors, since the driver may not be running in " + + "a pod unlike the executors. If this is set, an accompanying secret file must " + + "be specified for the executors. The fallback configuration allows the same path to be " + + "used for both the driver and the executors when running in cluster mode. File-based " + + "secret keys are only allowed when using Kubernetes.") + .fallbackConf(AUTH_SECRET_FILE) + + private[spark] val AUTH_SECRET_FILE_EXECUTOR = + ConfigBuilder("spark.authenticate.secret.executor.file") + .doc("Path to a file that contains the authentication secret to use. Loaded by the " + + "executors only. In Kubernetes client mode it is often useful to set a different " + + "secret path for the driver vs. the executors, since the driver may not be running " + + "in a pod unlike the executors. If this is set, an accompanying secret file must be " + + "specified for the executors. The fallback configuration allows the same path to be " + + "used for both the driver and the executors when running in cluster mode. File-based " + + "secret keys are only allowed when using Kubernetes.") + .fallbackConf(AUTH_SECRET_FILE) + private[spark] val NETWORK_ENCRYPTION_ENABLED = ConfigBuilder("spark.network.crypto.enabled") .booleanConf diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index eec8004fc94f..e9061f4e7beb 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark import java.io.File import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.Files import java.security.PrivilegedExceptionAction +import java.util.Base64 import org.apache.hadoop.security.UserGroupInformation @@ -395,9 +397,54 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(keyFromEnv === new SecurityManager(conf2).getSecretKey()) } + test("use executor-specific secret file configuration.") { + val secretFileFromDriver = createTempSecretFile("driver-secret") + val secretFileFromExecutor = createTempSecretFile("executor-secret") + val conf = new SparkConf() + .setMaster("k8s://127.0.0.1") + .set(AUTH_SECRET_FILE_DRIVER, Some(secretFileFromDriver.getAbsolutePath)) + .set(AUTH_SECRET_FILE_EXECUTOR, Some(secretFileFromExecutor.getAbsolutePath)) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr = new SecurityManager(conf, authSecretFileConf = AUTH_SECRET_FILE_EXECUTOR) + assert(encodeFileAsBase64(secretFileFromExecutor) === mgr.getSecretKey()) + } + + test("secret file must be defined in both driver and executor") { + val conf1 = new SparkConf() + .set(AUTH_SECRET_FILE_DRIVER, Some("/tmp/driver-secret.txt")) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr1 = new SecurityManager(conf1) + intercept[IllegalArgumentException] { + mgr1.initializeAuth() + } + + val conf2 = new SparkConf() + .set(AUTH_SECRET_FILE_EXECUTOR, Some("/tmp/executor-secret.txt")) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr2 = new SecurityManager(conf2) + intercept[IllegalArgumentException] { + mgr2.initializeAuth() + } + } + + Seq("yarn", "local", "local[*]", "local[1,2]", "mesos://localhost:8080").foreach { master => + test(s"master $master cannot use file mounted secrets") { + val conf = new SparkConf() + .set(AUTH_SECRET_FILE, "/tmp/secret.txt") + .set(SecurityManager.SPARK_AUTH_CONF, "true") + .setMaster(master) + intercept[IllegalArgumentException] { + new SecurityManager(conf).getSecretKey() + } + intercept[IllegalArgumentException] { + new SecurityManager(conf).initializeAuth() + } + } + } + // How is the secret expected to be generated and stored. object SecretTestType extends Enumeration { - val MANUAL, AUTO, UGI = Value + val MANUAL, AUTO, UGI, FILE = Value } import SecretTestType._ @@ -408,6 +455,7 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { ("local[*]", UGI), ("local[1, 2]", UGI), ("k8s://127.0.0.1", AUTO), + ("k8s://127.0.1.1", FILE), ("local-cluster[2, 1, 1024]", MANUAL), ("invalid", MANUAL) ).foreach { case (master, secretType) => @@ -440,6 +488,12 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { intercept[IllegalArgumentException] { mgr.getSecretKey() } + + case FILE => + val secretFile = createTempSecretFile() + conf.set(AUTH_SECRET_FILE, secretFile.getAbsolutePath) + mgr.initializeAuth() + assert(encodeFileAsBase64(secretFile) === mgr.getSecretKey()) } } } @@ -447,5 +501,15 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { } } + private def encodeFileAsBase64(secretFile: File) = { + Base64.getEncoder.encodeToString(Files.readAllBytes(secretFile.toPath)) + } + + private def createTempSecretFile(contents: String = "test-secret"): File = { + val secretDir = Utils.createTempDir("temp-secrets") + val secretFile = new File(secretDir, "temp-secret.txt") + Files.write(secretFile.toPath, contents.getBytes(UTF_8)) + secretFile + } } diff --git a/docs/security.md b/docs/security.md index 2a4f3c074c1e..8416ed91356a 100644 --- a/docs/security.md +++ b/docs/security.md @@ -66,6 +66,50 @@ Kubernetes admin to ensure that Spark authentication is secure.
Property NameDefaultMeaning
spark.files.maxPartitionBytes134217728 (128 MB)134217728 (128 MiB) The maximum number of bytes to pack into a single partition when reading files.
spark.files.openCostInBytes4194304 (4 MB)4194304 (4 MiB) The estimated cost to open a file, measured by the number of bytes could be scanned at the same time. This is used when putting multiple files into a partition. It is better to overestimate, @@ -1445,7 +1445,7 @@ Apart from these, the following properties are also available, and may be useful spark.rpc.message.maxSize 128 - Maximum message size (in MB) to allow in "control plane" communication; generally only applies to map + Maximum message size (in MiB) to allow in "control plane" communication; generally only applies to map output size information sent between executors and the driver. Increase this if you are running jobs with many thousands of map and reduce tasks and see messages about the RPC message size. spark.mesos.executor.memoryOverhead executor memory * 0.10, with minimum of 384 - The amount of additional memory, specified in MB, to be allocated per executor. By default, + The amount of additional memory, specified in MiB, to be allocated per executor. By default, the overhead will be larger of either 384 or 10% of spark.executor.memory. If set, the final overhead will be this value.
-m MEM, --memory MEMTotal amount of memory to allow Spark applications to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on workerTotal amount of memory to allow Spark applications to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GiB); only on worker
-d DIR, --work-dir DIR
SPARK_WORKER_MEMORYTotal amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property.Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GiB); note that each application's individual memory is configured using its spark.executor.memory property.
SPARK_WORKER_PORT
+Alternatively, one can mount authentication secrets using files and Kubernetes secrets that +the user mounts into their pods. + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.authenticate.secret.fileNone + Path pointing to the secret key to use for securing connections. Ensure that the + contents of the file have been securely generated. This file is loaded on both the driver + and the executors unless other settings override this (see below). +
spark.authenticate.secret.driver.fileThe value of spark.authenticate.secret.file + When specified, overrides the location that the Spark driver reads to load the secret. + Useful when in client mode, when the location of the secret file may differ in the pod versus + the node the driver is running in. When this is specified, + spark.authenticate.secret.executor.file must be specified so that the driver + and the executors can both use files to load the secret key. Ensure that the contents of the file + on the driver is identical to the contents of the file on the executors. +
spark.authenticate.secret.executor.fileThe value of spark.authenticate.secret.file + When specified, overrides the location that the Spark executors read to load the secret. + Useful in client mode, when the location of the secret file may differ in the pod versus + the node the driver is running in. When this is specified, + spark.authenticate.secret.driver.file must be specified so that the driver + and the executors can both use files to load the secret key. Ensure that the contents of the file + on the driver is identical to the contents of the file on the executors. +
+ +Note that when using files, Spark will not mount these files into the containers for you. It is up +you to ensure that the secret files are deployed securely into your containers and that the driver's +secret file agrees with the executors' secret file. + ## Encryption Spark supports AES-based encryption for RPC connections. For encryption to be enabled, RPC diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 939aa88b0797..4bcf4c9446aa 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -24,7 +24,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTIONS, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD, PYSPARK_EXECUTOR_MEMORY} +import org.apache.spark.internal.config._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -112,12 +112,14 @@ private[spark] class BasicExecutorFeatureStep( .build()) .build()) } ++ { - Option(secMgr.getSecretKey()).map { authSecret => - new EnvVarBuilder() - .withName(SecurityManager.ENV_AUTH_SECRET) - .withValue(authSecret) - .build() - } + if (kubernetesConf.get(AUTH_SECRET_FILE_EXECUTOR).isEmpty) { + Option(secMgr.getSecretKey()).map { authSecret => + new EnvVarBuilder() + .withName(SecurityManager.ENV_AUTH_SECRET) + .withValue(authSecret) + .build() + } + } else None } ++ { kubernetesConf.get(EXECUTOR_CLASS_PATH).map { cp => new EnvVarBuilder() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 6aa862643c78..05989d9be7ad 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -16,6 +16,10 @@ */ package org.apache.spark.deploy.k8s.features +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.Files + import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ @@ -158,6 +162,25 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { checkEnv(executor, conf, Map(SecurityManager.ENV_AUTH_SECRET -> secMgr.getSecretKey())) } + test("Auth secret shouldn't propagate if files are loaded.") { + val secretDir = Utils.createTempDir("temp-secret") + val secretFile = new File(secretDir, "secret-file.txt") + Files.write(secretFile.toPath, "some-secret".getBytes(StandardCharsets.UTF_8)) + val conf = baseConf.clone() + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET_FILE, secretFile.getAbsolutePath) + .set("spark.master", "k8s://127.0.0.1") + val secMgr = new SecurityManager(conf) + secMgr.initializeAuth() + + val step = new BasicExecutorFeatureStep(KubernetesTestConf.createExecutorConf(sparkConf = conf), + secMgr) + + val executor = step.configurePod(SparkPod.initialPod()) + assert(!KubernetesFeaturesTestUtils.containerHasEnvVar( + executor.container, SecurityManager.ENV_AUTH_SECRET)) + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) From bd8da3799dd160771ebb3ea55b7678b644248425 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 12 Dec 2018 10:03:50 +0800 Subject: [PATCH 0116/1072] [SPARK-26193][SQL][FOLLOW UP] Read metrics rename and display text changes ## What changes were proposed in this pull request? Follow up pr for #23207, include following changes: - Rename `SQLShuffleMetricsReporter` to `SQLShuffleReadMetricsReporter` to make it match with write side naming. - Display text changes for read side for naming consistent. - Rename function in `ShuffleWriteProcessor`. - Delete `private[spark]` in execution package. ## How was this patch tested? Existing tests. Closes #23286 from xuanyuanking/SPARK-26193-follow. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../spark/shuffle/ShuffleWriteProcessor.scala | 2 +- .../spark/sql/execution/ShuffledRowRDD.scala | 6 ++-- .../exchange/ShuffleExchangeExec.scala | 4 +-- .../apache/spark/sql/execution/limit.scala | 6 ++-- .../metric/SQLShuffleMetricsReporter.scala | 36 +++++++++---------- .../execution/UnsafeRowSerializerSuite.scala | 4 +-- .../execution/metric/SQLMetricsSuite.scala | 20 +++++------ 8 files changed, 40 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 2a8d1dd995e2..35664ff515d4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -92,7 +92,7 @@ private[spark] class ShuffleMapTask( threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime } else 0L - dep.shuffleWriterProcessor.writeProcess(rdd, dep, partitionId, context, partition) + dep.shuffleWriterProcessor.write(rdd, dep, partitionId, context, partition) } override def preferredLocations: Seq[TaskLocation] = preferredLocs diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index f5213157a9a8..5b0c7e9f2b0b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -41,7 +41,7 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { * get from [[ShuffleManager]] and triggers rdd compute, finally return the [[MapStatus]] for * this task. */ - def writeProcess( + def write( rdd: RDD[_], dep: ShuffleDependency[_, _, _], partitionId: Int, 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 9b05faaed045..079ff25fcb67 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 @@ -22,7 +22,7 @@ import java.util.Arrays import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleMetricsReporter} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} /** * The [[Partition]] used by [[ShuffledRowRDD]]. A post-shuffle partition @@ -157,9 +157,9 @@ class ShuffledRowRDD( override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val shuffledRowPartition = split.asInstanceOf[ShuffledRowRDDPartition] val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() - // `SQLShuffleMetricsReporter` will update its own metrics for SQL exchange operator, + // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, // as well as the `tempMetrics` for basic shuffle metrics. - val sqlMetricsReporter = new SQLShuffleMetricsReporter(tempMetrics, metrics) + val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) // The range of pre-shuffle partitions that we are fetching at here is // [startPreShufflePartitionIndex, endPreShufflePartitionIndex - 1]. val reader = 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 0c2020572e72..da7b0c6f43fb 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Uns import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleMetricsReporter, SQLShuffleWriteMetricsReporter} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.MutablePair @@ -50,7 +50,7 @@ case class ShuffleExchangeExec( private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = - SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size") ) ++ readMetrics ++ writeMetrics diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 1f2fdde53864..bfaf080292bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGe import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.sql.execution.metric.{SQLShuffleMetricsReporter, SQLShuffleWriteMetricsReporter} +import org.apache.spark.sql.execution.metric.{SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} /** * Take the first `limit` elements and collect them to a single partition. @@ -41,7 +41,7 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = - SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics = readMetrics ++ writeMetrics protected override def doExecute(): RDD[InternalRow] = { val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit)) @@ -165,7 +165,7 @@ case class TakeOrderedAndProjectExec( private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = - SQLShuffleMetricsReporter.createShuffleReadMetrics(sparkContext) + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics = readMetrics ++ writeMetrics protected override def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala index ff7941e3b3e8..2c0ea80495ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala @@ -27,23 +27,23 @@ import org.apache.spark.shuffle.ShuffleWriteMetricsReporter * @param metrics All metrics in current SparkPlan. This param should not empty and * contains all shuffle metrics defined in createShuffleReadMetrics. */ -private[spark] class SQLShuffleMetricsReporter( +class SQLShuffleReadMetricsReporter( tempMetrics: TempShuffleReadMetrics, metrics: Map[String, SQLMetric]) extends TempShuffleReadMetrics { private[this] val _remoteBlocksFetched = - metrics(SQLShuffleMetricsReporter.REMOTE_BLOCKS_FETCHED) + metrics(SQLShuffleReadMetricsReporter.REMOTE_BLOCKS_FETCHED) private[this] val _localBlocksFetched = - metrics(SQLShuffleMetricsReporter.LOCAL_BLOCKS_FETCHED) + metrics(SQLShuffleReadMetricsReporter.LOCAL_BLOCKS_FETCHED) private[this] val _remoteBytesRead = - metrics(SQLShuffleMetricsReporter.REMOTE_BYTES_READ) + metrics(SQLShuffleReadMetricsReporter.REMOTE_BYTES_READ) private[this] val _remoteBytesReadToDisk = - metrics(SQLShuffleMetricsReporter.REMOTE_BYTES_READ_TO_DISK) + metrics(SQLShuffleReadMetricsReporter.REMOTE_BYTES_READ_TO_DISK) private[this] val _localBytesRead = - metrics(SQLShuffleMetricsReporter.LOCAL_BYTES_READ) + metrics(SQLShuffleReadMetricsReporter.LOCAL_BYTES_READ) private[this] val _fetchWaitTime = - metrics(SQLShuffleMetricsReporter.FETCH_WAIT_TIME) + metrics(SQLShuffleReadMetricsReporter.FETCH_WAIT_TIME) private[this] val _recordsRead = - metrics(SQLShuffleMetricsReporter.RECORDS_READ) + metrics(SQLShuffleReadMetricsReporter.RECORDS_READ) override def incRemoteBlocksFetched(v: Long): Unit = { _remoteBlocksFetched.add(v) @@ -75,7 +75,7 @@ private[spark] class SQLShuffleMetricsReporter( } } -private[spark] object SQLShuffleMetricsReporter { +object SQLShuffleReadMetricsReporter { val REMOTE_BLOCKS_FETCHED = "remoteBlocksFetched" val LOCAL_BLOCKS_FETCHED = "localBlocksFetched" val REMOTE_BYTES_READ = "remoteBytesRead" @@ -88,8 +88,8 @@ private[spark] object SQLShuffleMetricsReporter { * Create all shuffle read relative metrics and return the Map. */ def createShuffleReadMetrics(sc: SparkContext): Map[String, SQLMetric] = Map( - REMOTE_BLOCKS_FETCHED -> SQLMetrics.createMetric(sc, "remote blocks fetched"), - LOCAL_BLOCKS_FETCHED -> SQLMetrics.createMetric(sc, "local blocks fetched"), + REMOTE_BLOCKS_FETCHED -> SQLMetrics.createMetric(sc, "remote blocks read"), + LOCAL_BLOCKS_FETCHED -> SQLMetrics.createMetric(sc, "local blocks read"), REMOTE_BYTES_READ -> SQLMetrics.createSizeMetric(sc, "remote bytes read"), REMOTE_BYTES_READ_TO_DISK -> SQLMetrics.createSizeMetric(sc, "remote bytes read to disk"), LOCAL_BYTES_READ -> SQLMetrics.createSizeMetric(sc, "local bytes read"), @@ -102,7 +102,7 @@ private[spark] object SQLShuffleMetricsReporter { * @param metricsReporter Other reporter need to be updated in this SQLShuffleWriteMetricsReporter. * @param metrics Shuffle write metrics in current SparkPlan. */ -private[spark] class SQLShuffleWriteMetricsReporter( +class SQLShuffleWriteMetricsReporter( metricsReporter: ShuffleWriteMetricsReporter, metrics: Map[String, SQLMetric]) extends ShuffleWriteMetricsReporter { private[this] val _bytesWritten = @@ -112,29 +112,29 @@ private[spark] class SQLShuffleWriteMetricsReporter( private[this] val _writeTime = metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_WRITE_TIME) - override private[spark] def incBytesWritten(v: Long): Unit = { + override def incBytesWritten(v: Long): Unit = { metricsReporter.incBytesWritten(v) _bytesWritten.add(v) } - override private[spark] def decRecordsWritten(v: Long): Unit = { + override def decRecordsWritten(v: Long): Unit = { metricsReporter.decBytesWritten(v) _recordsWritten.set(_recordsWritten.value - v) } - override private[spark] def incRecordsWritten(v: Long): Unit = { + override def incRecordsWritten(v: Long): Unit = { metricsReporter.incRecordsWritten(v) _recordsWritten.add(v) } - override private[spark] def incWriteTime(v: Long): Unit = { + override def incWriteTime(v: Long): Unit = { metricsReporter.incWriteTime(v) _writeTime.add(v) } - override private[spark] def decBytesWritten(v: Long): Unit = { + override def decBytesWritten(v: Long): Unit = { metricsReporter.decBytesWritten(v) _bytesWritten.set(_bytesWritten.value - v) } } -private[spark] object SQLShuffleWriteMetricsReporter { +object SQLShuffleWriteMetricsReporter { val SHUFFLE_BYTES_WRITTEN = "shuffleBytesWritten" val SHUFFLE_RECORDS_WRITTEN = "shuffleRecordsWritten" val SHUFFLE_WRITE_TIME = "shuffleWriteTime" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index 1ad5713ab8ae..ca8692290edb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{LocalSparkSession, Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.execution.metric.SQLShuffleMetricsReporter +import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.types._ import org.apache.spark.storage.ShuffleBlockId import org.apache.spark.util.collection.ExternalSorter @@ -140,7 +140,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { new UnsafeRowSerializer(2)) val shuffled = new ShuffledRowRDD( dependency, - SQLShuffleMetricsReporter.createShuffleReadMetrics(spark.sparkContext)) + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(spark.sparkContext)) shuffled.count() } } 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 f6495496a58e..47265df4831d 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 @@ -96,8 +96,8 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) val shuffleExpected1 = Map( "records read" -> 2L, - "local blocks fetched" -> 2L, - "remote blocks fetched" -> 0L, + "local blocks read" -> 2L, + "remote blocks read" -> 0L, "shuffle records written" -> 2L) testSparkPlanMetrics(df, 1, Map( 2L -> (("HashAggregate", expected1(0))), @@ -114,8 +114,8 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) val shuffleExpected2 = Map( "records read" -> 4L, - "local blocks fetched" -> 4L, - "remote blocks fetched" -> 0L, + "local blocks read" -> 4L, + "remote blocks read" -> 0L, "shuffle records written" -> 4L) testSparkPlanMetrics(df2, 1, Map( 2L -> (("HashAggregate", expected2(0))), @@ -175,8 +175,8 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared 1L -> (("Exchange", Map( "shuffle records written" -> 2L, "records read" -> 2L, - "local blocks fetched" -> 2L, - "remote blocks fetched" -> 0L))), + "local blocks read" -> 2L, + "remote blocks read" -> 0L))), 0L -> (("ObjectHashAggregate", Map("number of output rows" -> 1L)))) ) @@ -187,8 +187,8 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared 1L -> (("Exchange", Map( "shuffle records written" -> 4L, "records read" -> 4L, - "local blocks fetched" -> 4L, - "remote blocks fetched" -> 0L))), + "local blocks read" -> 4L, + "remote blocks read" -> 0L))), 0L -> (("ObjectHashAggregate", Map("number of output rows" -> 3L)))) ) } @@ -216,8 +216,8 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "number of output rows" -> 4L))), 2L -> (("Exchange", Map( "records read" -> 4L, - "local blocks fetched" -> 2L, - "remote blocks fetched" -> 0L, + "local blocks read" -> 2L, + "remote blocks read" -> 0L, "shuffle records written" -> 2L)))) ) } From 79e36e2c2ac01458b5baa3f3ee310fddd29e9c35 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 12 Dec 2018 09:03:13 -0600 Subject: [PATCH 0117/1072] [SPARK-19827][R][FOLLOWUP] spark.ml R API for PIC ## What changes were proposed in this pull request? Follow up style fixes to PIC in R; see #23072 ## How was this patch tested? Existing tests. Closes #23292 from srowen/SPARK-19827.2. Authored-by: Sean Owen Signed-off-by: Sean Owen --- R/pkg/R/mllib_clustering.R | 15 ++++++--------- R/pkg/R/mllib_fpm.R | 4 ++-- examples/src/main/r/ml/powerIterationClustering.R | 3 ++- 3 files changed, 10 insertions(+), 12 deletions(-) diff --git a/R/pkg/R/mllib_clustering.R b/R/pkg/R/mllib_clustering.R index 7d9dcebfe70d..9b32b71d34fe 100644 --- a/R/pkg/R/mllib_clustering.R +++ b/R/pkg/R/mllib_clustering.R @@ -621,11 +621,10 @@ setMethod("write.ml", signature(object = "LDAModel", path = "character"), #' #' A scalable graph clustering algorithm. Users can call \code{spark.assignClusters} to #' return a cluster assignment for each input vertex. -#' -# Run the PIC algorithm and returns a cluster assignment for each input vertex. +#' Run the PIC algorithm and returns a cluster assignment for each input vertex. #' @param data a SparkDataFrame. #' @param k the number of clusters to create. -#' @param initMode the initialization algorithm. +#' @param initMode the initialization algorithm; "random" or "degree" #' @param maxIter the maximum number of iterations. #' @param sourceCol the name of the input column for source vertex IDs. #' @param destinationCol the name of the input column for destination vertex IDs @@ -633,18 +632,16 @@ setMethod("write.ml", signature(object = "LDAModel", path = "character"), #' we treat all instance weights as 1.0. #' @param ... additional argument(s) passed to the method. #' @return A dataset that contains columns of vertex id and the corresponding cluster for the id. -#' The schema of it will be: -#' \code{id: Long} -#' \code{cluster: Int} +#' The schema of it will be: \code{id: integer}, \code{cluster: integer} #' @rdname spark.powerIterationClustering -#' @aliases assignClusters,PowerIterationClustering-method,SparkDataFrame-method +#' @aliases spark.assignClusters,SparkDataFrame-method #' @examples #' \dontrun{ #' df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), #' list(1L, 2L, 1.0), list(3L, 4L, 1.0), #' list(4L, 0L, 0.1)), #' schema = c("src", "dst", "weight")) -#' clusters <- spark.assignClusters(df, initMode="degree", weightCol="weight") +#' clusters <- spark.assignClusters(df, initMode = "degree", weightCol = "weight") #' showDF(clusters) #' } #' @note spark.assignClusters(SparkDataFrame) since 3.0.0 @@ -652,7 +649,7 @@ setMethod("spark.assignClusters", signature(data = "SparkDataFrame"), function(data, k = 2L, initMode = c("random", "degree"), maxIter = 20L, sourceCol = "src", destinationCol = "dst", weightCol = NULL) { - if (!is.numeric(k) || k < 1) { + if (!is.integer(k) || k < 1) { stop("k should be a number with value >= 1.") } if (!is.integer(maxIter) || maxIter <= 0) { diff --git a/R/pkg/R/mllib_fpm.R b/R/pkg/R/mllib_fpm.R index c248e9ec9be9..0cc7a16c302d 100644 --- a/R/pkg/R/mllib_fpm.R +++ b/R/pkg/R/mllib_fpm.R @@ -183,8 +183,8 @@ setMethod("write.ml", signature(object = "FPGrowthModel", path = "character"), #' @return A complete set of frequent sequential patterns in the input sequences of itemsets. #' The returned \code{SparkDataFrame} contains columns of sequence and corresponding #' frequency. The schema of it will be: -#' \code{sequence: ArrayType(ArrayType(T))} (T is the item type) -#' \code{freq: Long} +#' \code{sequence: ArrayType(ArrayType(T))}, \code{freq: integer} +#' where T is the item type #' @rdname spark.prefixSpan #' @aliases findFrequentSequentialPatterns,PrefixSpan,SparkDataFrame-method #' @examples diff --git a/examples/src/main/r/ml/powerIterationClustering.R b/examples/src/main/r/ml/powerIterationClustering.R index ba43037106d1..3530d88e5050 100644 --- a/examples/src/main/r/ml/powerIterationClustering.R +++ b/examples/src/main/r/ml/powerIterationClustering.R @@ -30,7 +30,8 @@ df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), list(4L, 0L, 0.1)), schema = c("src", "dst", "weight")) # assign clusters -clusters <- spark.assignClusters(df, k=2L, maxIter=20L, initMode="degree", weightCol="weight") +clusters <- spark.assignClusters(df, k = 2L, maxIter = 20L, + initMode = "degree", weightCol = "weight") showDF(arrange(clusters, clusters$id)) # $example off$ From 570b8f3d45ad8d6649ed633251a8194d910f1ab5 Mon Sep 17 00:00:00 2001 From: Ilya Matiach Date: Wed, 12 Dec 2018 10:06:41 -0600 Subject: [PATCH 0118/1072] [SPARK-24102][ML][MLLIB] ML Evaluators should use weight column - added weight column for regression evaluator ## What changes were proposed in this pull request? The evaluators BinaryClassificationEvaluator, RegressionEvaluator, and MulticlassClassificationEvaluator and the corresponding metrics classes BinaryClassificationMetrics, RegressionMetrics and MulticlassMetrics should use sample weight data. I've closed the PR: https://github.com/apache/spark/pull/16557 as recommended in favor of creating three pull requests, one for each of the evaluators (binary/regression/multiclass) to make it easier to review/update. The updates to the regression metrics were based on (and updated with new changes based on comments): https://issues.apache.org/jira/browse/SPARK-11520 ("RegressionMetrics should support instance weights") but the pull request was closed as the changes were never checked in. ## How was this patch tested? I added tests to the metrics class. Closes #17085 from imatiach-msft/ilmat/regression-evaluate. Authored-by: Ilya Matiach Signed-off-by: Sean Owen --- .../ml/evaluation/RegressionEvaluator.scala | 19 ++++--- .../mllib/evaluation/RegressionMetrics.scala | 30 ++++++----- .../stat/MultivariateOnlineSummarizer.scala | 25 ++++++---- .../stat/MultivariateStatisticalSummary.scala | 6 +++ .../evaluation/RegressionMetricsSuite.scala | 50 +++++++++++++++++++ project/MimaExcludes.scala | 5 +- 6 files changed, 106 insertions(+), 29 deletions(-) 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 031cd0d635bf..616569bb55e4 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 @@ -19,7 +19,7 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol} +import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol, HasWeightCol} import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.RegressionMetrics import org.apache.spark.sql.{Dataset, Row} @@ -33,7 +33,8 @@ import org.apache.spark.sql.types.{DoubleType, FloatType} @Since("1.4.0") @Experimental final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Evaluator with HasPredictionCol with HasLabelCol with DefaultParamsWritable { + extends Evaluator with HasPredictionCol with HasLabelCol + with HasWeightCol with DefaultParamsWritable { @Since("1.4.0") def this() = this(Identifiable.randomUID("regEval")) @@ -69,6 +70,10 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("1.4.0") def setLabelCol(value: String): this.type = set(labelCol, value) + /** @group setParam */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + setDefault(metricName -> "rmse") @Since("2.0.0") @@ -77,11 +82,13 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui SchemaUtils.checkColumnTypes(schema, $(predictionCol), Seq(DoubleType, FloatType)) SchemaUtils.checkNumericType(schema, $(labelCol)) - val predictionAndLabels = dataset - .select(col($(predictionCol)).cast(DoubleType), col($(labelCol)).cast(DoubleType)) + val predictionAndLabelsWithWeights = dataset + .select(col($(predictionCol)).cast(DoubleType), col($(labelCol)).cast(DoubleType), + if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol))) .rdd - .map { case Row(prediction: Double, label: Double) => (prediction, label) } - val metrics = new RegressionMetrics(predictionAndLabels) + .map { case Row(prediction: Double, label: Double, weight: Double) => + (prediction, label, weight) } + val metrics = new RegressionMetrics(predictionAndLabelsWithWeights) val metric = $(metricName) match { case "rmse" => metrics.rootMeanSquaredError case "mse" => metrics.meanSquaredError 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 020676cac5a6..525047973ad5 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 @@ -27,17 +27,18 @@ import org.apache.spark.sql.DataFrame /** * Evaluator for regression. * - * @param predictionAndObservations an RDD of (prediction, observation) pairs + * @param predAndObsWithOptWeight an RDD of either (prediction, observation, weight) + * or (prediction, observation) pairs * @param throughOrigin True if the regression is through the origin. For example, in linear * regression, it will be true without fitting intercept. */ @Since("1.2.0") class RegressionMetrics @Since("2.0.0") ( - predictionAndObservations: RDD[(Double, Double)], throughOrigin: Boolean) + predAndObsWithOptWeight: RDD[_ <: Product], throughOrigin: Boolean) extends Logging { @Since("1.2.0") - def this(predictionAndObservations: RDD[(Double, Double)]) = + def this(predictionAndObservations: RDD[_ <: Product]) = this(predictionAndObservations, false) /** @@ -52,10 +53,13 @@ class RegressionMetrics @Since("2.0.0") ( * Use MultivariateOnlineSummarizer to calculate summary statistics of observations and errors. */ private lazy val summary: MultivariateStatisticalSummary = { - val summary: MultivariateStatisticalSummary = predictionAndObservations.map { - case (prediction, observation) => Vectors.dense(observation, observation - prediction) + val summary: MultivariateStatisticalSummary = predAndObsWithOptWeight.map { + case (prediction: Double, observation: Double, weight: Double) => + (Vectors.dense(observation, observation - prediction), weight) + case (prediction: Double, observation: Double) => + (Vectors.dense(observation, observation - prediction), 1.0) }.treeAggregate(new MultivariateOnlineSummarizer())( - (summary, v) => summary.add(v), + (summary, sample) => summary.add(sample._1, sample._2), (sum1, sum2) => sum1.merge(sum2) ) summary @@ -63,11 +67,13 @@ class RegressionMetrics @Since("2.0.0") ( private lazy val SSy = math.pow(summary.normL2(0), 2) private lazy val SSerr = math.pow(summary.normL2(1), 2) - private lazy val SStot = summary.variance(0) * (summary.count - 1) + private lazy val SStot = summary.variance(0) * (summary.weightSum - 1) private lazy val SSreg = { val yMean = summary.mean(0) - predictionAndObservations.map { - case (prediction, _) => math.pow(prediction - yMean, 2) + predAndObsWithOptWeight.map { + case (prediction: Double, _: Double, weight: Double) => + math.pow(prediction - yMean, 2) * weight + case (prediction: Double, _: Double) => math.pow(prediction - yMean, 2) }.sum() } @@ -79,7 +85,7 @@ class RegressionMetrics @Since("2.0.0") ( */ @Since("1.2.0") def explainedVariance: Double = { - SSreg / summary.count + SSreg / summary.weightSum } /** @@ -88,7 +94,7 @@ class RegressionMetrics @Since("2.0.0") ( */ @Since("1.2.0") def meanAbsoluteError: Double = { - summary.normL1(1) / summary.count + summary.normL1(1) / summary.weightSum } /** @@ -97,7 +103,7 @@ class RegressionMetrics @Since("2.0.0") ( */ @Since("1.2.0") def meanSquaredError: Double = { - SSerr / summary.count + SSerr / summary.weightSum } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 0554b6d8ff5b..6d510e1633d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -52,7 +52,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S private var totalCnt: Long = 0 private var totalWeightSum: Double = 0.0 private var weightSquareSum: Double = 0.0 - private var weightSum: Array[Double] = _ + private var currWeightSum: Array[Double] = _ private var nnz: Array[Long] = _ private var currMax: Array[Double] = _ private var currMin: Array[Double] = _ @@ -78,7 +78,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currM2n = Array.ofDim[Double](n) currM2 = Array.ofDim[Double](n) currL1 = Array.ofDim[Double](n) - weightSum = Array.ofDim[Double](n) + currWeightSum = Array.ofDim[Double](n) nnz = Array.ofDim[Long](n) currMax = Array.fill[Double](n)(Double.MinValue) currMin = Array.fill[Double](n)(Double.MaxValue) @@ -91,7 +91,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val localCurrM2n = currM2n val localCurrM2 = currM2 val localCurrL1 = currL1 - val localWeightSum = weightSum + val localWeightSum = currWeightSum val localNumNonzeros = nnz val localCurrMax = currMax val localCurrMin = currMin @@ -139,8 +139,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S weightSquareSum += other.weightSquareSum var i = 0 while (i < n) { - val thisNnz = weightSum(i) - val otherNnz = other.weightSum(i) + val thisNnz = currWeightSum(i) + val otherNnz = other.currWeightSum(i) val totalNnz = thisNnz + otherNnz val totalCnnz = nnz(i) + other.nnz(i) if (totalNnz != 0.0) { @@ -157,7 +157,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currMax(i) = math.max(currMax(i), other.currMax(i)) currMin(i) = math.min(currMin(i), other.currMin(i)) } - weightSum(i) = totalNnz + currWeightSum(i) = totalNnz nnz(i) = totalCnnz i += 1 } @@ -170,7 +170,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S this.totalCnt = other.totalCnt this.totalWeightSum = other.totalWeightSum this.weightSquareSum = other.weightSquareSum - this.weightSum = other.weightSum.clone() + this.currWeightSum = other.currWeightSum.clone() this.nnz = other.nnz.clone() this.currMax = other.currMax.clone() this.currMin = other.currMin.clone() @@ -189,7 +189,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val realMean = Array.ofDim[Double](n) var i = 0 while (i < n) { - realMean(i) = currMean(i) * (weightSum(i) / totalWeightSum) + realMean(i) = currMean(i) * (currWeightSum(i) / totalWeightSum) i += 1 } Vectors.dense(realMean) @@ -214,8 +214,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val len = currM2n.length while (i < len) { // We prevent variance from negative value caused by numerical error. - realVariance(i) = math.max((currM2n(i) + deltaMean(i) * deltaMean(i) * weightSum(i) * - (totalWeightSum - weightSum(i)) / totalWeightSum) / denominator, 0.0) + realVariance(i) = math.max((currM2n(i) + deltaMean(i) * deltaMean(i) * currWeightSum(i) * + (totalWeightSum - currWeightSum(i)) / totalWeightSum) / denominator, 0.0) i += 1 } } @@ -229,6 +229,11 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S @Since("1.1.0") override def count: Long = totalCnt + /** + * Sum of weights. + */ + override def weightSum: Double = totalWeightSum + /** * Number of nonzero elements in each dimension. * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala index 39a16fb743d6..a4381032f8c0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala @@ -44,6 +44,12 @@ trait MultivariateStatisticalSummary { @Since("1.0.0") def count: Long + /** + * Sum of weights. + */ + @Since("3.0.0") + def weightSum: Double + /** * Number of nonzero elements (including explicitly presented zero values) in each column. */ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala index f1d517383643..23809777f7d3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala @@ -133,4 +133,54 @@ class RegressionMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { "root mean squared error mismatch") assert(metrics.r2 ~== 1.0 absTol eps, "r2 score mismatch") } + + test("regression metrics with same (1.0) weight samples") { + val predictionAndObservationWithWeight = sc.parallelize( + Seq((2.25, 3.0, 1.0), (-0.25, -0.5, 1.0), (1.75, 2.0, 1.0), (7.75, 7.0, 1.0)), 2) + val metrics = new RegressionMetrics(predictionAndObservationWithWeight, false) + assert(metrics.explainedVariance ~== 8.79687 absTol eps, + "explained variance regression score mismatch") + assert(metrics.meanAbsoluteError ~== 0.5 absTol eps, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.3125 absTol eps, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.55901 absTol eps, + "root mean squared error mismatch") + assert(metrics.r2 ~== 0.95717 absTol eps, "r2 score mismatch") + } + + /** + * The following values are hand calculated using the formula: + * [[https://en.wikipedia.org/wiki/Weighted_arithmetic_mean#Reliability_weights]] + * preds = c(2.25, -0.25, 1.75, 7.75) + * obs = c(3.0, -0.5, 2.0, 7.0) + * weights = c(0.1, 0.2, 0.15, 0.05) + * count = 4 + * + * Weighted metrics can be calculated with MultivariateStatisticalSummary. + * (observations, observations - predictions) + * mean (1.7, 0.05) + * variance (7.3, 0.3) + * numNonZeros (0.5, 0.5) + * max (7.0, 0.75) + * min (-0.5, -0.75) + * normL2 (2.0, 0.32596) + * normL1 (1.05, 0.2) + * + * explainedVariance: sum(pow((preds - 1.7),2)*weight) / weightedCount = 5.2425 + * meanAbsoluteError: normL1(1) / weightedCount = 0.4 + * meanSquaredError: pow(normL2(1),2) / weightedCount = 0.2125 + * rootMeanSquaredError: sqrt(meanSquaredError) = 0.46098 + * r2: 1 - pow(normL2(1),2) / (variance(0) * (weightedCount - 1)) = 1.02910 + */ + test("regression metrics with weighted samples") { + val predictionAndObservationWithWeight = sc.parallelize( + Seq((2.25, 3.0, 0.1), (-0.25, -0.5, 0.2), (1.75, 2.0, 0.15), (7.75, 7.0, 0.05)), 2) + val metrics = new RegressionMetrics(predictionAndObservationWithWeight, false) + assert(metrics.explainedVariance ~== 5.2425 absTol eps, + "explained variance regression score mismatch") + assert(metrics.meanAbsoluteError ~== 0.4 absTol eps, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.2125 absTol eps, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.46098 absTol eps, + "root mean squared error mismatch") + assert(metrics.r2 ~== 1.02910 absTol eps, "r2 score mismatch") + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b3252d70a80c..883913332ca1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -531,7 +531,10 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseColMajor"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseMatrix"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparseMatrix"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getSizeInBytes") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getSizeInBytes"), + + // [SPARK-18693] Added weightSum to trait MultivariateStatisticalSummary + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.stat.MultivariateStatisticalSummary.weightSum") ) ++ Seq( // [SPARK-17019] Expose on-heap and off-heap memory usage in various places ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.copy"), From a63e7b2a212bab94d080b00cf1c5f397800a276a Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 12 Dec 2018 12:01:21 -0800 Subject: [PATCH 0119/1072] [SPARK-25877][K8S] Move all feature logic to feature classes. This change makes the driver and executor builders a lot simpler by encapsulating almost all feature logic into the respective feature classes. The only logic that remains is the creation of the initial pod, which needs to happen before anything else so is better to be left in the builder class. Most feature classes already behave fine when the config has nothing they should handle, but a few minor tweaks had to be added. Unit tests were also updated or added to account for these. The builder suites were simplified a lot and just test the remaining pod-related code in the builders themselves. Author: Marcelo Vanzin Closes #23220 from vanzin/SPARK-25877. --- .../HadoopConfExecutorFeatureStep.scala | 10 +- .../HadoopSparkUserExecutorFeatureStep.scala | 5 +- .../KerberosConfExecutorFeatureStep.scala | 26 +-- .../features/PodTemplateConfigMapStep.scala | 82 +++++--- .../submit/KubernetesClientApplication.scala | 4 +- .../k8s/submit/KubernetesDriverBuilder.scala | 99 +++------ .../cluster/k8s/ExecutorPodsAllocator.scala | 3 +- .../k8s/KubernetesClusterManager.scala | 2 +- .../k8s/KubernetesExecutorBuilder.scala | 100 +++------ .../spark/deploy/k8s/PodBuilderSuite.scala | 177 ++++++++++++++++ .../PodTemplateConfigMapStepSuite.scala | 25 ++- .../spark/deploy/k8s/submit/ClientSuite.scala | 2 +- .../submit/KubernetesDriverBuilderSuite.scala | 194 +----------------- .../k8s/submit/PodBuilderSuiteUtils.scala | 142 ------------- .../k8s/ExecutorPodsAllocatorSuite.scala | 4 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 144 +------------ 16 files changed, 343 insertions(+), 676 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/PodBuilderSuiteUtils.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala index bca66759d586..da332881ae1a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala @@ -31,10 +31,10 @@ private[spark] class HadoopConfExecutorFeatureStep(conf: KubernetesExecutorConf) override def configurePod(pod: SparkPod): SparkPod = { val hadoopConfDirCMapName = conf.getOption(HADOOP_CONFIG_MAP_NAME) - require(hadoopConfDirCMapName.isDefined, - "Ensure that the env `HADOOP_CONF_DIR` is defined either in the client or " + - " using pre-existing ConfigMaps") - logInfo("HADOOP_CONF_DIR defined") - HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod) + if (hadoopConfDirCMapName.isDefined) { + HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod) + } else { + pod + } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala index e34211076319..c038e75491ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala @@ -28,7 +28,8 @@ private[spark] class HadoopSparkUserExecutorFeatureStep(conf: KubernetesExecutor extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - val sparkUserName = conf.get(KERBEROS_SPARK_USER_NAME) - HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) + conf.getOption(KERBEROS_SPARK_USER_NAME).map { user => + HadoopBootstrapUtil.bootstrapSparkUserPod(user, pod) + }.getOrElse(pod) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala index 32bb6a5d2bcb..907271b1cb48 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala @@ -27,18 +27,20 @@ import org.apache.spark.internal.Logging private[spark] class KerberosConfExecutorFeatureStep(conf: KubernetesExecutorConf) extends KubernetesFeatureConfigStep with Logging { - private val maybeKrb5CMap = conf.getOption(KRB5_CONFIG_MAP_NAME) - require(maybeKrb5CMap.isDefined, "HADOOP_CONF_DIR ConfigMap not found") - override def configurePod(pod: SparkPod): SparkPod = { - logInfo(s"Mounting Resources for Kerberos") - HadoopBootstrapUtil.bootstrapKerberosPod( - conf.get(KERBEROS_DT_SECRET_NAME), - conf.get(KERBEROS_DT_SECRET_KEY), - conf.get(KERBEROS_SPARK_USER_NAME), - None, - None, - maybeKrb5CMap, - pod) + val maybeKrb5CMap = conf.getOption(KRB5_CONFIG_MAP_NAME) + if (maybeKrb5CMap.isDefined) { + logInfo(s"Mounting Resources for Kerberos") + HadoopBootstrapUtil.bootstrapKerberosPod( + conf.get(KERBEROS_DT_SECRET_NAME), + conf.get(KERBEROS_DT_SECRET_KEY), + conf.get(KERBEROS_SPARK_USER_NAME), + None, + None, + maybeKrb5CMap, + pod) + } else { + pod + } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala index 09dcf93a54f8..7f41ca43589b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala @@ -28,44 +28,60 @@ import org.apache.spark.deploy.k8s.Constants._ private[spark] class PodTemplateConfigMapStep(conf: KubernetesConf) extends KubernetesFeatureConfigStep { + + private val hasTemplate = conf.contains(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) + def configurePod(pod: SparkPod): SparkPod = { - val podWithVolume = new PodBuilder(pod.pod) - .editSpec() - .addNewVolume() - .withName(POD_TEMPLATE_VOLUME) - .withNewConfigMap() - .withName(POD_TEMPLATE_CONFIGMAP) - .addNewItem() - .withKey(POD_TEMPLATE_KEY) - .withPath(EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME) - .endItem() - .endConfigMap() - .endVolume() - .endSpec() - .build() + if (hasTemplate) { + val podWithVolume = new PodBuilder(pod.pod) + .editSpec() + .addNewVolume() + .withName(POD_TEMPLATE_VOLUME) + .withNewConfigMap() + .withName(POD_TEMPLATE_CONFIGMAP) + .addNewItem() + .withKey(POD_TEMPLATE_KEY) + .withPath(EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME) + .endItem() + .endConfigMap() + .endVolume() + .endSpec() + .build() - val containerWithVolume = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(POD_TEMPLATE_VOLUME) - .withMountPath(EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH) - .endVolumeMount() - .build() - SparkPod(podWithVolume, containerWithVolume) + val containerWithVolume = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(POD_TEMPLATE_VOLUME) + .withMountPath(EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH) + .endVolumeMount() + .build() + SparkPod(podWithVolume, containerWithVolume) + } else { + pod + } } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map[String, String]( - KUBERNETES_EXECUTOR_PODTEMPLATE_FILE.key -> - (EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH + "/" + EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME)) + override def getAdditionalPodSystemProperties(): Map[String, String] = { + if (hasTemplate) { + Map[String, String]( + KUBERNETES_EXECUTOR_PODTEMPLATE_FILE.key -> + (EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH + "/" + EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME)) + } else { + Map.empty + } + } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - require(conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).isDefined) - val podTemplateFile = conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).get - val podTemplateString = Files.toString(new File(podTemplateFile), StandardCharsets.UTF_8) - Seq(new ConfigMapBuilder() - .withNewMetadata() - .withName(POD_TEMPLATE_CONFIGMAP) - .endMetadata() - .addToData(POD_TEMPLATE_KEY, podTemplateString) - .build()) + if (hasTemplate) { + val podTemplateFile = conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).get + val podTemplateString = Files.toString(new File(podTemplateFile), StandardCharsets.UTF_8) + Seq(new ConfigMapBuilder() + .withNewMetadata() + .withName(POD_TEMPLATE_CONFIGMAP) + .endMetadata() + .addToData(POD_TEMPLATE_KEY, podTemplateString) + .build()) + } else { + Nil + } } } 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 70a93c968795..3888778bf84c 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 @@ -104,7 +104,7 @@ private[spark] class Client( watcher: LoggingPodStatusWatcher) extends Logging { def run(): Unit = { - val resolvedDriverSpec = builder.buildFromFeatures(conf) + val resolvedDriverSpec = builder.buildFromFeatures(conf, kubernetesClient) val configMapName = s"${conf.resourceNamePrefix}-driver-conf-map" val configMap = buildConfigMap(configMapName, resolvedDriverSpec.systemProperties) // The include of the ENV_VAR for "SPARK_CONF_DIR" is to allow for the @@ -232,7 +232,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { None)) { kubernetesClient => val client = new Client( kubernetesConf, - KubernetesDriverBuilder(kubernetesClient, kubernetesConf.sparkConf), + new KubernetesDriverBuilder(), kubernetesClient, waitForAppCompletion, watcher) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index a5ad9729aee9..d2c0ced9fa2f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -20,90 +20,49 @@ import java.io.File import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.features._ -private[spark] class KubernetesDriverBuilder( - provideBasicStep: (KubernetesDriverConf => BasicDriverFeatureStep) = - new BasicDriverFeatureStep(_), - provideCredentialsStep: (KubernetesDriverConf => DriverKubernetesCredentialsFeatureStep) = - new DriverKubernetesCredentialsFeatureStep(_), - provideServiceStep: (KubernetesDriverConf => DriverServiceFeatureStep) = - new DriverServiceFeatureStep(_), - provideSecretsStep: (KubernetesConf => MountSecretsFeatureStep) = - new MountSecretsFeatureStep(_), - provideEnvSecretsStep: (KubernetesConf => EnvSecretsFeatureStep) = - new EnvSecretsFeatureStep(_), - provideLocalDirsStep: (KubernetesConf => LocalDirsFeatureStep) = - new LocalDirsFeatureStep(_), - provideVolumesStep: (KubernetesConf => MountVolumesFeatureStep) = - new MountVolumesFeatureStep(_), - provideDriverCommandStep: (KubernetesDriverConf => DriverCommandFeatureStep) = - new DriverCommandFeatureStep(_), - provideHadoopGlobalStep: (KubernetesDriverConf => KerberosConfDriverFeatureStep) = - new KerberosConfDriverFeatureStep(_), - providePodTemplateConfigMapStep: (KubernetesConf => PodTemplateConfigMapStep) = - new PodTemplateConfigMapStep(_), - provideInitialPod: () => SparkPod = () => SparkPod.initialPod) { +private[spark] class KubernetesDriverBuilder { - def buildFromFeatures(kubernetesConf: KubernetesDriverConf): KubernetesDriverSpec = { - val baseFeatures = Seq( - provideBasicStep(kubernetesConf), - provideCredentialsStep(kubernetesConf), - provideServiceStep(kubernetesConf), - provideLocalDirsStep(kubernetesConf)) - - val secretFeature = if (kubernetesConf.secretNamesToMountPaths.nonEmpty) { - Seq(provideSecretsStep(kubernetesConf)) - } else Nil - val envSecretFeature = if (kubernetesConf.secretEnvNamesToKeyRefs.nonEmpty) { - Seq(provideEnvSecretsStep(kubernetesConf)) - } else Nil - val volumesFeature = if (kubernetesConf.volumes.nonEmpty) { - Seq(provideVolumesStep(kubernetesConf)) - } else Nil - val podTemplateFeature = if ( - kubernetesConf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).isDefined) { - Seq(providePodTemplateConfigMapStep(kubernetesConf)) - } else Nil - - val driverCommandStep = provideDriverCommandStep(kubernetesConf) - - val hadoopConfigStep = Some(provideHadoopGlobalStep(kubernetesConf)) + def buildFromFeatures( + conf: KubernetesDriverConf, + client: KubernetesClient): KubernetesDriverSpec = { + val initialPod = conf.get(Config.KUBERNETES_DRIVER_PODTEMPLATE_FILE) + .map { file => + KubernetesUtils.loadPodFromTemplate( + client, + new File(file), + conf.get(Config.KUBERNETES_DRIVER_PODTEMPLATE_CONTAINER_NAME)) + } + .getOrElse(SparkPod.initialPod()) - val allFeatures: Seq[KubernetesFeatureConfigStep] = - baseFeatures ++ Seq(driverCommandStep) ++ - secretFeature ++ envSecretFeature ++ volumesFeature ++ - hadoopConfigStep ++ podTemplateFeature + val features = Seq( + new BasicDriverFeatureStep(conf), + new DriverKubernetesCredentialsFeatureStep(conf), + new DriverServiceFeatureStep(conf), + new MountSecretsFeatureStep(conf), + new EnvSecretsFeatureStep(conf), + new LocalDirsFeatureStep(conf), + new MountVolumesFeatureStep(conf), + new DriverCommandFeatureStep(conf), + new KerberosConfDriverFeatureStep(conf), + new PodTemplateConfigMapStep(conf)) - var spec = KubernetesDriverSpec( - provideInitialPod(), + val spec = KubernetesDriverSpec( + initialPod, driverKubernetesResources = Seq.empty, - kubernetesConf.sparkConf.getAll.toMap) - for (feature <- allFeatures) { + conf.sparkConf.getAll.toMap) + + features.foldLeft(spec) { case (spec, feature) => val configuredPod = feature.configurePod(spec.pod) val addedSystemProperties = feature.getAdditionalPodSystemProperties() val addedResources = feature.getAdditionalKubernetesResources() - spec = KubernetesDriverSpec( + KubernetesDriverSpec( configuredPod, spec.driverKubernetesResources ++ addedResources, spec.systemProperties ++ addedSystemProperties) } - spec } -} -private[spark] object KubernetesDriverBuilder { - def apply(kubernetesClient: KubernetesClient, conf: SparkConf): KubernetesDriverBuilder = { - conf.get(Config.KUBERNETES_DRIVER_PODTEMPLATE_FILE) - .map(new File(_)) - .map(file => new KubernetesDriverBuilder(provideInitialPod = () => - KubernetesUtils.loadPodFromTemplate( - kubernetesClient, - file, - conf.get(Config.KUBERNETES_DRIVER_PODTEMPLATE_CONTAINER_NAME)) - )) - .getOrElse(new KubernetesDriverBuilder()) - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index ac42554b1334..da3edfeca9b1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -136,7 +136,8 @@ private[spark] class ExecutorPodsAllocator( newExecutorId.toString, applicationId, driverPod) - val executorPod = executorBuilder.buildFromFeatures(executorConf, secMgr) + val executorPod = executorBuilder.buildFromFeatures(executorConf, secMgr, + kubernetesClient) val podWithAttachedContainer = new PodBuilder(executorPod.pod) .editOrNewSpec() .addToContainers(executorPod.container) 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 b31fbb420ed6..809bdf8ca8c2 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 @@ -95,7 +95,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val executorPodsAllocator = new ExecutorPodsAllocator( sc.conf, sc.env.securityManager, - KubernetesExecutorBuilder(kubernetesClient, sc.conf), + new KubernetesExecutorBuilder(), kubernetesClient, snapshotsStore, new SystemClock()) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index ba273cad6a8e..0b74966fe868 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -20,86 +20,36 @@ import java.io.File import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.SecurityManager import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ -private[spark] class KubernetesExecutorBuilder( - provideBasicStep: (KubernetesExecutorConf, SecurityManager) => BasicExecutorFeatureStep = - new BasicExecutorFeatureStep(_, _), - provideSecretsStep: (KubernetesConf => MountSecretsFeatureStep) = - new MountSecretsFeatureStep(_), - provideEnvSecretsStep: (KubernetesConf => EnvSecretsFeatureStep) = - new EnvSecretsFeatureStep(_), - provideLocalDirsStep: (KubernetesConf => LocalDirsFeatureStep) = - new LocalDirsFeatureStep(_), - provideVolumesStep: (KubernetesConf => MountVolumesFeatureStep) = - new MountVolumesFeatureStep(_), - provideHadoopConfStep: (KubernetesExecutorConf => HadoopConfExecutorFeatureStep) = - new HadoopConfExecutorFeatureStep(_), - provideKerberosConfStep: (KubernetesExecutorConf => KerberosConfExecutorFeatureStep) = - new KerberosConfExecutorFeatureStep(_), - provideHadoopSparkUserStep: (KubernetesExecutorConf => HadoopSparkUserExecutorFeatureStep) = - new HadoopSparkUserExecutorFeatureStep(_), - provideInitialPod: () => SparkPod = () => SparkPod.initialPod()) { +private[spark] class KubernetesExecutorBuilder { def buildFromFeatures( - kubernetesConf: KubernetesExecutorConf, - secMgr: SecurityManager): SparkPod = { - val sparkConf = kubernetesConf.sparkConf - val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) - val maybeDTSecretName = sparkConf.getOption(KERBEROS_DT_SECRET_NAME) - val maybeDTDataItem = sparkConf.getOption(KERBEROS_DT_SECRET_KEY) - - val baseFeatures = Seq(provideBasicStep(kubernetesConf, secMgr), - provideLocalDirsStep(kubernetesConf)) - val secretFeature = if (kubernetesConf.secretNamesToMountPaths.nonEmpty) { - Seq(provideSecretsStep(kubernetesConf)) - } else Nil - val secretEnvFeature = if (kubernetesConf.secretEnvNamesToKeyRefs.nonEmpty) { - Seq(provideEnvSecretsStep(kubernetesConf)) - } else Nil - val volumesFeature = if (kubernetesConf.volumes.nonEmpty) { - Seq(provideVolumesStep(kubernetesConf)) - } else Nil - - val maybeHadoopConfFeatureSteps = maybeHadoopConfigMap.map { _ => - val maybeKerberosStep = - if (maybeDTSecretName.isDefined && maybeDTDataItem.isDefined) { - provideKerberosConfStep(kubernetesConf) - } else { - provideHadoopSparkUserStep(kubernetesConf) - } - Seq(provideHadoopConfStep(kubernetesConf)) :+ - maybeKerberosStep - }.getOrElse(Seq.empty[KubernetesFeatureConfigStep]) - - val allFeatures: Seq[KubernetesFeatureConfigStep] = - baseFeatures ++ - secretFeature ++ - secretEnvFeature ++ - volumesFeature ++ - maybeHadoopConfFeatureSteps - - var executorPod = provideInitialPod() - for (feature <- allFeatures) { - executorPod = feature.configurePod(executorPod) - } - executorPod + conf: KubernetesExecutorConf, + secMgr: SecurityManager, + client: KubernetesClient): SparkPod = { + val initialPod = conf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) + .map { file => + KubernetesUtils.loadPodFromTemplate( + client, + new File(file), + conf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME)) + } + .getOrElse(SparkPod.initialPod()) + + val features = Seq( + new BasicExecutorFeatureStep(conf, secMgr), + new MountSecretsFeatureStep(conf), + new EnvSecretsFeatureStep(conf), + new LocalDirsFeatureStep(conf), + new MountVolumesFeatureStep(conf), + new HadoopConfExecutorFeatureStep(conf), + new KerberosConfExecutorFeatureStep(conf), + new HadoopSparkUserExecutorFeatureStep(conf)) + + features.foldLeft(initialPod) { case (pod, feature) => feature.configurePod(pod) } } -} -private[spark] object KubernetesExecutorBuilder { - def apply(kubernetesClient: KubernetesClient, conf: SparkConf): KubernetesExecutorBuilder = { - conf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) - .map(new File(_)) - .map(file => new KubernetesExecutorBuilder(provideInitialPod = () => - KubernetesUtils.loadPodFromTemplate( - kubernetesClient, - file, - conf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME)) - )) - .getOrElse(new KubernetesExecutorBuilder()) - } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala new file mode 100644 index 000000000000..7dde0c137716 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.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.deploy.k8s + +import java.io.File + +import io.fabric8.kubernetes.api.model.{Config => _, _} +import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} +import org.mockito.Matchers.any +import org.mockito.Mockito.{mock, never, verify, when} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.internal.config.ConfigEntry + +abstract class PodBuilderSuite extends SparkFunSuite { + + protected def templateFileConf: ConfigEntry[_] + + protected def buildPod(sparkConf: SparkConf, client: KubernetesClient): SparkPod + + private val baseConf = new SparkConf(false) + .set(Config.CONTAINER_IMAGE, "spark-executor:latest") + + test("use empty initial pod if template is not specified") { + val client = mock(classOf[KubernetesClient]) + buildPod(baseConf.clone(), client) + verify(client, never()).pods() + } + + test("load pod template if specified") { + val client = mockKubernetesClient() + val sparkConf = baseConf.clone().set(templateFileConf.key, "template-file.yaml") + val pod = buildPod(sparkConf, client) + verifyPod(pod) + } + + test("complain about misconfigured pod template") { + val client = mockKubernetesClient( + new PodBuilder() + .withNewMetadata() + .addToLabels("test-label-key", "test-label-value") + .endMetadata() + .build()) + val sparkConf = baseConf.clone().set(templateFileConf.key, "template-file.yaml") + val exception = intercept[SparkException] { + buildPod(sparkConf, client) + } + assert(exception.getMessage.contains("Could not load pod from template file.")) + } + + private def mockKubernetesClient(pod: Pod = podWithSupportedFeatures()): KubernetesClient = { + val kubernetesClient = mock(classOf[KubernetesClient]) + val pods = + mock(classOf[MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]]) + val podResource = mock(classOf[PodResource[Pod, DoneablePod]]) + when(kubernetesClient.pods()).thenReturn(pods) + when(pods.load(any(classOf[File]))).thenReturn(podResource) + when(podResource.get()).thenReturn(pod) + kubernetesClient + } + + private def verifyPod(pod: SparkPod): Unit = { + val metadata = pod.pod.getMetadata + assert(metadata.getLabels.containsKey("test-label-key")) + assert(metadata.getAnnotations.containsKey("test-annotation-key")) + assert(metadata.getNamespace === "namespace") + assert(metadata.getOwnerReferences.asScala.exists(_.getName == "owner-reference")) + val spec = pod.pod.getSpec + assert(!spec.getContainers.asScala.exists(_.getName == "executor-container")) + assert(spec.getDnsPolicy === "dns-policy") + assert(spec.getHostAliases.asScala.exists(_.getHostnames.asScala.exists(_ == "hostname"))) + assert(spec.getImagePullSecrets.asScala.exists(_.getName == "local-reference")) + assert(spec.getInitContainers.asScala.exists(_.getName == "init-container")) + assert(spec.getNodeName == "node-name") + assert(spec.getNodeSelector.get("node-selector-key") === "node-selector-value") + assert(spec.getSchedulerName === "scheduler") + assert(spec.getSecurityContext.getRunAsUser === 1000L) + assert(spec.getServiceAccount === "service-account") + assert(spec.getSubdomain === "subdomain") + assert(spec.getTolerations.asScala.exists(_.getKey == "toleration-key")) + assert(spec.getVolumes.asScala.exists(_.getName == "test-volume")) + val container = pod.container + assert(container.getName === "executor-container") + assert(container.getArgs.contains("arg")) + assert(container.getCommand.equals(List("command").asJava)) + assert(container.getEnv.asScala.exists(_.getName == "env-key")) + assert(container.getResources.getLimits.get("gpu") === + new QuantityBuilder().withAmount("1").build()) + assert(container.getSecurityContext.getRunAsNonRoot) + assert(container.getStdin) + assert(container.getTerminationMessagePath === "termination-message-path") + assert(container.getTerminationMessagePolicy === "termination-message-policy") + assert(pod.container.getVolumeMounts.asScala.exists(_.getName == "test-volume")) + } + + private def podWithSupportedFeatures(): Pod = { + new PodBuilder() + .withNewMetadata() + .addToLabels("test-label-key", "test-label-value") + .addToAnnotations("test-annotation-key", "test-annotation-value") + .withNamespace("namespace") + .addNewOwnerReference() + .withController(true) + .withName("owner-reference") + .endOwnerReference() + .endMetadata() + .withNewSpec() + .withDnsPolicy("dns-policy") + .withHostAliases(new HostAliasBuilder().withHostnames("hostname").build()) + .withImagePullSecrets( + new LocalObjectReferenceBuilder().withName("local-reference").build()) + .withInitContainers(new ContainerBuilder().withName("init-container").build()) + .withNodeName("node-name") + .withNodeSelector(Map("node-selector-key" -> "node-selector-value").asJava) + .withSchedulerName("scheduler") + .withNewSecurityContext() + .withRunAsUser(1000L) + .endSecurityContext() + .withServiceAccount("service-account") + .withSubdomain("subdomain") + .withTolerations(new TolerationBuilder() + .withKey("toleration-key") + .withOperator("Equal") + .withEffect("NoSchedule") + .build()) + .addNewVolume() + .withNewHostPath() + .withPath("/test") + .endHostPath() + .withName("test-volume") + .endVolume() + .addNewContainer() + .withArgs("arg") + .withCommand("command") + .addNewEnv() + .withName("env-key") + .withValue("env-value") + .endEnv() + .withImagePullPolicy("Always") + .withName("executor-container") + .withNewResources() + .withLimits(Map("gpu" -> new QuantityBuilder().withAmount("1").build()).asJava) + .endResources() + .withNewSecurityContext() + .withRunAsNonRoot(true) + .endSecurityContext() + .withStdin(true) + .withTerminationMessagePath("termination-message-path") + .withTerminationMessagePolicy("termination-message-policy") + .addToVolumeMounts( + new VolumeMountBuilder() + .withName("test-volume") + .withMountPath("/test") + .build()) + .endContainer() + .endSpec() + .build() + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala index 7295b82ca479..5e7388dc8e67 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala @@ -20,25 +20,32 @@ import java.io.{File, PrintWriter} import java.nio.file.Files import io.fabric8.kubernetes.api.model.ConfigMap -import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ -class PodTemplateConfigMapStepSuite extends SparkFunSuite with BeforeAndAfter { - private var kubernetesConf : KubernetesConf = _ - private var templateFile: File = _ +class PodTemplateConfigMapStepSuite extends SparkFunSuite { - before { - templateFile = Files.createTempFile("pod-template", "yml").toFile + test("Do nothing when executor template is not specified") { + val conf = KubernetesTestConf.createDriverConf() + val step = new PodTemplateConfigMapStep(conf) + + val initialPod = SparkPod.initialPod() + val configuredPod = step.configurePod(initialPod) + assert(configuredPod === initialPod) + + assert(step.getAdditionalKubernetesResources().isEmpty) + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + + test("Mounts executor template volume if config specified") { + val templateFile = Files.createTempFile("pod-template", "yml").toFile templateFile.deleteOnExit() val sparkConf = new SparkConf(false) .set(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, templateFile.getAbsolutePath) - kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) - } + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) - test("Mounts executor template volume if config specified") { val writer = new PrintWriter(templateFile) writer.write("pod-template-contents") writer.close() 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 e9c05fef6f5d..1bb926cbca23 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 @@ -126,7 +126,7 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { MockitoAnnotations.initMocks(this) kconf = KubernetesTestConf.createDriverConf( resourceNamePrefix = Some(KUBERNETES_RESOURCE_PREFIX)) - when(driverBuilder.buildFromFeatures(kconf)).thenReturn(BUILT_KUBERNETES_SPEC) + when(driverBuilder.buildFromFeatures(kconf, kubernetesClient)).thenReturn(BUILT_KUBERNETES_SPEC) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(POD_NAME)).thenReturn(namedPods) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index 7e7dc4763c2e..6518c91a1a1f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -16,201 +16,21 @@ */ package org.apache.spark.deploy.k8s.submit -import io.fabric8.kubernetes.api.model.PodBuilder import io.fabric8.kubernetes.client.KubernetesClient -import org.mockito.Mockito._ -import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.Config.{CONTAINER_IMAGE, KUBERNETES_DRIVER_PODTEMPLATE_FILE, KUBERNETES_EXECUTOR_PODTEMPLATE_FILE} -import org.apache.spark.deploy.k8s.features._ +import org.apache.spark.internal.config.ConfigEntry -class KubernetesDriverBuilderSuite extends SparkFunSuite { +class KubernetesDriverBuilderSuite extends PodBuilderSuite { - private val BASIC_STEP_TYPE = "basic" - private val CREDENTIALS_STEP_TYPE = "credentials" - private val SERVICE_STEP_TYPE = "service" - private val LOCAL_DIRS_STEP_TYPE = "local-dirs" - private val SECRETS_STEP_TYPE = "mount-secrets" - private val DRIVER_CMD_STEP_TYPE = "driver-command" - private val ENV_SECRETS_STEP_TYPE = "env-secrets" - private val HADOOP_GLOBAL_STEP_TYPE = "hadoop-global" - private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" - private val TEMPLATE_VOLUME_STEP_TYPE = "template-volume" - - private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - BASIC_STEP_TYPE, classOf[BasicDriverFeatureStep]) - - private val credentialsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - CREDENTIALS_STEP_TYPE, classOf[DriverKubernetesCredentialsFeatureStep]) - - private val serviceStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - SERVICE_STEP_TYPE, classOf[DriverServiceFeatureStep]) - - private val localDirsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - LOCAL_DIRS_STEP_TYPE, classOf[LocalDirsFeatureStep]) - - private val secretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - SECRETS_STEP_TYPE, classOf[MountSecretsFeatureStep]) - - private val driverCommandStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - DRIVER_CMD_STEP_TYPE, classOf[DriverCommandFeatureStep]) - - private val envSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) - - private val hadoopGlobalStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_GLOBAL_STEP_TYPE, classOf[KerberosConfDriverFeatureStep]) - - private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) - - private val templateVolumeStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - TEMPLATE_VOLUME_STEP_TYPE, classOf[PodTemplateConfigMapStep] - ) - - private val builderUnderTest: KubernetesDriverBuilder = - new KubernetesDriverBuilder( - _ => basicFeatureStep, - _ => credentialsStep, - _ => serviceStep, - _ => secretsStep, - _ => envSecretsStep, - _ => localDirsStep, - _ => mountVolumesStep, - _ => driverCommandStep, - _ => hadoopGlobalStep, - _ => templateVolumeStep) - - test("Apply fundamental steps all the time.") { - val conf = KubernetesTestConf.createDriverConf() - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) + override protected def templateFileConf: ConfigEntry[_] = { + Config.KUBERNETES_DRIVER_PODTEMPLATE_FILE } - test("Apply secrets step if secrets are present.") { - val conf = KubernetesTestConf.createDriverConf( - secretEnvNamesToKeyRefs = Map("EnvName" -> "SecretName:secretKey"), - secretNamesToMountPaths = Map("secret" -> "secretMountPath")) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - SECRETS_STEP_TYPE, - ENV_SECRETS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) - } - - test("Apply volumes step if mounts are present.") { - val volumeSpec = KubernetesVolumeSpec( - "volume", - "/tmp", - "", - false, - KubernetesHostPathVolumeConf("/path")) - val conf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeSpec)) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - MOUNT_VOLUMES_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) - } - - test("Apply volumes step if a mount subpath is present.") { - val volumeSpec = KubernetesVolumeSpec( - "volume", - "/tmp", - "foo", - false, - KubernetesHostPathVolumeConf("/path")) - val conf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeSpec)) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - MOUNT_VOLUMES_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) - } - - test("Apply template volume step if executor template is present.") { - val sparkConf = new SparkConf(false) - .set(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, "filename") + override protected def buildPod(sparkConf: SparkConf, client: KubernetesClient): SparkPod = { val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE, - TEMPLATE_VOLUME_STEP_TYPE) - } - - private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) - : Unit = { - val addedProperties = resolvedSpec.systemProperties - .filter { case (k, _) => !k.startsWith("spark.") } - .toMap - assert(addedProperties.keys.toSet === stepTypes.toSet) - stepTypes.foreach { stepType => - assert(resolvedSpec.pod.pod.getMetadata.getLabels.get(stepType) === stepType) - assert(resolvedSpec.driverKubernetesResources.containsSlice( - KubernetesFeaturesTestUtils.getSecretsForStepType(stepType))) - assert(resolvedSpec.systemProperties(stepType) === stepType) - } - } - - test("Start with empty pod if template is not specified") { - val kubernetesClient = mock(classOf[KubernetesClient]) - val driverBuilder = KubernetesDriverBuilder.apply(kubernetesClient, new SparkConf()) - verify(kubernetesClient, never()).pods() + new KubernetesDriverBuilder().buildFromFeatures(conf, client).pod } - test("Starts with template if specified") { - val kubernetesClient = PodBuilderSuiteUtils.loadingMockKubernetesClient() - val sparkConf = new SparkConf(false) - .set(CONTAINER_IMAGE, "spark-driver:latest") - .set(KUBERNETES_DRIVER_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) - val driverSpec = KubernetesDriverBuilder - .apply(kubernetesClient, sparkConf) - .buildFromFeatures(kubernetesConf) - PodBuilderSuiteUtils.verifyPodWithSupportedFeatures(driverSpec.pod) - } - - test("Throws on misconfigured pod template") { - val kubernetesClient = PodBuilderSuiteUtils.loadingMockKubernetesClient( - new PodBuilder() - .withNewMetadata() - .addToLabels("test-label-key", "test-label-value") - .endMetadata() - .build()) - val sparkConf = new SparkConf(false) - .set(CONTAINER_IMAGE, "spark-driver:latest") - .set(KUBERNETES_DRIVER_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) - val exception = intercept[SparkException] { - KubernetesDriverBuilder - .apply(kubernetesClient, sparkConf) - .buildFromFeatures(kubernetesConf) - } - assert(exception.getMessage.contains("Could not load pod from template file.")) - } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/PodBuilderSuiteUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/PodBuilderSuiteUtils.scala deleted file mode 100644 index c92e9e6e3b6b..000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/PodBuilderSuiteUtils.scala +++ /dev/null @@ -1,142 +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.deploy.k8s.submit - -import java.io.File - -import io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.KubernetesClient -import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} -import org.mockito.Matchers.any -import org.mockito.Mockito.{mock, when} -import org.scalatest.FlatSpec -import scala.collection.JavaConverters._ - -import org.apache.spark.deploy.k8s.SparkPod - -object PodBuilderSuiteUtils extends FlatSpec { - - def loadingMockKubernetesClient(pod: Pod = podWithSupportedFeatures()): KubernetesClient = { - val kubernetesClient = mock(classOf[KubernetesClient]) - val pods = - mock(classOf[MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]]) - val podResource = mock(classOf[PodResource[Pod, DoneablePod]]) - when(kubernetesClient.pods()).thenReturn(pods) - when(pods.load(any(classOf[File]))).thenReturn(podResource) - when(podResource.get()).thenReturn(pod) - kubernetesClient - } - - def verifyPodWithSupportedFeatures(pod: SparkPod): Unit = { - val metadata = pod.pod.getMetadata - assert(metadata.getLabels.containsKey("test-label-key")) - assert(metadata.getAnnotations.containsKey("test-annotation-key")) - assert(metadata.getNamespace === "namespace") - assert(metadata.getOwnerReferences.asScala.exists(_.getName == "owner-reference")) - val spec = pod.pod.getSpec - assert(!spec.getContainers.asScala.exists(_.getName == "executor-container")) - assert(spec.getDnsPolicy === "dns-policy") - assert(spec.getHostAliases.asScala.exists(_.getHostnames.asScala.exists(_ == "hostname"))) - assert(spec.getImagePullSecrets.asScala.exists(_.getName == "local-reference")) - assert(spec.getInitContainers.asScala.exists(_.getName == "init-container")) - assert(spec.getNodeName == "node-name") - assert(spec.getNodeSelector.get("node-selector-key") === "node-selector-value") - assert(spec.getSchedulerName === "scheduler") - assert(spec.getSecurityContext.getRunAsUser === 1000L) - assert(spec.getServiceAccount === "service-account") - assert(spec.getSubdomain === "subdomain") - assert(spec.getTolerations.asScala.exists(_.getKey == "toleration-key")) - assert(spec.getVolumes.asScala.exists(_.getName == "test-volume")) - val container = pod.container - assert(container.getName === "executor-container") - assert(container.getArgs.contains("arg")) - assert(container.getCommand.equals(List("command").asJava)) - assert(container.getEnv.asScala.exists(_.getName == "env-key")) - assert(container.getResources.getLimits.get("gpu") === - new QuantityBuilder().withAmount("1").build()) - assert(container.getSecurityContext.getRunAsNonRoot) - assert(container.getStdin) - assert(container.getTerminationMessagePath === "termination-message-path") - assert(container.getTerminationMessagePolicy === "termination-message-policy") - assert(pod.container.getVolumeMounts.asScala.exists(_.getName == "test-volume")) - - } - - - def podWithSupportedFeatures(): Pod = new PodBuilder() - .withNewMetadata() - .addToLabels("test-label-key", "test-label-value") - .addToAnnotations("test-annotation-key", "test-annotation-value") - .withNamespace("namespace") - .addNewOwnerReference() - .withController(true) - .withName("owner-reference") - .endOwnerReference() - .endMetadata() - .withNewSpec() - .withDnsPolicy("dns-policy") - .withHostAliases(new HostAliasBuilder().withHostnames("hostname").build()) - .withImagePullSecrets( - new LocalObjectReferenceBuilder().withName("local-reference").build()) - .withInitContainers(new ContainerBuilder().withName("init-container").build()) - .withNodeName("node-name") - .withNodeSelector(Map("node-selector-key" -> "node-selector-value").asJava) - .withSchedulerName("scheduler") - .withNewSecurityContext() - .withRunAsUser(1000L) - .endSecurityContext() - .withServiceAccount("service-account") - .withSubdomain("subdomain") - .withTolerations(new TolerationBuilder() - .withKey("toleration-key") - .withOperator("Equal") - .withEffect("NoSchedule") - .build()) - .addNewVolume() - .withNewHostPath() - .withPath("/test") - .endHostPath() - .withName("test-volume") - .endVolume() - .addNewContainer() - .withArgs("arg") - .withCommand("command") - .addNewEnv() - .withName("env-key") - .withValue("env-value") - .endEnv() - .withImagePullPolicy("Always") - .withName("executor-container") - .withNewResources() - .withLimits(Map("gpu" -> new QuantityBuilder().withAmount("1").build()).asJava) - .endResources() - .withNewSecurityContext() - .withRunAsNonRoot(true) - .endSecurityContext() - .withStdin(true) - .withTerminationMessagePath("termination-message-path") - .withTerminationMessagePolicy("termination-message-policy") - .addToVolumeMounts( - new VolumeMountBuilder() - .withName("test-volume") - .withMountPath("/test") - .build()) - .endContainer() - .endSpec() - .build() - -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index d4fa31af3d5c..278a3821a6f3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -80,8 +80,8 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) when(driverPodOperations.get).thenReturn(driverPod) - when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]), meq(secMgr))) - .thenAnswer(executorPodAnswer()) + when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]), meq(secMgr), + meq(kubernetesClient))).thenAnswer(executorPodAnswer()) snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() waitForExecutorPodsClock = new ManualClock(0L) podsAllocatorUnderTest = new ExecutorPodsAllocator( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index ef521fd801e9..bd716174a827 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -16,147 +16,23 @@ */ package org.apache.spark.scheduler.cluster.k8s -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.{Config => _, _} import io.fabric8.kubernetes.client.KubernetesClient -import org.mockito.Mockito.{mock, never, verify} -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features._ -import org.apache.spark.deploy.k8s.submit.PodBuilderSuiteUtils -import org.apache.spark.util.SparkConfWithEnv - -class KubernetesExecutorBuilderSuite extends SparkFunSuite { - private val BASIC_STEP_TYPE = "basic" - private val SECRETS_STEP_TYPE = "mount-secrets" - private val ENV_SECRETS_STEP_TYPE = "env-secrets" - private val LOCAL_DIRS_STEP_TYPE = "local-dirs" - private val HADOOP_CONF_STEP_TYPE = "hadoop-conf-step" - private val HADOOP_SPARK_USER_STEP_TYPE = "hadoop-spark-user" - private val KERBEROS_CONF_STEP_TYPE = "kerberos-step" - private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" - - private val secMgr = new SecurityManager(new SparkConf(false)) - - private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - BASIC_STEP_TYPE, classOf[BasicExecutorFeatureStep]) - private val mountSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - SECRETS_STEP_TYPE, classOf[MountSecretsFeatureStep]) - private val envSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) - private val localDirsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - LOCAL_DIRS_STEP_TYPE, classOf[LocalDirsFeatureStep]) - private val hadoopConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_CONF_STEP_TYPE, classOf[HadoopConfExecutorFeatureStep]) - private val hadoopSparkUser = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_SPARK_USER_STEP_TYPE, classOf[HadoopSparkUserExecutorFeatureStep]) - private val kerberosConf = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - KERBEROS_CONF_STEP_TYPE, classOf[KerberosConfExecutorFeatureStep]) - private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) +import org.apache.spark.internal.config.ConfigEntry - private val builderUnderTest = new KubernetesExecutorBuilder( - (_, _) => basicFeatureStep, - _ => mountSecretsStep, - _ => envSecretsStep, - _ => localDirsStep, - _ => mountVolumesStep, - _ => hadoopConfStep, - _ => kerberosConf, - _ => hadoopSparkUser) - - test("Basic steps are consistently applied.") { - val conf = KubernetesTestConf.createExecutorConf() - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf, secMgr), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) - } - - test("Apply secrets step if secrets are present.") { - val conf = KubernetesTestConf.createExecutorConf( - secretEnvNamesToKeyRefs = Map("secret-name" -> "secret-key"), - secretNamesToMountPaths = Map("secret" -> "secretMountPath")) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf, secMgr), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - SECRETS_STEP_TYPE, - ENV_SECRETS_STEP_TYPE) - } +class KubernetesExecutorBuilderSuite extends PodBuilderSuite { - test("Apply volumes step if mounts are present.") { - val volumeSpec = KubernetesVolumeSpec( - "volume", - "/tmp", - "", - false, - KubernetesHostPathVolumeConf("/checkpoint")) - val conf = KubernetesTestConf.createExecutorConf( - volumes = Seq(volumeSpec)) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf, secMgr), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - MOUNT_VOLUMES_STEP_TYPE) + override protected def templateFileConf: ConfigEntry[_] = { + Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE } - test("Apply basicHadoop step if HADOOP_CONF_DIR is defined") { - // HADOOP_DELEGATION_TOKEN - val conf = KubernetesTestConf.createExecutorConf( - sparkConf = new SparkConfWithEnv(Map("HADOOP_CONF_DIR" -> "/var/hadoop-conf")) - .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") - .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name")) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf, secMgr), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - HADOOP_CONF_STEP_TYPE, - HADOOP_SPARK_USER_STEP_TYPE) + override protected def buildPod(sparkConf: SparkConf, client: KubernetesClient): SparkPod = { + sparkConf.set("spark.driver.host", "https://driver.host.com") + val conf = KubernetesTestConf.createExecutorConf(sparkConf = sparkConf) + val secMgr = new SecurityManager(sparkConf) + new KubernetesExecutorBuilder().buildFromFeatures(conf, secMgr, client) } - test("Apply kerberos step if DT secrets created") { - val conf = KubernetesTestConf.createExecutorConf( - sparkConf = new SparkConf(false) - .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") - .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") - .set(KERBEROS_SPARK_USER_NAME, "spark-user") - .set(KERBEROS_DT_SECRET_NAME, "dt-secret") - .set(KERBEROS_DT_SECRET_KEY, "dt-key" )) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf, secMgr), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - HADOOP_CONF_STEP_TYPE, - KERBEROS_CONF_STEP_TYPE) - } - - private def validateStepTypesApplied(resolvedPod: SparkPod, stepTypes: String*): Unit = { - assert(resolvedPod.pod.getMetadata.getLabels.asScala.keys.toSet === stepTypes.toSet) - } - - test("Starts with empty executor pod if template is not specified") { - val kubernetesClient = mock(classOf[KubernetesClient]) - val executorBuilder = KubernetesExecutorBuilder.apply(kubernetesClient, new SparkConf()) - verify(kubernetesClient, never()).pods() - } - - test("Starts with executor template if specified") { - val kubernetesClient = PodBuilderSuiteUtils.loadingMockKubernetesClient() - val sparkConf = new SparkConf(false) - .set("spark.driver.host", "https://driver.host.com") - .set(Config.CONTAINER_IMAGE, "spark-executor:latest") - .set(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = KubernetesTestConf.createExecutorConf( - sparkConf = sparkConf, - driverPod = Some(new PodBuilder() - .withNewMetadata() - .withName("driver") - .endMetadata() - .build())) - val sparkPod = KubernetesExecutorBuilder(kubernetesClient, sparkConf) - .buildFromFeatures(kubernetesConf, secMgr) - PodBuilderSuiteUtils.verifyPodWithSupportedFeatures(sparkPod) - } } From 2920438c43ade38e62442b0ba8937b716a05f7ad Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Wed, 12 Dec 2018 16:18:22 -0800 Subject: [PATCH 0120/1072] [SPARK-25277][YARN] YARN applicationMaster metrics should not register static metrics ## What changes were proposed in this pull request? YARN applicationMaster metrics registration introduced in SPARK-24594 causes further registration of static metrics (Codegenerator and HiveExternalCatalog) and of JVM metrics, which I believe do not belong in this context. This looks like an unintended side effect of using the start method of [[MetricsSystem]]. A possible solution proposed here, is to introduce startNoRegisterSources to avoid these additional registrations of static sources and of JVM sources in the case of YARN applicationMaster metrics (this could be useful for other metrics that may be added in the future). ## How was this patch tested? Manually tested on a YARN cluster, Closes #22279 from LucaCanali/YarnMetricsRemoveExtraSourceRegistration. Lead-authored-by: Luca Canali Co-authored-by: LucaCanali Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/metrics/MetricsSystem.scala | 8 +++++--- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 ++- 2 files changed, 7 insertions(+), 4 deletions(-) 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 bb7b434e9a11..301317a79dfc 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -94,11 +94,13 @@ private[spark] class MetricsSystem private ( metricsConfig.initialize() - def start() { + def start(registerStaticSources: Boolean = true) { require(!running, "Attempting to start a MetricsSystem that is already running") running = true - StaticSources.allSources.foreach(registerSource) - registerSources() + if (registerStaticSources) { + StaticSources.allSources.foreach(registerSource) + registerSources() + } registerSinks() sinks.foreach(_.start) } 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 c1f3211bcab2..e46c4f970c4a 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 @@ -449,7 +449,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends val ms = MetricsSystem.createMetricsSystem("applicationMaster", sparkConf, securityMgr) val prefix = _sparkConf.get(YARN_METRICS_NAMESPACE).getOrElse(appId) ms.registerSource(new ApplicationMasterSource(prefix, allocator)) - ms.start() + // do not register static sources in this case as per SPARK-25277 + ms.start(false) metricsSystem = Some(ms) reporterThread = launchReporterThread() } From 6daa78309460e338dd688cf6cdbd46a12666f72e Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Wed, 12 Dec 2018 16:45:50 -0800 Subject: [PATCH 0121/1072] [SPARK-26322][SS] Add spark.kafka.sasl.token.mechanism to ease delegation token configuration. ## What changes were proposed in this pull request? When Kafka delegation token obtained, SCRAM `sasl.mechanism` has to be configured for authentication. This can be configured on the related source/sink which is inconvenient from user perspective. Such granularity is not required and this configuration can be implemented with one central parameter. In this PR `spark.kafka.sasl.token.mechanism` added to configure this centrally (default: `SCRAM-SHA-512`). ## How was this patch tested? Existing unit tests + on cluster. Closes #23274 from gaborgsomogyi/SPARK-26322. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../apache/spark/internal/config/Kafka.scala | 9 ++ .../structured-streaming-kafka-integration.md | 144 +----------------- .../sql/kafka010/KafkaSourceProvider.scala | 15 +- 3 files changed, 21 insertions(+), 147 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala b/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala index 064fc93cb8ed..e91ddd3e9741 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Kafka.scala @@ -79,4 +79,13 @@ private[spark] object Kafka { "For further details please see kafka documentation. Only used to obtain delegation token.") .stringConf .createOptional + + val TOKEN_SASL_MECHANISM = + ConfigBuilder("spark.kafka.sasl.token.mechanism") + .doc("SASL mechanism used for client connections with delegation token. Because SCRAM " + + "login module used for authentication a compatible mechanism has to be set here. " + + "For further details please see kafka documentation (sasl.mechanism). Only used to " + + "authenticate against Kafka broker with delegation token.") + .stringConf + .createWithDefault("SCRAM-SHA-512") } diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 7040f8da2c61..3d64ec4cb55f 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -642,9 +642,9 @@ This way the application can be configured via Spark parameters and may not need configuration (Spark can use Kafka's dynamic JAAS configuration feature). For further information about delegation tokens, see [Kafka delegation token docs](http://kafka.apache.org/documentation/#security_delegation_token). -The process is initiated by Spark's Kafka delegation token provider. When `spark.kafka.bootstrap.servers`, +The process is initiated by Spark's Kafka delegation token provider. When `spark.kafka.bootstrap.servers` is set, Spark considers the following log in options, in order of preference: -- **JAAS login configuration** +- **JAAS login configuration**, please see example below. - **Keytab file**, such as, ./bin/spark-submit \ @@ -669,144 +669,8 @@ Kafka broker configuration): After obtaining delegation token successfully, Spark distributes it across nodes and renews it accordingly. Delegation token uses `SCRAM` login module for authentication and because of that the appropriate -`sasl.mechanism` has to be configured on source/sink (it must match with Kafka broker configuration): - -
-
-{% highlight scala %} - -// Setting on Kafka Source for Streaming Queries -val df = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") - .option("subscribe", "topic1") - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] - -// Setting on Kafka Source for Batch Queries -val df = spark - .read - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") - .option("subscribe", "topic1") - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] - -// Setting on Kafka Sink for Streaming Queries -val ds = df - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .writeStream - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") - .option("topic", "topic1") - .start() - -// Setting on Kafka Sink for Batch Queries -val ds = df - .selectExpr("topic1", "CAST(key AS STRING)", "CAST(value AS STRING)") - .write - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") - .save() - -{% endhighlight %} -
-
-{% highlight java %} - -// Setting on Kafka Source for Streaming Queries -Dataset df = spark - .readStream() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") - .option("subscribe", "topic1") - .load(); -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); - -// Setting on Kafka Source for Batch Queries -Dataset df = spark - .read() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") - .option("subscribe", "topic1") - .load(); -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); - -// Setting on Kafka Sink for Streaming Queries -StreamingQuery ds = df - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .writeStream() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") - .option("topic", "topic1") - .start(); - -// Setting on Kafka Sink for Batch Queries -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .write() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") - .option("topic", "topic1") - .save(); - -{% endhighlight %} -
-
-{% highlight python %} - -// Setting on Kafka Source for Streaming Queries -df = spark \ - .readStream \ - .format("kafka") \ - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") \ - .option("subscribe", "topic1") \ - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - -// Setting on Kafka Source for Batch Queries -df = spark \ - .read \ - .format("kafka") \ - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") \ - .option("subscribe", "topic1") \ - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - -// Setting on Kafka Sink for Streaming Queries -ds = df \ - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") \ - .writeStream \ - .format("kafka") \ - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") \ - .option("topic", "topic1") \ - .start() - -// Setting on Kafka Sink for Batch Queries -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") \ - .write \ - .format("kafka") \ - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ - .option("kafka.sasl.mechanism", "SCRAM-SHA-512") \ - .option("topic", "topic1") \ - .save() - -{% endhighlight %} -
-
+`spark.kafka.sasl.token.mechanism` (default: `SCRAM-SHA-512`) has to be configured. Also, this parameter +must match with Kafka broker configuration. When delegation token is available on an executor it can be overridden with JAAS login configuration. diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 0ac330435e5c..6a0c2088ac3d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -30,6 +30,7 @@ import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySe import org.apache.spark.SparkEnv import org.apache.spark.deploy.security.KafkaTokenUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ @@ -501,7 +502,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { // If buffer config is not set, set it to reasonable value to work around // buffer issues (see KAFKA-3135) .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) - .setTokenJaasConfigIfNeeded() + .setAuthenticationConfigIfNeeded() .build() def kafkaParamsForExecutors( @@ -523,7 +524,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { // If buffer config is not set, set it to reasonable value to work around // buffer issues (see KAFKA-3135) .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) - .setTokenJaasConfigIfNeeded() + .setAuthenticationConfigIfNeeded() .build() /** @@ -556,7 +557,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { this } - def setTokenJaasConfigIfNeeded(): ConfigUpdater = { + def setAuthenticationConfigIfNeeded(): ConfigUpdater = { // There are multiple possibilities to log in and applied in the following order: // - JVM global security provided -> try to log in with JVM global security configuration // which can be configured for example with 'java.security.auth.login.config'. @@ -568,11 +569,11 @@ private[kafka010] object KafkaSourceProvider extends Logging { } else if (KafkaSecurityHelper.isTokenAvailable()) { logDebug("Delegation token detected, using it for login.") val jaasParams = KafkaSecurityHelper.getTokenJaasParams(SparkEnv.get.conf) - val mechanism = kafkaParams - .getOrElse(SaslConfigs.SASL_MECHANISM, SaslConfigs.DEFAULT_SASL_MECHANISM) + set(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + val mechanism = SparkEnv.get.conf.get(Kafka.TOKEN_SASL_MECHANISM) require(mechanism.startsWith("SCRAM"), "Delegation token works only with SCRAM mechanism.") - set(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + set(SaslConfigs.SASL_MECHANISM, mechanism) } this } @@ -600,7 +601,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { ConfigUpdater("executor", specifiedKafkaParams) .set(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, serClassName) .set(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, serClassName) - .setTokenJaasConfigIfNeeded() + .setAuthenticationConfigIfNeeded() .build() } From 05b68d5cc92e46bd701cd01b4179cd13397eaf90 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 13 Dec 2018 11:13:15 +0800 Subject: [PATCH 0122/1072] [SPARK-26297][SQL] improve the doc of Distribution/Partitioning ## What changes were proposed in this pull request? Some documents of `Distribution/Partitioning` are stale and misleading, this PR fixes them: 1. `Distribution` never have intra-partition requirement 2. `OrderedDistribution` does not require tuples that share the same value being colocated in the same partition. 3. `RangePartitioning` can provide a weaker guarantee for a prefix of its `ordering` expressions. ## How was this patch tested? comment-only PR. Closes #23249 from cloud-fan/doc. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../plans/physical/partitioning.scala | 54 ++++++++++++------- 1 file changed, 34 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index cc1a5e835d9c..17e1cb416fc8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -22,13 +22,11 @@ import org.apache.spark.sql.types.{DataType, IntegerType} /** * Specifies how tuples that share common expressions will be distributed when a query is executed - * in parallel on many machines. Distribution can be used to refer to two distinct physical - * properties: - * - Inter-node partitioning of data: In this case the distribution describes how tuples are - * partitioned across physical machines in a cluster. Knowing this property allows some - * operators (e.g., Aggregate) to perform partition local operations instead of global ones. - * - Intra-partition ordering of data: In this case the distribution describes guarantees made - * about how tuples are distributed within a single partition. + * in parallel on many machines. + * + * Distribution here refers to inter-node partitioning of data. That is, it describes how tuples + * are partitioned across physical machines in a cluster. Knowing this property allows some + * operators (e.g., Aggregate) to perform partition local operations instead of global ones. */ sealed trait Distribution { /** @@ -70,9 +68,7 @@ case object AllTuples extends Distribution { /** * Represents data where tuples that share the same values for the `clustering` - * [[Expression Expressions]] will be co-located. Based on the context, this - * can mean such tuples are either co-located in the same partition or they will be contiguous - * within a single partition. + * [[Expression Expressions]] will be co-located in the same partition. */ case class ClusteredDistribution( clustering: Seq[Expression], @@ -118,10 +114,12 @@ case class HashClusteredDistribution( /** * Represents data where tuples have been ordered according to the `ordering` - * [[Expression Expressions]]. This is a strictly stronger guarantee than - * [[ClusteredDistribution]] as an ordering will ensure that tuples that share the - * same value for the ordering expressions are contiguous and will never be split across - * partitions. + * [[Expression Expressions]]. Its requirement is defined as the following: + * - Given any 2 adjacent partitions, all the rows of the second partition must be larger than or + * equal to any row in the first partition, according to the `ordering` expressions. + * + * In other words, this distribution requires the rows to be ordered across partitions, but not + * necessarily within a partition. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { require( @@ -241,12 +239,12 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) /** * Represents a partitioning where rows are split across partitions based on some total ordering of - * the expressions specified in `ordering`. When data is partitioned in this manner the following - * two conditions are guaranteed to hold: - * - All row where the expressions in `ordering` evaluate to the same values will be in the same - * partition. - * - Each partition will have a `min` and `max` row, relative to the given ordering. All rows - * that are in between `min` and `max` in this `ordering` will reside in this partition. + * the expressions specified in `ordering`. When data is partitioned in this manner, it guarantees: + * Given any 2 adjacent partitions, all the rows of the second partition must be larger than any row + * in the first partition, according to the `ordering` expressions. + * + * This is a strictly stronger guarantee than what `OrderedDistribution(ordering)` requires, as + * there is no overlap between partitions. * * This class extends expression primarily so that transformations over expression will descend * into its child. @@ -262,6 +260,22 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) super.satisfies0(required) || { required match { case OrderedDistribution(requiredOrdering) => + // If `ordering` is a prefix of `requiredOrdering`: + // Let's say `ordering` is [a, b] and `requiredOrdering` is [a, b, c]. According to the + // RangePartitioning definition, any [a, b] in a previous partition must be smaller + // than any [a, b] in the following partition. This also means any [a, b, c] in a + // previous partition must be smaller than any [a, b, c] in the following partition. + // Thus `RangePartitioning(a, b)` satisfies `OrderedDistribution(a, b, c)`. + // + // If `requiredOrdering` is a prefix of `ordering`: + // Let's say `ordering` is [a, b, c] and `requiredOrdering` is [a, b]. According to the + // RangePartitioning definition, any [a, b, c] in a previous partition must be smaller + // than any [a, b, c] in the following partition. If there is a [a1, b1] from a previous + // partition which is larger than a [a2, b2] from the following partition, then there + // must be a [a1, b1 c1] larger than [a2, b2, c2], which violates RangePartitioning + // definition. So it's guaranteed that, any [a, b] in a previous partition must not be + // greater(i.e. smaller or equal to) than any [a, b] in the following partition. Thus + // `RangePartitioning(a, b, c)` satisfies `OrderedDistribution(a, b)`. val minSize = Seq(requiredOrdering.size, ordering.size).min requiredOrdering.take(minSize) == ordering.take(minSize) case ClusteredDistribution(requiredClustering, _) => From 3238e3d1c0d9be5c43a72705e18afbbb4c512e15 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 13 Dec 2018 12:50:15 +0800 Subject: [PATCH 0123/1072] [SPARK-26348][SQL][TEST] make sure expression is resolved during test ## What changes were proposed in this pull request? cleanup some tests to make sure expression is resolved during test. ## How was this patch tested? test-only PR Closes #23297 from cloud-fan/test. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../expressions/ExpressionEvalHelper.scala | 10 ++++---- .../expressions/JsonExpressionsSuite.scala | 11 ++++----- .../catalyst/expressions/PredicateSuite.scala | 23 ++++++------------- .../expressions/StringExpressionsSuite.scala | 7 ++---- 5 files changed, 20 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 176ea823b1fc..151481c80ee9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -136,7 +136,7 @@ package object dsl { implicit def longToLiteral(l: Long): Literal = Literal(l) implicit def floatToLiteral(f: Float): Literal = Literal(f) implicit def doubleToLiteral(d: Double): Literal = Literal(d) - implicit def stringToLiteral(s: String): Literal = Literal(s) + implicit def stringToLiteral(s: String): Literal = Literal.create(s, StringType) implicit def dateToLiteral(d: Date): Literal = Literal(d) implicit def bigDecimalToLiteral(d: BigDecimal): Literal = Literal(d.underlying()) implicit def bigDecimalToLiteral(d: java.math.BigDecimal): Literal = Literal(d) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index b4fd170467d8..1c91adab7137 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -28,7 +28,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.analysis.{ResolveTimeZone, SimpleAnalyzer} +import org.apache.spark.sql.catalyst.analysis.ResolveTimeZone import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.optimizer.SimpleTestOptimizer import org.apache.spark.sql.catalyst.plans.PlanTestBase @@ -70,7 +70,9 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa private def prepareEvaluation(expression: Expression): Expression = { val serializer = new JavaSerializer(new SparkConf()).newInstance val resolver = ResolveTimeZone(new SQLConf) - resolver.resolveTimeZones(serializer.deserialize(serializer.serialize(expression))) + val expr = resolver.resolveTimeZones(expression) + assert(expr.resolved) + serializer.deserialize(serializer.serialize(expr)) } protected def checkEvaluation( @@ -296,9 +298,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa expected: Any, inputRow: InternalRow = EmptyRow): Unit = { val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, OneRowRelation()) - // We should analyze the plan first, otherwise we possibly optimize an unresolved plan. - val analyzedPlan = SimpleAnalyzer.execute(plan) - val optimizedPlan = SimpleTestOptimizer.execute(analyzedPlan) + val optimizedPlan = SimpleTestOptimizer.execute(plan) checkEvaluationWithoutCodegen(optimizedPlan.expressions.head, expected, inputRow) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 5d60cefc1389..238e6e34b4ae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.exceptions.TestFailedException import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf @@ -694,11 +694,10 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val mapType2 = MapType(IntegerType, CalendarIntervalType) val schema2 = StructType(StructField("a", mapType2) :: Nil) val struct2 = Literal.create(null, schema2) - intercept[TreeNodeException[_]] { - checkEvaluation( - StructsToJson(Map.empty, struct2, gmtId), - null - ) + StructsToJson(Map.empty, struct2, gmtId).checkInputDataTypes() match { + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("Unable to convert column a of type calendarinterval to JSON")) + case _ => fail("from_json should not work on interval map value type.") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index 0f63717f9daf..3541afcd2144 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -24,6 +24,7 @@ import scala.collection.immutable.HashSet import org.apache.spark.SparkFunSuite import org.apache.spark.sql.RandomDataGenerator import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.encoders.ExamplePointUDT import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} @@ -231,22 +232,12 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { testWithRandomDataGeneration(structType, nullable) } - // Map types: not supported - for ( - keyType <- atomicTypes; - valueType <- atomicTypes; - nullable <- Seq(true, false)) { - val mapType = MapType(keyType, valueType) - val e = intercept[Exception] { - testWithRandomDataGeneration(mapType, nullable) - } - if (e.getMessage.contains("Code generation of")) { - // If the `value` expression is null, `eval` will be short-circuited. - // Codegen version evaluation will be run then. - assert(e.getMessage.contains("cannot generate equality code for un-comparable type")) - } else { - assert(e.getMessage.contains("Exception evaluating")) - } + // In doesn't support map type and will fail the analyzer. + val map = Literal.create(create_map(1 -> 1), MapType(IntegerType, IntegerType)) + In(map, Seq(map)).checkInputDataTypes() match { + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("function in does not support ordering on type map")) + case _ => fail("In should not work on map type") } } 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 aa334e040d5f..e95f2dff231b 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 @@ -744,16 +744,14 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("ParseUrl") { def checkParseUrl(expected: String, urlStr: String, partToExtract: String): Unit = { - checkEvaluation( - ParseUrl(Seq(Literal(urlStr), Literal(partToExtract))), expected) + checkEvaluation(ParseUrl(Seq(urlStr, partToExtract)), expected) } def checkParseUrlWithKey( expected: String, urlStr: String, partToExtract: String, key: String): Unit = { - checkEvaluation( - ParseUrl(Seq(Literal(urlStr), Literal(partToExtract), Literal(key))), expected) + checkEvaluation(ParseUrl(Seq(urlStr, partToExtract, key)), expected) } checkParseUrl("spark.apache.org", "http://spark.apache.org/path?query=1", "HOST") @@ -798,7 +796,6 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Sentences(nullString, nullString, nullString), null) checkEvaluation(Sentences(nullString, nullString), null) checkEvaluation(Sentences(nullString), null) - checkEvaluation(Sentences(Literal.create(null, NullType)), null) checkEvaluation(Sentences("", nullString, nullString), Seq.empty) checkEvaluation(Sentences("", nullString), Seq.empty) checkEvaluation(Sentences(""), Seq.empty) From 8edae94fa7ec1a1cc2c69e0924da0da85d4aac83 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 13 Dec 2018 13:14:59 +0800 Subject: [PATCH 0124/1072] [SPARK-26355][PYSPARK] Add a workaround for PyArrow 0.11. ## What changes were proposed in this pull request? In PyArrow 0.11, there is a API breaking change. - [ARROW-1949](https://issues.apache.org/jira/browse/ARROW-1949) - [Python/C++] Add option to Array.from_pandas and pyarrow.array to perform unsafe casts. This causes test failures in `ScalarPandasUDFTests.test_vectorized_udf_null_(byte|short|int|long)`: ``` File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/worker.py", line 377, in main process() File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/worker.py", line 372, in process serializer.dump_stream(func(split_index, iterator), outfile) File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 317, in dump_stream batch = _create_batch(series, self._timezone) File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 286, in _create_batch arrs = [create_array(s, t) for s, t in series] File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 284, in create_array return pa.Array.from_pandas(s, mask=mask, type=t) File "pyarrow/array.pxi", line 474, in pyarrow.lib.Array.from_pandas return array(obj, mask=mask, type=type, safe=safe, from_pandas=True, File "pyarrow/array.pxi", line 169, in pyarrow.lib.array return _ndarray_to_array(values, mask, type, from_pandas, safe, File "pyarrow/array.pxi", line 69, in pyarrow.lib._ndarray_to_array check_status(NdarrayToArrow(pool, values, mask, from_pandas, File "pyarrow/error.pxi", line 81, in pyarrow.lib.check_status raise ArrowInvalid(message) ArrowInvalid: Floating point value truncated ``` We should add a workaround to support PyArrow 0.11. ## How was this patch tested? In my local environment. Closes #23305 from ueshin/issues/SPARK-26355/pyarrow_0.11. Authored-by: Takuya UESHIN Signed-off-by: Hyukjin Kwon --- python/pyspark/serializers.py | 5 ++++- .../pyspark/sql/tests/test_pandas_udf_grouped_map.py | 11 +++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index f3ebd3767a0a..fd4695210fb7 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -281,7 +281,10 @@ def create_array(s, t): # TODO: see ARROW-2432. Remove when the minimum PyArrow version becomes 0.10.0. return pa.Array.from_pandas(s.apply( lambda v: decimal.Decimal('NaN') if v is None else v), mask=mask, type=t) - return pa.Array.from_pandas(s, mask=mask, type=t) + elif LooseVersion(pa.__version__) < LooseVersion("0.11.0"): + # TODO: see ARROW-1949. Remove when the minimum PyArrow version becomes 0.11.0. + return pa.Array.from_pandas(s, mask=mask, type=t) + return pa.Array.from_pandas(s, mask=mask, type=t, safe=False) arrs = [create_array(s, t) for s, t in series] return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in xrange(len(arrs))]) diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py index bfecc071386e..a12c608dff9d 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py @@ -468,8 +468,15 @@ def invalid_positional_types(pdf): with QuietTest(self.sc): with self.assertRaisesRegexp(Exception, "KeyError: 'id'"): grouped_df.apply(column_name_typo).collect() - with self.assertRaisesRegexp(Exception, "No cast implemented"): - grouped_df.apply(invalid_positional_types).collect() + from distutils.version import LooseVersion + import pyarrow as pa + if LooseVersion(pa.__version__) < LooseVersion("0.11.0"): + # TODO: see ARROW-1949. Remove when the minimum PyArrow version becomes 0.11.0. + with self.assertRaisesRegexp(Exception, "No cast implemented"): + grouped_df.apply(invalid_positional_types).collect() + else: + with self.assertRaisesRegexp(Exception, "an integer is required"): + grouped_df.apply(invalid_positional_types).collect() def test_positional_assignment_conf(self): import pandas as pd From 19b63c560d92a0e30b2dfc523bcce4f1c9daf851 Mon Sep 17 00:00:00 2001 From: Qi Shao Date: Thu, 13 Dec 2018 20:05:49 +0800 Subject: [PATCH 0125/1072] [MINOR][R] Fix indents of sparkR welcome message to be consistent with pyspark and spark-shell ## What changes were proposed in this pull request? 1. Removed empty space at the beginning of welcome message lines of sparkR to be consistent with welcome message of `pyspark` and `spark-shell` 2. Setting indent of logo message lines to 3 to be consistent with welcome message of `pyspark` and `spark-shell` Output of `pyspark`: ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /__ / .__/\_,_/_/ /_/\_\ version 2.4.0 /_/ Using Python version 3.6.6 (default, Jun 28 2018 11:07:29) SparkSession available as 'spark'. ``` Output of `spark-shell`: ``` Spark session available as 'spark'. Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.4.0 /_/ Using Scala version 2.11.12 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_161) Type in expressions to have them evaluated. Type :help for more information. ``` ## How was this patch tested? Before: Output of `sparkR`: ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.4.0 /_/ SparkSession available as 'spark'. ``` After: ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.4.0 /_/ SparkSession available as 'spark'. ``` Closes #23293 from AzureQ/master. Authored-by: Qi Shao Signed-off-by: Hyukjin Kwon --- R/pkg/inst/profile/shell.R | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R index 32eb3671b594..e4e0d032997d 100644 --- a/R/pkg/inst/profile/shell.R +++ b/R/pkg/inst/profile/shell.R @@ -33,19 +33,19 @@ sc <- SparkR:::callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", spark) assign("sc", sc, envir = .GlobalEnv) sparkVer <- SparkR:::callJMethod(sc, "version") - cat("\n Welcome to") + cat("\nWelcome to") cat("\n") - cat(" ____ __", "\n") - cat(" / __/__ ___ _____/ /__", "\n") - cat(" _\\ \\/ _ \\/ _ `/ __/ '_/", "\n") - cat(" /___/ .__/\\_,_/_/ /_/\\_\\") + cat(" ____ __", "\n") + cat(" / __/__ ___ _____/ /__", "\n") + cat(" _\\ \\/ _ \\/ _ `/ __/ '_/", "\n") + cat(" /___/ .__/\\_,_/_/ /_/\\_\\") if (nchar(sparkVer) == 0) { cat("\n") } else { - cat(" version ", sparkVer, "\n") + cat(" version", sparkVer, "\n") } - cat(" /_/", "\n") + cat(" /_/", "\n") cat("\n") - cat("\n SparkSession available as 'spark'.\n") + cat("\nSparkSession available as 'spark'.\n") } From f3726092169406979849b3cb5afeb52be106fd68 Mon Sep 17 00:00:00 2001 From: seancxmao Date: Thu, 13 Dec 2018 07:40:13 -0600 Subject: [PATCH 0126/1072] [MINOR][DOC] Fix comments of ConvertToLocalRelation rule ## What changes were proposed in this pull request? There are some comments issues left when `ConvertToLocalRelation` rule was added (see #22205/[SPARK-25212](https://issues.apache.org/jira/browse/SPARK-25212)). This PR fixes those comments issues. ## How was this patch tested? N/A Closes #23273 from seancxmao/ConvertToLocalRelation-doc. Authored-by: seancxmao Signed-off-by: Sean Owen --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 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 8d251eeab848..f615757a837a 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 @@ -131,11 +131,11 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) // since the other rules might make two separate Unions operators adjacent. Batch("Union", Once, CombineUnions) :: - // run this once earlier. this might simplify the plan and reduce cost of optimizer. - // for example, a query such as Filter(LocalRelation) would go through all the heavy + // Run this once earlier. This might simplify the plan and reduce cost of optimizer. + // For example, a query such as Filter(LocalRelation) would go through all the heavy // optimizer rules that are triggered when there is a filter - // (e.g. InferFiltersFromConstraints). if we run this batch earlier, the query becomes just - // LocalRelation and does not trigger many rules + // (e.g. InferFiltersFromConstraints). If we run this batch earlier, the query becomes just + // LocalRelation and does not trigger many rules. Batch("LocalRelation early", fixedPoint, ConvertToLocalRelation, PropagateEmptyRelation) :: @@ -1370,10 +1370,8 @@ object DecimalAggregates extends Rule[LogicalPlan] { } /** - * Converts local operations (i.e. ones that don't require data exchange) on LocalRelation to - * another LocalRelation. - * - * This is relatively simple as it currently handles only 2 single case: Project and Limit. + * Converts local operations (i.e. ones that don't require data exchange) on `LocalRelation` to + * another `LocalRelation`. */ object ConvertToLocalRelation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { From f69998ace6b2e0665e45839c3e98c77a18be442a Mon Sep 17 00:00:00 2001 From: lichaoqun Date: Thu, 13 Dec 2018 07:42:17 -0600 Subject: [PATCH 0127/1072] [MINOR][DOC] update the condition description of BypassMergeSortShuffle MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? These three condition descriptions should be updated, follow #23228 :
  • no Ordering is specified,
  • no Aggregator is specified, and
  • the number of partitions is less than spark.shuffle.sort.bypassMergeThreshold.
  • 1、If the shuffle dependency specifies aggregation, but it only aggregates at the reduce-side, BypassMergeSortShuffle can still be used. 2、If the number of output partitions is spark.shuffle.sort.bypassMergeThreshold(eg.200), we can use BypassMergeSortShuffle. ## How was this patch tested? N/A Closes #23281 from lcqzte10192193/wid-lcq-1211. Authored-by: lichaoqun Signed-off-by: Sean Owen --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index fda33cd8293d..997bc9e3f043 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 @@ -58,9 +58,8 @@ * simultaneously opens separate serializers and file streams for all partitions. As a result, * {@link SortShuffleManager} only selects this write path when *
      - *
    • no Ordering is specified,
    • - *
    • no Aggregator is specified, and
    • - *
    • the number of partitions is less than + *
    • no map-side combine is specified, and
    • + *
    • the number of partitions is less than or equal to * spark.shuffle.sort.bypassMergeThreshold.
    • *
    * From 29b3eb6fedd8f90495046da598eacc4ac00944c3 Mon Sep 17 00:00:00 2001 From: "n.fraison" Date: Thu, 13 Dec 2018 08:34:47 -0600 Subject: [PATCH 0128/1072] [SPARK-26340][CORE] Ensure cores per executor is greater than cpu per task Currently this check is only performed for dynamic allocation use case in ExecutorAllocationManager. ## What changes were proposed in this pull request? Checks that cpu per task is lower than number of cores per executor otherwise throw an exception ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23290 from ashangit/master. Authored-by: n.fraison Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/SparkConf.scala | 9 +++++++++ .../src/test/scala/org/apache/spark/SparkConfSuite.scala | 7 +++++++ 2 files changed, 16 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 21c5cbc04d81..8d135d3e083d 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -605,6 +605,15 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } + if (contains("spark.executor.cores") && contains("spark.task.cpus")) { + val executorCores = getInt("spark.executor.cores", 1) + val taskCpus = getInt("spark.task.cpus", 1) + + if (executorCores < taskCpus) { + throw new SparkException("spark.executor.cores must not be less than spark.task.cpus.") + } + } + val encryptionEnabled = get(NETWORK_ENCRYPTION_ENABLED) || get(SASL_ENCRYPTION_ENABLED) require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED), s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.") diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index df274d949bae..7cb03deae139 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -138,6 +138,13 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(sc.appName === "My other app") } + test("creating SparkContext with cpus per tasks bigger than cores per executors") { + val conf = new SparkConf(false) + .set("spark.executor.cores", "1") + .set("spark.task.cpus", "2") + intercept[SparkException] { sc = new SparkContext(conf) } + } + test("nested property names") { // This wasn't supported by some external conf parsing libraries System.setProperty("spark.test.a", "a") From 6c1f7ba8f627a69cac74f11400066dd9871d9102 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 13 Dec 2018 23:03:26 +0800 Subject: [PATCH 0129/1072] [SPARK-26313][SQL] move `newScanBuilder` from Table to read related mix-in traits ## What changes were proposed in this pull request? As discussed in https://github.com/apache/spark/pull/23208/files#r239684490 , we should put `newScanBuilder` in read related mix-in traits like `SupportsBatchRead`, to support write-only table. In the `Append` operator, we should skip schema validation if not necessary. In the future we would introduce a capability API, so that data source can tell Spark that it doesn't want to do validation. ## How was this patch tested? existing tests. Closes #23266 from cloud-fan/ds-read. Authored-by: Wenchen Fan Signed-off-by: Hyukjin Kwon --- .../sql/sources/v2/SupportsBatchRead.java | 8 ++--- .../spark/sql/sources/v2/SupportsRead.java | 35 +++++++++++++++++++ .../apache/spark/sql/sources/v2/Table.java | 15 ++------ 3 files changed, 41 insertions(+), 17 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchRead.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchRead.java index 0df89dbb608a..6c5a95d2a75b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchRead.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchRead.java @@ -24,10 +24,10 @@ /** * An empty mix-in interface for {@link Table}, to indicate this table supports batch scan. *

    - * If a {@link Table} implements this interface, its {@link Table#newScanBuilder(DataSourceOptions)} - * must return a {@link ScanBuilder} that builds {@link Scan} with {@link Scan#toBatch()} - * implemented. + * If a {@link Table} implements this interface, the + * {@link SupportsRead#newScanBuilder(DataSourceOptions)} must return a {@link ScanBuilder} that + * builds {@link Scan} with {@link Scan#toBatch()} implemented. *

    */ @Evolving -public interface SupportsBatchRead extends Table { } +public interface SupportsBatchRead extends SupportsRead { } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java new file mode 100644 index 000000000000..e22738d20d50 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.sources.v2.reader.ScanBuilder; + +/** + * An internal base interface of mix-in interfaces for readable {@link Table}. This adds + * {@link #newScanBuilder(DataSourceOptions)} that is used to create a scan for batch, micro-batch, + * or continuous processing. + */ +interface SupportsRead extends Table { + + /** + * Returns a {@link ScanBuilder} which can be used to build a {@link Scan}. Spark will call this + * method to configure each scan. + */ + ScanBuilder newScanBuilder(DataSourceOptions options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java index 0c65fe0f9e76..08664859b8de 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java @@ -18,8 +18,6 @@ package org.apache.spark.sql.sources.v2; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.Scan; -import org.apache.spark.sql.sources.v2.reader.ScanBuilder; import org.apache.spark.sql.types.StructType; /** @@ -43,17 +41,8 @@ public interface Table { String name(); /** - * Returns the schema of this table. + * Returns the schema of this table. If the table is not readable and doesn't have a schema, an + * empty schema can be returned here. */ StructType schema(); - - /** - * Returns a {@link ScanBuilder} which can be used to build a {@link Scan} later. Spark will call - * this method for each data scanning query. - *

    - * The builder can take some query specific information to do operators pushdown, and keep these - * information in the created {@link Scan}. - *

    - */ - ScanBuilder newScanBuilder(DataSourceOptions options); } From 524d1be6d2920674eb871b5f0f25e7496a374090 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 13 Dec 2018 09:07:33 -0800 Subject: [PATCH 0130/1072] [SPARK-26098][WEBUI] Show associated SQL query in Job page ## What changes were proposed in this pull request? For jobs associated to SQL queries, it would be easier to understand the context to showing the SQL query in Job detail page. Before code change, it is hard to tell what the job is about from the job page: ![image](https://user-images.githubusercontent.com/1097932/48659359-96baa180-ea8a-11e8-8419-a0a87c3f30fc.png) After code change: ![image](https://user-images.githubusercontent.com/1097932/48659390-26f8e680-ea8b-11e8-8fdd-3b58909ea364.png) After navigating to the associated SQL detail page, We can see the whole context : ![image](https://user-images.githubusercontent.com/1097932/48659463-9fac7280-ea8c-11e8-9dfe-244e849f72a5.png) **For Jobs don't have associated SQL query, the text won't be shown.** ## How was this patch tested? Manual test Closes #23068 from gengliangwang/addSQLID. Authored-by: Gengliang Wang Signed-off-by: gatorsmile --- .../apache/spark/status/AppStatusListener.scala | 7 ++++++- .../org/apache/spark/status/AppStatusStore.scala | 7 +++++++ .../scala/org/apache/spark/status/LiveEntity.scala | 5 +++-- .../scala/org/apache/spark/status/storeTypes.scala | 3 ++- .../scala/org/apache/spark/ui/jobs/JobPage.scala | 14 +++++++++++++- 5 files changed, 31 insertions(+), 5 deletions(-) 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 bd3f58b6182c..262ff6547faa 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -70,6 +70,8 @@ private[spark] class AppStatusListener( private val liveTasks = new HashMap[Long, LiveTask]() private val liveRDDs = new HashMap[Int, LiveRDD]() private val pools = new HashMap[String, SchedulerPool]() + + private val SQL_EXECUTION_ID_KEY = "spark.sql.execution.id" // Keep the active executor count as a separate variable to avoid having to do synchronization // around liveExecutors. @volatile private var activeExecutorCount = 0 @@ -318,6 +320,8 @@ private[spark] class AppStatusListener( val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") val jobGroup = Option(event.properties) .flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) } + val sqlExecutionId = Option(event.properties) + .flatMap(p => Option(p.getProperty(SQL_EXECUTION_ID_KEY)).map(_.toLong)) val job = new LiveJob( event.jobId, @@ -325,7 +329,8 @@ private[spark] class AppStatusListener( if (event.time > 0) Some(new Date(event.time)) else None, event.stageIds, jobGroup, - numTasks) + numTasks, + sqlExecutionId) liveJobs.put(event.jobId, job) liveUpdate(job, 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 b35781cb36e8..312bcccb1cca 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -56,6 +56,13 @@ private[spark] class AppStatusStore( store.read(classOf[JobDataWrapper], jobId).info } + // Returns job data and associated SQL execution ID of certain Job ID. + // If there is no related SQL execution, the SQL execution ID part will be None. + def jobWithAssociatedSql(jobId: Int): (v1.JobData, Option[Long]) = { + val data = store.read(classOf[JobDataWrapper], jobId) + (data.info, data.sqlExecutionId) + } + def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = { val base = store.view(classOf[ExecutorSummaryWrapper]) val filtered = if (activeOnly) { 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 47e45a66eccc..7f7b83a54d79 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -64,7 +64,8 @@ private class LiveJob( val submissionTime: Option[Date], val stageIds: Seq[Int], jobGroup: Option[String], - numTasks: Int) extends LiveEntity { + numTasks: Int, + sqlExecutionId: Option[Long]) extends LiveEntity { var activeTasks = 0 var completedTasks = 0 @@ -108,7 +109,7 @@ private class LiveJob( skippedStages.size, failedStages, killedSummary) - new JobDataWrapper(info, skippedStages) + new JobDataWrapper(info, skippedStages, sqlExecutionId) } } 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 ef19e86f3135..eea47b3b1709 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -68,7 +68,8 @@ private[spark] class ExecutorSummaryWrapper(val info: ExecutorSummary) { */ private[spark] class JobDataWrapper( val info: JobData, - val skippedStages: Set[Int]) { + val skippedStages: Set[Int], + val sqlExecutionId: Option[Long]) { @JsonIgnore @KVIndex private def id: Int = info.jobId 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 55444a2c0c9a..b58a6ca447ed 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 @@ -189,7 +189,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") val jobId = parameterId.toInt - val jobData = store.asOption(store.job(jobId)).getOrElse { + val (jobData, sqlExecutionId) = store.asOption(store.jobWithAssociatedSql(jobId)).getOrElse { val content =

    No information to display for job {jobId}

    @@ -197,6 +197,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP return UIUtils.headerSparkPage( request, s"Details for Job $jobId", content, parent) } + val isComplete = jobData.status != JobExecutionStatus.RUNNING val stages = jobData.stageIds.map { stageId => // This could be empty if the listener hasn't received information about the @@ -278,6 +279,17 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP Status: {jobData.status} + { + if (sqlExecutionId.isDefined) { +
  • + Associated SQL Query: + {{sqlExecutionId.get}} +
  • + } + } { if (jobData.jobGroup.isDefined) {
  • From 362e472831e0609f88fdeb01d8e14badc812b0f4 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Thu, 13 Dec 2018 16:12:55 -0800 Subject: [PATCH 0131/1072] [SPARK-23886][SS] Update query status for ContinuousExecution ## What changes were proposed in this pull request? Added query status updates to ContinuousExecution. ## How was this patch tested? Existing unit tests + added ContinuousQueryStatusAndProgressSuite. Closes #23095 from gaborgsomogyi/SPARK-23886. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../streaming/MicroBatchExecution.scala | 6 ++ .../streaming/ProgressReporter.scala | 1 - .../continuous/ContinuousExecution.scala | 6 ++ .../sql/streaming/StreamingQueryStatus.scala | 6 +- ...ontinuousQueryStatusAndProgressSuite.scala | 55 +++++++++++++++++++ 5 files changed, 71 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 64e09edf27f5..03beefeca269 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -147,6 +147,12 @@ class MicroBatchExecution( logInfo(s"Query $prettyIdString was stopped") } + /** Begins recording statistics about query progress for a given trigger. */ + override protected def startTrigger(): Unit = { + super.startTrigger() + currentStatus = currentStatus.copy(isTriggerActive = true) + } + /** * Repeatedly attempts to run batches as data arrives. */ 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 6a22f0cc8431..39ab702ee083 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 @@ -114,7 +114,6 @@ trait ProgressReporter extends Logging { logDebug("Starting Trigger Calculation") lastTriggerStartTimestamp = currentTriggerStartTimestamp currentTriggerStartTimestamp = triggerClock.getTimeMillis() - currentStatus = currentStatus.copy(isTriggerActive = true) currentTriggerStartOffsets = null currentTriggerEndOffsets = null currentDurationsMs.clear() 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 4d42428fd189..f0859aaaa304 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 @@ -118,6 +118,8 @@ class ContinuousExecution( // For at least once, we can just ignore those reports and risk duplicates. commitLog.getLatest() match { case Some((latestEpochId, _)) => + updateStatusMessage("Starting new streaming query " + + s"and getting offsets from latest epoch $latestEpochId") val nextOffsets = offsetLog.get(latestEpochId).getOrElse { throw new IllegalStateException( s"Batch $latestEpochId was committed without end epoch offsets!") @@ -129,6 +131,7 @@ class ContinuousExecution( nextOffsets case None => // We are starting this stream for the first time. Offsets are all None. + updateStatusMessage("Starting new streaming query") logInfo(s"Starting new streaming query.") currentBatchId = 0 OffsetSeq.fill(continuousSources.map(_ => null): _*) @@ -263,6 +266,7 @@ class ContinuousExecution( epochUpdateThread.setDaemon(true) epochUpdateThread.start() + updateStatusMessage("Running") reportTimeTaken("runContinuous") { SQLExecution.withNewExecutionId( sparkSessionForQuery, lastExecution) { @@ -322,6 +326,8 @@ class ContinuousExecution( * before this is called. */ def commit(epoch: Long): Unit = { + updateStatusMessage(s"Committing epoch $epoch") + assert(continuousSources.length == 1, "only one continuous source supported currently") assert(offsetLog.get(epoch).isDefined, s"offset for epoch $epoch not reported before commit") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index 9dc62b7aac89..6ca9aacab724 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -28,9 +28,11 @@ import org.apache.spark.annotation.Evolving * Reports information about the instantaneous status of a streaming query. * * @param message A human readable description of what the stream is currently doing. - * @param isDataAvailable True when there is new data to be processed. + * @param isDataAvailable True when there is new data to be processed. Doesn't apply + * to ContinuousExecution where it is always false. * @param isTriggerActive True when the trigger is actively firing, false when waiting for the - * next trigger time. + * next trigger time. Doesn't apply to ContinuousExecution where it is + * always false. * * @since 2.1.0 */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala new file mode 100644 index 000000000000..10bea7f09057 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.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.streaming.continuous + +import org.apache.spark.sql.execution.streaming.StreamExecution +import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream +import org.apache.spark.sql.streaming.Trigger + +class ContinuousQueryStatusAndProgressSuite extends ContinuousSuiteBase { + test("StreamingQueryStatus - ContinuousExecution isDataAvailable and isTriggerActive " + + "should be false") { + import testImplicits._ + + val input = ContinuousMemoryStream[Int] + + def assertStatus(stream: StreamExecution): Unit = { + assert(stream.status.isDataAvailable === false) + assert(stream.status.isTriggerActive === false) + } + + val trigger = Trigger.Continuous(100) + testStream(input.toDF(), useV2Sink = true)( + StartStream(trigger), + Execute(assertStatus), + AddData(input, 0, 1, 2), + Execute(assertStatus), + CheckAnswer(0, 1, 2), + Execute(assertStatus), + StopStream, + Execute(assertStatus), + AddData(input, 3, 4, 5), + Execute(assertStatus), + StartStream(trigger), + Execute(assertStatus), + CheckAnswer(0, 1, 2, 3, 4, 5), + Execute(assertStatus), + StopStream, + Execute(assertStatus)) + } +} From 160e583a17235318c06b95992941a772ff782fae Mon Sep 17 00:00:00 2001 From: Li Jin Date: Fri, 14 Dec 2018 10:45:24 +0800 Subject: [PATCH 0132/1072] [SPARK-26364][PYTHON][TESTING] Clean up imports in test_pandas_udf* ## What changes were proposed in this pull request? Clean up unconditional import statements and move them to the top. Conditional imports (pandas, numpy, pyarrow) are left as-is. ## How was this patch tested? Exising tests. Closes #23314 from icexelloss/clean-up-test-imports. Authored-by: Li Jin Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/tests/test_pandas_udf.py | 16 +--- .../sql/tests/test_pandas_udf_grouped_agg.py | 39 +--------- .../sql/tests/test_pandas_udf_grouped_map.py | 40 +++------- .../sql/tests/test_pandas_udf_scalar.py | 75 +++++-------------- .../sql/tests/test_pandas_udf_window.py | 29 +------ 5 files changed, 36 insertions(+), 163 deletions(-) diff --git a/python/pyspark/sql/tests/test_pandas_udf.py b/python/pyspark/sql/tests/test_pandas_udf.py index c4b5478a7e89..d4d9679649ee 100644 --- a/python/pyspark/sql/tests/test_pandas_udf.py +++ b/python/pyspark/sql/tests/test_pandas_udf.py @@ -17,12 +17,16 @@ import unittest +from pyspark.sql.functions import udf, pandas_udf, PandasUDFType from pyspark.sql.types import * from pyspark.sql.utils import ParseException +from pyspark.rdd import PythonEvalType from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ pandas_requirement_message, pyarrow_requirement_message from pyspark.testing.utils import QuietTest +from py4j.protocol import Py4JJavaError + @unittest.skipIf( not have_pandas or not have_pyarrow, @@ -30,9 +34,6 @@ class PandasUDFTests(ReusedSQLTestCase): def test_pandas_udf_basic(self): - from pyspark.rdd import PythonEvalType - from pyspark.sql.functions import pandas_udf, PandasUDFType - udf = pandas_udf(lambda x: x, DoubleType()) self.assertEqual(udf.returnType, DoubleType()) self.assertEqual(udf.evalType, PythonEvalType.SQL_SCALAR_PANDAS_UDF) @@ -65,10 +66,6 @@ def test_pandas_udf_basic(self): self.assertEqual(udf.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF) def test_pandas_udf_decorator(self): - from pyspark.rdd import PythonEvalType - from pyspark.sql.functions import pandas_udf, PandasUDFType - from pyspark.sql.types import StructType, StructField, DoubleType - @pandas_udf(DoubleType()) def foo(x): return x @@ -114,8 +111,6 @@ def foo(x): self.assertEqual(foo.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF) def test_udf_wrong_arg(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - with QuietTest(self.sc): with self.assertRaises(ParseException): @pandas_udf('blah') @@ -151,9 +146,6 @@ def foo(k, v, w): return k def test_stopiteration_in_udf(self): - from pyspark.sql.functions import udf, pandas_udf, PandasUDFType - from py4j.protocol import Py4JJavaError - def foo(x): raise StopIteration() diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py index 5383704434c8..18264ead2fd0 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py @@ -17,6 +17,9 @@ import unittest +from pyspark.rdd import PythonEvalType +from pyspark.sql.functions import array, explode, col, lit, mean, sum, \ + udf, pandas_udf, PandasUDFType from pyspark.sql.types import * from pyspark.sql.utils import AnalysisException from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ @@ -31,7 +34,6 @@ class GroupedAggPandasUDFTests(ReusedSQLTestCase): @property def data(self): - from pyspark.sql.functions import array, explode, col, lit return self.spark.range(10).toDF('id') \ .withColumn("vs", array([lit(i * 1.0) + col('id') for i in range(20, 30)])) \ .withColumn("v", explode(col('vs'))) \ @@ -40,8 +42,6 @@ def data(self): @property def python_plus_one(self): - from pyspark.sql.functions import udf - @udf('double') def plus_one(v): assert isinstance(v, (int, float)) @@ -51,7 +51,6 @@ def plus_one(v): @property def pandas_scalar_plus_two(self): import pandas as pd - from pyspark.sql.functions import pandas_udf, PandasUDFType @pandas_udf('double', PandasUDFType.SCALAR) def plus_two(v): @@ -61,8 +60,6 @@ def plus_two(v): @property def pandas_agg_mean_udf(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - @pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() @@ -70,8 +67,6 @@ def avg(v): @property def pandas_agg_sum_udf(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - @pandas_udf('double', PandasUDFType.GROUPED_AGG) def sum(v): return v.sum() @@ -80,7 +75,6 @@ def sum(v): @property def pandas_agg_weighted_mean_udf(self): import numpy as np - from pyspark.sql.functions import pandas_udf, PandasUDFType @pandas_udf('double', PandasUDFType.GROUPED_AGG) def weighted_mean(v, w): @@ -88,8 +82,6 @@ def weighted_mean(v, w): return weighted_mean def test_manual(self): - from pyspark.sql.functions import pandas_udf, array - df = self.data sum_udf = self.pandas_agg_sum_udf mean_udf = self.pandas_agg_mean_udf @@ -118,8 +110,6 @@ def test_manual(self): self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) def test_basic(self): - from pyspark.sql.functions import col, lit, mean - df = self.data weighted_mean_udf = self.pandas_agg_weighted_mean_udf @@ -150,9 +140,6 @@ def test_basic(self): self.assertPandasEqual(expected4.toPandas(), result4.toPandas()) def test_unsupported_types(self): - from pyspark.sql.types import DoubleType, MapType - from pyspark.sql.functions import pandas_udf, PandasUDFType - with QuietTest(self.sc): with self.assertRaisesRegexp(NotImplementedError, 'not supported'): pandas_udf( @@ -173,8 +160,6 @@ def mean_and_std_udf(v): return {v.mean(): v.std()} def test_alias(self): - from pyspark.sql.functions import mean - df = self.data mean_udf = self.pandas_agg_mean_udf @@ -187,8 +172,6 @@ def test_mixed_sql(self): """ Test mixing group aggregate pandas UDF with sql expression. """ - from pyspark.sql.functions import sum - df = self.data sum_udf = self.pandas_agg_sum_udf @@ -225,8 +208,6 @@ def test_mixed_udfs(self): """ Test mixing group aggregate pandas UDF with python UDF and scalar pandas UDF. """ - from pyspark.sql.functions import sum - df = self.data plus_one = self.python_plus_one plus_two = self.pandas_scalar_plus_two @@ -292,8 +273,6 @@ def test_multiple_udfs(self): """ Test multiple group aggregate pandas UDFs in one agg function. """ - from pyspark.sql.functions import sum, mean - df = self.data mean_udf = self.pandas_agg_mean_udf sum_udf = self.pandas_agg_sum_udf @@ -315,8 +294,6 @@ def test_multiple_udfs(self): self.assertPandasEqual(expected1, result1) def test_complex_groupby(self): - from pyspark.sql.functions import sum - df = self.data sum_udf = self.pandas_agg_sum_udf plus_one = self.python_plus_one @@ -359,8 +336,6 @@ def test_complex_groupby(self): self.assertPandasEqual(expected7.toPandas(), result7.toPandas()) def test_complex_expressions(self): - from pyspark.sql.functions import col, sum - df = self.data plus_one = self.python_plus_one plus_two = self.pandas_scalar_plus_two @@ -434,7 +409,6 @@ def test_complex_expressions(self): self.assertPandasEqual(expected3, result3) def test_retain_group_columns(self): - from pyspark.sql.functions import sum with self.sql_conf({"spark.sql.retainGroupColumns": False}): df = self.data sum_udf = self.pandas_agg_sum_udf @@ -444,8 +418,6 @@ def test_retain_group_columns(self): self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) def test_array_type(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - df = self.data array_udf = pandas_udf(lambda x: [1.0, 2.0], 'array', PandasUDFType.GROUPED_AGG) @@ -453,8 +425,6 @@ def test_array_type(self): self.assertEquals(result1.first()['v2'], [1.0, 2.0]) def test_invalid_args(self): - from pyspark.sql.functions import mean - df = self.data plus_one = self.python_plus_one mean_udf = self.pandas_agg_mean_udf @@ -478,9 +448,6 @@ def test_invalid_args(self): df.groupby(df.id).agg(mean_udf(df.v), mean(df.v)).collect() def test_register_vectorized_udf_basic(self): - from pyspark.sql.functions import pandas_udf - from pyspark.rdd import PythonEvalType - sum_pandas_udf = pandas_udf( lambda v: v.sum(), "integer", PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF) diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py index a12c608dff9d..80e70349b78d 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py @@ -18,7 +18,12 @@ import datetime import unittest +from collections import OrderedDict +from decimal import Decimal +from distutils.version import LooseVersion + from pyspark.sql import Row +from pyspark.sql.functions import array, explode, col, lit, udf, sum, pandas_udf, PandasUDFType from pyspark.sql.types import * from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ pandas_requirement_message, pyarrow_requirement_message @@ -32,16 +37,12 @@ class GroupedMapPandasUDFTests(ReusedSQLTestCase): @property def data(self): - from pyspark.sql.functions import array, explode, col, lit return self.spark.range(10).toDF('id') \ .withColumn("vs", array([lit(i) for i in range(20, 30)])) \ .withColumn("v", explode(col('vs'))).drop('vs') def test_supported_types(self): - from decimal import Decimal - from distutils.version import LooseVersion import pyarrow as pa - from pyspark.sql.functions import pandas_udf, PandasUDFType values = [ 1, 2, 3, @@ -131,8 +132,6 @@ def test_supported_types(self): self.assertPandasEqual(expected3, result3) def test_array_type_correct(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType, array, col - df = self.data.withColumn("arr", array(col("id"))).repartition(1, "id") output_schema = StructType( @@ -151,8 +150,6 @@ def test_array_type_correct(self): self.assertPandasEqual(expected, result) def test_register_grouped_map_udf(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - foo_udf = pandas_udf(lambda x: x, "id long", PandasUDFType.GROUPED_MAP) with QuietTest(self.sc): with self.assertRaisesRegexp( @@ -161,7 +158,6 @@ def test_register_grouped_map_udf(self): self.spark.catalog.registerFunction("foo_udf", foo_udf) def test_decorator(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType df = self.data @pandas_udf( @@ -176,7 +172,6 @@ def foo(pdf): self.assertPandasEqual(expected, result) def test_coerce(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType df = self.data foo = pandas_udf( @@ -191,7 +186,6 @@ def test_coerce(self): self.assertPandasEqual(expected, result) def test_complex_groupby(self): - from pyspark.sql.functions import pandas_udf, col, PandasUDFType df = self.data @pandas_udf( @@ -210,7 +204,6 @@ def normalize(pdf): self.assertPandasEqual(expected, result) def test_empty_groupby(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType df = self.data @pandas_udf( @@ -229,7 +222,6 @@ def normalize(pdf): self.assertPandasEqual(expected, result) def test_datatype_string(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType df = self.data foo_udf = pandas_udf( @@ -243,8 +235,6 @@ def test_datatype_string(self): self.assertPandasEqual(expected, result) def test_wrong_return_type(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - with QuietTest(self.sc): with self.assertRaisesRegexp( NotImplementedError, @@ -255,7 +245,6 @@ def test_wrong_return_type(self): PandasUDFType.GROUPED_MAP) def test_wrong_args(self): - from pyspark.sql.functions import udf, pandas_udf, sum, PandasUDFType df = self.data with QuietTest(self.sc): @@ -277,9 +266,7 @@ def test_wrong_args(self): pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) def test_unsupported_types(self): - from distutils.version import LooseVersion import pyarrow as pa - from pyspark.sql.functions import pandas_udf, PandasUDFType common_err_msg = 'Invalid returnType.*grouped map Pandas UDF.*' unsupported_types = [ @@ -300,7 +287,6 @@ def test_unsupported_types(self): # Regression test for SPARK-23314 def test_timestamp_dst(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType # Daylight saving time for Los Angeles for 2015 is Sun, Nov 1 at 2:00 am dt = [datetime.datetime(2015, 11, 1, 0, 30), datetime.datetime(2015, 11, 1, 1, 30), @@ -311,12 +297,12 @@ def test_timestamp_dst(self): self.assertPandasEqual(df.toPandas(), result.toPandas()) def test_udf_with_key(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType + import numpy as np + df = self.data pdf = df.toPandas() def foo1(key, pdf): - import numpy as np assert type(key) == tuple assert type(key[0]) == np.int64 @@ -326,7 +312,6 @@ def foo1(key, pdf): v4=pdf.v * pdf.id.mean()) def foo2(key, pdf): - import numpy as np assert type(key) == tuple assert type(key[0]) == np.int64 assert type(key[1]) == np.int32 @@ -385,9 +370,7 @@ def foo3(key, pdf): self.assertPandasEqual(expected4, result4) def test_column_order(self): - from collections import OrderedDict import pandas as pd - from pyspark.sql.functions import pandas_udf, PandasUDFType # Helper function to set column names from a list def rename_pdf(pdf, names): @@ -468,7 +451,6 @@ def invalid_positional_types(pdf): with QuietTest(self.sc): with self.assertRaisesRegexp(Exception, "KeyError: 'id'"): grouped_df.apply(column_name_typo).collect() - from distutils.version import LooseVersion import pyarrow as pa if LooseVersion(pa.__version__) < LooseVersion("0.11.0"): # TODO: see ARROW-1949. Remove when the minimum PyArrow version becomes 0.11.0. @@ -480,7 +462,6 @@ def invalid_positional_types(pdf): def test_positional_assignment_conf(self): import pandas as pd - from pyspark.sql.functions import pandas_udf, PandasUDFType with self.sql_conf({ "spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName": False}): @@ -496,9 +477,7 @@ def foo(_): self.assertEqual(r.b, 1) def test_self_join_with_pandas(self): - import pyspark.sql.functions as F - - @F.pandas_udf('key long, col string', F.PandasUDFType.GROUPED_MAP) + @pandas_udf('key long, col string', PandasUDFType.GROUPED_MAP) def dummy_pandas_udf(df): return df[['key', 'col']] @@ -508,12 +487,11 @@ def dummy_pandas_udf(df): # this was throwing an AnalysisException before SPARK-24208 res = df_with_pandas.alias('temp0').join(df_with_pandas.alias('temp1'), - F.col('temp0.key') == F.col('temp1.key')) + col('temp0.key') == col('temp1.key')) self.assertEquals(res.count(), 5) def test_mixed_scalar_udfs_followed_by_grouby_apply(self): import pandas as pd - from pyspark.sql.functions import udf, pandas_udf, PandasUDFType df = self.spark.range(0, 10).toDF('v1') df = df.withColumn('v2', udf(lambda x: x + 1, 'int')(df['v1'])) \ diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index 2f585a372598..6a6865a9fb16 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -16,12 +16,20 @@ # import datetime import os +import random import shutil import sys import tempfile import time import unittest +from datetime import date, datetime +from decimal import Decimal +from distutils.version import LooseVersion + +from pyspark.rdd import PythonEvalType +from pyspark.sql import Column +from pyspark.sql.functions import array, col, expr, lit, sum, udf, pandas_udf from pyspark.sql.types import Row from pyspark.sql.types import * from pyspark.sql.utils import AnalysisException @@ -59,18 +67,16 @@ def tearDownClass(cls): @property def nondeterministic_vectorized_udf(self): - from pyspark.sql.functions import pandas_udf + import pandas as pd + import numpy as np @pandas_udf('double') def random_udf(v): - import pandas as pd - import numpy as np return pd.Series(np.random.random(len(v))) random_udf = random_udf.asNondeterministic() return random_udf def test_pandas_udf_tokenize(self): - from pyspark.sql.functions import pandas_udf tokenize = pandas_udf(lambda s: s.apply(lambda str: str.split(' ')), ArrayType(StringType())) self.assertEqual(tokenize.returnType, ArrayType(StringType())) @@ -79,7 +85,6 @@ def test_pandas_udf_tokenize(self): self.assertEqual([Row(hi=[u'hi', u'boo']), Row(hi=[u'bye', u'boo'])], result.collect()) def test_pandas_udf_nested_arrays(self): - from pyspark.sql.functions import pandas_udf tokenize = pandas_udf(lambda s: s.apply(lambda str: [str.split(' ')]), ArrayType(ArrayType(StringType()))) self.assertEqual(tokenize.returnType, ArrayType(ArrayType(StringType()))) @@ -88,7 +93,6 @@ def test_pandas_udf_nested_arrays(self): self.assertEqual([Row(hi=[[u'hi', u'boo']]), Row(hi=[[u'bye', u'boo']])], result.collect()) def test_vectorized_udf_basic(self): - from pyspark.sql.functions import pandas_udf, col, array df = self.spark.range(10).select( col('id').cast('string').alias('str'), col('id').cast('int').alias('int'), @@ -114,9 +118,6 @@ def test_vectorized_udf_basic(self): self.assertEquals(df.collect(), res.collect()) def test_register_nondeterministic_vectorized_udf_basic(self): - from pyspark.sql.functions import pandas_udf - from pyspark.rdd import PythonEvalType - import random random_pandas_udf = pandas_udf( lambda x: random.randint(6, 6) + x, IntegerType()).asNondeterministic() self.assertEqual(random_pandas_udf.deterministic, False) @@ -129,7 +130,6 @@ def test_register_nondeterministic_vectorized_udf_basic(self): self.assertEqual(row[0], 7) def test_vectorized_udf_null_boolean(self): - from pyspark.sql.functions import pandas_udf, col data = [(True,), (True,), (None,), (False,)] schema = StructType().add("bool", BooleanType()) df = self.spark.createDataFrame(data, schema) @@ -138,7 +138,6 @@ def test_vectorized_udf_null_boolean(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_byte(self): - from pyspark.sql.functions import pandas_udf, col data = [(None,), (2,), (3,), (4,)] schema = StructType().add("byte", ByteType()) df = self.spark.createDataFrame(data, schema) @@ -147,7 +146,6 @@ def test_vectorized_udf_null_byte(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_short(self): - from pyspark.sql.functions import pandas_udf, col data = [(None,), (2,), (3,), (4,)] schema = StructType().add("short", ShortType()) df = self.spark.createDataFrame(data, schema) @@ -156,7 +154,6 @@ def test_vectorized_udf_null_short(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_int(self): - from pyspark.sql.functions import pandas_udf, col data = [(None,), (2,), (3,), (4,)] schema = StructType().add("int", IntegerType()) df = self.spark.createDataFrame(data, schema) @@ -165,7 +162,6 @@ def test_vectorized_udf_null_int(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_long(self): - from pyspark.sql.functions import pandas_udf, col data = [(None,), (2,), (3,), (4,)] schema = StructType().add("long", LongType()) df = self.spark.createDataFrame(data, schema) @@ -174,7 +170,6 @@ def test_vectorized_udf_null_long(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_float(self): - from pyspark.sql.functions import pandas_udf, col data = [(3.0,), (5.0,), (-1.0,), (None,)] schema = StructType().add("float", FloatType()) df = self.spark.createDataFrame(data, schema) @@ -183,7 +178,6 @@ def test_vectorized_udf_null_float(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_double(self): - from pyspark.sql.functions import pandas_udf, col data = [(3.0,), (5.0,), (-1.0,), (None,)] schema = StructType().add("double", DoubleType()) df = self.spark.createDataFrame(data, schema) @@ -192,8 +186,6 @@ def test_vectorized_udf_null_double(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_decimal(self): - from decimal import Decimal - from pyspark.sql.functions import pandas_udf, col data = [(Decimal(3.0),), (Decimal(5.0),), (Decimal(-1.0),), (None,)] schema = StructType().add("decimal", DecimalType(38, 18)) df = self.spark.createDataFrame(data, schema) @@ -202,7 +194,6 @@ def test_vectorized_udf_null_decimal(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_string(self): - from pyspark.sql.functions import pandas_udf, col data = [("foo",), (None,), ("bar",), ("bar",)] schema = StructType().add("str", StringType()) df = self.spark.createDataFrame(data, schema) @@ -211,7 +202,6 @@ def test_vectorized_udf_null_string(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_string_in_udf(self): - from pyspark.sql.functions import pandas_udf, col import pandas as pd df = self.spark.range(10) str_f = pandas_udf(lambda x: pd.Series(map(str, x)), StringType()) @@ -220,7 +210,6 @@ def test_vectorized_udf_string_in_udf(self): self.assertEquals(expected.collect(), actual.collect()) def test_vectorized_udf_datatype_string(self): - from pyspark.sql.functions import pandas_udf, col df = self.spark.range(10).select( col('id').cast('string').alias('str'), col('id').cast('int').alias('int'), @@ -244,9 +233,8 @@ def test_vectorized_udf_datatype_string(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_null_binary(self): - from distutils.version import LooseVersion import pyarrow as pa - from pyspark.sql.functions import pandas_udf, col + if LooseVersion(pa.__version__) < LooseVersion("0.10.0"): with QuietTest(self.sc): with self.assertRaisesRegexp( @@ -262,7 +250,6 @@ def test_vectorized_udf_null_binary(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_array_type(self): - from pyspark.sql.functions import pandas_udf, col data = [([1, 2],), ([3, 4],)] array_schema = StructType([StructField("array", ArrayType(IntegerType()))]) df = self.spark.createDataFrame(data, schema=array_schema) @@ -271,7 +258,6 @@ def test_vectorized_udf_array_type(self): self.assertEquals(df.collect(), result.collect()) def test_vectorized_udf_null_array(self): - from pyspark.sql.functions import pandas_udf, col data = [([1, 2],), (None,), (None,), ([3, 4],), (None,)] array_schema = StructType([StructField("array", ArrayType(IntegerType()))]) df = self.spark.createDataFrame(data, schema=array_schema) @@ -280,7 +266,6 @@ def test_vectorized_udf_null_array(self): self.assertEquals(df.collect(), result.collect()) def test_vectorized_udf_complex(self): - from pyspark.sql.functions import pandas_udf, col, expr df = self.spark.range(10).select( col('id').cast('int').alias('a'), col('id').cast('int').alias('b'), @@ -293,7 +278,6 @@ def test_vectorized_udf_complex(self): self.assertEquals(expected.collect(), res.collect()) def test_vectorized_udf_exception(self): - from pyspark.sql.functions import pandas_udf, col df = self.spark.range(10) raise_exception = pandas_udf(lambda x: x * (1 / 0), LongType()) with QuietTest(self.sc): @@ -301,8 +285,8 @@ def test_vectorized_udf_exception(self): df.select(raise_exception(col('id'))).collect() def test_vectorized_udf_invalid_length(self): - from pyspark.sql.functions import pandas_udf, col import pandas as pd + df = self.spark.range(10) raise_exception = pandas_udf(lambda _: pd.Series(1), LongType()) with QuietTest(self.sc): @@ -312,7 +296,6 @@ def test_vectorized_udf_invalid_length(self): df.select(raise_exception(col('id'))).collect() def test_vectorized_udf_chained(self): - from pyspark.sql.functions import pandas_udf, col df = self.spark.range(10) f = pandas_udf(lambda x: x + 1, LongType()) g = pandas_udf(lambda x: x - 1, LongType()) @@ -320,7 +303,6 @@ def test_vectorized_udf_chained(self): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_wrong_return_type(self): - from pyspark.sql.functions import pandas_udf with QuietTest(self.sc): with self.assertRaisesRegexp( NotImplementedError, @@ -328,7 +310,6 @@ def test_vectorized_udf_wrong_return_type(self): pandas_udf(lambda x: x * 1.0, MapType(LongType(), LongType())) def test_vectorized_udf_return_scalar(self): - from pyspark.sql.functions import pandas_udf, col df = self.spark.range(10) f = pandas_udf(lambda x: 1.0, DoubleType()) with QuietTest(self.sc): @@ -336,7 +317,6 @@ def test_vectorized_udf_return_scalar(self): df.select(f(col('id'))).collect() def test_vectorized_udf_decorator(self): - from pyspark.sql.functions import pandas_udf, col df = self.spark.range(10) @pandas_udf(returnType=LongType()) @@ -346,21 +326,18 @@ def identity(x): self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_empty_partition(self): - from pyspark.sql.functions import pandas_udf, col df = self.spark.createDataFrame(self.sc.parallelize([Row(id=1)], 2)) f = pandas_udf(lambda x: x, LongType()) res = df.select(f(col('id'))) self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_varargs(self): - from pyspark.sql.functions import pandas_udf, col df = self.spark.createDataFrame(self.sc.parallelize([Row(id=1)], 2)) f = pandas_udf(lambda *v: v[0], LongType()) res = df.select(f(col('id'))) self.assertEquals(df.collect(), res.collect()) def test_vectorized_udf_unsupported_types(self): - from pyspark.sql.functions import pandas_udf with QuietTest(self.sc): with self.assertRaisesRegexp( NotImplementedError, @@ -368,8 +345,6 @@ def test_vectorized_udf_unsupported_types(self): pandas_udf(lambda x: x, MapType(StringType(), IntegerType())) def test_vectorized_udf_dates(self): - from pyspark.sql.functions import pandas_udf, col - from datetime import date schema = StructType().add("idx", LongType()).add("date", DateType()) data = [(0, date(1969, 1, 1),), (1, date(2012, 2, 2),), @@ -405,8 +380,6 @@ def check_data(idx, date, date_copy): self.assertIsNone(result[i][3]) # "check_data" col def test_vectorized_udf_timestamps(self): - from pyspark.sql.functions import pandas_udf, col - from datetime import datetime schema = StructType([ StructField("idx", LongType(), True), StructField("timestamp", TimestampType(), True)]) @@ -447,8 +420,8 @@ def check_data(idx, timestamp, timestamp_copy): self.assertIsNone(result[i][3]) # "check_data" col def test_vectorized_udf_return_timestamp_tz(self): - from pyspark.sql.functions import pandas_udf, col import pandas as pd + df = self.spark.range(10) @pandas_udf(returnType=TimestampType()) @@ -465,8 +438,8 @@ def gen_timestamps(id): self.assertEquals(expected, ts) def test_vectorized_udf_check_config(self): - from pyspark.sql.functions import pandas_udf, col import pandas as pd + with self.sql_conf({"spark.sql.execution.arrow.maxRecordsPerBatch": 3}): df = self.spark.range(10, numPartitions=1) @@ -479,9 +452,8 @@ def check_records_per_batch(x): self.assertTrue(r <= 3) def test_vectorized_udf_timestamps_respect_session_timezone(self): - from pyspark.sql.functions import pandas_udf, col - from datetime import datetime import pandas as pd + schema = StructType([ StructField("idx", LongType(), True), StructField("timestamp", TimestampType(), True)]) @@ -519,8 +491,6 @@ def test_vectorized_udf_timestamps_respect_session_timezone(self): def test_nondeterministic_vectorized_udf(self): # Test that nondeterministic UDFs are evaluated only once in chained UDF evaluations - from pyspark.sql.functions import pandas_udf, col - @pandas_udf('double') def plus_ten(v): return v + 10 @@ -533,8 +503,6 @@ def plus_ten(v): self.assertTrue(result1['plus_ten(rand)'].equals(result1['rand'] + 10)) def test_nondeterministic_vectorized_udf_in_aggregate(self): - from pyspark.sql.functions import sum - df = self.spark.range(10) random_udf = self.nondeterministic_vectorized_udf @@ -545,8 +513,6 @@ def test_nondeterministic_vectorized_udf_in_aggregate(self): df.agg(sum(random_udf(df.id))).collect() def test_register_vectorized_udf_basic(self): - from pyspark.rdd import PythonEvalType - from pyspark.sql.functions import pandas_udf, col, expr df = self.spark.range(10).select( col('id').cast('int').alias('a'), col('id').cast('int').alias('b')) @@ -563,11 +529,10 @@ def test_register_vectorized_udf_basic(self): # Regression test for SPARK-23314 def test_timestamp_dst(self): - from pyspark.sql.functions import pandas_udf # Daylight saving time for Los Angeles for 2015 is Sun, Nov 1 at 2:00 am - dt = [datetime.datetime(2015, 11, 1, 0, 30), - datetime.datetime(2015, 11, 1, 1, 30), - datetime.datetime(2015, 11, 1, 2, 30)] + dt = [datetime(2015, 11, 1, 0, 30), + datetime(2015, 11, 1, 1, 30), + datetime(2015, 11, 1, 2, 30)] df = self.spark.createDataFrame(dt, 'timestamp').toDF('time') foo_udf = pandas_udf(lambda x: x, 'timestamp') result = df.withColumn('time', foo_udf(df.time)) @@ -593,7 +558,6 @@ def test_type_annotation(self): def test_mixed_udf(self): import pandas as pd - from pyspark.sql.functions import col, udf, pandas_udf df = self.spark.range(0, 1).toDF('v') @@ -696,8 +660,6 @@ def f4(x): def test_mixed_udf_and_sql(self): import pandas as pd - from pyspark.sql import Column - from pyspark.sql.functions import udf, pandas_udf df = self.spark.range(0, 1).toDF('v') @@ -758,7 +720,6 @@ def test_datasource_with_udf(self): # This needs to a separate test because Arrow dependency is optional import pandas as pd import numpy as np - from pyspark.sql.functions import pandas_udf, lit, col path = tempfile.mkdtemp() shutil.rmtree(path) diff --git a/python/pyspark/sql/tests/test_pandas_udf_window.py b/python/pyspark/sql/tests/test_pandas_udf_window.py index f0e6d2696df6..0a7a19c1c081 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_window.py +++ b/python/pyspark/sql/tests/test_pandas_udf_window.py @@ -18,6 +18,8 @@ import unittest from pyspark.sql.utils import AnalysisException +from pyspark.sql.functions import array, explode, col, lit, mean, min, max, rank, \ + udf, pandas_udf, PandasUDFType from pyspark.sql.window import Window from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ pandas_requirement_message, pyarrow_requirement_message @@ -30,7 +32,6 @@ class WindowPandasUDFTests(ReusedSQLTestCase): @property def data(self): - from pyspark.sql.functions import array, explode, col, lit return self.spark.range(10).toDF('id') \ .withColumn("vs", array([lit(i * 1.0) + col('id') for i in range(20, 30)])) \ .withColumn("v", explode(col('vs'))) \ @@ -39,18 +40,14 @@ def data(self): @property def python_plus_one(self): - from pyspark.sql.functions import udf return udf(lambda v: v + 1, 'double') @property def pandas_scalar_time_two(self): - from pyspark.sql.functions import pandas_udf return pandas_udf(lambda v: v * 2, 'double') @property def pandas_agg_mean_udf(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - @pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() @@ -58,8 +55,6 @@ def avg(v): @property def pandas_agg_max_udf(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - @pandas_udf('double', PandasUDFType.GROUPED_AGG) def max(v): return v.max() @@ -67,8 +62,6 @@ def max(v): @property def pandas_agg_min_udf(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - @pandas_udf('double', PandasUDFType.GROUPED_AGG) def min(v): return v.min() @@ -88,8 +81,6 @@ def unpartitioned_window(self): return Window.partitionBy() def test_simple(self): - from pyspark.sql.functions import mean - df = self.data w = self.unbounded_window @@ -105,8 +96,6 @@ def test_simple(self): self.assertPandasEqual(expected2.toPandas(), result2.toPandas()) def test_multiple_udfs(self): - from pyspark.sql.functions import max, min, mean - df = self.data w = self.unbounded_window @@ -121,8 +110,6 @@ def test_multiple_udfs(self): self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) def test_replace_existing(self): - from pyspark.sql.functions import mean - df = self.data w = self.unbounded_window @@ -132,8 +119,6 @@ def test_replace_existing(self): self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) def test_mixed_sql(self): - from pyspark.sql.functions import mean - df = self.data w = self.unbounded_window mean_udf = self.pandas_agg_mean_udf @@ -144,8 +129,6 @@ def test_mixed_sql(self): self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) def test_mixed_udf(self): - from pyspark.sql.functions import mean - df = self.data w = self.unbounded_window @@ -171,8 +154,6 @@ def test_mixed_udf(self): self.assertPandasEqual(expected2.toPandas(), result2.toPandas()) def test_without_partitionBy(self): - from pyspark.sql.functions import mean - df = self.data w = self.unpartitioned_window mean_udf = self.pandas_agg_mean_udf @@ -187,8 +168,6 @@ def test_without_partitionBy(self): self.assertPandasEqual(expected2.toPandas(), result2.toPandas()) def test_mixed_sql_and_udf(self): - from pyspark.sql.functions import max, min, rank, col - df = self.data w = self.unbounded_window ow = self.ordered_window @@ -221,8 +200,6 @@ def test_mixed_sql_and_udf(self): self.assertPandasEqual(expected4.toPandas(), result4.toPandas()) def test_array_type(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - df = self.data w = self.unbounded_window @@ -231,8 +208,6 @@ def test_array_type(self): self.assertEquals(result1.first()['v2'], [1.0, 2.0]) def test_invalid_args(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType - df = self.data w = self.unbounded_window ow = self.ordered_window From 9c481c7a6b8019c569a08b2645bf9c19ff84a9e5 Mon Sep 17 00:00:00 2001 From: jasonwayne Date: Fri, 14 Dec 2018 10:47:58 +0800 Subject: [PATCH 0133/1072] [SPARK-26360] remove redundant validateQuery call ## What changes were proposed in this pull request? remove a redundant `KafkaWriter.validateQuery` call in `KafkaSourceProvider ` ## How was this patch tested? Just removing duplicate codes, so I just build and run unit tests. Closes #23309 from JasonWayne/SPARK-26360. Authored-by: jasonwayne Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/kafka010/KafkaSourceProvider.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 6a0c2088ac3d..4b8b5c0019b4 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -266,8 +266,6 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister // We convert the options argument from V2 -> Java map -> scala mutable -> scala immutable. val producerParams = kafkaParamsForProducer(options.asMap.asScala.toMap) - KafkaWriter.validateQuery(schema.toAttributes, producerParams, topic) - new KafkaStreamingWriteSupport(topic, producerParams, schema) } From 93139afb072d14870fb4eab01cb11df28eb0f8dd Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 14 Dec 2018 10:50:48 +0800 Subject: [PATCH 0134/1072] [SPARK-26337][SQL][TEST] Add benchmark for LongToUnsafeRowMap ## What changes were proposed in this pull request? Regarding the performance issue of SPARK-26155, it reports the issue on TPC-DS. I think it is better to add a benchmark for `LongToUnsafeRowMap` which is the root cause of performance regression. It can be easier to show performance difference between different metric implementations in `LongToUnsafeRowMap`. ## How was this patch tested? Manually run added benchmark. Closes #23284 from viirya/SPARK-26337. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- ...HashedRelationMetricsBenchmark-results.txt | 11 +++ .../HashedRelationMetricsBenchmark.scala | 84 +++++++++++++++++++ 2 files changed, 95 insertions(+) create mode 100644 sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala diff --git a/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt new file mode 100644 index 000000000000..338244ad542f --- /dev/null +++ b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt @@ -0,0 +1,11 @@ +================================================================================================ +LongToUnsafeRowMap metrics +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-7700HQ CPU @ 2.80GHz +LongToUnsafeRowMap metrics: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +LongToUnsafeRowMap 234 / 315 2.1 467.3 1.0X + + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala new file mode 100644 index 000000000000..bdf753debe62 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED +import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{BoundReference, UnsafeProjection} +import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap +import org.apache.spark.sql.types.LongType + +/** + * Benchmark to measure metrics performance at HashedRelation. + * To run this benchmark: + * {{{ + * 1. without sbt: bin/spark-submit --class + * 2. build/sbt "sql/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/HashedRelationMetricsBenchmark-results.txt". + * }}} + */ +object HashedRelationMetricsBenchmark extends SqlBasedBenchmark { + + def benchmarkLongToUnsafeRowMapMetrics(numRows: Int): Unit = { + runBenchmark("LongToUnsafeRowMap metrics") { + val benchmark = new Benchmark("LongToUnsafeRowMap metrics", numRows, output = output) + benchmark.addCase("LongToUnsafeRowMap") { iter => + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) + + val keys = Range.Long(0, numRows, 1) + val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + keys.foreach { k => + map.append(k, unsafeProj(InternalRow(k))) + } + map.optimize() + + val threads = (0 to 100).map { _ => + val thread = new Thread { + override def run: Unit = { + val row = unsafeProj(InternalRow(0L)).copy() + keys.foreach { k => + assert(map.getValue(k, row) eq row) + assert(row.getLong(0) == k) + } + } + } + thread.start() + thread + } + threads.map(_.join()) + map.free() + } + benchmark.run() + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + benchmarkLongToUnsafeRowMapMetrics(500000) + } +} From 2d8838dccde6d77b4ff1a15fdd6a0d4da2fda8c7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 13 Dec 2018 20:55:12 -0800 Subject: [PATCH 0135/1072] [SPARK-26368][SQL] Make it clear that getOrInferFileFormatSchema doesn't create InMemoryFileIndex ## What changes were proposed in this pull request? I was looking at the code and it was a bit difficult to see the life cycle of InMemoryFileIndex passed into getOrInferFileFormatSchema, because once it is passed in, and another time it was created in getOrInferFileFormatSchema. It'd be easier to understand the life cycle if we move the creation of it out. ## How was this patch tested? This is a simple code move and should be covered by existing tests. Closes #23317 from rxin/SPARK-26368. Authored-by: Reynold Xin Signed-off-by: gatorsmile --- .../execution/datasources/DataSource.scala | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) 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 795a6d0b6b04..fefff68c4ba8 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 @@ -122,21 +122,14 @@ case class DataSource( * be any further inference in any triggers. * * @param format the file format object for this DataSource - * @param fileIndex optional [[InMemoryFileIndex]] for getting partition schema and file list + * @param getFileIndex [[InMemoryFileIndex]] for getting partition schema and file list * @return A pair of the data schema (excluding partition columns) and the schema of the partition * columns. */ private def getOrInferFileFormatSchema( format: FileFormat, - fileIndex: Option[InMemoryFileIndex] = None): (StructType, StructType) = { - // The operations below are expensive therefore try not to do them if we don't need to, e.g., - // in streaming mode, we have already inferred and registered partition columns, we will - // never have to materialize the lazy val below - lazy val tempFileIndex = fileIndex.getOrElse { - val globbedPaths = - checkAndGlobPathIfNecessary(checkEmptyGlobPath = false, checkFilesExist = false) - createInMemoryFileIndex(globbedPaths) - } + getFileIndex: () => InMemoryFileIndex): (StructType, StructType) = { + lazy val tempFileIndex = getFileIndex() val partitionSchema = if (partitionColumns.isEmpty) { // Try to infer partitioning, because no DataSource in the read path provides the partitioning @@ -236,7 +229,15 @@ case class DataSource( "you may be able to create a static DataFrame on that directory with " + "'spark.read.load(directory)' and infer schema from it.") } - val (dataSchema, partitionSchema) = getOrInferFileFormatSchema(format) + + val (dataSchema, partitionSchema) = getOrInferFileFormatSchema(format, () => { + // The operations below are expensive therefore try not to do them if we don't need to, + // e.g., in streaming mode, we have already inferred and registered partition columns, + // we will never have to materialize the lazy val below + val globbedPaths = + checkAndGlobPathIfNecessary(checkEmptyGlobPath = false, checkFilesExist = false) + createInMemoryFileIndex(globbedPaths) + }) SourceInfo( s"FileSource[$path]", StructType(dataSchema ++ partitionSchema), @@ -370,7 +371,7 @@ case class DataSource( } else { val index = createInMemoryFileIndex(globbedPaths) val (resultDataSchema, resultPartitionSchema) = - getOrInferFileFormatSchema(format, Some(index)) + getOrInferFileFormatSchema(format, () => index) (index, resultDataSchema, resultPartitionSchema) } From 3dda58af2b7f42beab736d856bf17b4d35c8866c Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 15 Dec 2018 00:23:28 +0800 Subject: [PATCH 0136/1072] [SPARK-26370][SQL] Fix resolution of higher-order function for the same identifier. ## What changes were proposed in this pull request? When using a higher-order function with the same variable name as the existing columns in `Filter` or something which uses `Analyzer.resolveExpressionBottomUp` during the resolution, e.g.,: ```scala val df = Seq( (Seq(1, 9, 8, 7), 1, 2), (Seq(5, 9, 7), 2, 2), (Seq.empty, 3, 2), (null, 4, 2) ).toDF("i", "x", "d") checkAnswer(df.filter("exists(i, x -> x % d == 0)"), Seq(Row(Seq(1, 9, 8, 7), 1, 2))) checkAnswer(df.select("x").filter("exists(i, x -> x % d == 0)"), Seq(Row(1))) ``` the following exception happens: ``` java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.BoundReference cannot be cast to org.apache.spark.sql.catalyst.expressions.NamedExpression at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at scala.collection.TraversableLike.map(TraversableLike.scala:237) at scala.collection.TraversableLike.map$(TraversableLike.scala:230) at scala.collection.AbstractTraversable.map(Traversable.scala:108) at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.$anonfun$functionsForEval$1(higherOrderFunctions.scala:147) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237) at scala.collection.immutable.List.foreach(List.scala:392) at scala.collection.TraversableLike.map(TraversableLike.scala:237) at scala.collection.TraversableLike.map$(TraversableLike.scala:230) at scala.collection.immutable.List.map(List.scala:298) at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.functionsForEval(higherOrderFunctions.scala:145) at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.functionsForEval$(higherOrderFunctions.scala:145) at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionsForEval$lzycompute(higherOrderFunctions.scala:369) at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionsForEval(higherOrderFunctions.scala:369) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.functionForEval(higherOrderFunctions.scala:176) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.functionForEval$(higherOrderFunctions.scala:176) at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionForEval(higherOrderFunctions.scala:369) at org.apache.spark.sql.catalyst.expressions.ArrayExists.nullSafeEval(higherOrderFunctions.scala:387) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.eval(higherOrderFunctions.scala:190) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.eval$(higherOrderFunctions.scala:185) at org.apache.spark.sql.catalyst.expressions.ArrayExists.eval(higherOrderFunctions.scala:369) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source) at org.apache.spark.sql.execution.FilterExec.$anonfun$doExecute$3(basicPhysicalOperators.scala:216) at org.apache.spark.sql.execution.FilterExec.$anonfun$doExecute$3$adapted(basicPhysicalOperators.scala:215) ... ``` because the `UnresolvedAttribute`s in `LambdaFunction` are unexpectedly resolved by the rule. This pr modified to use a placeholder `UnresolvedNamedLambdaVariable` to prevent unexpected resolution. ## How was this patch tested? Added a test and modified some tests. Closes #23320 from ueshin/issues/SPARK-26370/hof_resolution. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan --- .../analysis/higherOrderFunctions.scala | 5 ++-- .../expressions/higherOrderFunctions.scala | 26 +++++++++++++++++-- .../sql/catalyst/parser/AstBuilder.scala | 7 +++-- .../ResolveLambdaVariablesSuite.scala | 10 ++++--- ...ReplaceNullWithFalseInPredicateSuite.scala | 14 +++++----- .../parser/ExpressionParserSuite.scala | 6 +++-- .../typeCoercion/native/mapZipWith.sql.out | 4 +-- .../spark/sql/DataFrameFunctionsSuite.scala | 20 ++++++++++++++ 8 files changed, 72 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala index a8a7bbd9f9cd..1cd7f412bb67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala @@ -150,13 +150,14 @@ case class ResolveLambdaVariables(conf: SQLConf) extends Rule[LogicalPlan] { val lambdaMap = l.arguments.map(v => canonicalizer(v.name) -> v).toMap l.mapChildren(resolve(_, parentLambdaMap ++ lambdaMap)) - case u @ UnresolvedAttribute(name +: nestedFields) => + case u @ UnresolvedNamedLambdaVariable(name +: nestedFields) => parentLambdaMap.get(canonicalizer(name)) match { case Some(lambda) => nestedFields.foldLeft(lambda: Expression) { (expr, fieldName) => ExtractValue(expr, Literal(fieldName), conf.resolver) } - case None => u + case None => + UnresolvedAttribute(u.nameParts) } case _ => 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 a8639d29f964..7141b6e99638 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 @@ -22,12 +22,34 @@ import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods +/** + * A placeholder of lambda variables to prevent unexpected resolution of [[LambdaFunction]]. + */ +case class UnresolvedNamedLambdaVariable(nameParts: Seq[String]) + extends LeafExpression with NamedExpression with Unevaluable { + + override def name: String = + nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override def qualifier: Seq[String] = throw new UnresolvedException(this, "qualifier") + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") + override def newInstance(): NamedExpression = throw new UnresolvedException(this, "newInstance") + override lazy val resolved = false + + override def toString: String = s"lambda '$name" + + override def sql: String = name +} + /** * A named lambda variable. */ @@ -79,7 +101,7 @@ case class LambdaFunction( object LambdaFunction { val identity: LambdaFunction = { - val id = UnresolvedAttribute.quoted("id") + val id = UnresolvedNamedLambdaVariable(Seq("id")) LambdaFunction(id, Seq(id)) } } 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 672bffcfc0ca..8959f78b656d 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 @@ -1338,9 +1338,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitLambda(ctx: LambdaContext): Expression = withOrigin(ctx) { val arguments = ctx.IDENTIFIER().asScala.map { name => - UnresolvedAttribute.quoted(name.getText) + UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(name.getText).nameParts) } - LambdaFunction(expression(ctx.expression), arguments) + val function = expression(ctx.expression).transformUp { + case a: UnresolvedAttribute => UnresolvedNamedLambdaVariable(a.nameParts) + } + LambdaFunction(function, arguments) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala index c4171c75ecd0..a5847ba7c522 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala @@ -49,19 +49,21 @@ class ResolveLambdaVariablesSuite extends PlanTest { comparePlans(Analyzer.execute(plan(e1)), plan(e2)) } + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + test("resolution - no op") { checkExpression(key, key) } test("resolution - simple") { - val in = ArrayTransform(values1, LambdaFunction('x.attr + 1, 'x.attr :: Nil)) + val in = ArrayTransform(values1, LambdaFunction(lv('x) + 1, lv('x) :: Nil)) val out = ArrayTransform(values1, LambdaFunction(lvInt + 1, lvInt :: Nil)) checkExpression(in, out) } test("resolution - nested") { val in = ArrayTransform(values2, LambdaFunction( - ArrayTransform('x.attr, LambdaFunction('x.attr + 1, 'x.attr :: Nil)), 'x.attr :: Nil)) + ArrayTransform(lv('x), LambdaFunction(lv('x) + 1, lv('x) :: Nil)), lv('x) :: Nil)) val out = ArrayTransform(values2, LambdaFunction( ArrayTransform(lvArray, LambdaFunction(lvInt + 1, lvInt :: Nil)), lvArray :: Nil)) checkExpression(in, out) @@ -75,14 +77,14 @@ class ResolveLambdaVariablesSuite extends PlanTest { test("fail - name collisions") { val p = plan(ArrayTransform(values1, - LambdaFunction('x.attr + 'X.attr, 'x.attr :: 'X.attr :: Nil))) + LambdaFunction(lv('x) + lv('X), lv('x) :: lv('X) :: Nil))) val msg = intercept[AnalysisException](Analyzer.execute(p)).getMessage assert(msg.contains("arguments should not have names that are semantically the same")) } test("fail - lambda arguments") { val p = plan(ArrayTransform(values1, - LambdaFunction('x.attr + 'y.attr + 'z.attr, 'x.attr :: 'y.attr :: 'z.attr :: Nil))) + LambdaFunction(lv('x) + lv('y) + lv('z), lv('x) :: lv('y) :: lv('z) :: Nil))) val msg = intercept[AnalysisException](Analyzer.execute(p)).getMessage assert(msg.contains("does not match the number of arguments expected")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala index ee0d04da3e46..748075bfd6a6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{And, ArrayExists, ArrayFilter, ArrayTransform, CaseWhen, Expression, GreaterThan, If, LambdaFunction, Literal, MapFilter, NamedExpression, Or} +import org.apache.spark.sql.catalyst.expressions.{And, ArrayExists, ArrayFilter, ArrayTransform, CaseWhen, Expression, GreaterThan, If, LambdaFunction, Literal, MapFilter, NamedExpression, Or, UnresolvedNamedLambdaVariable} import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} @@ -306,22 +306,24 @@ class ReplaceNullWithFalseInPredicateSuite extends PlanTest { testProjection(originalExpr = column, expectedExpr = column) } + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + test("replace nulls in lambda function of ArrayFilter") { - testHigherOrderFunc('a, ArrayFilter, Seq('e)) + testHigherOrderFunc('a, ArrayFilter, Seq(lv('e))) } test("replace nulls in lambda function of ArrayExists") { - testHigherOrderFunc('a, ArrayExists, Seq('e)) + testHigherOrderFunc('a, ArrayExists, Seq(lv('e))) } test("replace nulls in lambda function of MapFilter") { - testHigherOrderFunc('m, MapFilter, Seq('k, 'v)) + testHigherOrderFunc('m, MapFilter, Seq(lv('k), lv('v))) } test("inability to replace nulls in arbitrary higher-order function") { val lambdaFunc = LambdaFunction( - function = If('e > 0, Literal(null, BooleanType), TrueLiteral), - arguments = Seq[NamedExpression]('e)) + function = If(lv('e) > 0, Literal(null, BooleanType), TrueLiteral), + arguments = Seq[NamedExpression](lv('e))) val column = ArrayTransform('a, lambdaFunc) testProjection(originalExpr = column, expectedExpr = column) } 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 b4df22c5b29f..8bcc69d580d8 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 @@ -246,9 +246,11 @@ class ExpressionParserSuite extends PlanTest { intercept("foo(a x)", "extraneous input 'x'") } + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + test("lambda functions") { - assertEqual("x -> x + 1", LambdaFunction('x + 1, Seq('x.attr))) - assertEqual("(x, y) -> x + y", LambdaFunction('x + 'y, Seq('x.attr, 'y.attr))) + assertEqual("x -> x + 1", LambdaFunction(lv('x) + 1, Seq(lv('x)))) + assertEqual("(x, y) -> x + y", LambdaFunction(lv('x) + lv('y), Seq(lv('x), lv('y)))) } test("window function expressions") { diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out index 35740094ba53..86a578ca013d 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out @@ -85,7 +85,7 @@ FROM various_maps struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), `k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7 +cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7 -- !query 6 @@ -113,7 +113,7 @@ FROM various_maps struct<> -- !query 8 output org.apache.spark.sql.AnalysisException -cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), `k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7 +cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7 -- !query 9 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 e6d1a038a591..b7fc9570af91 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 @@ -2908,6 +2908,26 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { } assert(ex.getMessage.contains("Cannot use null as map key")) } + + test("SPARK-26370: Fix resolution of higher-order function for the same identifier") { + val df = Seq( + (Seq(1, 9, 8, 7), 1, 2), + (Seq(5, 9, 7), 2, 2), + (Seq.empty, 3, 2), + (null, 4, 2) + ).toDF("i", "x", "d") + + checkAnswer(df.selectExpr("x", "exists(i, x -> x % d == 0)"), + Seq( + Row(1, true), + Row(2, false), + Row(3, false), + Row(4, null))) + checkAnswer(df.filter("exists(i, x -> x % d == 0)"), + Seq(Row(Seq(1, 9, 8, 7), 1, 2))) + checkAnswer(df.select("x").filter("exists(i, x -> x % d == 0)"), + Seq(Row(1))) + } } object DataFrameFunctionsSuite { From d25e443eec6efc9172eade6ac11be7b3ff04759d Mon Sep 17 00:00:00 2001 From: CarolinPeng <00244106@zte.intra> Date: Fri, 14 Dec 2018 14:23:21 -0600 Subject: [PATCH 0137/1072] [MINOR][SQL] Some errors in the notes. ## What changes were proposed in this pull request? When using ordinals to access linked list, the time cost is O(n). ## How was this patch tested? Existing tests. Closes #23280 from CarolinePeng/update_Two. Authored-by: CarolinPeng <00244106@zte.intra> Signed-off-by: Sean Owen --- .../org/apache/spark/sql/catalyst/expressions/package.scala | 2 +- .../apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 0083ee64653e..bf18e8bcb52d 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 @@ -101,7 +101,7 @@ package object expressions { StructType(attrs.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) } - // It's possible that `attrs` is a linked list, which can lead to bad O(n^2) loops when + // It's possible that `attrs` is a linked list, which can lead to bad O(n) loops when // accessing attributes by their ordinals. To avoid this performance penalty, convert the input // to an array. @transient private lazy val attrsArray = attrs.toArray diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index a520eba001af..3ad2ee692361 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -93,7 +93,7 @@ abstract class LogicalPlan /** * Optionally resolves the given strings to a [[NamedExpression]] using the input from all child * nodes of this LogicalPlan. The attribute is expressed as - * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. + * string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ def resolveChildren( nameParts: Seq[String], From 1b604c1fd0b9ef17b394818fbd6c546bc01cdd8c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 15 Dec 2018 13:52:07 +0800 Subject: [PATCH 0138/1072] [SPARK-26265][CORE][FOLLOWUP] Put freePage into a finally block ## What changes were proposed in this pull request? Based on the [comment](https://github.com/apache/spark/pull/23272#discussion_r240735509), it seems to be better to put `freePage` into a `finally` block. This patch as a follow-up to do so. ## How was this patch tested? Existing tests. Closes #23294 from viirya/SPARK-26265-followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Hyukjin Kwon --- .../spark/unsafe/map/BytesToBytesMap.java | 57 ++++++++++--------- 1 file changed, 30 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index fbba002f1f80..7df8aafb2b67 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -262,36 +262,39 @@ private void advanceToNextPage() { // reference to the page to free and free it after releasing the lock of `MapIterator`. MemoryBlock pageToFree = null; - synchronized (this) { - int nextIdx = dataPages.indexOf(currentPage) + 1; - if (destructive && currentPage != null) { - dataPages.remove(currentPage); - pageToFree = currentPage; - nextIdx --; - } - if (dataPages.size() > nextIdx) { - currentPage = dataPages.get(nextIdx); - pageBaseObject = currentPage.getBaseObject(); - offsetInPage = currentPage.getBaseOffset(); - recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); - offsetInPage += UnsafeAlignedOffset.getUaoSize(); - } else { - currentPage = null; - if (reader != null) { - handleFailedDelete(); + try { + synchronized (this) { + int nextIdx = dataPages.indexOf(currentPage) + 1; + if (destructive && currentPage != null) { + dataPages.remove(currentPage); + pageToFree = currentPage; + nextIdx--; } - try { - Closeables.close(reader, /* swallowIOException = */ false); - reader = spillWriters.getFirst().getReader(serializerManager); - recordsInPage = -1; - } catch (IOException e) { - // Scala iterator does not handle exception - Platform.throwException(e); + if (dataPages.size() > nextIdx) { + currentPage = dataPages.get(nextIdx); + pageBaseObject = currentPage.getBaseObject(); + offsetInPage = currentPage.getBaseOffset(); + recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); + offsetInPage += UnsafeAlignedOffset.getUaoSize(); + } else { + currentPage = null; + if (reader != null) { + handleFailedDelete(); + } + try { + Closeables.close(reader, /* swallowIOException = */ false); + reader = spillWriters.getFirst().getReader(serializerManager); + recordsInPage = -1; + } catch (IOException e) { + // Scala iterator does not handle exception + Platform.throwException(e); + } } } - } - if (pageToFree != null) { - freePage(pageToFree); + } finally { + if (pageToFree != null) { + freePage(pageToFree); + } } } From 9ccae0c9e7d1a0a704e8cd7574ba508419e05e30 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Sat, 15 Dec 2018 13:55:24 +0800 Subject: [PATCH 0139/1072] [SPARK-26362][CORE] Remove 'spark.driver.allowMultipleContexts' to disallow multiple creation of SparkContexts ## What changes were proposed in this pull request? Multiple SparkContexts are discouraged and it has been warning for last 4 years, see SPARK-4180. It could cause arbitrary and mysterious error cases, see SPARK-2243. Honestly, I didn't even know Spark still allows it, which looks never officially supported, see SPARK-2243. I believe It should be good timing now to remove this configuration. ## How was this patch tested? Each doc was manually checked and manually tested: ``` $ ./bin/spark-shell --conf=spark.driver.allowMultipleContexts=true ... scala> new SparkContext() org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at: org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:939) ... org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2435) at scala.Option.foreach(Option.scala:274) at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2432) at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2509) at org.apache.spark.SparkContext.(SparkContext.scala:80) at org.apache.spark.SparkContext.(SparkContext.scala:112) ... 49 elided ``` Closes #23311 from HyukjinKwon/SPARK-26362. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/SparkContext.scala | 65 +++++++------------ .../spark/api/java/JavaSparkContext.scala | 4 +- .../org/apache/spark/SparkContextSuite.scala | 19 +----- .../ExternalClusterManagerSuite.scala | 3 +- docs/rdd-programming-guide.md | 2 +- project/MimaExcludes.scala | 4 ++ python/pyspark/context.py | 3 + .../execution/ExchangeCoordinatorSuite.scala | 1 - 8 files changed, 34 insertions(+), 67 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 696dafda6d1e..09cc346db0ed 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -64,9 +64,8 @@ import org.apache.spark.util.logging.DriverLogger * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * - * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before - * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details. - * + * @note Only one `SparkContext` should be active per JVM. You must `stop()` the + * active `SparkContext` before creating a new one. * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ @@ -75,14 +74,10 @@ class SparkContext(config: SparkConf) extends Logging { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() - // If true, log warnings instead of throwing exceptions when multiple SparkContexts are active - private val allowMultipleContexts: Boolean = - config.getBoolean("spark.driver.allowMultipleContexts", false) - // 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. - SparkContext.markPartiallyConstructed(this, allowMultipleContexts) + SparkContext.markPartiallyConstructed(this) val startTime = System.currentTimeMillis() @@ -2392,7 +2387,7 @@ class SparkContext(config: SparkConf) extends Logging { // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having finished construction. // NOTE: this must be placed at the end of the SparkContext constructor. - SparkContext.setActiveContext(this, allowMultipleContexts) + SparkContext.setActiveContext(this) } /** @@ -2409,18 +2404,18 @@ object SparkContext extends Logging { private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object() /** - * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `null`. + * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `null`. * - * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK. + * Access to this field is guarded by `SPARK_CONTEXT_CONSTRUCTOR_LOCK`. */ private val activeContext: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) /** - * Points to a partially-constructed SparkContext if some thread is in the SparkContext + * Points to a partially-constructed SparkContext if another thread is in the SparkContext * constructor, or `None` if no SparkContext is being constructed. * - * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK + * Access to this field is guarded by `SPARK_CONTEXT_CONSTRUCTOR_LOCK`. */ private var contextBeingConstructed: Option[SparkContext] = None @@ -2428,24 +2423,16 @@ object SparkContext extends Logging { * Called to ensure that no other SparkContext is running in this JVM. * * Throws an exception if a running context is detected and logs a warning if another thread is - * constructing a SparkContext. This warning is necessary because the current locking scheme + * constructing a SparkContext. This warning is necessary because the current locking scheme * prevents us from reliably distinguishing between cases where another context is being * constructed and cases where another constructor threw an exception. */ - private def assertNoOtherContextIsRunning( - sc: SparkContext, - allowMultipleContexts: Boolean): Unit = { + private def assertNoOtherContextIsRunning(sc: SparkContext): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { Option(activeContext.get()).filter(_ ne sc).foreach { ctx => - val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + - " To ignore this error, set spark.driver.allowMultipleContexts = true. " + + val errMsg = "Only one SparkContext should be running in this JVM (see SPARK-2243)." + s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}" - val exception = new SparkException(errMsg) - if (allowMultipleContexts) { - logWarning("Multiple running SparkContexts detected in the same JVM!", exception) - } else { - throw exception - } + throw new SparkException(errMsg) } contextBeingConstructed.filter(_ ne sc).foreach { otherContext => @@ -2454,7 +2441,7 @@ object SparkContext extends Logging { val otherContextCreationSite = Option(otherContext.creationSite).map(_.longForm).getOrElse("unknown location") val warnMsg = "Another SparkContext is being constructed (or threw an exception in its" + - " constructor). This may indicate an error, since only one SparkContext may be" + + " constructor). This may indicate an error, since only one SparkContext should be" + " running in this JVM (see SPARK-2243)." + s" The other SparkContext was created at:\n$otherContextCreationSite" logWarning(warnMsg) @@ -2467,8 +2454,6 @@ object SparkContext extends Logging { * singleton object. Because we can only have one active SparkContext per JVM, * this is useful when applications may wish to share a SparkContext. * - * @note This function cannot be used to create multiple SparkContext instances - * even if multiple contexts are allowed. * @param config `SparkConfig` that will be used for initialisation of the `SparkContext` * @return current `SparkContext` (or a new one if it wasn't created before the function call) */ @@ -2477,7 +2462,7 @@ object SparkContext extends Logging { // from assertNoOtherContextIsRunning within setActiveContext SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { if (activeContext.get() == null) { - setActiveContext(new SparkContext(config), allowMultipleContexts = false) + setActiveContext(new SparkContext(config)) } else { if (config.getAll.nonEmpty) { logWarning("Using an existing SparkContext; some configuration may not take effect.") @@ -2494,14 +2479,12 @@ object SparkContext extends Logging { * * This method allows not passing a SparkConf (useful if just retrieving). * - * @note This function cannot be used to create multiple SparkContext instances - * even if multiple contexts are allowed. * @return current `SparkContext` (or a new one if wasn't created before the function call) */ def getOrCreate(): SparkContext = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { if (activeContext.get() == null) { - setActiveContext(new SparkContext(), allowMultipleContexts = false) + setActiveContext(new SparkContext()) } activeContext.get() } @@ -2516,16 +2499,14 @@ object SparkContext extends Logging { /** * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is - * running. Throws an exception if a running context is detected and logs a warning if another - * thread is constructing a SparkContext. This warning is necessary because the current locking + * running. Throws an exception if a running context is detected and logs a warning if another + * thread is constructing a SparkContext. This warning is necessary because the current locking * scheme prevents us from reliably distinguishing between cases where another context is being * constructed and cases where another constructor threw an exception. */ - private[spark] def markPartiallyConstructed( - sc: SparkContext, - allowMultipleContexts: Boolean): Unit = { + private[spark] def markPartiallyConstructed(sc: SparkContext): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - assertNoOtherContextIsRunning(sc, allowMultipleContexts) + assertNoOtherContextIsRunning(sc) contextBeingConstructed = Some(sc) } } @@ -2534,18 +2515,16 @@ object SparkContext extends Logging { * Called at the end of the SparkContext constructor to ensure that no other SparkContext has * raced with this constructor and started. */ - private[spark] def setActiveContext( - sc: SparkContext, - allowMultipleContexts: Boolean): Unit = { + private[spark] def setActiveContext(sc: SparkContext): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - assertNoOtherContextIsRunning(sc, allowMultipleContexts) + assertNoOtherContextIsRunning(sc) contextBeingConstructed = None activeContext.set(sc) } } /** - * Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's + * Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's * also called in unit tests to prevent a flood of warnings from test suites that don't / can't * properly clean up their SparkContexts. */ 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 03f259d73e97..2f74d09b3a2b 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 @@ -40,8 +40,8 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD} * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. * - * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before - * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details. + * @note Only one `SparkContext` should be active per JVM. You must `stop()` the + * active `SparkContext` before creating a new one. */ class JavaSparkContext(val sc: SparkContext) extends Closeable { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index ec4c7efb5835..66de2f2ac86a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -44,7 +44,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("Only one SparkContext may be active at a time") { // Regression test for SPARK-4180 val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.driver.allowMultipleContexts", "false") sc = new SparkContext(conf) val envBefore = SparkEnv.get // A SparkContext is already running, so we shouldn't be able to create a second one @@ -58,7 +57,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test("Can still construct a new SparkContext after failing to construct a previous one") { - val conf = new SparkConf().set("spark.driver.allowMultipleContexts", "false") + val conf = new SparkConf() // This is an invalid configuration (no app name or master URL) intercept[SparkException] { new SparkContext(conf) @@ -67,18 +66,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf.setMaster("local").setAppName("test")) } - test("Check for multiple SparkContexts can be disabled via undocumented debug option") { - var secondSparkContext: SparkContext = null - try { - val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.driver.allowMultipleContexts", "true") - sc = new SparkContext(conf) - secondSparkContext = new SparkContext(conf) - } finally { - Option(secondSparkContext).foreach(_.stop()) - } - } - test("Test getOrCreate") { var sc2: SparkContext = null SparkContext.clearActiveContext() @@ -92,10 +79,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc === sc2) assert(sc eq sc2) - // Try creating second context to confirm that it's still possible, if desired - sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") - .set("spark.driver.allowMultipleContexts", "true")) - sc2.stop() } 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 0621c98d4118..30d0966691a3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -25,8 +25,7 @@ import org.apache.spark.util.AccumulatorV2 class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext { test("launch of backend and scheduler") { - val conf = new SparkConf().setMaster("myclusterManager"). - setAppName("testcm").set("spark.driver.allowMultipleContexts", "true") + val conf = new SparkConf().setMaster("myclusterManager").setAppName("testcm") sc = new SparkContext(conf) // check if the scheduler components are created and initialized sc.schedulerBackend match { diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index 2d1ddae5780d..308a8ea65390 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -138,7 +138,7 @@ The first thing a Spark program must do is to create a [SparkContext](api/scala/ how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object that contains information about your application. -Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one. +Only one SparkContext should be active per JVM. You must `stop()` the active SparkContext before creating a new one. {% highlight scala %} val conf = new SparkConf().setAppName(appName).setMaster(master) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 883913332ca1..7bb70a29195d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -220,6 +220,10 @@ object MimaExcludes { // [SPARK-26139] Implement shuffle write metrics in SQL ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ShuffleDependency.this"), + // [SPARK-26362][CORE] Remove 'spark.driver.allowMultipleContexts' to disallow multiple creation of SparkContexts + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.setActiveContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.markPartiallyConstructed"), + // Data Source V2 API changes (problem: Problem) => problem match { case MissingClassProblem(cls) => diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 1180bf91baa5..6137ed25a0dd 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -63,6 +63,9 @@ class SparkContext(object): Main entry point for Spark functionality. A SparkContext represents the connection to a Spark cluster, and can be used to create L{RDD} and broadcast variables on that cluster. + + .. note:: Only one :class:`SparkContext` should be active per JVM. You must `stop()` + the active :class:`SparkContext` before creating a new one. """ _gateway = None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 6ad025f37e44..4a439940beb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -263,7 +263,6 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { .setMaster("local[*]") .setAppName("test") .set("spark.ui.enabled", "false") - .set("spark.driver.allowMultipleContexts", "true") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") From 860f4497f2a59b21d455ec8bfad9ae15d2fd4d2e Mon Sep 17 00:00:00 2001 From: Jing Chen He Date: Sat, 15 Dec 2018 08:41:16 -0600 Subject: [PATCH 0140/1072] [SPARK-26315][PYSPARK] auto cast threshold from Integer to Float in approxSimilarityJoin of BucketedRandomProjectionLSHModel ## What changes were proposed in this pull request? If the input parameter 'threshold' to the function approxSimilarityJoin is not a float, we would get an exception. The fix is to convert the 'threshold' into a float before calling the java implementation method. ## How was this patch tested? Added a new test case. Without this fix, the test will throw an exception as reported in the JIRA. With the fix, the test passes. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23313 from jerryjch/SPARK-26315. Authored-by: Jing Chen He Signed-off-by: Sean Owen --- python/pyspark/ml/feature.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index c9507c20918e..08ae58246adb 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -192,6 +192,7 @@ def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol") "datasetA" and "datasetB", and a column "distCol" is added to show the distance between each pair. """ + threshold = TypeConverters.toFloat(threshold) return self._call_java("approxSimilarityJoin", datasetA, datasetB, threshold, distCol) @@ -239,6 +240,16 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp | 3| 6| 2.23606797749979| +---+---+-----------------+ ... + >>> model.approxSimilarityJoin(df, df2, 3, distCol="EuclideanDistance").select( + ... col("datasetA.id").alias("idA"), + ... col("datasetB.id").alias("idB"), + ... col("EuclideanDistance")).show() + +---+---+-----------------+ + |idA|idB|EuclideanDistance| + +---+---+-----------------+ + | 3| 6| 2.23606797749979| + +---+---+-----------------+ + ... >>> brpPath = temp_path + "/brp" >>> brp.save(brpPath) >>> brp2 = BucketedRandomProjectionLSH.load(brpPath) From 8a27952cdbf492939d9bda59e2f516f574581636 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 16 Dec 2018 09:32:13 +0800 Subject: [PATCH 0141/1072] [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON ## What changes were proposed in this pull request? In the PR, I propose to switch on **java.time API** for parsing timestamps and dates from JSON inputs with microseconds precision. The SQL config `spark.sql.legacy.timeParser.enabled` allow to switch back to previous behavior with using `java.text.SimpleDateFormat`/`FastDateFormat` for parsing/generating timestamps/dates. ## How was this patch tested? It was tested by `JsonExpressionsSuite`, `JsonFunctionsSuite` and `JsonSuite`. Closes #23196 from MaxGekk/json-time-parser. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- docs/sql-migration-guide-upgrade.md | 2 +- .../sql/catalyst/csv/CSVInferSchema.scala | 6 +- .../sql/catalyst/csv/UnivocityGenerator.scala | 8 +- .../sql/catalyst/csv/UnivocityParser.scala | 6 +- .../spark/sql/catalyst/json/JSONOptions.scala | 10 +- .../sql/catalyst/json/JacksonGenerator.scala | 14 +- .../sql/catalyst/json/JacksonParser.scala | 35 +-- ...rmatter.scala => TimestampFormatter.scala} | 93 ++++---- .../sql/util/DateTimeFormatterSuite.scala | 103 --------- .../util/DateTimestampFormatterSuite.scala | 174 +++++++++++++++ .../datasources/json/JsonSuite.scala | 201 ++++++++++-------- .../sql/sources/HadoopFsRelationTest.scala | 105 ++++----- 12 files changed, 422 insertions(+), 335 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/{DateTimeFormatter.scala => TimestampFormatter.scala} (63%) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimeFormatterSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimestampFormatterSuite.scala diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 8834e8991d8c..115fc6516fb4 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -35,7 +35,7 @@ displayTitle: Spark SQL Upgrading Guide - Spark applications which are built with Spark version 2.4 and prior, and call methods of `UserDefinedFunction`, need to be re-compiled with Spark 3.0, as they are not binary compatible with Spark 3.0. - - Since Spark 3.0, CSV datasource uses java.time API for parsing and generating CSV content. New formatting implementation supports date/timestamp patterns conformed to ISO 8601. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. + - Since Spark 3.0, CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpuse with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. 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 345dc4d41993..35ade136cc60 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 @@ -22,13 +22,13 @@ import scala.util.control.Exception.allCatch import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.expressions.ExprUtils -import org.apache.spark.sql.catalyst.util.DateTimeFormatter +import org.apache.spark.sql.catalyst.util.TimestampFormatter import org.apache.spark.sql.types._ class CSVInferSchema(val options: CSVOptions) extends Serializable { @transient - private lazy val timeParser = DateTimeFormatter( + private lazy val timestampParser = TimestampFormatter( options.timestampFormat, options.timeZone, options.locale) @@ -160,7 +160,7 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { private def tryParseTimestamp(field: String): DataType = { // This case infers a custom `dataFormat` is set. - if ((allCatch opt timeParser.parse(field)).isDefined) { + if ((allCatch opt timestampParser.parse(field)).isDefined) { TimestampType } else { tryParseBoolean(field) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala index af09cd6c8449..f012d96138f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala @@ -22,7 +22,7 @@ import java.io.Writer import com.univocity.parsers.csv.CsvWriter import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeFormatter} +import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} import org.apache.spark.sql.types._ class UnivocityGenerator( @@ -41,18 +41,18 @@ class UnivocityGenerator( private val valueConverters: Array[ValueConverter] = schema.map(_.dataType).map(makeConverter).toArray - private val timeFormatter = DateTimeFormatter( + private val timestampFormatter = TimestampFormatter( options.timestampFormat, options.timeZone, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.timeZone, options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.locale) private def makeConverter(dataType: DataType): ValueConverter = dataType match { case DateType => (row: InternalRow, ordinal: Int) => dateFormatter.format(row.getInt(ordinal)) case TimestampType => - (row: InternalRow, ordinal: Int) => timeFormatter.format(row.getLong(ordinal)) + (row: InternalRow, ordinal: Int) => timestampFormatter.format(row.getLong(ordinal)) case udt: UserDefinedType[_] => makeConverter(udt.sqlType) 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 0f375e036029..ed089120055e 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 @@ -74,11 +74,11 @@ class UnivocityParser( private val row = new GenericInternalRow(requiredSchema.length) - private val timeFormatter = DateTimeFormatter( + private val timestampFormatter = TimestampFormatter( options.timestampFormat, options.timeZone, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.timeZone, options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.locale) // Retrieve the raw record string. private def getCurrentInput: UTF8String = { @@ -158,7 +158,7 @@ class UnivocityParser( } case _: TimestampType => (d: String) => - nullSafeDatum(d, name, nullable, options)(timeFormatter.parse) + nullSafeDatum(d, name, nullable, options)(timestampFormatter.parse) case _: DateType => (d: String) => nullSafeDatum(d, name, nullable, options)(dateFormatter.parse) 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 e10b8a327c01..eaff3fa7bec2 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 @@ -21,7 +21,6 @@ import java.nio.charset.{Charset, StandardCharsets} import java.util.{Locale, TimeZone} import com.fasterxml.jackson.core.{JsonFactory, JsonParser} -import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ @@ -82,13 +81,10 @@ private[sql] class JSONOptions( val timeZone: TimeZone = DateTimeUtils.getTimeZone( parameters.getOrElse(DateTimeUtils.TIMEZONE_OPTION, defaultTimeZoneId)) - // Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe. - val dateFormat: FastDateFormat = - FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd"), locale) + val dateFormat: String = parameters.getOrElse("dateFormat", "yyyy-MM-dd") - val timestampFormat: FastDateFormat = - FastDateFormat.getInstance( - parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), timeZone, locale) + val timestampFormat: String = + parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX") val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index d02a2be8ddad..951f5190cd50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -23,7 +23,7 @@ import com.fasterxml.jackson.core._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters -import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, MapData} +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ /** @@ -77,6 +77,12 @@ private[sql] class JacksonGenerator( private val lineSeparator: String = options.lineSeparatorInWrite + private val timestampFormatter = TimestampFormatter( + options.timestampFormat, + options.timeZone, + options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private def makeWriter(dataType: DataType): ValueWriter = dataType match { case NullType => (row: SpecializedGetters, ordinal: Int) => @@ -116,14 +122,12 @@ private[sql] class JacksonGenerator( case TimestampType => (row: SpecializedGetters, ordinal: Int) => - val timestampString = - options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal))) + val timestampString = timestampFormatter.format(row.getLong(ordinal)) gen.writeString(timestampString) case DateType => (row: SpecializedGetters, ordinal: Int) => - val dateString = - options.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal))) + val dateString = dateFormatter.format(row.getInt(ordinal)) gen.writeString(dateString) case BinaryType => 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 7e3bd4df51bb..3f245e1400fa 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 @@ -55,6 +55,12 @@ class JacksonParser( private val factory = new JsonFactory() options.setJacksonOptions(factory) + private val timestampFormatter = TimestampFormatter( + options.timestampFormat, + options.timeZone, + options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + /** * Create a converter which converts the JSON documents held by the `JsonParser` * to a value according to a desired schema. This is a wrapper for the method @@ -218,17 +224,7 @@ class JacksonParser( case TimestampType => (parser: JsonParser) => parseJsonToken[java.lang.Long](parser, dataType) { case VALUE_STRING if parser.getTextLength >= 1 => - val stringValue = parser.getText - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681. - Long.box { - Try(options.timestampFormat.parse(stringValue).getTime * 1000L) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.stringToTime(stringValue).getTime * 1000L - } - } + timestampFormatter.parse(parser.getText) case VALUE_NUMBER_INT => parser.getLongValue * 1000000L @@ -237,22 +233,7 @@ class JacksonParser( case DateType => (parser: JsonParser) => parseJsonToken[java.lang.Integer](parser, dataType) { case VALUE_STRING if parser.getTextLength >= 1 => - val stringValue = parser.getText - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681.x - Int.box { - Try(DateTimeUtils.millisToDays(options.dateFormat.parse(stringValue).getTime)) - .orElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(stringValue).getTime)) - } - .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. - stringValue.toInt - } - } + dateFormatter.parse(parser.getText) } case BinaryType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala similarity index 63% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index ad1f4131de2f..2b8d22dde926 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.time._ import java.time.format.DateTimeFormatterBuilder -import java.time.temporal.{ChronoField, TemporalQueries} +import java.time.temporal.{ChronoField, TemporalAccessor, TemporalQueries} import java.util.{Locale, TimeZone} import scala.util.Try @@ -28,31 +28,44 @@ import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.sql.internal.SQLConf -sealed trait DateTimeFormatter { +sealed trait TimestampFormatter { def parse(s: String): Long // returns microseconds since epoch def format(us: Long): String } -class Iso8601DateTimeFormatter( +trait FormatterUtils { + protected def zoneId: ZoneId + protected def buildFormatter( + pattern: String, + locale: Locale): java.time.format.DateTimeFormatter = { + new DateTimeFormatterBuilder() + .appendPattern(pattern) + .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970) + .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) + .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) + .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) + .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) + .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) + .toFormatter(locale) + } + protected def toInstantWithZoneId(temporalAccessor: TemporalAccessor): java.time.Instant = { + val localDateTime = LocalDateTime.from(temporalAccessor) + val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) + Instant.from(zonedDateTime) + } +} + +class Iso8601TimestampFormatter( pattern: String, timeZone: TimeZone, - locale: Locale) extends DateTimeFormatter { - val formatter = new DateTimeFormatterBuilder() - .appendPattern(pattern) - .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970) - .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) - .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) - .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) - .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) - .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) - .toFormatter(locale) + locale: Locale) extends TimestampFormatter with FormatterUtils { + val zoneId = timeZone.toZoneId + val formatter = buildFormatter(pattern, locale) def toInstant(s: String): Instant = { val temporalAccessor = formatter.parse(s) if (temporalAccessor.query(TemporalQueries.offset()) == null) { - val localDateTime = LocalDateTime.from(temporalAccessor) - val zonedDateTime = ZonedDateTime.of(localDateTime, timeZone.toZoneId) - Instant.from(zonedDateTime) + toInstantWithZoneId(temporalAccessor) } else { Instant.from(temporalAccessor) } @@ -75,10 +88,10 @@ class Iso8601DateTimeFormatter( } } -class LegacyDateTimeFormatter( +class LegacyTimestampFormatter( pattern: String, timeZone: TimeZone, - locale: Locale) extends DateTimeFormatter { + locale: Locale) extends TimestampFormatter { val format = FastDateFormat.getInstance(pattern, timeZone, locale) protected def toMillis(s: String): Long = format.parse(s).getTime @@ -90,21 +103,21 @@ class LegacyDateTimeFormatter( } } -class LegacyFallbackDateTimeFormatter( +class LegacyFallbackTimestampFormatter( pattern: String, timeZone: TimeZone, - locale: Locale) extends LegacyDateTimeFormatter(pattern, timeZone, locale) { + locale: Locale) extends LegacyTimestampFormatter(pattern, timeZone, locale) { override def toMillis(s: String): Long = { Try {super.toMillis(s)}.getOrElse(DateTimeUtils.stringToTime(s).getTime) } } -object DateTimeFormatter { - def apply(format: String, timeZone: TimeZone, locale: Locale): DateTimeFormatter = { +object TimestampFormatter { + def apply(format: String, timeZone: TimeZone, locale: Locale): TimestampFormatter = { if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyFallbackDateTimeFormatter(format, timeZone, locale) + new LegacyFallbackTimestampFormatter(format, timeZone, locale) } else { - new Iso8601DateTimeFormatter(format, timeZone, locale) + new Iso8601TimestampFormatter(format, timeZone, locale) } } } @@ -116,13 +129,19 @@ sealed trait DateFormatter { class Iso8601DateFormatter( pattern: String, - timeZone: TimeZone, - locale: Locale) extends DateFormatter { + locale: Locale) extends DateFormatter with FormatterUtils { + + val zoneId = ZoneId.of("UTC") + + val formatter = buildFormatter(pattern, locale) - val dateTimeFormatter = new Iso8601DateTimeFormatter(pattern, timeZone, locale) + def toInstant(s: String): Instant = { + val temporalAccessor = formatter.parse(s) + toInstantWithZoneId(temporalAccessor) + } override def parse(s: String): Int = { - val seconds = dateTimeFormatter.toInstant(s).getEpochSecond + val seconds = toInstant(s).getEpochSecond val days = Math.floorDiv(seconds, DateTimeUtils.SECONDS_PER_DAY) days.toInt @@ -130,15 +149,12 @@ class Iso8601DateFormatter( override def format(days: Int): String = { val instant = Instant.ofEpochSecond(days * DateTimeUtils.SECONDS_PER_DAY) - dateTimeFormatter.formatter.withZone(timeZone.toZoneId).format(instant) + formatter.withZone(zoneId).format(instant) } } -class LegacyDateFormatter( - pattern: String, - timeZone: TimeZone, - locale: Locale) extends DateFormatter { - val format = FastDateFormat.getInstance(pattern, timeZone, locale) +class LegacyDateFormatter(pattern: String, locale: Locale) extends DateFormatter { + val format = FastDateFormat.getInstance(pattern, locale) def parse(s: String): Int = { val milliseconds = format.parse(s).getTime @@ -153,8 +169,7 @@ class LegacyDateFormatter( class LegacyFallbackDateFormatter( pattern: String, - timeZone: TimeZone, - locale: Locale) extends LegacyDateFormatter(pattern, timeZone, locale) { + locale: Locale) extends LegacyDateFormatter(pattern, locale) { override def parse(s: String): Int = { Try(super.parse(s)).orElse { // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards @@ -169,11 +184,11 @@ class LegacyFallbackDateFormatter( } object DateFormatter { - def apply(format: String, timeZone: TimeZone, locale: Locale): DateFormatter = { + def apply(format: String, locale: Locale): DateFormatter = { if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyFallbackDateFormatter(format, timeZone, locale) + new LegacyFallbackDateFormatter(format, locale) } else { - new Iso8601DateFormatter(format, timeZone, locale) + new Iso8601DateFormatter(format, locale) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimeFormatterSuite.scala deleted file mode 100644 index 02d4ee049060..000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimeFormatterSuite.scala +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.util - -import java.util.{Locale, TimeZone} - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeFormatter, DateTimeTestUtils} - -class DateTimeFormatterSuite extends SparkFunSuite { - test("parsing dates using time zones") { - val localDate = "2018-12-02" - val expectedDays = Map( - "UTC" -> 17867, - "PST" -> 17867, - "CET" -> 17866, - "Africa/Dakar" -> 17867, - "America/Los_Angeles" -> 17867, - "Antarctica/Vostok" -> 17866, - "Asia/Hong_Kong" -> 17866, - "Europe/Amsterdam" -> 17866) - DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => - val formatter = DateFormatter("yyyy-MM-dd", TimeZone.getTimeZone(timeZone), Locale.US) - val daysSinceEpoch = formatter.parse(localDate) - assert(daysSinceEpoch === expectedDays(timeZone)) - } - } - - test("parsing timestamps using time zones") { - val localDate = "2018-12-02T10:11:12.001234" - val expectedMicros = Map( - "UTC" -> 1543745472001234L, - "PST" -> 1543774272001234L, - "CET" -> 1543741872001234L, - "Africa/Dakar" -> 1543745472001234L, - "America/Los_Angeles" -> 1543774272001234L, - "Antarctica/Vostok" -> 1543723872001234L, - "Asia/Hong_Kong" -> 1543716672001234L, - "Europe/Amsterdam" -> 1543741872001234L) - DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => - val formatter = DateTimeFormatter( - "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", - TimeZone.getTimeZone(timeZone), - Locale.US) - val microsSinceEpoch = formatter.parse(localDate) - assert(microsSinceEpoch === expectedMicros(timeZone)) - } - } - - test("format dates using time zones") { - val daysSinceEpoch = 17867 - val expectedDate = Map( - "UTC" -> "2018-12-02", - "PST" -> "2018-12-01", - "CET" -> "2018-12-02", - "Africa/Dakar" -> "2018-12-02", - "America/Los_Angeles" -> "2018-12-01", - "Antarctica/Vostok" -> "2018-12-02", - "Asia/Hong_Kong" -> "2018-12-02", - "Europe/Amsterdam" -> "2018-12-02") - DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => - val formatter = DateFormatter("yyyy-MM-dd", TimeZone.getTimeZone(timeZone), Locale.US) - val date = formatter.format(daysSinceEpoch) - assert(date === expectedDate(timeZone)) - } - } - - test("format timestamps using time zones") { - val microsSinceEpoch = 1543745472001234L - val expectedTimestamp = Map( - "UTC" -> "2018-12-02T10:11:12.001234", - "PST" -> "2018-12-02T02:11:12.001234", - "CET" -> "2018-12-02T11:11:12.001234", - "Africa/Dakar" -> "2018-12-02T10:11:12.001234", - "America/Los_Angeles" -> "2018-12-02T02:11:12.001234", - "Antarctica/Vostok" -> "2018-12-02T16:11:12.001234", - "Asia/Hong_Kong" -> "2018-12-02T18:11:12.001234", - "Europe/Amsterdam" -> "2018-12-02T11:11:12.001234") - DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => - val formatter = DateTimeFormatter( - "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", - TimeZone.getTimeZone(timeZone), - Locale.US) - val timestamp = formatter.format(microsSinceEpoch) - assert(timestamp === expectedTimestamp(timeZone)) - } - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimestampFormatterSuite.scala new file mode 100644 index 000000000000..43e348c7eebf --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimestampFormatterSuite.scala @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.util + +import java.util.{Locale, TimeZone} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf + +class DateTimestampFormatterSuite extends SparkFunSuite with SQLHelper { + test("parsing dates") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val daysSinceEpoch = formatter.parse("2018-12-02") + assert(daysSinceEpoch === 17867) + } + } + } + + test("parsing timestamps using time zones") { + val localDate = "2018-12-02T10:11:12.001234" + val expectedMicros = Map( + "UTC" -> 1543745472001234L, + "PST" -> 1543774272001234L, + "CET" -> 1543741872001234L, + "Africa/Dakar" -> 1543745472001234L, + "America/Los_Angeles" -> 1543774272001234L, + "Antarctica/Vostok" -> 1543723872001234L, + "Asia/Hong_Kong" -> 1543716672001234L, + "Europe/Amsterdam" -> 1543741872001234L) + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + val formatter = TimestampFormatter( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + TimeZone.getTimeZone(timeZone), + Locale.US) + val microsSinceEpoch = formatter.parse(localDate) + assert(microsSinceEpoch === expectedMicros(timeZone)) + } + } + + test("format dates") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val date = formatter.format(17867) + assert(date === "2018-12-02") + } + } + } + + test("format timestamps using time zones") { + val microsSinceEpoch = 1543745472001234L + val expectedTimestamp = Map( + "UTC" -> "2018-12-02T10:11:12.001234", + "PST" -> "2018-12-02T02:11:12.001234", + "CET" -> "2018-12-02T11:11:12.001234", + "Africa/Dakar" -> "2018-12-02T10:11:12.001234", + "America/Los_Angeles" -> "2018-12-02T02:11:12.001234", + "Antarctica/Vostok" -> "2018-12-02T16:11:12.001234", + "Asia/Hong_Kong" -> "2018-12-02T18:11:12.001234", + "Europe/Amsterdam" -> "2018-12-02T11:11:12.001234") + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + val formatter = TimestampFormatter( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + TimeZone.getTimeZone(timeZone), + Locale.US) + val timestamp = formatter.format(microsSinceEpoch) + assert(timestamp === expectedTimestamp(timeZone)) + } + } + + test("roundtrip timestamp -> micros -> timestamp using timezones") { + Seq( + -58710115316212000L, + -18926315945345679L, + -9463427405253013L, + -244000001L, + 0L, + 99628200102030L, + 1543749753123456L, + 2177456523456789L, + 11858049903010203L).foreach { micros => + DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone, Locale.US) + val timestamp = formatter.format(micros) + val parsed = formatter.parse(timestamp) + assert(micros === parsed) + } + } + } + + test("roundtrip micros -> timestamp -> micros using timezones") { + Seq( + "0109-07-20T18:38:03.788000", + "1370-04-01T10:00:54.654321", + "1670-02-11T14:09:54.746987", + "1969-12-31T23:55:55.999999", + "1970-01-01T00:00:00.000000", + "1973-02-27T02:30:00.102030", + "2018-12-02T11:22:33.123456", + "2039-01-01T01:02:03.456789", + "2345-10-07T22:45:03.010203").foreach { timestamp => + DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone, Locale.US) + val micros = formatter.parse(timestamp) + val formatted = formatter.format(micros) + assert(timestamp === formatted) + } + } + } + + test("roundtrip date -> days -> date") { + Seq( + "0050-01-01", + "0953-02-02", + "1423-03-08", + "1969-12-31", + "1972-08-25", + "1975-09-26", + "2018-12-12", + "2038-01-01", + "5010-11-17").foreach { date => + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val days = formatter.parse(date) + val formatted = formatter.format(days) + assert(date === formatted) + } + } + } + } + + test("roundtrip days -> date -> days") { + Seq( + -701265, + -371419, + -199722, + -1, + 0, + 967, + 2094, + 17877, + 24837, + 1110657).foreach { days => + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val date = formatter.format(days) + val parsed = formatter.parse(date) + assert(days === parsed) + } + } + } + } +} 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 3330de3584eb..786335b42e3c 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 @@ -57,14 +57,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } val factory = new JsonFactory() - def enforceCorrectType(value: Any, dataType: DataType): Any = { + def enforceCorrectType( + value: Any, + dataType: DataType, + options: Map[String, String] = Map.empty): Any = { val writer = new StringWriter() Utils.tryWithResource(factory.createGenerator(writer)) { generator => generator.writeObject(value) generator.flush() } - val dummyOption = new JSONOptions(Map.empty[String, String], "GMT") + val dummyOption = new JSONOptions(options, SQLConf.get.sessionLocalTimeZone) val dummySchema = StructType(Seq.empty) val parser = new JacksonParser(dummySchema, dummyOption, allowArrayAsStructs = true) @@ -96,19 +99,27 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong * 1000L)), enforceCorrectType(intNumber.toLong, TimestampType)) val strTime = "2014-09-30 12:34:56" - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)), - enforceCorrectType(strTime, TimestampType)) + checkTypePromotion( + expected = DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)), + enforceCorrectType(strTime, TimestampType, + Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"))) val strDate = "2014-10-15" checkTypePromotion( DateTimeUtils.fromJavaDate(Date.valueOf(strDate)), enforceCorrectType(strDate, DateType)) val ISO8601Time1 = "1970-01-01T01:00:01.0Z" - val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(3601000)), - enforceCorrectType(ISO8601Time1, TimestampType)) + enforceCorrectType( + ISO8601Time1, + TimestampType, + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SX"))) + val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(10801000)), - enforceCorrectType(ISO8601Time2, TimestampType)) + enforceCorrectType( + ISO8601Time2, + TimestampType, + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ssXXX"))) val ISO8601Date = "1970-01-01" checkTypePromotion(DateTimeUtils.millisToDays(32400000), @@ -1440,103 +1451,105 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("backward compatibility") { - // This test we make sure our JSON support can read JSON data generated by previous version - // of Spark generated through toJSON method and JSON data source. - // The data is generated by the following program. - // Here are a few notes: - // - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13) - // in the JSON object. - // - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to - // JSON objects generated by those Spark versions (col17). - // - If the type is NullType, we do not write data out. - - // Create the schema. - val struct = - StructType( - StructField("f1", FloatType, true) :: - StructField("f2", ArrayType(BooleanType), true) :: Nil) + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> "true") { + // This test we make sure our JSON support can read JSON data generated by previous version + // of Spark generated through toJSON method and JSON data source. + // The data is generated by the following program. + // Here are a few notes: + // - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13) + // in the JSON object. + // - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to + // JSON objects generated by those Spark versions (col17). + // - If the type is NullType, we do not write data out. + + // Create the schema. + val struct = + StructType( + StructField("f1", FloatType, true) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) - val dataTypes = - Seq( - StringType, BinaryType, NullType, BooleanType, - ByteType, ShortType, IntegerType, LongType, - FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), - DateType, TimestampType, - ArrayType(IntegerType), MapType(StringType, LongType), struct, - new TestUDT.MyDenseVectorUDT()) - val fields = dataTypes.zipWithIndex.map { case (dataType, index) => - StructField(s"col$index", dataType, nullable = true) - } - val schema = StructType(fields) + val dataTypes = + Seq( + StringType, BinaryType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), + DateType, TimestampType, + ArrayType(IntegerType), MapType(StringType, LongType), struct, + new TestUDT.MyDenseVectorUDT()) + val fields = dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, nullable = true) + } + val schema = StructType(fields) - val constantValues = - Seq( - "a string in binary".getBytes(StandardCharsets.UTF_8), - null, - true, - 1.toByte, - 2.toShort, - 3, - Long.MaxValue, - 0.25.toFloat, - 0.75, - new java.math.BigDecimal(s"1234.23456"), - new java.math.BigDecimal(s"1.23456"), - java.sql.Date.valueOf("2015-01-01"), - java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"), - Seq(2, 3, 4), - Map("a string" -> 2000L), - Row(4.75.toFloat, Seq(false, true)), - new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25))) - val data = - Row.fromSeq(Seq("Spark " + spark.sparkContext.version) ++ constantValues) :: Nil + val constantValues = + Seq( + "a string in binary".getBytes(StandardCharsets.UTF_8), + null, + true, + 1.toByte, + 2.toShort, + 3, + Long.MaxValue, + 0.25.toFloat, + 0.75, + new java.math.BigDecimal(s"1234.23456"), + new java.math.BigDecimal(s"1.23456"), + java.sql.Date.valueOf("2015-01-01"), + java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"), + Seq(2, 3, 4), + Map("a string" -> 2000L), + Row(4.75.toFloat, Seq(false, true)), + new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25))) + val data = + Row.fromSeq(Seq("Spark " + spark.sparkContext.version) ++ constantValues) :: Nil - // Data generated by previous versions. - // scalastyle:off - val existingJSONData = + // Data generated by previous versions. + // scalastyle:off + val existingJSONData = """{"col0":"Spark 1.2.2","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"16436","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil - // scalastyle:on - - // Generate data for the current version. - val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1), schema) - withTempPath { path => - df.write.format("json").mode("overwrite").save(path.getCanonicalPath) + """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: + """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: + """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: + """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: + """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: + """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"16436","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil + // scalastyle:on + + // Generate data for the current version. + val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1), schema) + withTempPath { path => + df.write.format("json").mode("overwrite").save(path.getCanonicalPath) - // df.toJSON will convert internal rows to external rows first and then generate - // JSON objects. While, df.write.format("json") will write internal rows directly. - val allJSON = + // df.toJSON will convert internal rows to external rows first and then generate + // JSON objects. While, df.write.format("json") will write internal rows directly. + val allJSON = existingJSONData ++ df.toJSON.collect() ++ sparkContext.textFile(path.getCanonicalPath).collect() - Utils.deleteRecursively(path) - sparkContext.parallelize(allJSON, 1).saveAsTextFile(path.getCanonicalPath) - - // Read data back with the schema specified. - val col0Values = - Seq( - "Spark 1.2.2", - "Spark 1.3.1", - "Spark 1.3.1", - "Spark 1.4.1", - "Spark 1.4.1", - "Spark 1.5.0", - "Spark 1.5.0", - "Spark " + spark.sparkContext.version, - "Spark " + spark.sparkContext.version) - val expectedResult = col0Values.map { v => - Row.fromSeq(Seq(v) ++ constantValues) + Utils.deleteRecursively(path) + sparkContext.parallelize(allJSON, 1).saveAsTextFile(path.getCanonicalPath) + + // Read data back with the schema specified. + val col0Values = + Seq( + "Spark 1.2.2", + "Spark 1.3.1", + "Spark 1.3.1", + "Spark 1.4.1", + "Spark 1.4.1", + "Spark 1.5.0", + "Spark 1.5.0", + "Spark " + spark.sparkContext.version, + "Spark " + spark.sparkContext.version) + val expectedResult = col0Values.map { v => + Row.fromSeq(Seq(v) ++ constantValues) + } + checkAnswer( + spark.read.format("json").schema(schema).load(path.getCanonicalPath), + expectedResult + ) } - checkAnswer( - spark.read.format("json").schema(schema).load(path.getCanonicalPath), - expectedResult - ) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 6075f2c8877d..f0f62b608785 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import java.io.File +import java.util.TimeZone import scala.util.Random @@ -125,56 +126,62 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } else { Seq(false) } - for (dataType <- supportedDataTypes) { - for (parquetDictionaryEncodingEnabled <- parquetDictionaryEncodingEnabledConfs) { - val extraMessage = if (isParquetDataSource) { - s" with parquet.enable.dictionary = $parquetDictionaryEncodingEnabled" - } else { - "" - } - logInfo(s"Testing $dataType data type$extraMessage") - - val extraOptions = Map[String, String]( - "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString - ) - - withTempPath { file => - val path = file.getCanonicalPath - - val dataGenerator = RandomDataGenerator.forType( - dataType = dataType, - nullable = true, - new Random(System.nanoTime()) - ).getOrElse { - fail(s"Failed to create data generator for schema $dataType") + // TODO: Support new parser too, see SPARK-26374. + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> "true") { + for (dataType <- supportedDataTypes) { + for (parquetDictionaryEncodingEnabled <- parquetDictionaryEncodingEnabledConfs) { + val extraMessage = if (isParquetDataSource) { + s" with parquet.enable.dictionary = $parquetDictionaryEncodingEnabled" + } else { + "" + } + logInfo(s"Testing $dataType data type$extraMessage") + + val extraOptions = Map[String, String]( + "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString + ) + + withTempPath { file => + val path = file.getCanonicalPath + + val seed = System.nanoTime() + withClue(s"Random data generated with the seed: ${seed}") { + val dataGenerator = RandomDataGenerator.forType( + dataType = dataType, + nullable = true, + new Random(seed) + ).getOrElse { + fail(s"Failed to create data generator for schema $dataType") + } + + // Create a DF for the schema with random data. The index field is used to sort the + // DataFrame. This is a workaround for SPARK-10591. + val schema = new StructType() + .add("index", IntegerType, nullable = false) + .add("col", dataType, nullable = true) + val rdd = + spark.sparkContext.parallelize((1 to 10).map(i => Row(i, dataGenerator()))) + val df = spark.createDataFrame(rdd, schema).orderBy("index").coalesce(1) + + df.write + .mode("overwrite") + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .options(extraOptions) + .save(path) + + val loadedDF = spark + .read + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .schema(df.schema) + .options(extraOptions) + .load(path) + .orderBy("index") + + checkAnswer(loadedDF, df) + } } - - // Create a DF for the schema with random data. The index field is used to sort the - // DataFrame. This is a workaround for SPARK-10591. - val schema = new StructType() - .add("index", IntegerType, nullable = false) - .add("col", dataType, nullable = true) - val rdd = - spark.sparkContext.parallelize((1 to 10).map(i => Row(i, dataGenerator()))) - val df = spark.createDataFrame(rdd, schema).orderBy("index").coalesce(1) - - df.write - .mode("overwrite") - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .options(extraOptions) - .save(path) - - val loadedDF = spark - .read - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .schema(df.schema) - .options(extraOptions) - .load(path) - .orderBy("index") - - checkAnswer(loadedDF, df) } } } From cd815ae6c5ce3edb8aec3add942549f76a20e586 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Sun, 16 Dec 2018 10:57:11 +0800 Subject: [PATCH 0142/1072] [SPARK-26078][SQL] Dedup self-join attributes on IN subqueries ## What changes were proposed in this pull request? When there is a self-join as result of a IN subquery, the join condition may be invalid, resulting in trivially true predicates and return wrong results. The PR deduplicates the subquery output in order to avoid the issue. ## How was this patch tested? added UT Closes #23057 from mgaido91/SPARK-26078. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/subquery.scala | 99 ++++++++++++------- .../org/apache/spark/sql/SubquerySuite.scala | 37 +++++++ 2 files changed, 98 insertions(+), 38 deletions(-) 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 e9b7a8b76e68..34840c6c977a 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -43,31 +43,53 @@ import org.apache.spark.sql.types._ * condition. */ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { - private def dedupJoin(joinPlan: LogicalPlan): LogicalPlan = joinPlan match { + + private def buildJoin( + outerPlan: LogicalPlan, + subplan: LogicalPlan, + joinType: JoinType, + condition: Option[Expression]): Join = { + // Deduplicate conflicting attributes if any. + val dedupSubplan = dedupSubqueryOnSelfJoin(outerPlan, subplan, None, condition) + Join(outerPlan, dedupSubplan, joinType, condition) + } + + private def dedupSubqueryOnSelfJoin( + outerPlan: LogicalPlan, + subplan: LogicalPlan, + valuesOpt: Option[Seq[Expression]], + condition: Option[Expression] = None): LogicalPlan = { // SPARK-21835: It is possibly that the two sides of the join have conflicting attributes, // the produced join then becomes unresolved and break structural integrity. We should - // de-duplicate conflicting attributes. We don't use transformation here because we only - // care about the most top join converted from correlated predicate subquery. - case j @ Join(left, right, joinType @ (LeftSemi | LeftAnti | ExistenceJoin(_)), joinCond) => - val duplicates = right.outputSet.intersect(left.outputSet) - if (duplicates.nonEmpty) { - val aliasMap = AttributeMap(duplicates.map { dup => - dup -> Alias(dup, dup.toString)() - }.toSeq) - val aliasedExpressions = right.output.map { ref => - aliasMap.getOrElse(ref, ref) - } - val newRight = Project(aliasedExpressions, right) - val newJoinCond = joinCond.map { condExpr => - condExpr transform { - case a: Attribute => aliasMap.getOrElse(a, a).toAttribute + // de-duplicate conflicting attributes. + // SPARK-26078: it may also happen that the subquery has conflicting attributes with the outer + // values. In this case, the resulting join would contain trivially true conditions (eg. + // id#3 = id#3) which cannot be de-duplicated after. In this method, if there are conflicting + // attributes in the join condition, the subquery's conflicting attributes are changed using + // a projection which aliases them and resolves the problem. + val outerReferences = valuesOpt.map(values => + AttributeSet.fromAttributeSets(values.map(_.references))).getOrElse(AttributeSet.empty) + val outerRefs = outerPlan.outputSet ++ outerReferences + val duplicates = outerRefs.intersect(subplan.outputSet) + if (duplicates.nonEmpty) { + condition.foreach { e => + val conflictingAttrs = e.references.intersect(duplicates) + if (conflictingAttrs.nonEmpty) { + throw new AnalysisException("Found conflicting attributes " + + s"${conflictingAttrs.mkString(",")} in the condition joining outer plan:\n " + + s"$outerPlan\nand subplan:\n $subplan") } - } - Join(left, newRight, joinType, newJoinCond) - } else { - j } - case _ => joinPlan + val rewrites = AttributeMap(duplicates.map { dup => + dup -> Alias(dup, dup.toString)() + }.toSeq) + val aliasedExpressions = subplan.output.map { ref => + rewrites.getOrElse(ref, ref) + } + Project(aliasedExpressions, subplan) + } else { + subplan + } } def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -85,17 +107,16 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { withSubquery.foldLeft(newFilter) { case (p, Exists(sub, conditions, _)) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftSemi, joinCond)) + buildJoin(outerPlan, sub, LeftSemi, joinCond) case (p, Not(Exists(sub, conditions, _))) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftAnti, joinCond)) + buildJoin(outerPlan, sub, LeftAnti, joinCond) case (p, InSubquery(values, ListQuery(sub, conditions, _, _))) => - val inConditions = values.zip(sub.output).map(EqualTo.tupled) - val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p) // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftSemi, joinCond)) + val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) + val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p) + Join(outerPlan, newSub, LeftSemi, joinCond) case (p, Not(InSubquery(values, ListQuery(sub, conditions, _, _)))) => // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr // Construct the condition. A NULL in one of the conditions is regarded as a positive @@ -103,7 +124,10 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Note that will almost certainly be planned as a Broadcast Nested Loop join. // Use EXISTS if performance matters to you. - val inConditions = values.zip(sub.output).map(EqualTo.tupled) + + // Deduplicate conflicting attributes if any. + val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions, p) // Expand the NOT IN expression with the NULL-aware semantic // to its full form. That is from: @@ -118,8 +142,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // will have the final conditions in the LEFT ANTI as // (A.A1 = B.B1 OR ISNULL(A.A1 = B.B1)) AND (B.B2 = A.A2) AND B.B3 > 1 val finalJoinCond = (nullAwareJoinConds ++ conditions).reduceLeft(And) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftAnti, Option(finalJoinCond))) + Join(outerPlan, newSub, LeftAnti, Option(finalJoinCond)) case (p, predicate) => val (newCond, inputPlan) = rewriteExistentialExpr(Seq(predicate), p) Project(p.output, Filter(newCond.get, inputPlan)) @@ -140,16 +163,16 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { e transformUp { case Exists(sub, conditions, _) => val exists = AttributeReference("exists", BooleanType, nullable = false)() - // Deduplicate conflicting attributes if any. - newPlan = dedupJoin( - Join(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And))) + newPlan = + buildJoin(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) exists case InSubquery(values, ListQuery(sub, conditions, _, _)) => val exists = AttributeReference("exists", BooleanType, nullable = false)() - val inConditions = values.zip(sub.output).map(EqualTo.tupled) - val newConditions = (inConditions ++ conditions).reduceLeftOption(And) // Deduplicate conflicting attributes if any. - newPlan = dedupJoin(Join(newPlan, sub, ExistenceJoin(exists), newConditions)) + val newSub = dedupSubqueryOnSelfJoin(newPlan, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) + val newConditions = (inConditions ++ conditions).reduceLeftOption(And) + newPlan = Join(newPlan, newSub, ExistenceJoin(exists), newConditions) exists } } 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 5088821ad736..c95c52f1d3a9 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 @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ class SubquerySuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -1280,4 +1281,40 @@ class SubquerySuite extends QueryTest with SharedSQLContext { assert(subqueries.length == 1) } } + + test("SPARK-26078: deduplicate fake self joins for IN subqueries") { + withTempView("a", "b") { + Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("a") + Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("b") + + val df1 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id IN (SELECT id FROM b WHERE num = 2) + """.stripMargin) + checkAnswer(df1, Seq(Row("a", 2, "a"), Row("a", 2, "b"))) + val df2 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id NOT IN (SELECT id FROM b WHERE num = 2) + """.stripMargin) + checkAnswer(df2, Seq(Row("b", 1, "a"), Row("b", 1, "b"))) + val df3 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id IN (SELECT id FROM b WHERE num = 2) OR + |c.id IN (SELECT id FROM b WHERE num = 3) + """.stripMargin) + checkAnswer(df3, Seq(Row("a", 2, "a"), Row("a", 2, "b"))) + } + } } From e3e33d8794da5f3597b8d706b734af5025360939 Mon Sep 17 00:00:00 2001 From: Bruce Robbins Date: Sun, 16 Dec 2018 11:02:00 +0800 Subject: [PATCH 0143/1072] [SPARK-26372][SQL] Don't reuse value from previous row when parsing bad CSV input field ## What changes were proposed in this pull request? CSV parsing accidentally uses the previous good value for a bad input field. See example in Jira. This PR ensures that the associated column is set to null when an input field cannot be converted. ## How was this patch tested? Added new test. Ran all SQL unit tests (testOnly org.apache.spark.sql.*). Ran pyspark tests for pyspark-sql Closes #23323 from bersprockets/csv-bad-field. Authored-by: Bruce Robbins Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/csv/UnivocityParser.scala | 1 + .../resources/test-data/bad_after_good.csv | 2 ++ .../execution/datasources/csv/CSVSuite.scala | 19 +++++++++++++++++++ 3 files changed, 22 insertions(+) create mode 100644 sql/core/src/test/resources/test-data/bad_after_good.csv 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 ed089120055e..82a5b3c302b1 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 @@ -239,6 +239,7 @@ class UnivocityParser( } catch { case NonFatal(e) => badRecordException = badRecordException.orElse(Some(e)) + row.setNullAt(i) } i += 1 } diff --git a/sql/core/src/test/resources/test-data/bad_after_good.csv b/sql/core/src/test/resources/test-data/bad_after_good.csv new file mode 100644 index 000000000000..4621a7d23714 --- /dev/null +++ b/sql/core/src/test/resources/test-data/bad_after_good.csv @@ -0,0 +1,2 @@ +"good record",1999-08-01 +"bad record",1999-088-01 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 3b977d74053e..d9e5d7af1967 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 @@ -63,6 +63,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te private val datesFile = "test-data/dates.csv" private val unescapedQuotesFile = "test-data/unescaped-quotes.csv" private val valueMalformedFile = "test-data/value-malformed.csv" + private val badAfterGoodFile = "test-data/bad_after_good.csv" /** Verifies data and schema. */ private def verifyCars( @@ -2012,4 +2013,22 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te assert(!files.exists(_.getName.endsWith("csv"))) } } + + test("Do not reuse last good value for bad input field") { + val schema = StructType( + StructField("col1", StringType) :: + StructField("col2", DateType) :: + Nil + ) + val rows = spark.read + .schema(schema) + .format("csv") + .load(testFile(badAfterGoodFile)) + + val expectedRows = Seq( + Row("good record", java.sql.Date.valueOf("1999-08-01")), + Row("bad record", null)) + + checkAnswer(rows, expectedRows) + } } From 5217f7b2263c7aaeadf60ef602776bb3777269cd Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 17 Dec 2018 08:24:51 +0800 Subject: [PATCH 0144/1072] [SPARK-26248][SQL] Infer date type from CSV ## What changes were proposed in this pull request? The `CSVInferSchema` class is extended to support inferring of `DateType` from CSV input. The attempt to infer `DateType` is performed after inferring `TimestampType`. ## How was this patch tested? Added new test for inferring date types from CSV . It was also tested by existing suites like `CSVInferSchemaSuite`, `CsvExpressionsSuite`, `CsvFunctionsSuite` and `CsvSuite`. Closes #23202 from MaxGekk/csv-date-inferring. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/csv/CSVInferSchema.scala | 20 +++++++++++++++---- .../catalyst/csv/CSVInferSchemaSuite.scala | 18 +++++++++++++++++ 2 files changed, 34 insertions(+), 4 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 35ade136cc60..11f3740d99a7 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 @@ -22,16 +22,20 @@ import scala.util.control.Exception.allCatch import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.expressions.ExprUtils -import org.apache.spark.sql.catalyst.util.TimestampFormatter +import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} import org.apache.spark.sql.types._ class CSVInferSchema(val options: CSVOptions) extends Serializable { @transient - private lazy val timestampParser = TimestampFormatter( + private lazy val timestampFormatter = TimestampFormatter( options.timestampFormat, options.timeZone, options.locale) + @transient + private lazy val dateFormatter = DateFormatter( + options.dateFormat, + options.locale) private val decimalParser = { ExprUtils.getDecimalParser(options.locale) @@ -104,6 +108,7 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { compatibleType(typeSoFar, tryParseDecimal(field)).getOrElse(StringType) case DoubleType => tryParseDouble(field) case TimestampType => tryParseTimestamp(field) + case DateType => tryParseDate(field) case BooleanType => tryParseBoolean(field) case StringType => StringType case other: DataType => @@ -159,9 +164,16 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { } private def tryParseTimestamp(field: String): DataType = { - // This case infers a custom `dataFormat` is set. - if ((allCatch opt timestampParser.parse(field)).isDefined) { + if ((allCatch opt timestampFormatter.parse(field)).isDefined) { TimestampType + } else { + tryParseDate(field) + } + } + + private def tryParseDate(field: String): DataType = { + if ((allCatch opt dateFormatter.parse(field)).isDefined) { + DateType } else { tryParseBoolean(field) } 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 c2b525ad1a9f..84b2e616a442 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 @@ -187,4 +187,22 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach(checkDecimalInfer(_, DecimalType(7, 0))) } + + test("inferring date type") { + var options = new CSVOptions(Map("dateFormat" -> "yyyy/MM/dd"), false, "GMT") + var inferSchema = new CSVInferSchema(options) + assert(inferSchema.inferField(NullType, "2018/12/02") == DateType) + + options = new CSVOptions(Map("dateFormat" -> "MMM yyyy"), false, "GMT") + inferSchema = new CSVInferSchema(options) + assert(inferSchema.inferField(NullType, "Dec 2018") == DateType) + + options = new CSVOptions( + Map("dateFormat" -> "yyyy-MM-dd", "timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"), + columnPruning = false, + defaultTimeZoneId = "GMT") + inferSchema = new CSVInferSchema(options) + assert(inferSchema.inferField(NullType, "2018-12-03T11:00:00") == TimestampType) + assert(inferSchema.inferField(NullType, "2018-12-03") == DateType) + } } From e408e05322ac4e31de4d9bc58687c86882e3944a Mon Sep 17 00:00:00 2001 From: Keiji Yoshida Date: Sun, 16 Dec 2018 17:11:58 -0800 Subject: [PATCH 0145/1072] [MINOR][DOCS] Fix the "not found: value Row" error on the "programmatic_schema" example ## What changes were proposed in this pull request? Print `import org.apache.spark.sql.Row` of `SparkSQLExample.scala` on the `programmatic_schema` example to fix the `not found: value Row` error on it. ``` scala> val rowRDD = peopleRDD.map(_.split(",")).map(attributes => Row(attributes(0), attributes(1).trim)) :28: error: not found: value Row val rowRDD = peopleRDD.map(_.split(",")).map(attributes => Row(attributes(0), attributes(1).trim)) ``` ## How was this patch tested? NA Closes #23326 from kjmrknsn/fix-sql-getting-started. Authored-by: Keiji Yoshida Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/examples/sql/SparkSQLExample.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala index 958361a6684c..678cbc64aff1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -16,7 +16,9 @@ */ package org.apache.spark.examples.sql +// $example on:programmatic_schema$ import org.apache.spark.sql.Row +// $example off:programmatic_schema$ // $example on:init_session$ import org.apache.spark.sql.SparkSession // $example off:init_session$ From db1c5b1839598eada81e4709ab4d25e799bb1810 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 17 Dec 2018 11:53:14 +0800 Subject: [PATCH 0146/1072] Revert "[SPARK-26248][SQL] Infer date type from CSV" This reverts commit 5217f7b2263c7aaeadf60ef602776bb3777269cd. --- .../sql/catalyst/csv/CSVInferSchema.scala | 20 ++++--------------- .../catalyst/csv/CSVInferSchemaSuite.scala | 18 ----------------- 2 files changed, 4 insertions(+), 34 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 11f3740d99a7..35ade136cc60 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 @@ -22,20 +22,16 @@ import scala.util.control.Exception.allCatch import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.expressions.ExprUtils -import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.TimestampFormatter import org.apache.spark.sql.types._ class CSVInferSchema(val options: CSVOptions) extends Serializable { @transient - private lazy val timestampFormatter = TimestampFormatter( + private lazy val timestampParser = TimestampFormatter( options.timestampFormat, options.timeZone, options.locale) - @transient - private lazy val dateFormatter = DateFormatter( - options.dateFormat, - options.locale) private val decimalParser = { ExprUtils.getDecimalParser(options.locale) @@ -108,7 +104,6 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { compatibleType(typeSoFar, tryParseDecimal(field)).getOrElse(StringType) case DoubleType => tryParseDouble(field) case TimestampType => tryParseTimestamp(field) - case DateType => tryParseDate(field) case BooleanType => tryParseBoolean(field) case StringType => StringType case other: DataType => @@ -164,16 +159,9 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { } private def tryParseTimestamp(field: String): DataType = { - if ((allCatch opt timestampFormatter.parse(field)).isDefined) { + // This case infers a custom `dataFormat` is set. + if ((allCatch opt timestampParser.parse(field)).isDefined) { TimestampType - } else { - tryParseDate(field) - } - } - - private def tryParseDate(field: String): DataType = { - if ((allCatch opt dateFormatter.parse(field)).isDefined) { - DateType } else { tryParseBoolean(field) } 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 84b2e616a442..c2b525ad1a9f 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 @@ -187,22 +187,4 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach(checkDecimalInfer(_, DecimalType(7, 0))) } - - test("inferring date type") { - var options = new CSVOptions(Map("dateFormat" -> "yyyy/MM/dd"), false, "GMT") - var inferSchema = new CSVInferSchema(options) - assert(inferSchema.inferField(NullType, "2018/12/02") == DateType) - - options = new CSVOptions(Map("dateFormat" -> "MMM yyyy"), false, "GMT") - inferSchema = new CSVInferSchema(options) - assert(inferSchema.inferField(NullType, "Dec 2018") == DateType) - - options = new CSVOptions( - Map("dateFormat" -> "yyyy-MM-dd", "timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"), - columnPruning = false, - defaultTimeZoneId = "GMT") - inferSchema = new CSVInferSchema(options) - assert(inferSchema.inferField(NullType, "2018-12-03T11:00:00") == TimestampType) - assert(inferSchema.inferField(NullType, "2018-12-03") == DateType) - } } From 56448c662398f4c5319a337e6601450270a6a27c Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Mon, 17 Dec 2018 13:41:20 +0800 Subject: [PATCH 0147/1072] [SPARK-26352][SQL] join reorder should not change the order of output attributes ## What changes were proposed in this pull request? The optimizer rule `org.apache.spark.sql.catalyst.optimizer.ReorderJoin` performs join reordering on inner joins. This was introduced from SPARK-12032 (https://github.com/apache/spark/pull/10073) in 2015-12. After it had reordered the joins, though, it didn't check whether or not the output attribute order is still the same as before. Thus, it's possible to have a mismatch between the reordered output attributes order vs the schema that a DataFrame thinks it has. The same problem exists in the CBO version of join reordering (`CostBasedJoinReorder`) too. This can be demonstrated with the example: ```scala spark.sql("create table table_a (x int, y int) using parquet") spark.sql("create table table_b (i int, j int) using parquet") spark.sql("create table table_c (a int, b int) using parquet") val df = spark.sql(""" with df1 as (select * from table_a cross join table_b) select * from df1 join table_c on a = x and b = i """) ``` here's what the DataFrame thinks: ``` scala> df.printSchema root |-- x: integer (nullable = true) |-- y: integer (nullable = true) |-- i: integer (nullable = true) |-- j: integer (nullable = true) |-- a: integer (nullable = true) |-- b: integer (nullable = true) ``` here's what the optimized plan thinks, after join reordering: ``` scala> df.queryExecution.optimizedPlan.output.foreach(a => println(s"|-- ${a.name}: ${a.dataType.typeName}")) |-- x: integer |-- y: integer |-- a: integer |-- b: integer |-- i: integer |-- j: integer ``` If we exclude the `ReorderJoin` rule (using Spark 2.4's optimizer rule exclusion feature), it's back to normal: ``` scala> spark.conf.set("spark.sql.optimizer.excludedRules", "org.apache.spark.sql.catalyst.optimizer.ReorderJoin") scala> val df = spark.sql("with df1 as (select * from table_a cross join table_b) select * from df1 join table_c on a = x and b = i") df: org.apache.spark.sql.DataFrame = [x: int, y: int ... 4 more fields] scala> df.queryExecution.optimizedPlan.output.foreach(a => println(s"|-- ${a.name}: ${a.dataType.typeName}")) |-- x: integer |-- y: integer |-- i: integer |-- j: integer |-- a: integer |-- b: integer ``` Note that this output attribute ordering problem leads to data corruption, and can manifest itself in various symptoms: * Silently corrupting data, if the reordered columns happen to either have matching types or have sufficiently-compatible types (e.g. all fixed length primitive types are considered as "sufficiently compatible" in an `UnsafeRow`), then only the resulting data is going to be wrong but it might not trigger any alarms immediately. Or * Weird Java-level exceptions like `java.lang.NegativeArraySizeException`, or even SIGSEGVs. ## How was this patch tested? Added new unit test in `JoinReorderSuite` and new end-to-end test in `JoinSuite`. Also made `JoinReorderSuite` and `StarJoinReorderSuite` assert more strongly on maintaining output attribute order. Closes #23303 from rednaxelafx/fix-join-reorder. Authored-by: Kris Mok Signed-off-by: Wenchen Fan --- .../optimizer/CostBasedJoinReorder.scala | 10 +++++ .../spark/sql/catalyst/optimizer/joins.scala | 12 +++++- .../optimizer/JoinOptimizationSuite.scala | 3 ++ .../catalyst/optimizer/JoinReorderSuite.scala | 38 +++++++++++++++++-- .../StarJoinCostBasedReorderSuite.scala | 21 +++++++++- .../optimizer/StarJoinReorderSuite.scala | 28 ++++++++++++-- .../org/apache/spark/sql/JoinSuite.scala | 14 +++++++ 7 files changed, 116 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 064ca68b7a62..01634a9d852c 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 @@ -48,6 +48,7 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { if projectList.forall(_.isInstanceOf[Attribute]) => reorder(p, p.output) } + // After reordering is finished, convert OrderedJoin back to Join result transformDown { case OrderedJoin(left, right, jt, cond) => Join(left, right, jt, cond) @@ -175,11 +176,20 @@ object JoinReorderDP extends PredicateHelper with Logging { assert(topOutputSet == p.outputSet) // Keep the same order of final output attributes. p.copy(projectList = output) + case finalPlan if !sameOutput(finalPlan, output) => + Project(output, finalPlan) case finalPlan => finalPlan } } + private def sameOutput(plan: LogicalPlan, expectedOutput: Seq[Attribute]): Boolean = { + val thisOutput = plan.output + thisOutput.length == expectedOutput.length && thisOutput.zip(expectedOutput).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + } + /** Find all possible plans at the next level, based on existing levels. */ private def searchLevel( existingLevels: Seq[JoinPlanMap], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 6ebb194d71c2..0b6471289a47 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -86,9 +86,9 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case ExtractFiltersAndInnerJoins(input, conditions) + case p @ ExtractFiltersAndInnerJoins(input, conditions) if input.size > 2 && conditions.nonEmpty => - if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { + val reordered = if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { val starJoinPlan = StarSchemaDetection.reorderStarJoins(input, conditions) if (starJoinPlan.nonEmpty) { val rest = input.filterNot(starJoinPlan.contains(_)) @@ -99,6 +99,14 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { } else { createOrderedJoin(input, conditions) } + + if (p.sameOutput(reordered)) { + reordered + } else { + // Reordering the joins have changed the order of the columns. + // Inject a projection to make sure we restore to the expected ordering. + Project(p.output, reordered) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala index ccd9d8dd4d21..e9438b2eee55 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala @@ -102,16 +102,19 @@ class JoinOptimizationSuite extends PlanTest { x.join(y).join(z).where(("x.b".attr === "z.b".attr) && ("y.d".attr === "z.a".attr)), x.join(z, condition = Some("x.b".attr === "z.b".attr)) .join(y, condition = Some("y.d".attr === "z.a".attr)) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ), ( x.join(y, Cross).join(z, Cross) .where(("x.b".attr === "z.b".attr) && ("y.d".attr === "z.a".attr)), x.join(z, Cross, Some("x.b".attr === "z.b".attr)) .join(y, Cross, Some("y.d".attr === "z.a".attr)) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ), ( x.join(y, Inner).join(z, Cross).where("x.b".attr === "z.a".attr), x.join(z, Cross, Some("x.b".attr === "z.a".attr)).join(y, Inner) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ) ) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala index 565b0a10154a..c94a8b9e318f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala @@ -20,8 +20,8 @@ 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.{Attribute, AttributeMap} -import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} -import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.{Cross, Inner, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} import org.apache.spark.sql.internal.SQLConf.{CBO_ENABLED, JOIN_REORDER_ENABLED} @@ -124,7 +124,8 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { // the original order (t1 J t2) J t3. val bestPlan = t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) - .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -139,7 +140,9 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { val bestPlan = t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) // this is redundant but we'll take it for now .join(t4) + .select(outputsOf(t1, t2, t4, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -202,6 +205,7 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) .join(t4.join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + .select(outputsOf(t1, t4, t2, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -219,6 +223,23 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { } } + test("SPARK-26352: join reordering should not change the order of attributes") { + // This test case does not rely on CBO. + // It's similar to the test case above, but catches a reordering bug that the one above doesn't + val tab1 = LocalRelation('x.int, 'y.int) + val tab2 = LocalRelation('i.int, 'j.int) + val tab3 = LocalRelation('a.int, 'b.int) + val original = + tab1.join(tab2, Cross) + .join(tab3, Inner, Some('a === 'x && 'b === 'i)) + val expected = + tab1.join(tab3, Inner, Some('a === 'x)) + .join(tab2, Cross, Some('b === 'i)) + .select(outputsOf(tab1, tab2, tab3): _*) + + assertEqualPlans(original, expected) + } + test("reorder recursively") { // Original order: // Join @@ -266,8 +287,17 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { private def assertEqualPlans( originalPlan: LogicalPlan, groundTruthBestPlan: LogicalPlan): Unit = { - val optimized = Optimize.execute(originalPlan.analyze) + val analyzed = originalPlan.analyze + val optimized = Optimize.execute(analyzed) val expected = groundTruthBestPlan.analyze + + assert(analyzed.sameOutput(expected)) // if this fails, the expected plan itself is incorrect + assert(analyzed.sameOutput(optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala index d4d23ad69b2c..baae934e1e4f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala @@ -218,6 +218,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) .join(t2, Inner, Some(nameToAttr("f1_c2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("f1_c1") === nameToAttr("t1_c1"))) + .select(outputsOf(f1, t1, t2, d1, d2): _*) assertEqualPlans(query, expected) } @@ -256,6 +257,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas .join(t3.join(t2, Inner, Some(nameToAttr("t2_c2") === nameToAttr("t3_c1"))), Inner, Some(nameToAttr("d1_c2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("t1_c1") === nameToAttr("f1_c1"))) + .select(outputsOf(d1, t1, t2, f1, d2, t3): _*) assertEqualPlans(query, expected) } @@ -297,6 +299,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) .join(t1.join(t2, Inner, Some(nameToAttr("t1_c1") === nameToAttr("t2_c1"))), Inner, Some(nameToAttr("t1_c2") === nameToAttr("t4_c2"))) + .select(outputsOf(d1, t1, t2, t3, t4, f1, d2): _*) assertEqualPlans(query, expected) } @@ -347,6 +350,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas Some(nameToAttr("d3_c2") === nameToAttr("t1_c1"))) .join(t5.join(t6, Inner, Some(nameToAttr("t5_c2") === nameToAttr("t6_c2"))), Inner, Some(nameToAttr("d2_c2") === nameToAttr("t5_c1"))) + .select(outputsOf(d1, t3, t4, f1, d2, t5, t6, d3, t1, t2): _*) assertEqualPlans(query, expected) } @@ -375,6 +379,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas f1.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) + .select(outputsOf(d1, d2, f1, d3): _*) assertEqualPlans(query, expected) } @@ -400,13 +405,27 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas f1.join(t3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("t3_c1"))) .join(t2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("t1_c1"))) + .select(outputsOf(t1, f1, t2, t3): _*) assertEqualPlans(query, expected) } private def assertEqualPlans( plan1: LogicalPlan, plan2: LogicalPlan): Unit = { - val optimized = Optimize.execute(plan1.analyze) + val analyzed = plan1.analyze + val optimized = Optimize.execute(analyzed) val expected = plan2.analyze + + assert(equivalentOutput(analyzed, expected)) // if this fails, the expected itself is incorrect + assert(equivalentOutput(analyzed, optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } + + private def equivalentOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + normalizeExprIds(plan1).output == normalizeExprIds(plan2).output + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala index 4e0883e91e84..9dc653b9d6c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala @@ -182,6 +182,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk1"))) .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d2, f1, d3, s3): _*) assertEqualPlans(query, expected) } @@ -220,6 +221,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") < nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -255,7 +257,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("s3_c2"))) - + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -292,6 +294,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_c2"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") < nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -395,6 +398,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("f11_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f11, f1, d2, s3): _*) assertEqualPlans(query, equivQuery) } @@ -430,6 +434,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_c4"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -465,6 +470,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -499,6 +505,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -532,6 +539,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") < nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") < nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") < nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -565,13 +573,27 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } - private def assertEqualPlans( plan1: LogicalPlan, plan2: LogicalPlan): Unit = { - val optimized = Optimize.execute(plan1.analyze) + private def assertEqualPlans(plan1: LogicalPlan, plan2: LogicalPlan): Unit = { + val analyzed = plan1.analyze + val optimized = Optimize.execute(analyzed) val expected = plan2.analyze + + assert(equivalentOutput(analyzed, expected)) // if this fails, the expected itself is incorrect + assert(equivalentOutput(analyzed, optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } + + private def equivalentOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + normalizeExprIds(plan1).output == normalizeExprIds(plan2).output + } } 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 aa2162c9d2cd..91445c8d96d8 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 @@ -895,4 +895,18 @@ class JoinSuite extends QueryTest with SharedSQLContext { checkAnswer(res, Row(0, 0, 0)) } } + + test("SPARK-26352: join reordering should not change the order of columns") { + withTable("tab1", "tab2", "tab3") { + spark.sql("select 1 as x, 100 as y").write.saveAsTable("tab1") + spark.sql("select 42 as i, 200 as j").write.saveAsTable("tab2") + spark.sql("select 1 as a, 42 as b").write.saveAsTable("tab3") + + val df = spark.sql(""" + with tmp as (select * from tab1 cross join tab2) + select * from tmp join tab3 on a = x and b = i + """) + checkAnswer(df, Row(1, 100, 42, 200, 1, 42)) + } + } } From 5960a8297ca06a4c62f39a8821dba4ba172f2bfc Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sun, 16 Dec 2018 23:40:06 -0800 Subject: [PATCH 0148/1072] [SPARK-26327][SQL][FOLLOW-UP] Refactor the code and restore the metrics name ## What changes were proposed in this pull request? - The original comment about `updateDriverMetrics` is not right. - Refactor the code to ensure `selectedPartitions ` has been set before sending the driver-side metrics. - Restore the original name, which is more general and extendable. ## How was this patch tested? The existing tests. Closes #23328 from gatorsmile/followupSpark-26142. Authored-by: gatorsmile Signed-off-by: gatorsmile --- .../sql/execution/DataSourceScanExec.scala | 39 +++++++++---------- 1 file changed, 19 insertions(+), 20 deletions(-) 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 c0fa4e777b49..322ffffca564 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.commons.lang3.StringUtils import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} @@ -167,14 +167,26 @@ case class FileSourceScanExec( partitionSchema = relation.partitionSchema, relation.sparkSession.sessionState.conf) - private var fileListingTime = 0L + val driverMetrics: HashMap[String, Long] = HashMap.empty + + /** + * Send the driver-side metrics. Before calling this function, selectedPartitions has + * been initialized. See SPARK-26327 for more details. + */ + private def sendDriverMetrics(): Unit = { + driverMetrics.foreach(e => metrics(e._1).add(e._2)) + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, + metrics.filter(e => driverMetrics.contains(e._1)).values.toSeq) + } @transient private lazy val selectedPartitions: Seq[PartitionDirectory] = { val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) val startTime = System.nanoTime() val ret = relation.location.listFiles(partitionFilters, dataFilters) + driverMetrics("numFiles") = ret.map(_.files.size.toLong).sum val timeTakenMs = ((System.nanoTime() - startTime) + optimizerMetadataTimeNs) / 1000 / 1000 - fileListingTime = timeTakenMs + driverMetrics("metadataTime") = timeTakenMs ret } @@ -286,8 +298,6 @@ case class FileSourceScanExec( } private lazy val inputRDD: RDD[InternalRow] = { - // Update metrics for taking effect in both code generation node and normal node. - updateDriverMetrics() val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( sparkSession = relation.sparkSession, @@ -298,12 +308,14 @@ case class FileSourceScanExec( options = relation.options, hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options)) - relation.bucketSpec match { + val readRDD = relation.bucketSpec match { case Some(bucketing) if relation.sparkSession.sessionState.conf.bucketingEnabled => createBucketedReadRDD(bucketing, readFile, selectedPartitions, relation) case _ => createNonBucketedReadRDD(readFile, selectedPartitions, relation) } + sendDriverMetrics() + readRDD } override def inputRDDs(): Seq[RDD[InternalRow]] = { @@ -313,7 +325,7 @@ case class FileSourceScanExec( override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of files"), - "fileListingTime" -> SQLMetrics.createMetric(sparkContext, "file listing time (ms)"), + "metadataTime" -> SQLMetrics.createMetric(sparkContext, "metadata time"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) protected override def doExecute(): RDD[InternalRow] = { @@ -504,19 +516,6 @@ case class FileSourceScanExec( } } - /** - * Send the updated metrics to driver, while this function calling, selectedPartitions has - * been initialized. See SPARK-26327 for more detail. - */ - private def updateDriverMetrics() = { - metrics("numFiles").add(selectedPartitions.map(_.files.size.toLong).sum) - metrics("fileListingTime").add(fileListingTime) - - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, - metrics("numFiles") :: metrics("fileListingTime") :: Nil) - } - override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, From f6888f7c944daff3d7c88b37e883673866eb148e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 17 Dec 2018 00:13:51 -0800 Subject: [PATCH 0149/1072] [SPARK-20636] Add the rule TransposeWindow to the optimization batch ## What changes were proposed in this pull request? This PR is a follow-up of the PR https://github.com/apache/spark/pull/17899. It is to add the rule TransposeWindow the optimizer batch. ## How was this patch tested? The existing tests. Closes #23222 from gatorsmile/followupSPARK-20636. Authored-by: gatorsmile Signed-off-by: gatorsmile --- .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../sql/DataFrameWindowFunctionsSuite.scala | 38 +++++++++++++------ 2 files changed, 27 insertions(+), 12 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 f615757a837a..3eb6bca6ec97 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 @@ -73,6 +73,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) CombineLimits, CombineUnions, // Constant folding and strength reduction + TransposeWindow, NullPropagation, ConstantPropagation, FoldablePropagation, 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 9a5d5a9966ab..9277dc685924 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql import org.scalatest.Matchers.the import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} +import org.apache.spark.sql.catalyst.optimizer.TransposeWindow +import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -668,18 +670,30 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { ("S2", "P2", 300) ).toDF("sno", "pno", "qty") - val w1 = Window.partitionBy("sno") - val w2 = Window.partitionBy("sno", "pno") - - checkAnswer( - df.select($"sno", $"pno", $"qty", sum($"qty").over(w2).alias("sum_qty_2")) - .select($"sno", $"pno", $"qty", col("sum_qty_2"), sum("qty").over(w1).alias("sum_qty_1")), - Seq( - Row("S1", "P1", 100, 800, 800), - Row("S1", "P1", 700, 800, 800), - Row("S2", "P1", 200, 200, 500), - Row("S2", "P2", 300, 300, 500))) - + Seq(true, false).foreach { transposeWindowEnabled => + val excludedRules = if (transposeWindowEnabled) "" else TransposeWindow.ruleName + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> excludedRules) { + val w1 = Window.partitionBy("sno") + val w2 = Window.partitionBy("sno", "pno") + + val select = df.select($"sno", $"pno", $"qty", sum($"qty").over(w2).alias("sum_qty_2")) + .select($"sno", $"pno", $"qty", col("sum_qty_2"), sum("qty").over(w1).alias("sum_qty_1")) + + val expectedNumExchanges = if (transposeWindowEnabled) 1 else 2 + val actualNumExchanges = select.queryExecution.executedPlan.collect { + case e: Exchange => e + }.length + assert(actualNumExchanges == expectedNumExchanges) + + checkAnswer( + select, + Seq( + Row("S1", "P1", 100, 800, 800), + Row("S1", "P1", 700, 800, 800), + Row("S2", "P1", 200, 200, 500), + Row("S2", "P2", 300, 300, 500))) + } + } } test("NaN and -0.0 in window partition keys") { From 12640d674b0af6716023fad30fe12cee728bfe34 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 17 Dec 2018 21:47:38 +0800 Subject: [PATCH 0150/1072] [SPARK-26243][SQL][FOLLOWUP] fix code style issues in TimestampFormatter.scala ## What changes were proposed in this pull request? 1. rename `FormatterUtils` to `DateTimeFormatterHelper`, and move it to a separated file 2. move `DateFormatter` and its implementation to a separated file 3. mark some methods as private 4. add `override` to some methods ## How was this patch tested? existing tests Closes #23329 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/util/DateFormatter.scala | 96 +++++++++++++++ .../util/DateTimeFormatterHelper.scala | 44 +++++++ .../catalyst/util/TimestampFormatter.scala | 115 ++---------------- .../spark/sql/util/DateFormatterSuite.scala | 92 ++++++++++++++ ...te.scala => TimestampFormatterSuite.scala} | 73 +---------- 5 files changed, 246 insertions(+), 174 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala rename sql/catalyst/src/test/scala/org/apache/spark/sql/util/{DateTimestampFormatterSuite.scala => TimestampFormatterSuite.scala} (66%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala new file mode 100644 index 000000000000..9e8d51cc65f0 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import java.time.{Instant, ZoneId} +import java.util.Locale + +import scala.util.Try + +import org.apache.commons.lang3.time.FastDateFormat + +import org.apache.spark.sql.internal.SQLConf + +sealed trait DateFormatter { + def parse(s: String): Int // returns days since epoch + def format(days: Int): String +} + +class Iso8601DateFormatter( + pattern: String, + locale: Locale) extends DateFormatter with DateTimeFormatterHelper { + + private val formatter = buildFormatter(pattern, locale) + private val UTC = ZoneId.of("UTC") + + private def toInstant(s: String): Instant = { + val temporalAccessor = formatter.parse(s) + toInstantWithZoneId(temporalAccessor, UTC) + } + + override def parse(s: String): Int = { + val seconds = toInstant(s).getEpochSecond + val days = Math.floorDiv(seconds, DateTimeUtils.SECONDS_PER_DAY) + days.toInt + } + + override def format(days: Int): String = { + val instant = Instant.ofEpochSecond(days * DateTimeUtils.SECONDS_PER_DAY) + formatter.withZone(UTC).format(instant) + } +} + +class LegacyDateFormatter(pattern: String, locale: Locale) extends DateFormatter { + private val format = FastDateFormat.getInstance(pattern, locale) + + override def parse(s: String): Int = { + val milliseconds = format.parse(s).getTime + DateTimeUtils.millisToDays(milliseconds) + } + + override def format(days: Int): String = { + val date = DateTimeUtils.toJavaDate(days) + format.format(date) + } +} + +class LegacyFallbackDateFormatter( + pattern: String, + locale: Locale) extends LegacyDateFormatter(pattern, locale) { + override def parse(s: String): Int = { + Try(super.parse(s)).orElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(s).getTime)) + }.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. + s.toInt + } + } +} + +object DateFormatter { + def apply(format: String, locale: Locale): DateFormatter = { + if (SQLConf.get.legacyTimeParserEnabled) { + new LegacyFallbackDateFormatter(format, locale) + } else { + new Iso8601DateFormatter(format, locale) + } + } +} 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 new file mode 100644 index 000000000000..b85101d38d9e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import java.time.{Instant, LocalDateTime, ZonedDateTime, ZoneId} +import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder} +import java.time.temporal.{ChronoField, TemporalAccessor} +import java.util.Locale + +trait DateTimeFormatterHelper { + + protected def buildFormatter(pattern: String, locale: Locale): DateTimeFormatter = { + new DateTimeFormatterBuilder() + .appendPattern(pattern) + .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970) + .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) + .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) + .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) + .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) + .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) + .toFormatter(locale) + } + + protected def toInstantWithZoneId(temporalAccessor: TemporalAccessor, zoneId: ZoneId): Instant = { + val localDateTime = LocalDateTime.from(temporalAccessor) + val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) + Instant.from(zonedDateTime) + } +} 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 2b8d22dde926..eb1303303463 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 @@ -18,8 +18,7 @@ package org.apache.spark.sql.catalyst.util import java.time._ -import java.time.format.DateTimeFormatterBuilder -import java.time.temporal.{ChronoField, TemporalAccessor, TemporalQueries} +import java.time.temporal.TemporalQueries import java.util.{Locale, TimeZone} import scala.util.Try @@ -33,39 +32,16 @@ sealed trait TimestampFormatter { def format(us: Long): String } -trait FormatterUtils { - protected def zoneId: ZoneId - protected def buildFormatter( - pattern: String, - locale: Locale): java.time.format.DateTimeFormatter = { - new DateTimeFormatterBuilder() - .appendPattern(pattern) - .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970) - .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) - .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) - .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) - .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) - .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) - .toFormatter(locale) - } - protected def toInstantWithZoneId(temporalAccessor: TemporalAccessor): java.time.Instant = { - val localDateTime = LocalDateTime.from(temporalAccessor) - val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) - Instant.from(zonedDateTime) - } -} - class Iso8601TimestampFormatter( pattern: String, timeZone: TimeZone, - locale: Locale) extends TimestampFormatter with FormatterUtils { - val zoneId = timeZone.toZoneId - val formatter = buildFormatter(pattern, locale) + locale: Locale) extends TimestampFormatter with DateTimeFormatterHelper { + private val formatter = buildFormatter(pattern, locale) - def toInstant(s: String): Instant = { + private def toInstant(s: String): Instant = { val temporalAccessor = formatter.parse(s) if (temporalAccessor.query(TemporalQueries.offset()) == null) { - toInstantWithZoneId(temporalAccessor) + toInstantWithZoneId(temporalAccessor, timeZone.toZoneId) } else { Instant.from(temporalAccessor) } @@ -77,9 +53,9 @@ class Iso8601TimestampFormatter( result } - def parse(s: String): Long = instantToMicros(toInstant(s)) + override def parse(s: String): Long = instantToMicros(toInstant(s)) - def format(us: Long): String = { + override def format(us: Long): String = { val secs = Math.floorDiv(us, DateTimeUtils.MICROS_PER_SECOND) val mos = Math.floorMod(us, DateTimeUtils.MICROS_PER_SECOND) val instant = Instant.ofEpochSecond(secs, mos * DateTimeUtils.NANOS_PER_MICROS) @@ -92,13 +68,13 @@ class LegacyTimestampFormatter( pattern: String, timeZone: TimeZone, locale: Locale) extends TimestampFormatter { - val format = FastDateFormat.getInstance(pattern, timeZone, locale) + private val format = FastDateFormat.getInstance(pattern, timeZone, locale) protected def toMillis(s: String): Long = format.parse(s).getTime - def parse(s: String): Long = toMillis(s) * DateTimeUtils.MICROS_PER_MILLIS + override def parse(s: String): Long = toMillis(s) * DateTimeUtils.MICROS_PER_MILLIS - def format(us: Long): String = { + override def format(us: Long): String = { format.format(DateTimeUtils.toJavaTimestamp(us)) } } @@ -121,74 +97,3 @@ object TimestampFormatter { } } } - -sealed trait DateFormatter { - def parse(s: String): Int // returns days since epoch - def format(days: Int): String -} - -class Iso8601DateFormatter( - pattern: String, - locale: Locale) extends DateFormatter with FormatterUtils { - - val zoneId = ZoneId.of("UTC") - - val formatter = buildFormatter(pattern, locale) - - def toInstant(s: String): Instant = { - val temporalAccessor = formatter.parse(s) - toInstantWithZoneId(temporalAccessor) - } - - override def parse(s: String): Int = { - val seconds = toInstant(s).getEpochSecond - val days = Math.floorDiv(seconds, DateTimeUtils.SECONDS_PER_DAY) - - days.toInt - } - - override def format(days: Int): String = { - val instant = Instant.ofEpochSecond(days * DateTimeUtils.SECONDS_PER_DAY) - formatter.withZone(zoneId).format(instant) - } -} - -class LegacyDateFormatter(pattern: String, locale: Locale) extends DateFormatter { - val format = FastDateFormat.getInstance(pattern, locale) - - def parse(s: String): Int = { - val milliseconds = format.parse(s).getTime - DateTimeUtils.millisToDays(milliseconds) - } - - def format(days: Int): String = { - val date = DateTimeUtils.toJavaDate(days) - format.format(date) - } -} - -class LegacyFallbackDateFormatter( - pattern: String, - locale: Locale) extends LegacyDateFormatter(pattern, locale) { - override def parse(s: String): Int = { - Try(super.parse(s)).orElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(s).getTime)) - }.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. - s.toInt - } - } -} - -object DateFormatter { - def apply(format: String, locale: Locale): DateFormatter = { - if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyFallbackDateFormatter(format, locale) - } else { - new Iso8601DateFormatter(format, locale) - } - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala new file mode 100644 index 000000000000..019615b81101 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.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.sql.util + +import java.util.Locale + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf + +class DateFormatterSuite extends SparkFunSuite with SQLHelper { + test("parsing dates") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val daysSinceEpoch = formatter.parse("2018-12-02") + assert(daysSinceEpoch === 17867) + } + } + } + + test("format dates") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val date = formatter.format(17867) + assert(date === "2018-12-02") + } + } + } + + test("roundtrip date -> days -> date") { + Seq( + "0050-01-01", + "0953-02-02", + "1423-03-08", + "1969-12-31", + "1972-08-25", + "1975-09-26", + "2018-12-12", + "2038-01-01", + "5010-11-17").foreach { date => + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val days = formatter.parse(date) + val formatted = formatter.format(days) + assert(date === formatted) + } + } + } + } + + test("roundtrip days -> date -> days") { + Seq( + -701265, + -371419, + -199722, + -1, + 0, + 967, + 2094, + 17877, + 24837, + 1110657).foreach { days => + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val date = formatter.format(days) + val parsed = formatter.parse(date) + assert(days === parsed) + } + } + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala similarity index 66% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimestampFormatterSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index 43e348c7eebf..c110ffa01f73 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateTimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -21,19 +21,9 @@ import java.util.{Locale, TimeZone} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, TimestampFormatter} -class DateTimestampFormatterSuite extends SparkFunSuite with SQLHelper { - test("parsing dates") { - DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter("yyyy-MM-dd", Locale.US) - val daysSinceEpoch = formatter.parse("2018-12-02") - assert(daysSinceEpoch === 17867) - } - } - } +class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { test("parsing timestamps using time zones") { val localDate = "2018-12-02T10:11:12.001234" @@ -56,16 +46,6 @@ class DateTimestampFormatterSuite extends SparkFunSuite with SQLHelper { } } - test("format dates") { - DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter("yyyy-MM-dd", Locale.US) - val date = formatter.format(17867) - assert(date === "2018-12-02") - } - } - } - test("format timestamps using time zones") { val microsSinceEpoch = 1543745472001234L val expectedTimestamp = Map( @@ -87,7 +67,7 @@ class DateTimestampFormatterSuite extends SparkFunSuite with SQLHelper { } } - test("roundtrip timestamp -> micros -> timestamp using timezones") { + test("roundtrip micros -> timestamp -> micros using timezones") { Seq( -58710115316212000L, -18926315945345679L, @@ -107,7 +87,7 @@ class DateTimestampFormatterSuite extends SparkFunSuite with SQLHelper { } } - test("roundtrip micros -> timestamp -> micros using timezones") { + test("roundtrip timestamp -> micros -> timestamp using timezones") { Seq( "0109-07-20T18:38:03.788000", "1370-04-01T10:00:54.654321", @@ -126,49 +106,4 @@ class DateTimestampFormatterSuite extends SparkFunSuite with SQLHelper { } } } - - test("roundtrip date -> days -> date") { - Seq( - "0050-01-01", - "0953-02-02", - "1423-03-08", - "1969-12-31", - "1972-08-25", - "1975-09-26", - "2018-12-12", - "2038-01-01", - "5010-11-17").foreach { date => - DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter("yyyy-MM-dd", Locale.US) - val days = formatter.parse(date) - val formatted = formatter.format(days) - assert(date === formatted) - } - } - } - } - - test("roundtrip days -> date -> days") { - Seq( - -701265, - -371419, - -199722, - -1, - 0, - 967, - 2094, - 17877, - 24837, - 1110657).foreach { days => - DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter("yyyy-MM-dd", Locale.US) - val date = formatter.format(days) - val parsed = formatter.parse(date) - assert(days === parsed) - } - } - } - } } From c04ad17ccf14a07ffdb2bf637124492a341075f2 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Mon, 17 Dec 2018 09:28:23 -0600 Subject: [PATCH 0151/1072] [SPARK-20351][ML] Add trait hasTrainingSummary to replace the duplicate code ## What changes were proposed in this pull request? Add a trait HasTrainingSummary to avoid code duplicate related to training summary. Currently all the training summary use the similar pattern which can be generalized, ``` private[ml] final var trainingSummary: Option[T] = None def hasSummary: Boolean = trainingSummary.isDefined def summary: T = trainingSummary.getOrElse... private[ml] def setSummary(summary: Option[T]): ... ``` Classes with the trait need to override `setSummry`. And for Java compatibility, they will also have to override `summary` method, otherwise the java code will regard all the summary class as Object due to a known issue with Scala. ## How was this patch tested? existing Java and Scala unit tests Closes #17654 from hhbyyh/hassummary. Authored-by: Yuhao Yang Signed-off-by: Sean Owen --- .../classification/LogisticRegression.scala | 24 ++------- .../spark/ml/clustering/BisectingKMeans.scala | 25 ++------- .../spark/ml/clustering/GaussianMixture.scala | 24 ++------- .../apache/spark/ml/clustering/KMeans.scala | 23 ++------ .../GeneralizedLinearRegression.scala | 22 ++------ .../ml/regression/LinearRegression.scala | 21 ++------ .../spark/ml/util/HasTrainingSummary.scala | 52 +++++++++++++++++++ 7 files changed, 78 insertions(+), 113 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/util/HasTrainingSummary.scala 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 27a7db0b2f5d..f2a5c11a3486 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 @@ -934,8 +934,8 @@ class LogisticRegressionModel private[spark] ( @Since("2.1.0") val interceptVector: Vector, @Since("1.3.0") override val numClasses: Int, private val isMultinomial: Boolean) - extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] - with LogisticRegressionParams with MLWritable { + extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] with MLWritable + with LogisticRegressionParams with HasTrainingSummary[LogisticRegressionTrainingSummary] { require(coefficientMatrix.numRows == interceptVector.size, s"Dimension mismatch! Expected " + s"coefficientMatrix.numRows == interceptVector.size, but ${coefficientMatrix.numRows} != " + @@ -1018,20 +1018,16 @@ class LogisticRegressionModel private[spark] ( @Since("1.6.0") override val numFeatures: Int = coefficientMatrix.numCols - private var trainingSummary: Option[LogisticRegressionTrainingSummary] = None - /** * Gets summary of model on training set. An exception is thrown - * if `trainingSummary == None`. + * if `hasSummary` is false. */ @Since("1.5.0") - def summary: LogisticRegressionTrainingSummary = trainingSummary.getOrElse { - throw new SparkException("No training summary available for this LogisticRegressionModel") - } + override def summary: LogisticRegressionTrainingSummary = super.summary /** * Gets summary of model on training set. An exception is thrown - * if `trainingSummary == None` or it is a multiclass model. + * if `hasSummary` is false or it is a multiclass model. */ @Since("2.3.0") def binarySummary: BinaryLogisticRegressionTrainingSummary = summary match { @@ -1062,16 +1058,6 @@ class LogisticRegressionModel private[spark] ( (model, model.getProbabilityCol, model.getPredictionCol) } - private[classification] - def setSummary(summary: Option[LogisticRegressionTrainingSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** Indicates whether a training summary exists for this model instance. */ - @Since("1.5.0") - def hasSummary: Boolean = trainingSummary.isDefined - /** * Evaluates the model on a test dataset. * 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 1a94aefa3f56..49e9f5136813 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 @@ -87,8 +87,9 @@ private[clustering] trait BisectingKMeansParams extends Params with HasMaxIter @Since("2.0.0") class BisectingKMeansModel private[ml] ( @Since("2.0.0") override val uid: String, - private val parentModel: MLlibBisectingKMeansModel - ) extends Model[BisectingKMeansModel] with BisectingKMeansParams with MLWritable { + private val parentModel: MLlibBisectingKMeansModel) + extends Model[BisectingKMeansModel] with BisectingKMeansParams with MLWritable + with HasTrainingSummary[BisectingKMeansSummary] { @Since("2.0.0") override def copy(extra: ParamMap): BisectingKMeansModel = { @@ -143,28 +144,12 @@ class BisectingKMeansModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new BisectingKMeansModel.BisectingKMeansModelWriter(this) - private var trainingSummary: Option[BisectingKMeansSummary] = None - - private[clustering] def setSummary(summary: Option[BisectingKMeansSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** - * Return true if there exists summary of model. - */ - @Since("2.1.0") - def hasSummary: Boolean = trainingSummary.nonEmpty - /** * Gets summary of model on training set. An exception is - * thrown if `trainingSummary == None`. + * thrown if `hasSummary` is false. */ @Since("2.1.0") - def summary: BisectingKMeansSummary = trainingSummary.getOrElse { - throw new SparkException( - s"No training summary available for the ${this.getClass.getSimpleName}") - } + override def summary: BisectingKMeansSummary = super.summary } object BisectingKMeansModel extends MLReadable[BisectingKMeansModel] { 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 88abc1605d69..bb10b3228b93 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 @@ -85,7 +85,8 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override val uid: String, @Since("2.0.0") val weights: Array[Double], @Since("2.0.0") val gaussians: Array[MultivariateGaussian]) - extends Model[GaussianMixtureModel] with GaussianMixtureParams with MLWritable { + extends Model[GaussianMixtureModel] with GaussianMixtureParams with MLWritable + with HasTrainingSummary[GaussianMixtureSummary] { /** @group setParam */ @Since("2.1.0") @@ -160,28 +161,13 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new GaussianMixtureModel.GaussianMixtureModelWriter(this) - private var trainingSummary: Option[GaussianMixtureSummary] = None - - private[clustering] def setSummary(summary: Option[GaussianMixtureSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** - * Return true if there exists summary of model. - */ - @Since("2.0.0") - def hasSummary: Boolean = trainingSummary.nonEmpty - /** * Gets summary of model on training set. An exception is - * thrown if `trainingSummary == None`. + * thrown if `hasSummary` is false. */ @Since("2.0.0") - def summary: GaussianMixtureSummary = trainingSummary.getOrElse { - throw new RuntimeException( - s"No training summary available for the ${this.getClass.getSimpleName}") - } + override def summary: GaussianMixtureSummary = super.summary + } @Since("2.0.0") 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 2eed84d51782..319747d4a193 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 @@ -107,7 +107,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe class KMeansModel private[ml] ( @Since("1.5.0") override val uid: String, private[clustering] val parentModel: MLlibKMeansModel) - extends Model[KMeansModel] with KMeansParams with GeneralMLWritable { + extends Model[KMeansModel] with KMeansParams with GeneralMLWritable + with HasTrainingSummary[KMeansSummary] { @Since("1.5.0") override def copy(extra: ParamMap): KMeansModel = { @@ -153,28 +154,12 @@ class KMeansModel private[ml] ( @Since("1.6.0") override def write: GeneralMLWriter = new GeneralMLWriter(this) - private var trainingSummary: Option[KMeansSummary] = None - - private[clustering] def setSummary(summary: Option[KMeansSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** - * Return true if there exists summary of model. - */ - @Since("2.0.0") - def hasSummary: Boolean = trainingSummary.nonEmpty - /** * Gets summary of model on training set. An exception is - * thrown if `trainingSummary == None`. + * thrown if `hasSummary` is false. */ @Since("2.0.0") - def summary: KMeansSummary = trainingSummary.getOrElse { - throw new SparkException( - s"No training summary available for the ${this.getClass.getSimpleName}") - } + override def summary: KMeansSummary = super.summary } /** Helper class for storing model data */ 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 abb60ea20575..885b13bf8dac 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 @@ -1001,7 +1001,8 @@ class GeneralizedLinearRegressionModel private[ml] ( @Since("2.0.0") val coefficients: Vector, @Since("2.0.0") val intercept: Double) extends RegressionModel[Vector, GeneralizedLinearRegressionModel] - with GeneralizedLinearRegressionBase with MLWritable { + with GeneralizedLinearRegressionBase with MLWritable + with HasTrainingSummary[GeneralizedLinearRegressionTrainingSummary] { /** * Sets the link prediction (linear predictor) column name. @@ -1054,29 +1055,12 @@ class GeneralizedLinearRegressionModel private[ml] ( output.toDF() } - private var trainingSummary: Option[GeneralizedLinearRegressionTrainingSummary] = None - /** * Gets R-like summary of model on training set. An exception is * thrown if there is no summary available. */ @Since("2.0.0") - def summary: GeneralizedLinearRegressionTrainingSummary = trainingSummary.getOrElse { - throw new SparkException( - "No training summary available for this GeneralizedLinearRegressionModel") - } - - /** - * Indicates if [[summary]] is available. - */ - @Since("2.0.0") - def hasSummary: Boolean = trainingSummary.nonEmpty - - private[regression] - def setSummary(summary: Option[GeneralizedLinearRegressionTrainingSummary]): this.type = { - this.trainingSummary = summary - this - } + override def summary: GeneralizedLinearRegressionTrainingSummary = super.summary /** * Evaluate the model on the given dataset, returning a summary of the results. 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 ce6c12cc368d..197828762d16 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 @@ -647,33 +647,20 @@ class LinearRegressionModel private[ml] ( @Since("1.3.0") val intercept: Double, @Since("2.3.0") val scale: Double) extends RegressionModel[Vector, LinearRegressionModel] - with LinearRegressionParams with GeneralMLWritable { + with LinearRegressionParams with GeneralMLWritable + with HasTrainingSummary[LinearRegressionTrainingSummary] { private[ml] def this(uid: String, coefficients: Vector, intercept: Double) = this(uid, coefficients, intercept, 1.0) - private var trainingSummary: Option[LinearRegressionTrainingSummary] = None - override val numFeatures: Int = coefficients.size /** * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is - * thrown if `trainingSummary == None`. + * thrown if `hasSummary` is false. */ @Since("1.5.0") - def summary: LinearRegressionTrainingSummary = trainingSummary.getOrElse { - throw new SparkException("No training summary available for this LinearRegressionModel") - } - - private[regression] - def setSummary(summary: Option[LinearRegressionTrainingSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** Indicates whether a training summary exists for this model instance. */ - @Since("1.5.0") - def hasSummary: Boolean = trainingSummary.isDefined + override def summary: LinearRegressionTrainingSummary = super.summary /** * Evaluates the model on a test dataset. diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/HasTrainingSummary.scala b/mllib/src/main/scala/org/apache/spark/ml/util/HasTrainingSummary.scala new file mode 100644 index 000000000000..edb0208144e1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/HasTrainingSummary.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import org.apache.spark.SparkException +import org.apache.spark.annotation.Since + + +/** + * Trait for models that provides Training summary. + * + * @tparam T Summary instance type + */ +@Since("3.0.0") +private[ml] trait HasTrainingSummary[T] { + + private[ml] final var trainingSummary: Option[T] = None + + /** Indicates whether a training summary exists for this model instance. */ + @Since("3.0.0") + def hasSummary: Boolean = trainingSummary.isDefined + + /** + * Gets summary of model on training set. An exception is + * thrown if if `hasSummary` is false. + */ + @Since("3.0.0") + def summary: T = trainingSummary.getOrElse { + throw new SparkException( + s"No training summary available for this ${this.getClass.getSimpleName}") + } + + private[ml] def setSummary(summary: Option[T]): this.type = { + this.trainingSummary = summary + this + } +} From 6d45e6ea1507943f6ee833af8ad7969294b0356a Mon Sep 17 00:00:00 2001 From: chakravarthi Date: Mon, 17 Dec 2018 09:46:50 -0800 Subject: [PATCH 0152/1072] [SPARK-26255][YARN] Apply user provided UI filters to SQL tab in yarn mode MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? User specified filters are not applied to SQL tab in yarn mode, as it is overridden by the yarn AmIp filter. So we need to append user provided filters (spark.ui.filters) with yarn filter. ## How was this patch tested? 【Test step】: 1) Launch spark sql with authentication filter as below: 2) spark-sql --master yarn --conf spark.ui.filters=org.apache.hadoop.security.authentication.server.AuthenticationFilter --conf spark.org.apache.hadoop.security.authentication.server.AuthenticationFilter.params="type=simple" 3) Go to Yarn application list UI link 4) Launch the application master for the Spark-SQL app ID and access all the tabs by appending tab name. 5) It will display an error for all tabs including SQL tab.(before able to access SQL tab,as Authentication filter is not applied for SQL tab) 6) Also can be verified with info logs,that Authentication filter applied to SQL tab.(before it is not applied). I have attached the behaviour below in following order.. 1) Command used 2) Before fix (logs and UI) 3) After fix (logs and UI) **1) COMMAND USED**: launching spark-sql with authentication filter. ![image](https://user-images.githubusercontent.com/45845595/49947295-e7e97400-ff16-11e8-8c9a-10659487ddee.png) **2) BEFORE FIX:** **UI result:** able to access SQL tab. ![image](https://user-images.githubusercontent.com/45845595/49948398-62b38e80-ff19-11e8-95dc-e74f9e3c2ba7.png) **logs**: authentication filter not applied to SQL tab. ![image](https://user-images.githubusercontent.com/45845595/49947343-ff286180-ff16-11e8-9de0-3f8db140bc32.png) **3) AFTER FIX:** **UI result**: Not able to access SQL tab. ![image](https://user-images.githubusercontent.com/45845595/49947360-0d767d80-ff17-11e8-9e9e-a95311949164.png) **in logs**: Both yarn filter and Authentication filter applied to SQL tab. ![image](https://user-images.githubusercontent.com/45845595/49947377-1a936c80-ff17-11e8-9f44-700eb3dc0ded.png) Closes #23312 from chakravarthiT/SPARK-26255_ui. Authored-by: chakravarthi Signed-off-by: Marcelo Vanzin --- .../apache/spark/scheduler/cluster/YarnSchedulerBackend.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 67c36aac4926..1289d4be79ea 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 @@ -168,8 +168,10 @@ private[spark] abstract class YarnSchedulerBackend( filterName != null && filterName.nonEmpty && filterParams != null && filterParams.nonEmpty if (hasFilter) { + // SPARK-26255: Append user provided filters(spark.ui.filters) with yarn filter. + val allFilters = filterName + "," + conf.get("spark.ui.filters", "") logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") - conf.set("spark.ui.filters", filterName) + conf.set("spark.ui.filters", allFilters) filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } } From 5a116e669cb196f59ab3f8d06477f675cd0400f9 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Mon, 17 Dec 2018 10:07:35 -0800 Subject: [PATCH 0153/1072] [SPARK-26371][SS] Increase kafka ConfigUpdater test coverage. ## What changes were proposed in this pull request? As Kafka delegation token added logic into ConfigUpdater it would be good to test it. This PR contains the following changes: * ConfigUpdater extracted to a separate file and renamed to KafkaConfigUpdater * mockito-core dependency added to kafka-0-10-sql * Unit tests added ## How was this patch tested? Existing + new unit tests + on cluster. Closes #23321 from gaborgsomogyi/SPARK-26371. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- external/kafka-0-10-sql/pom.xml | 5 + .../sql/kafka010/KafkaConfigUpdater.scala | 74 ++++++++++++ .../sql/kafka010/KafkaSourceProvider.scala | 52 +------- .../kafka010/KafkaConfigUpdaterSuite.scala | 113 ++++++++++++++++++ .../kafka010/KafkaDelegationTokenTest.scala | 90 ++++++++++++++ .../kafka010/KafkaSecurityHelperSuite.scala | 46 +------ 6 files changed, 287 insertions(+), 93 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdater.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdaterSuite.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index de8731c4b774..1c77906f43b1 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -106,6 +106,11 @@ ${jetty.version} test + + org.mockito + mockito-core + test + org.scalacheck scalacheck_${scala.binary.version} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdater.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdater.scala new file mode 100644 index 000000000000..bc1b8019f6a6 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdater.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} + +import scala.collection.JavaConverters._ + +import org.apache.kafka.common.config.SaslConfigs + +import org.apache.spark.SparkEnv +import org.apache.spark.deploy.security.KafkaTokenUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kafka + +/** + * Class to conveniently update Kafka config params, while logging the changes + */ +private[kafka010] case class KafkaConfigUpdater(module: String, kafkaParams: Map[String, String]) + extends Logging { + private val map = new ju.HashMap[String, Object](kafkaParams.asJava) + + def set(key: String, value: Object): this.type = { + map.put(key, value) + logDebug(s"$module: Set $key to $value, earlier value: ${kafkaParams.getOrElse(key, "")}") + this + } + + def setIfUnset(key: String, value: Object): this.type = { + if (!map.containsKey(key)) { + map.put(key, value) + logDebug(s"$module: Set $key to $value") + } + this + } + + def setAuthenticationConfigIfNeeded(): this.type = { + // There are multiple possibilities to log in and applied in the following order: + // - JVM global security provided -> try to log in with JVM global security configuration + // which can be configured for example with 'java.security.auth.login.config'. + // For this no additional parameter needed. + // - Token is provided -> try to log in with scram module using kafka's dynamic JAAS + // configuration. + if (KafkaTokenUtil.isGlobalJaasConfigurationProvided) { + logDebug("JVM global security configuration detected, using it for login.") + } else if (KafkaSecurityHelper.isTokenAvailable()) { + logDebug("Delegation token detected, using it for login.") + val jaasParams = KafkaSecurityHelper.getTokenJaasParams(SparkEnv.get.conf) + set(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + val mechanism = SparkEnv.get.conf.get(Kafka.TOKEN_SASL_MECHANISM) + require(mechanism.startsWith("SCRAM"), + "Delegation token works only with SCRAM mechanism.") + set(SaslConfigs.SASL_MECHANISM, mechanism) + } + this + } + + def build(): ju.Map[String, Object] = map +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 4b8b5c0019b4..5774ee7a1c94 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -24,13 +24,9 @@ import scala.collection.JavaConverters._ import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} -import org.apache.spark.SparkEnv -import org.apache.spark.deploy.security.KafkaTokenUtil import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ @@ -483,7 +479,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { } def kafkaParamsForDriver(specifiedKafkaParams: Map[String, String]): ju.Map[String, Object] = - ConfigUpdater("source", specifiedKafkaParams) + KafkaConfigUpdater("source", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -506,7 +502,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { def kafkaParamsForExecutors( specifiedKafkaParams: Map[String, String], uniqueGroupId: String): ju.Map[String, Object] = - ConfigUpdater("executor", specifiedKafkaParams) + KafkaConfigUpdater("executor", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -537,48 +533,6 @@ private[kafka010] object KafkaSourceProvider extends Logging { s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}" } - /** Class to conveniently update Kafka config params, while logging the changes */ - private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { - private val map = new ju.HashMap[String, Object](kafkaParams.asJava) - - def set(key: String, value: Object): this.type = { - map.put(key, value) - logDebug(s"$module: Set $key to $value, earlier value: ${kafkaParams.getOrElse(key, "")}") - this - } - - def setIfUnset(key: String, value: Object): ConfigUpdater = { - if (!map.containsKey(key)) { - map.put(key, value) - logDebug(s"$module: Set $key to $value") - } - this - } - - def setAuthenticationConfigIfNeeded(): ConfigUpdater = { - // There are multiple possibilities to log in and applied in the following order: - // - JVM global security provided -> try to log in with JVM global security configuration - // which can be configured for example with 'java.security.auth.login.config'. - // For this no additional parameter needed. - // - Token is provided -> try to log in with scram module using kafka's dynamic JAAS - // configuration. - if (KafkaTokenUtil.isGlobalJaasConfigurationProvided) { - logDebug("JVM global security configuration detected, using it for login.") - } else if (KafkaSecurityHelper.isTokenAvailable()) { - logDebug("Delegation token detected, using it for login.") - val jaasParams = KafkaSecurityHelper.getTokenJaasParams(SparkEnv.get.conf) - set(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) - val mechanism = SparkEnv.get.conf.get(Kafka.TOKEN_SASL_MECHANISM) - require(mechanism.startsWith("SCRAM"), - "Delegation token works only with SCRAM mechanism.") - set(SaslConfigs.SASL_MECHANISM, mechanism) - } - this - } - - def build(): ju.Map[String, Object] = map - } - private[kafka010] def kafkaParamsForProducer( parameters: Map[String, String]): ju.Map[String, Object] = { val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } @@ -596,7 +550,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { val specifiedKafkaParams = convertToSpecifiedParams(parameters) - ConfigUpdater("executor", specifiedKafkaParams) + KafkaConfigUpdater("executor", specifiedKafkaParams) .set(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, serClassName) .set(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, serClassName) .setAuthenticationConfigIfNeeded() diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdaterSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdaterSuite.scala new file mode 100644 index 000000000000..25ccca3cb984 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdaterSuite.scala @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.common.config.SaslConfigs + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config._ + +class KafkaConfigUpdaterSuite extends SparkFunSuite with KafkaDelegationTokenTest { + private val testModule = "testModule" + private val testKey = "testKey" + private val testValue = "testValue" + private val otherTestValue = "otherTestValue" + + test("set should always set value") { + val params = Map.empty[String, String] + + val updatedParams = KafkaConfigUpdater(testModule, params) + .set(testKey, testValue) + .build() + + assert(updatedParams.size() === 1) + assert(updatedParams.get(testKey) === testValue) + } + + test("setIfUnset without existing key should set value") { + val params = Map.empty[String, String] + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setIfUnset(testKey, testValue) + .build() + + assert(updatedParams.size() === 1) + assert(updatedParams.get(testKey) === testValue) + } + + test("setIfUnset with existing key should not set value") { + val params = Map[String, String](testKey -> testValue) + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setIfUnset(testKey, otherTestValue) + .build() + + assert(updatedParams.size() === 1) + assert(updatedParams.get(testKey) === testValue) + } + + test("setAuthenticationConfigIfNeeded with global security should not set values") { + val params = Map.empty[String, String] + setGlobalKafkaClientConfig() + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setAuthenticationConfigIfNeeded() + .build() + + assert(updatedParams.size() === 0) + } + + test("setAuthenticationConfigIfNeeded with token should set values") { + val params = Map.empty[String, String] + setSparkEnv(Map.empty) + addTokenToUGI() + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setAuthenticationConfigIfNeeded() + .build() + + assert(updatedParams.size() === 2) + assert(updatedParams.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) + assert(updatedParams.get(SaslConfigs.SASL_MECHANISM) === + Kafka.TOKEN_SASL_MECHANISM.defaultValueString) + } + + test("setAuthenticationConfigIfNeeded with token and invalid mechanism should throw exception") { + val params = Map.empty[String, String] + setSparkEnv(Map[String, String](Kafka.TOKEN_SASL_MECHANISM.key -> "INVALID")) + addTokenToUGI() + + val e = intercept[IllegalArgumentException] { + KafkaConfigUpdater(testModule, params) + .setAuthenticationConfigIfNeeded() + .build() + } + + assert(e.getMessage.contains("Delegation token works only with SCRAM mechanism.")) + } + + test("setAuthenticationConfigIfNeeded without security should not set values") { + val params = Map.empty[String, String] + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setAuthenticationConfigIfNeeded() + .build() + + assert(updatedParams.size() === 0) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala new file mode 100644 index 000000000000..1899c65c721b --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} +import javax.security.auth.login.{AppConfigurationEntry, Configuration} + +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.Token +import org.mockito.Mockito.{doReturn, mock} +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} +import org.apache.spark.deploy.security.KafkaTokenUtil +import org.apache.spark.deploy.security.KafkaTokenUtil.KafkaDelegationTokenIdentifier + +/** + * This is a trait which provides functionalities for Kafka delegation token related test suites. + */ +trait KafkaDelegationTokenTest extends BeforeAndAfterEach { + self: SparkFunSuite => + + protected val tokenId = "tokenId" + ju.UUID.randomUUID().toString + protected val tokenPassword = "tokenPassword" + ju.UUID.randomUUID().toString + + private class KafkaJaasConfiguration extends Configuration { + val entry = + new AppConfigurationEntry( + "DummyModule", + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + ju.Collections.emptyMap[String, Object]() + ) + + override def getAppConfigurationEntry(name: String): Array[AppConfigurationEntry] = { + if (name.equals("KafkaClient")) { + Array(entry) + } else { + null + } + } + } + + override def afterEach(): Unit = { + try { + Configuration.setConfiguration(null) + UserGroupInformation.setLoginUser(null) + SparkEnv.set(null) + } finally { + super.afterEach() + } + } + + protected def setGlobalKafkaClientConfig(): Unit = { + Configuration.setConfiguration(new KafkaJaasConfiguration) + } + + protected def addTokenToUGI(): Unit = { + val token = new Token[KafkaDelegationTokenIdentifier]( + tokenId.getBytes, + tokenPassword.getBytes, + KafkaTokenUtil.TOKEN_KIND, + KafkaTokenUtil.TOKEN_SERVICE + ) + val creds = new Credentials() + creds.addToken(KafkaTokenUtil.TOKEN_SERVICE, token) + UserGroupInformation.getCurrentUser.addCredentials(creds) + } + + protected def setSparkEnv(settings: Traversable[(String, String)]): Unit = { + val conf = new SparkConf().setAll(settings) + val env = mock(classOf[SparkEnv]) + doReturn(conf).when(env).conf + SparkEnv.set(env) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala index fd9dee390d18..d908bbfc2c5f 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala @@ -17,51 +17,9 @@ package org.apache.spark.sql.kafka010 -import java.util.UUID - -import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.hadoop.security.token.Token -import org.scalatest.BeforeAndAfterEach - import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.security.KafkaTokenUtil -import org.apache.spark.deploy.security.KafkaTokenUtil.KafkaDelegationTokenIdentifier - -class KafkaSecurityHelperSuite extends SparkFunSuite with BeforeAndAfterEach { - private val tokenId = "tokenId" + UUID.randomUUID().toString - private val tokenPassword = "tokenPassword" + UUID.randomUUID().toString - - private var sparkConf: SparkConf = null - - override def beforeEach(): Unit = { - super.beforeEach() - sparkConf = new SparkConf() - } - - override def afterEach(): Unit = { - try { - resetUGI - } finally { - super.afterEach() - } - } - - private def addTokenToUGI(): Unit = { - val token = new Token[KafkaDelegationTokenIdentifier]( - tokenId.getBytes, - tokenPassword.getBytes, - KafkaTokenUtil.TOKEN_KIND, - KafkaTokenUtil.TOKEN_SERVICE - ) - val creds = new Credentials() - creds.addToken(KafkaTokenUtil.TOKEN_SERVICE, token) - UserGroupInformation.getCurrentUser.addCredentials(creds) - } - - private def resetUGI: Unit = { - UserGroupInformation.setLoginUser(null) - } +class KafkaSecurityHelperSuite extends SparkFunSuite with KafkaDelegationTokenTest { test("isTokenAvailable without token should return false") { assert(!KafkaSecurityHelper.isTokenAvailable()) } @@ -75,7 +33,7 @@ class KafkaSecurityHelperSuite extends SparkFunSuite with BeforeAndAfterEach { test("getTokenJaasParams with token should return scram module") { addTokenToUGI() - val jaasParams = KafkaSecurityHelper.getTokenJaasParams(sparkConf) + val jaasParams = KafkaSecurityHelper.getTokenJaasParams(new SparkConf()) assert(jaasParams.contains("ScramLoginModule required")) assert(jaasParams.contains("tokenauth=true")) From 81d377d772a527d9ae3311be0480e6403769e919 Mon Sep 17 00:00:00 2001 From: Vaclav Kosar Date: Mon, 17 Dec 2018 11:50:24 -0800 Subject: [PATCH 0154/1072] [SPARK-24933][SS] Report numOutputRows in SinkProgress ## What changes were proposed in this pull request? SinkProgress should report similar properties like SourceProgress as long as they are available for given Sink. Count of written rows is metric availble for all Sinks. Since relevant progress information is with respect to commited rows, ideal object to carry this info is WriterCommitMessage. For brevity the implementation will focus only on Sinks with API V2 and on Micro Batch mode. Implemention for Continuous mode will be provided at later date. ### Before ``` {"description":"org.apache.spark.sql.kafka010.KafkaSourceProvider3c0bd317"} ``` ### After ``` {"description":"org.apache.spark.sql.kafka010.KafkaSourceProvider3c0bd317","numOutputRows":5000} ``` ### This PR is related to: - https://issues.apache.org/jira/browse/SPARK-24647 - https://issues.apache.org/jira/browse/SPARK-21313 ## How was this patch tested? Existing and new unit tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #21919 from vackosar/feature/SPARK-24933-numOutputRows. Lead-authored-by: Vaclav Kosar Co-authored-by: Kosar, Vaclav: Functions Transformation Signed-off-by: gatorsmile --- .../spark/sql/kafka010/KafkaSinkSuite.scala | 21 +++++++++++++ .../v2/WriteToDataSourceV2Exec.scala | 30 +++++++++++++++---- .../streaming/MicroBatchExecution.scala | 11 +++++-- .../streaming/ProgressReporter.scala | 7 +++-- .../execution/streaming/StreamExecution.scala | 4 +++ .../apache/spark/sql/streaming/progress.scala | 21 +++++++++++-- ...StreamingQueryStatusAndProgressSuite.scala | 10 ++++--- 7 files changed, 88 insertions(+), 16 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index d46c4139011d..07d2b8a5dc42 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -232,6 +232,27 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { } } + test("streaming - sink progress is produced") { + /* ensure sink progress is correctly produced. */ + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF(), + withTopic = Some(topic), + withOutputMode = Some(OutputMode.Update()))() + + try { + input.addData("1", "2", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + assert(writer.lastProgress.sink.numOutputRows == 3L) + } finally { + writer.stop() + } + } test("streaming - write data with bad schema") { val input = MemoryStream[String] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 9a1fe1e0a328..d7e20eed4cbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{LongAccumulator, Utils} /** * Deprecated logical plan for writing data into data source v2. This is being replaced by more @@ -47,6 +47,8 @@ case class WriteToDataSourceV2(writeSupport: BatchWriteSupport, query: LogicalPl case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: SparkPlan) extends UnaryExecNode { + var commitProgress: Option[StreamWriterCommitProgress] = None + override def child: SparkPlan = query override def output: Seq[Attribute] = Nil @@ -55,6 +57,7 @@ case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: Spark val useCommitCoordinator = writeSupport.useCommitCoordinator val rdd = query.execute() val messages = new Array[WriterCommitMessage](rdd.partitions.length) + val totalNumRowsAccumulator = new LongAccumulator() logInfo(s"Start processing data source write support: $writeSupport. " + s"The input RDD has ${messages.length} partitions.") @@ -65,15 +68,18 @@ case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: Spark (context: TaskContext, iter: Iterator[InternalRow]) => DataWritingSparkTask.run(writerFactory, context, iter, useCommitCoordinator), rdd.partitions.indices, - (index, message: WriterCommitMessage) => { - messages(index) = message - writeSupport.onDataWriterCommit(message) + (index, result: DataWritingSparkTaskResult) => { + val commitMessage = result.writerCommitMessage + messages(index) = commitMessage + totalNumRowsAccumulator.add(result.numRows) + writeSupport.onDataWriterCommit(commitMessage) } ) logInfo(s"Data source write support $writeSupport is committing.") writeSupport.commit(messages) logInfo(s"Data source write support $writeSupport committed.") + commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value)) } catch { case cause: Throwable => logError(s"Data source write support $writeSupport is aborting.") @@ -102,7 +108,7 @@ object DataWritingSparkTask extends Logging { writerFactory: DataWriterFactory, context: TaskContext, iter: Iterator[InternalRow], - useCommitCoordinator: Boolean): WriterCommitMessage = { + useCommitCoordinator: Boolean): DataWritingSparkTaskResult = { val stageId = context.stageId() val stageAttempt = context.stageAttemptNumber() val partId = context.partitionId() @@ -110,9 +116,12 @@ object DataWritingSparkTask extends Logging { val attemptId = context.attemptNumber() val dataWriter = writerFactory.createWriter(partId, taskId) + var count = 0L // write the data and commit this writer. Utils.tryWithSafeFinallyAndFailureCallbacks(block = { while (iter.hasNext) { + // Count is here. + count += 1 dataWriter.write(iter.next()) } @@ -139,7 +148,7 @@ object DataWritingSparkTask extends Logging { logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId" + s"stage $stageId.$stageAttempt)") - msg + DataWritingSparkTaskResult(count, msg) })(catchBlock = { // If there is an error, abort this writer @@ -151,3 +160,12 @@ object DataWritingSparkTask extends Logging { }) } } + +private[v2] case class DataWritingSparkTaskResult( + numRows: Long, + writerCommitMessage: WriterCommitMessage) + +/** + * Sink progress information collected after commit. + */ +private[sql] case class StreamWriterCommitProgress(numOutputRows: Long) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 03beefeca269..8ad436a4ff57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentBatchTimestamp, import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, WriteToDataSourceV2} +import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamWriterCommitProgress, WriteToDataSourceV2, WriteToDataSourceV2Exec} import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWritSupport, RateControlMicroBatchReadSupport} import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} @@ -246,6 +246,7 @@ class MicroBatchExecution( * DONE */ private def populateStartOffsets(sparkSessionToRunBatches: SparkSession): Unit = { + sinkCommitProgress = None offsetLog.getLatest() match { case Some((latestBatchId, nextOffsets)) => /* First assume that we are re-executing the latest known batch @@ -537,7 +538,8 @@ class MicroBatchExecution( val nextBatch = new Dataset(sparkSessionToRunBatch, lastExecution, RowEncoder(lastExecution.analyzed.schema)) - reportTimeTaken("addBatch") { + val batchSinkProgress: Option[StreamWriterCommitProgress] = + reportTimeTaken("addBatch") { SQLExecution.withNewExecutionId(sparkSessionToRunBatch, lastExecution) { sink match { case s: Sink => s.addBatch(currentBatchId, nextBatch) @@ -545,10 +547,15 @@ class MicroBatchExecution( // This doesn't accumulate any data - it just forces execution of the microbatch writer. nextBatch.collect() } + lastExecution.executedPlan match { + case w: WriteToDataSourceV2Exec => w.commitProgress + case _ => None + } } } withProgressLocked { + sinkCommitProgress = batchSinkProgress watermarkTracker.updateWatermark(lastExecution.executedPlan) commitLog.add(currentBatchId, CommitMetadata(watermarkTracker.currentWatermark)) committedOffsets ++= availableOffsets 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 39ab702ee083..d1f3f74c5e73 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2StreamingScanExec +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2StreamingScanExec, StreamWriterCommitProgress} import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport import org.apache.spark.sql.streaming._ import org.apache.spark.sql.streaming.StreamingQueryListener.QueryProgressEvent @@ -56,6 +56,7 @@ trait ProgressReporter extends Logging { protected def logicalPlan: LogicalPlan protected def lastExecution: QueryExecution protected def newData: Map[BaseStreamingSource, LogicalPlan] + protected def sinkCommitProgress: Option[StreamWriterCommitProgress] protected def sources: Seq[BaseStreamingSource] protected def sink: BaseStreamingSink protected def offsetSeqMetadata: OffsetSeqMetadata @@ -167,7 +168,9 @@ trait ProgressReporter extends Logging { ) } - val sinkProgress = new SinkProgress(sink.toString) + val sinkProgress = SinkProgress( + sink.toString, + sinkCommitProgress.map(_.numOutputRows)) val newProgress = new StreamingQueryProgress( id = id, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 89b4f40c9c0b..83824f40ab90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.command.StreamingExplainCommand +import org.apache.spark.sql.execution.datasources.v2.StreamWriterCommitProgress import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} @@ -114,6 +115,9 @@ abstract class StreamExecution( @volatile var availableOffsets = new StreamProgress + @volatile + var sinkCommitProgress: Option[StreamWriterCommitProgress] = None + /** The current batchId or -1 if execution has not yet been initialized. */ protected var currentBatchId: Long = -1 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 3cd6700efef5..0b3945cbd132 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 @@ -30,6 +30,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.streaming.SinkProgress.DEFAULT_NUM_OUTPUT_ROWS /** * Information about updates made to stateful operators in a [[StreamingQuery]] during a trigger. @@ -207,11 +208,19 @@ class SourceProgress protected[sql]( * during a trigger. See [[StreamingQueryProgress]] for more information. * * @param description Description of the source corresponding to this status. + * @param numOutputRows Number of rows written to the sink or -1 for Continuous Mode (temporarily) + * or Sink V1 (until decommissioned). * @since 2.1.0 */ @Evolving class SinkProgress protected[sql]( - val description: String) extends Serializable { + val description: String, + val numOutputRows: Long) extends Serializable { + + /** SinkProgress without custom metrics. */ + protected[sql] def this(description: String) { + this(description, DEFAULT_NUM_OUTPUT_ROWS) + } /** The compact JSON representation of this progress. */ def json: String = compact(render(jsonValue)) @@ -222,6 +231,14 @@ class SinkProgress protected[sql]( override def toString: String = prettyJson private[sql] def jsonValue: JValue = { - ("description" -> JString(description)) + ("description" -> JString(description)) ~ + ("numOutputRows" -> JInt(numOutputRows)) } } + +private[sql] object SinkProgress { + val DEFAULT_NUM_OUTPUT_ROWS: Long = -1L + + def apply(description: String, numOutputRows: Option[Long]): SinkProgress = + new SinkProgress(description, numOutputRows.getOrElse(DEFAULT_NUM_OUTPUT_ROWS)) +} 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 7bef687e7e43..2f460b044b23 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 @@ -73,7 +73,8 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "inputRowsPerSecond" : 10.0 | } ], | "sink" : { - | "description" : "sink" + | "description" : "sink", + | "numOutputRows" : -1 | } |} """.stripMargin.trim) @@ -105,7 +106,8 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "numInputRows" : 678 | } ], | "sink" : { - | "description" : "sink" + | "description" : "sink", + | "numOutputRows" : -1 | } |} """.stripMargin.trim) @@ -250,7 +252,7 @@ object StreamingQueryStatusAndProgressSuite { processedRowsPerSecond = Double.PositiveInfinity // should not be present in the json ) ), - sink = new SinkProgress("sink") + sink = SinkProgress("sink", None) ) val testProgress2 = new StreamingQueryProgress( @@ -274,7 +276,7 @@ object StreamingQueryStatusAndProgressSuite { processedRowsPerSecond = Double.NegativeInfinity // should not be present in the json ) ), - sink = new SinkProgress("sink") + sink = SinkProgress("sink", None) ) val testStatus = new StreamingQueryStatus("active", true, false) From 114d0de14c441f06d98ab1bcf6c8375c58ecd9ab Mon Sep 17 00:00:00 2001 From: suxingfate Date: Mon, 17 Dec 2018 13:36:57 -0800 Subject: [PATCH 0155/1072] [SPARK-25922][K8] Spark Driver/Executor "spark-app-selector" label mismatch ## What changes were proposed in this pull request? In K8S Cluster mode, the algorithm to generate spark-app-selector/spark.app.id of spark driver is different with spark executor. This patch makes sure spark driver and executor to use the same spark-app-selector/spark.app.id if spark.app.id is set, otherwise it will use superclass applicationId. In K8S Client mode, spark-app-selector/spark.app.id for executors will use superclass applicationId. ## How was this patch tested? Manually run." Closes #23322 from suxingfate/SPARK-25922. Lead-authored-by: suxingfate Co-authored-by: xinglwang Signed-off-by: Yinan Li --- .../KubernetesClusterSchedulerBackend.scala | 28 ++++++++++++++----- ...bernetesClusterSchedulerBackendSuite.scala | 14 +++++----- 2 files changed, 28 insertions(+), 14 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 68f6f2e46e31..03f5da2bb0bc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -18,9 +18,10 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.ExecutorService -import io.fabric8.kubernetes.client.KubernetesClient import scala.concurrent.{ExecutionContext, Future} +import io.fabric8.kubernetes.client.KubernetesClient + import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -39,10 +40,10 @@ private[spark] class KubernetesClusterSchedulerBackend( lifecycleEventHandler: ExecutorPodsLifecycleManager, watchEvents: ExecutorPodsWatchSnapshotSource, pollEvents: ExecutorPodsPollingSnapshotSource) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { - private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( - requestExecutorsService) + private implicit val requestExecutorContext = + ExecutionContext.fromExecutorService(requestExecutorsService) protected override val minRegisteredRatio = if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { @@ -60,6 +61,17 @@ private[spark] class KubernetesClusterSchedulerBackend( removeExecutor(executorId, reason) } + /** + * Get an application ID associated with the job. + * This returns the string value of spark.app.id if set, otherwise + * the locally-generated ID from the superclass. + * + * @return The application ID + */ + override def applicationId(): String = { + conf.getOption("spark.app.id").map(_.toString).getOrElse(super.applicationId) + } + override def start(): Unit = { super.start() if (!Utils.isDynamicAllocationEnabled(conf)) { @@ -88,7 +100,8 @@ private[spark] class KubernetesClusterSchedulerBackend( if (shouldDeleteExecutors) { Utils.tryLogNonFatalError { - kubernetesClient.pods() + kubernetesClient + .pods() .withLabel(SPARK_APP_ID_LABEL, applicationId()) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .delete() @@ -120,7 +133,8 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { - kubernetesClient.pods() + kubernetesClient + .pods() .withLabel(SPARK_APP_ID_LABEL, applicationId()) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .withLabelIn(SPARK_EXECUTOR_ID_LABEL, executorIds: _*) @@ -133,7 +147,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } private class KubernetesDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) - extends DriverEndpoint(rpcEnv, sparkProperties) { + extends DriverEndpoint(rpcEnv, sparkProperties) { override def onDisconnected(rpcAddress: RpcAddress): Unit = { // Don't do anything besides disabling the executor - allow the Kubernetes API events to diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 75232f7b98b0..6e182bed459f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -37,6 +37,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private val requestExecutorsService = new DeterministicScheduler() private val sparkConf = new SparkConf(false) .set("spark.executor.instances", "3") + .set("spark.app.id", TEST_SPARK_APP_ID) @Mock private var sc: SparkContext = _ @@ -87,8 +88,10 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn when(sc.env).thenReturn(env) when(env.rpcEnv).thenReturn(rpcEnv) driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) - when(rpcEnv.setupEndpoint( - mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) + when( + rpcEnv.setupEndpoint( + mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), + driverEndpoint.capture())) .thenReturn(driverEndpointRef) when(kubernetesClient.pods()).thenReturn(podOperations) schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( @@ -100,9 +103,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn podAllocator, lifecycleEventHandler, watchEvents, - pollEvents) { - override def applicationId(): String = TEST_SPARK_APP_ID - } + pollEvents) } test("Start all components") { @@ -127,8 +128,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn test("Remove executor") { schedulerBackendUnderTest.start() - schedulerBackendUnderTest.doRemoveExecutor( - "1", ExecutorKilled) + schedulerBackendUnderTest.doRemoveExecutor("1", ExecutorKilled) verify(driverEndpointRef).send(RemoveExecutor("1", ExecutorKilled)) } From 86100df54ba8413bebd6ca243b55a6007bc7a2de Mon Sep 17 00:00:00 2001 From: Li Jin Date: Tue, 18 Dec 2018 09:15:21 +0800 Subject: [PATCH 0156/1072] [SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions.py | 21 +- .../sql/tests/test_pandas_udf_window.py | 157 ++++++++- python/pyspark/worker.py | 57 ++- .../sql/catalyst/analysis/CheckAnalysis.scala | 5 - .../execution/python/WindowInPandasExec.scala | 329 +++++++++++++++--- .../sql/execution/window/WindowExec.scala | 189 +--------- .../sql/execution/window/WindowExecBase.scala | 230 ++++++++++++ .../window/WindowFunctionFrame.scala | 108 ++++-- 8 files changed, 792 insertions(+), 304 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index f98e550e39da..d188de39e21c 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2982,8 +2982,7 @@ def pandas_udf(f=None, returnType=None, functionType=None): | 2| 6.0| +---+-----------+ - This example shows using grouped aggregated UDFs as window functions. Note that only - unbounded window frame is supported at the moment: + This example shows using grouped aggregated UDFs as window functions. >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> from pyspark.sql import Window @@ -2993,20 +2992,24 @@ def pandas_udf(f=None, returnType=None, functionType=None): >>> @pandas_udf("double", PandasUDFType.GROUPED_AGG) # doctest: +SKIP ... def mean_udf(v): ... return v.mean() - >>> w = Window \\ - ... .partitionBy('id') \\ - ... .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing) + >>> w = (Window.partitionBy('id') + ... .orderBy('v') + ... .rowsBetween(-1, 0)) >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show() # doctest: +SKIP +---+----+------+ | id| v|mean_v| +---+----+------+ - | 1| 1.0| 1.5| + | 1| 1.0| 1.0| | 1| 2.0| 1.5| - | 2| 3.0| 6.0| - | 2| 5.0| 6.0| - | 2|10.0| 6.0| + | 2| 3.0| 3.0| + | 2| 5.0| 4.0| + | 2|10.0| 7.5| +---+----+------+ + .. note:: For performance reasons, the input series to window functions are not copied. + Therefore, mutating the input series is not allowed and will cause incorrect results. + For the same reason, users should also not rely on the index of the input series. + .. seealso:: :meth:`pyspark.sql.GroupedData.agg` and :class:`pyspark.sql.Window` .. note:: The user-defined functions are considered deterministic by default. Due to diff --git a/python/pyspark/sql/tests/test_pandas_udf_window.py b/python/pyspark/sql/tests/test_pandas_udf_window.py index 0a7a19c1c081..3ba98e76468b 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_window.py +++ b/python/pyspark/sql/tests/test_pandas_udf_window.py @@ -46,6 +46,15 @@ def python_plus_one(self): def pandas_scalar_time_two(self): return pandas_udf(lambda v: v * 2, 'double') + @property + def pandas_agg_count_udf(self): + from pyspark.sql.functions import pandas_udf, PandasUDFType + + @pandas_udf('long', PandasUDFType.GROUPED_AGG) + def count(v): + return len(v) + return count + @property def pandas_agg_mean_udf(self): @pandas_udf('double', PandasUDFType.GROUPED_AGG) @@ -70,7 +79,7 @@ def min(v): @property def unbounded_window(self): return Window.partitionBy('id') \ - .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing) + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing).orderBy('v') @property def ordered_window(self): @@ -80,6 +89,32 @@ def ordered_window(self): def unpartitioned_window(self): return Window.partitionBy() + @property + def sliding_row_window(self): + return Window.partitionBy('id').orderBy('v').rowsBetween(-2, 1) + + @property + def sliding_range_window(self): + return Window.partitionBy('id').orderBy('v').rangeBetween(-2, 4) + + @property + def growing_row_window(self): + return Window.partitionBy('id').orderBy('v').rowsBetween(Window.unboundedPreceding, 3) + + @property + def growing_range_window(self): + return Window.partitionBy('id').orderBy('v') \ + .rangeBetween(Window.unboundedPreceding, 4) + + @property + def shrinking_row_window(self): + return Window.partitionBy('id').orderBy('v').rowsBetween(-2, Window.unboundedFollowing) + + @property + def shrinking_range_window(self): + return Window.partitionBy('id').orderBy('v') \ + .rangeBetween(-3, Window.unboundedFollowing) + def test_simple(self): df = self.data w = self.unbounded_window @@ -100,12 +135,12 @@ def test_multiple_udfs(self): w = self.unbounded_window result1 = df.withColumn('mean_v', self.pandas_agg_mean_udf(df['v']).over(w)) \ - .withColumn('max_v', self.pandas_agg_max_udf(df['v']).over(w)) \ - .withColumn('min_w', self.pandas_agg_min_udf(df['w']).over(w)) + .withColumn('max_v', self.pandas_agg_max_udf(df['v']).over(w)) \ + .withColumn('min_w', self.pandas_agg_min_udf(df['w']).over(w)) expected1 = df.withColumn('mean_v', mean(df['v']).over(w)) \ - .withColumn('max_v', max(df['v']).over(w)) \ - .withColumn('min_w', min(df['w']).over(w)) + .withColumn('max_v', max(df['v']).over(w)) \ + .withColumn('min_w', min(df['w']).over(w)) self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) @@ -183,16 +218,16 @@ def test_mixed_sql_and_udf(self): # Test chaining sql aggregate function and udf result3 = df.withColumn('max_v', max_udf(df['v']).over(w)) \ - .withColumn('min_v', min(df['v']).over(w)) \ - .withColumn('v_diff', col('max_v') - col('min_v')) \ - .drop('max_v', 'min_v') + .withColumn('min_v', min(df['v']).over(w)) \ + .withColumn('v_diff', col('max_v') - col('min_v')) \ + .drop('max_v', 'min_v') expected3 = expected1 # Test mixing sql window function and udf result4 = df.withColumn('max_v', max_udf(df['v']).over(w)) \ - .withColumn('rank', rank().over(ow)) + .withColumn('rank', rank().over(ow)) expected4 = df.withColumn('max_v', max(df['v']).over(w)) \ - .withColumn('rank', rank().over(ow)) + .withColumn('rank', rank().over(ow)) self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) self.assertPandasEqual(expected2.toPandas(), result2.toPandas()) @@ -210,8 +245,6 @@ def test_array_type(self): def test_invalid_args(self): df = self.data w = self.unbounded_window - ow = self.ordered_window - mean_udf = self.pandas_agg_mean_udf with QuietTest(self.sc): with self.assertRaisesRegexp( @@ -220,11 +253,101 @@ def test_invalid_args(self): foo_udf = pandas_udf(lambda x: x, 'v double', PandasUDFType.GROUPED_MAP) df.withColumn('v2', foo_udf(df['v']).over(w)) - with QuietTest(self.sc): - with self.assertRaisesRegexp( - AnalysisException, - '.*Only unbounded window frame is supported.*'): - df.withColumn('mean_v', mean_udf(df['v']).over(ow)) + def test_bounded_simple(self): + from pyspark.sql.functions import mean, max, min, count + + df = self.data + w1 = self.sliding_row_window + w2 = self.shrinking_range_window + + plus_one = self.python_plus_one + count_udf = self.pandas_agg_count_udf + mean_udf = self.pandas_agg_mean_udf + max_udf = self.pandas_agg_max_udf + min_udf = self.pandas_agg_min_udf + + result1 = df.withColumn('mean_v', mean_udf(plus_one(df['v'])).over(w1)) \ + .withColumn('count_v', count_udf(df['v']).over(w2)) \ + .withColumn('max_v', max_udf(df['v']).over(w2)) \ + .withColumn('min_v', min_udf(df['v']).over(w1)) + + expected1 = df.withColumn('mean_v', mean(plus_one(df['v'])).over(w1)) \ + .withColumn('count_v', count(df['v']).over(w2)) \ + .withColumn('max_v', max(df['v']).over(w2)) \ + .withColumn('min_v', min(df['v']).over(w1)) + + self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) + + def test_growing_window(self): + from pyspark.sql.functions import mean + + df = self.data + w1 = self.growing_row_window + w2 = self.growing_range_window + + mean_udf = self.pandas_agg_mean_udf + + result1 = df.withColumn('m1', mean_udf(df['v']).over(w1)) \ + .withColumn('m2', mean_udf(df['v']).over(w2)) + + expected1 = df.withColumn('m1', mean(df['v']).over(w1)) \ + .withColumn('m2', mean(df['v']).over(w2)) + + self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) + + def test_sliding_window(self): + from pyspark.sql.functions import mean + + df = self.data + w1 = self.sliding_row_window + w2 = self.sliding_range_window + + mean_udf = self.pandas_agg_mean_udf + + result1 = df.withColumn('m1', mean_udf(df['v']).over(w1)) \ + .withColumn('m2', mean_udf(df['v']).over(w2)) + + expected1 = df.withColumn('m1', mean(df['v']).over(w1)) \ + .withColumn('m2', mean(df['v']).over(w2)) + + self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) + + def test_shrinking_window(self): + from pyspark.sql.functions import mean + + df = self.data + w1 = self.shrinking_row_window + w2 = self.shrinking_range_window + + mean_udf = self.pandas_agg_mean_udf + + result1 = df.withColumn('m1', mean_udf(df['v']).over(w1)) \ + .withColumn('m2', mean_udf(df['v']).over(w2)) + + expected1 = df.withColumn('m1', mean(df['v']).over(w1)) \ + .withColumn('m2', mean(df['v']).over(w2)) + + self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) + + def test_bounded_mixed(self): + from pyspark.sql.functions import mean, max + + df = self.data + w1 = self.sliding_row_window + w2 = self.unbounded_window + + mean_udf = self.pandas_agg_mean_udf + max_udf = self.pandas_agg_max_udf + + result1 = df.withColumn('mean_v', mean_udf(df['v']).over(w1)) \ + .withColumn('max_v', max_udf(df['v']).over(w2)) \ + .withColumn('mean_unbounded_v', mean_udf(df['v']).over(w1)) + + expected1 = df.withColumn('mean_v', mean(df['v']).over(w1)) \ + .withColumn('max_v', max(df['v']).over(w2)) \ + .withColumn('mean_unbounded_v', mean(df['v']).over(w1)) + + self.assertPandasEqual(expected1.toPandas(), result1.toPandas()) if __name__ == "__main__": diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 953b468e9651..bf007b0c62d8 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -145,7 +145,18 @@ def wrapped(*series): return lambda *a: (wrapped(*a), arrow_return_type) -def wrap_window_agg_pandas_udf(f, return_type): +def wrap_window_agg_pandas_udf(f, return_type, runner_conf, udf_index): + window_bound_types_str = runner_conf.get('pandas_window_bound_types') + window_bound_type = [t.strip().lower() for t in window_bound_types_str.split(',')][udf_index] + if window_bound_type == 'bounded': + return wrap_bounded_window_agg_pandas_udf(f, return_type) + elif window_bound_type == 'unbounded': + return wrap_unbounded_window_agg_pandas_udf(f, return_type) + else: + raise RuntimeError("Invalid window bound type: {} ".format(window_bound_type)) + + +def wrap_unbounded_window_agg_pandas_udf(f, return_type): # This is similar to grouped_agg_pandas_udf, the only difference # is that window_agg_pandas_udf needs to repeat the return value # to match window length, where grouped_agg_pandas_udf just returns @@ -160,7 +171,41 @@ def wrapped(*series): return lambda *a: (wrapped(*a), arrow_return_type) -def read_single_udf(pickleSer, infile, eval_type, runner_conf): +def wrap_bounded_window_agg_pandas_udf(f, return_type): + arrow_return_type = to_arrow_type(return_type) + + def wrapped(begin_index, end_index, *series): + import pandas as pd + result = [] + + # Index operation is faster on np.ndarray, + # So we turn the index series into np array + # here for performance + begin_array = begin_index.values + end_array = end_index.values + + for i in range(len(begin_array)): + # Note: Create a slice from a series for each window is + # actually pretty expensive. However, there + # is no easy way to reduce cost here. + # Note: s.iloc[i : j] is about 30% faster than s[i: j], with + # the caveat that the created slices shares the same + # memory with s. Therefore, user are not allowed to + # change the value of input series inside the window + # function. It is rare that user needs to modify the + # input series in the window function, and therefore, + # it is be a reasonable restriction. + # Note: Calling reset_index on the slices will increase the cost + # of creating slices by about 100%. Therefore, for performance + # reasons we don't do it here. + series_slices = [s.iloc[begin_array[i]: end_array[i]] for s in series] + result.append(f(*series_slices)) + return pd.Series(result) + + return lambda *a: (wrapped(*a), arrow_return_type) + + +def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index): num_arg = read_int(infile) arg_offsets = [read_int(infile) for i in range(num_arg)] row_func = None @@ -184,7 +229,7 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf): elif eval_type == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: return arg_offsets, wrap_grouped_agg_pandas_udf(func, return_type) elif eval_type == PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF: - return arg_offsets, wrap_window_agg_pandas_udf(func, return_type) + return arg_offsets, wrap_window_agg_pandas_udf(func, return_type, runner_conf, udf_index) elif eval_type == PythonEvalType.SQL_BATCHED_UDF: return arg_offsets, wrap_udf(func, return_type) else: @@ -226,7 +271,8 @@ def read_udfs(pickleSer, infile, eval_type): # See FlatMapGroupsInPandasExec for how arg_offsets are used to # distinguish between grouping attributes and data attributes - arg_offsets, udf = read_single_udf(pickleSer, infile, eval_type, runner_conf) + arg_offsets, udf = read_single_udf( + pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf split_offset = arg_offsets[0] + 1 arg0 = ["a[%d]" % o for o in arg_offsets[1: split_offset]] @@ -238,7 +284,8 @@ def read_udfs(pickleSer, infile, eval_type): # In the special case of a single UDF this will return a single result rather # than a tuple of results; this is the format that the JVM side expects. for i in range(num_udfs): - arg_offsets, udf = read_single_udf(pickleSer, infile, eval_type, runner_conf) + arg_offsets, udf = read_single_udf( + pickleSer, infile, eval_type, runner_conf, udf_index=i) udfs['f%d' % i] = udf args = ["a[%d]" % o for o in arg_offsets] call_udf.append("f%d(%s)" % (i, ", ".join(args))) 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 6a91d556b2f3..88d41e882440 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 @@ -134,11 +134,6 @@ trait CheckAnalysis extends PredicateHelper { failAnalysis("An offset window function can only be evaluated in an ordered " + s"row-based window frame with a single offset: $w") - case _ @ WindowExpression(_: PythonUDF, - WindowSpecDefinition(_, _, frame: SpecifiedWindowFrame)) - if !frame.isUnbounded => - failAnalysis("Only unbounded window frame is supported with Pandas UDFs.") - case w @ WindowExpression(e, s) => // Only allow window functions with an aggregate expression or an offset window // function or a Pandas window UDF. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala index 82973307feef..1ce1215bfdd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala @@ -27,17 +27,64 @@ import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} 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.plans.physical._ -import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.execution.{ExternalAppendOnlyUnsafeRowArray, SparkPlan} import org.apache.spark.sql.execution.arrow.ArrowUtils -import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.execution.window._ +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils +/** + * This class calculates and outputs windowed aggregates over the rows in a single partition. + * + * This is similar to [[WindowExec]]. The main difference is that this node does not compute + * any window aggregation values. Instead, it computes the lower and upper bound for each window + * (i.e. window bounds) and pass the data and indices to Python worker to do the actual window + * aggregation. + * + * It currently materializes all data associated with the same partition key and passes them to + * Python worker. This is not strictly necessary for sliding windows and can be improved (by + * possibly slicing data into overlapping chunks and stitching them together). + * + * This class groups window expressions by their window boundaries so that window expressions + * with the same window boundaries can share the same window bounds. The window bounds are + * prepended to the data passed to the python worker. + * + * For example, if we have: + * avg(v) over specifiedwindowframe(RowFrame, -5, 5), + * avg(v) over specifiedwindowframe(RowFrame, UnboundedPreceding, UnboundedFollowing), + * avg(v) over specifiedwindowframe(RowFrame, -3, 3), + * max(v) over specifiedwindowframe(RowFrame, -3, 3) + * + * The python input will look like: + * (lower_bound_w1, upper_bound_w1, lower_bound_w3, upper_bound_w3, v) + * + * where w1 is specifiedwindowframe(RowFrame, -5, 5) + * w2 is specifiedwindowframe(RowFrame, UnboundedPreceding, UnboundedFollowing) + * w3 is specifiedwindowframe(RowFrame, -3, 3) + * + * Note that w2 doesn't have bound indices in the python input because it's unbounded window + * so it's bound indices will always be the same. + * + * Bounded window and Unbounded window are evaluated differently in Python worker: + * (1) Bounded window takes the window bound indices in addition to the input columns. + * Unbounded window takes only input columns. + * (2) Bounded window evaluates the udf once per input row. + * Unbounded window evaluates the udf once per window partition. + * This is controlled by Python runner conf "pandas_window_bound_types" + * + * The logic to compute window bounds is delegated to [[WindowFunctionFrame]] and shared with + * [[WindowExec]] + * + * Note this doesn't support partial aggregation and all aggregation is computed from the entire + * window. + */ case class WindowInPandasExec( windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], - child: SparkPlan) extends UnaryExecNode { + child: SparkPlan) + extends WindowExecBase(windowExpression, partitionSpec, orderSpec, child) { override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) @@ -60,6 +107,26 @@ case class WindowInPandasExec( override def outputPartitioning: Partitioning = child.outputPartitioning + /** + * Helper functions and data structures for window bounds + * + * It contains: + * (1) Total number of window bound indices in the python input row + * (2) Function from frame index to its lower bound column index in the python input row + * (3) Function from frame index to its upper bound column index in the python input row + * (4) Seq from frame index to its window bound type + */ + private type WindowBoundHelpers = (Int, Int => Int, Int => Int, Seq[WindowBoundType]) + + /** + * Enum for window bound types. Used only inside this class. + */ + private sealed case class WindowBoundType(value: String) + private object UnboundedWindow extends WindowBoundType("unbounded") + private object BoundedWindow extends WindowBoundType("bounded") + + private val windowBoundTypeConf = "pandas_window_bound_types" + private def collectFunctions(udf: PythonUDF): (ChainedPythonFunctions, Seq[Expression]) = { udf.children match { case Seq(u: PythonUDF) => @@ -73,68 +140,150 @@ case class WindowInPandasExec( } /** - * Create the resulting projection. - * - * This method uses Code Generation. It can only be used on the executor side. - * - * @param expressions unbound ordered function expressions. - * @return the final resulting projection. + * See [[WindowBoundHelpers]] for details. */ - private[this] def createResultProjection(expressions: Seq[Expression]): UnsafeProjection = { - val references = expressions.zipWithIndex.map { case (e, i) => - // Results of window expressions will be on the right side of child's output - BoundReference(child.output.size + i, e.dataType, e.nullable) + private def computeWindowBoundHelpers( + factories: Seq[InternalRow => WindowFunctionFrame] + ): WindowBoundHelpers = { + val functionFrames = factories.map(_(EmptyRow)) + + val windowBoundTypes = functionFrames.map { + case _: UnboundedWindowFunctionFrame => UnboundedWindow + case _: UnboundedFollowingWindowFunctionFrame | + _: SlidingWindowFunctionFrame | + _: UnboundedPrecedingWindowFunctionFrame => BoundedWindow + // It should be impossible to get other types of window function frame here + case frame => throw new RuntimeException(s"Unexpected window function frame $frame.") } - val unboundToRefMap = expressions.zip(references).toMap - val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap)) - UnsafeProjection.create( - child.output ++ patchedWindowExpression, - child.output) + + val requiredIndices = functionFrames.map { + case _: UnboundedWindowFunctionFrame => 0 + case _ => 2 + } + + val upperBoundIndices = requiredIndices.scan(0)(_ + _).tail + + val boundIndices = requiredIndices.zip(upperBoundIndices).map { case (num, upperBoundIndex) => + if (num == 0) { + // Sentinel values for unbounded window + (-1, -1) + } else { + (upperBoundIndex - 2, upperBoundIndex - 1) + } + } + + def lowerBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._1 + def upperBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._2 + + (requiredIndices.sum, lowerBoundIndex, upperBoundIndex, windowBoundTypes) } protected override def doExecute(): RDD[InternalRow] = { - val inputRDD = child.execute() + // Unwrap the expressions and factories from the map. + val expressionsWithFrameIndex = + windowFrameExpressionFactoryPairs.map(_._1).zipWithIndex.flatMap { + case (buffer, frameIndex) => buffer.map(expr => (expr, frameIndex)) + } + + val expressions = expressionsWithFrameIndex.map(_._1) + val expressionIndexToFrameIndex = + expressionsWithFrameIndex.map(_._2).zipWithIndex.map(_.swap).toMap + + val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray + // Helper functions + val (numBoundIndices, lowerBoundIndex, upperBoundIndex, frameWindowBoundTypes) = + computeWindowBoundHelpers(factories) + val isBounded = { frameIndex: Int => lowerBoundIndex(frameIndex) >= 0 } + val numFrames = factories.length + + val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold + val spillThreshold = conf.windowExecBufferSpillThreshold val sessionLocalTimeZone = conf.sessionLocalTimeZone - val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) // Extract window expressions and window functions - val expressions = windowExpression.flatMap(_.collect { case e: WindowExpression => e }) - - val udfExpressions = expressions.map(_.windowFunction.asInstanceOf[PythonUDF]) + val windowExpressions = expressions.flatMap(_.collect { case e: WindowExpression => e }) + val udfExpressions = windowExpressions.map(_.windowFunction.asInstanceOf[PythonUDF]) + // We shouldn't be chaining anything here. + // All chained python functions should only contain one function. val (pyFuncs, inputs) = udfExpressions.map(collectFunctions).unzip + require(pyFuncs.length == expressions.length) + + val udfWindowBoundTypes = pyFuncs.indices.map(i => + frameWindowBoundTypes(expressionIndexToFrameIndex(i))) + val pythonRunnerConf: Map[String, String] = (ArrowUtils.getPythonRunnerConfMap(conf) + + (windowBoundTypeConf -> udfWindowBoundTypes.map(_.value).mkString(","))) // Filter child output attributes down to only those that are UDF inputs. - // Also eliminate duplicate UDF inputs. - val allInputs = new ArrayBuffer[Expression] - val dataTypes = new ArrayBuffer[DataType] + // Also eliminate duplicate UDF inputs. This is similar to how other Python UDF node + // handles UDF inputs. + val dataInputs = new ArrayBuffer[Expression] + val dataInputTypes = new ArrayBuffer[DataType] val argOffsets = inputs.map { input => input.map { e => - if (allInputs.exists(_.semanticEquals(e))) { - allInputs.indexWhere(_.semanticEquals(e)) + if (dataInputs.exists(_.semanticEquals(e))) { + dataInputs.indexWhere(_.semanticEquals(e)) } else { - allInputs += e - dataTypes += e.dataType - allInputs.length - 1 + dataInputs += e + dataInputTypes += e.dataType + dataInputs.length - 1 } }.toArray }.toArray - // Schema of input rows to the python runner - val windowInputSchema = StructType(dataTypes.zipWithIndex.map { case (dt, i) => - StructField(s"_$i", dt) - }) + // In addition to UDF inputs, we will prepend window bounds for each UDFs. + // For bounded windows, we prepend lower bound and upper bound. For unbounded windows, + // we no not add window bounds. (strictly speaking, we only need to lower or upper bound + // if the window is bounded only on one side, this can be improved in the future) - inputRDD.mapPartitionsInternal { iter => - val context = TaskContext.get() + // Setting window bounds for each window frames. Each window frame has different bounds so + // each has its own window bound columns. + val windowBoundsInput = factories.indices.flatMap { frameIndex => + if (isBounded(frameIndex)) { + Seq( + BoundReference(lowerBoundIndex(frameIndex), IntegerType, nullable = false), + BoundReference(upperBoundIndex(frameIndex), IntegerType, nullable = false) + ) + } else { + Seq.empty + } + } - val grouped = if (partitionSpec.isEmpty) { - // Use an empty unsafe row as a place holder for the grouping key - Iterator((new UnsafeRow(), iter)) + // Setting the window bounds argOffset for each UDF. For UDFs with bounded window, argOffset + // for the UDF is (lowerBoundOffet, upperBoundOffset, inputOffset1, inputOffset2, ...) + // For UDFs with unbounded window, argOffset is (inputOffset1, inputOffset2, ...) + pyFuncs.indices.foreach { exprIndex => + val frameIndex = expressionIndexToFrameIndex(exprIndex) + if (isBounded(frameIndex)) { + argOffsets(exprIndex) = + Array(lowerBoundIndex(frameIndex), upperBoundIndex(frameIndex)) ++ + argOffsets(exprIndex).map(_ + windowBoundsInput.length) } else { - GroupedIterator(iter, partitionSpec, child.output) + argOffsets(exprIndex) = argOffsets(exprIndex).map(_ + windowBoundsInput.length) } + } + + val allInputs = windowBoundsInput ++ dataInputs + val allInputTypes = allInputs.map(_.dataType) + + // Start processing. + child.execute().mapPartitions { iter => + val context = TaskContext.get() + + // Get all relevant projections. + val resultProj = createResultProjection(expressions) + val pythonInputProj = UnsafeProjection.create( + allInputs, + windowBoundsInput.map(ref => + AttributeReference(s"i_${ref.ordinal}", ref.dataType)()) ++ child.output + ) + val pythonInputSchema = StructType( + allInputTypes.zipWithIndex.map { case (dt, i) => + StructField(s"_$i", dt) + } + ) + val grouping = UnsafeProjection.create(partitionSpec, child.output) // The queue used to buffer input rows so we can drain it to // combine input with output from Python. @@ -144,11 +293,94 @@ case class WindowInPandasExec( queue.close() } - val inputProj = UnsafeProjection.create(allInputs, child.output) - val pythonInput = grouped.map { case (_, rows) => - rows.map { row => - queue.add(row.asInstanceOf[UnsafeRow]) - inputProj(row) + val stream = iter.map { row => + queue.add(row.asInstanceOf[UnsafeRow]) + row + } + + val pythonInput = new Iterator[Iterator[UnsafeRow]] { + + // Manage the stream and the grouping. + var nextRow: UnsafeRow = null + var nextGroup: UnsafeRow = null + var nextRowAvailable: Boolean = false + private[this] def fetchNextRow() { + nextRowAvailable = stream.hasNext + if (nextRowAvailable) { + nextRow = stream.next().asInstanceOf[UnsafeRow] + nextGroup = grouping(nextRow) + } else { + nextRow = null + nextGroup = null + } + } + fetchNextRow() + + // Manage the current partition. + val buffer: ExternalAppendOnlyUnsafeRowArray = + new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold) + var bufferIterator: Iterator[UnsafeRow] = _ + + val indexRow = new SpecificInternalRow(Array.fill(numBoundIndices)(IntegerType)) + + val frames = factories.map(_(indexRow)) + + private[this] def fetchNextPartition() { + // Collect all the rows in the current partition. + // Before we start to fetch new input rows, make a copy of nextGroup. + val currentGroup = nextGroup.copy() + + // clear last partition + buffer.clear() + + while (nextRowAvailable && nextGroup == currentGroup) { + buffer.add(nextRow) + fetchNextRow() + } + + // Setup the frames. + var i = 0 + while (i < numFrames) { + frames(i).prepare(buffer) + i += 1 + } + + // Setup iteration + rowIndex = 0 + bufferIterator = buffer.generateIterator() + } + + // Iteration + var rowIndex = 0 + + override final def hasNext: Boolean = + (bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable + + override final def next(): Iterator[UnsafeRow] = { + // Load the next partition if we need to. + if ((bufferIterator == null || !bufferIterator.hasNext) && nextRowAvailable) { + fetchNextPartition() + } + + val join = new JoinedRow + + bufferIterator.zipWithIndex.map { + case (current, index) => + var frameIndex = 0 + while (frameIndex < numFrames) { + frames(frameIndex).write(index, current) + // If the window is unbounded we don't need to write out window bounds. + if (isBounded(frameIndex)) { + indexRow.setInt( + lowerBoundIndex(frameIndex), frames(frameIndex).currentLowerBound()) + indexRow.setInt( + upperBoundIndex(frameIndex), frames(frameIndex).currentUpperBound()) + } + frameIndex += 1 + } + + pythonInputProj(join(indexRow, current)) + } } } @@ -156,12 +388,11 @@ case class WindowInPandasExec( pyFuncs, PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF, argOffsets, - windowInputSchema, + pythonInputSchema, sessionLocalTimeZone, pythonRunnerConf).compute(pythonInput, context.partitionId(), context) val joined = new JoinedRow - val resultProj = createResultProjection(expressions) windowFunctionResult.flatMap(_.rowIterator.asScala).map { windowOutput => val leftRow = queue.remove() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala index 729b8bdb3dae..89f6edda2ef5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala @@ -83,7 +83,7 @@ case class WindowExec( partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], child: SparkPlan) - extends UnaryExecNode { + extends WindowExecBase(windowExpression, partitionSpec, orderSpec, child) { override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) @@ -104,193 +104,6 @@ case class WindowExec( override def outputPartitioning: Partitioning = child.outputPartitioning - /** - * Create a bound ordering object for a given frame type and offset. A bound ordering object is - * used to determine which input row lies within the frame boundaries of an output row. - * - * This method uses Code Generation. It can only be used on the executor side. - * - * @param frame to evaluate. This can either be a Row or Range frame. - * @param bound with respect to the row. - * @param timeZone the session local timezone for time related calculations. - * @return a bound ordering object. - */ - private[this] def createBoundOrdering( - frame: FrameType, bound: Expression, timeZone: String): BoundOrdering = { - (frame, bound) match { - case (RowFrame, CurrentRow) => - RowBoundOrdering(0) - - case (RowFrame, IntegerLiteral(offset)) => - RowBoundOrdering(offset) - - case (RangeFrame, CurrentRow) => - val ordering = newOrdering(orderSpec, child.output) - RangeBoundOrdering(ordering, IdentityProjection, IdentityProjection) - - case (RangeFrame, offset: Expression) if orderSpec.size == 1 => - // Use only the first order expression when the offset is non-null. - val sortExpr = orderSpec.head - val expr = sortExpr.child - - // Create the projection which returns the current 'value'. - val current = newMutableProjection(expr :: Nil, child.output) - - // Flip the sign of the offset when processing the order is descending - val boundOffset = sortExpr.direction match { - case Descending => UnaryMinus(offset) - case Ascending => offset - } - - // Create the projection which returns the current 'value' modified by adding the offset. - val boundExpr = (expr.dataType, boundOffset.dataType) match { - case (DateType, IntegerType) => DateAdd(expr, boundOffset) - case (TimestampType, CalendarIntervalType) => - TimeAdd(expr, boundOffset, Some(timeZone)) - case (a, b) if a== b => Add(expr, boundOffset) - } - val bound = newMutableProjection(boundExpr :: Nil, child.output) - - // Construct the ordering. This is used to compare the result of current value projection - // to the result of bound value projection. This is done manually because we want to use - // Code Generation (if it is enabled). - val boundSortExprs = sortExpr.copy(BoundReference(0, expr.dataType, expr.nullable)) :: Nil - val ordering = newOrdering(boundSortExprs, Nil) - RangeBoundOrdering(ordering, current, bound) - - case (RangeFrame, _) => - sys.error("Non-Zero range offsets are not supported for windows " + - "with multiple order expressions.") - } - } - - /** - * Collection containing an entry for each window frame to process. Each entry contains a frame's - * [[WindowExpression]]s and factory function for the WindowFrameFunction. - */ - private[this] lazy val windowFrameExpressionFactoryPairs = { - type FrameKey = (String, FrameType, Expression, Expression) - type ExpressionBuffer = mutable.Buffer[Expression] - val framedFunctions = mutable.Map.empty[FrameKey, (ExpressionBuffer, ExpressionBuffer)] - - // Add a function and its function to the map for a given frame. - def collect(tpe: String, fr: SpecifiedWindowFrame, e: Expression, fn: Expression): Unit = { - val key = (tpe, fr.frameType, fr.lower, fr.upper) - val (es, fns) = framedFunctions.getOrElseUpdate( - key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression])) - es += e - fns += fn - } - - // Collect all valid window functions and group them by their frame. - windowExpression.foreach { x => - x.foreach { - case e @ WindowExpression(function, spec) => - val frame = spec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] - function match { - case AggregateExpression(f, _, _, _) => collect("AGGREGATE", frame, e, f) - case f: AggregateWindowFunction => collect("AGGREGATE", frame, e, f) - case f: OffsetWindowFunction => collect("OFFSET", frame, e, f) - case f => sys.error(s"Unsupported window function: $f") - } - case _ => - } - } - - // Map the groups to a (unbound) expression and frame factory pair. - var numExpressions = 0 - val timeZone = conf.sessionLocalTimeZone - framedFunctions.toSeq.map { - case (key, (expressions, functionSeq)) => - val ordinal = numExpressions - val functions = functionSeq.toArray - - // Construct an aggregate processor if we need one. - def processor = AggregateProcessor( - functions, - ordinal, - child.output, - (expressions, schema) => - newMutableProjection(expressions, schema, subexpressionEliminationEnabled)) - - // Create the factory - val factory = key match { - // Offset Frame - case ("OFFSET", _, IntegerLiteral(offset), _) => - target: InternalRow => - new OffsetWindowFunctionFrame( - target, - ordinal, - // OFFSET frame functions are guaranteed be OffsetWindowFunctions. - functions.map(_.asInstanceOf[OffsetWindowFunction]), - child.output, - (expressions, schema) => - newMutableProjection(expressions, schema, subexpressionEliminationEnabled), - offset) - - // Entire Partition Frame. - case ("AGGREGATE", _, UnboundedPreceding, UnboundedFollowing) => - target: InternalRow => { - new UnboundedWindowFunctionFrame(target, processor) - } - - // Growing Frame. - case ("AGGREGATE", frameType, UnboundedPreceding, upper) => - target: InternalRow => { - new UnboundedPrecedingWindowFunctionFrame( - target, - processor, - createBoundOrdering(frameType, upper, timeZone)) - } - - // Shrinking Frame. - case ("AGGREGATE", frameType, lower, UnboundedFollowing) => - target: InternalRow => { - new UnboundedFollowingWindowFunctionFrame( - target, - processor, - createBoundOrdering(frameType, lower, timeZone)) - } - - // Moving Frame. - case ("AGGREGATE", frameType, lower, upper) => - target: InternalRow => { - new SlidingWindowFunctionFrame( - target, - processor, - createBoundOrdering(frameType, lower, timeZone), - createBoundOrdering(frameType, upper, timeZone)) - } - } - - // Keep track of the number of expressions. This is a side-effect in a map... - numExpressions += expressions.size - - // Create the Frame Expression - Factory pair. - (expressions, factory) - } - } - - /** - * Create the resulting projection. - * - * This method uses Code Generation. It can only be used on the executor side. - * - * @param expressions unbound ordered function expressions. - * @return the final resulting projection. - */ - private[this] def createResultProjection(expressions: Seq[Expression]): UnsafeProjection = { - val references = expressions.zipWithIndex.map{ case (e, i) => - // Results of window expressions will be on the right side of child's output - BoundReference(child.output.size + i, e.dataType, e.nullable) - } - val unboundToRefMap = expressions.zip(references).toMap - val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap)) - UnsafeProjection.create( - child.output ++ patchedWindowExpression, - child.output) - } - protected override def doExecute(): RDD[InternalRow] = { // Unwrap the expressions and factories from the map. val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala new file mode 100644 index 000000000000..dcb86f48bdf3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.window + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType} + +abstract class WindowExecBase( + windowExpression: Seq[NamedExpression], + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + child: SparkPlan) extends UnaryExecNode { + + /** + * Create the resulting projection. + * + * This method uses Code Generation. It can only be used on the executor side. + * + * @param expressions unbound ordered function expressions. + * @return the final resulting projection. + */ + protected def createResultProjection(expressions: Seq[Expression]): UnsafeProjection = { + val references = expressions.zipWithIndex.map { case (e, i) => + // Results of window expressions will be on the right side of child's output + BoundReference(child.output.size + i, e.dataType, e.nullable) + } + val unboundToRefMap = expressions.zip(references).toMap + val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap)) + UnsafeProjection.create( + child.output ++ patchedWindowExpression, + child.output) + } + + /** + * Create a bound ordering object for a given frame type and offset. A bound ordering object is + * used to determine which input row lies within the frame boundaries of an output row. + * + * This method uses Code Generation. It can only be used on the executor side. + * + * @param frame to evaluate. This can either be a Row or Range frame. + * @param bound with respect to the row. + * @param timeZone the session local timezone for time related calculations. + * @return a bound ordering object. + */ + private def createBoundOrdering( + frame: FrameType, bound: Expression, timeZone: String): BoundOrdering = { + (frame, bound) match { + case (RowFrame, CurrentRow) => + RowBoundOrdering(0) + + case (RowFrame, IntegerLiteral(offset)) => + RowBoundOrdering(offset) + + case (RangeFrame, CurrentRow) => + val ordering = newOrdering(orderSpec, child.output) + RangeBoundOrdering(ordering, IdentityProjection, IdentityProjection) + + case (RangeFrame, offset: Expression) if orderSpec.size == 1 => + // Use only the first order expression when the offset is non-null. + val sortExpr = orderSpec.head + val expr = sortExpr.child + + // Create the projection which returns the current 'value'. + val current = newMutableProjection(expr :: Nil, child.output) + + // Flip the sign of the offset when processing the order is descending + val boundOffset = sortExpr.direction match { + case Descending => UnaryMinus(offset) + case Ascending => offset + } + + // Create the projection which returns the current 'value' modified by adding the offset. + val boundExpr = (expr.dataType, boundOffset.dataType) match { + case (DateType, IntegerType) => DateAdd(expr, boundOffset) + case (TimestampType, CalendarIntervalType) => + TimeAdd(expr, boundOffset, Some(timeZone)) + case (a, b) if a == b => Add(expr, boundOffset) + } + val bound = newMutableProjection(boundExpr :: Nil, child.output) + + // Construct the ordering. This is used to compare the result of current value projection + // to the result of bound value projection. This is done manually because we want to use + // Code Generation (if it is enabled). + val boundSortExprs = sortExpr.copy(BoundReference(0, expr.dataType, expr.nullable)) :: Nil + val ordering = newOrdering(boundSortExprs, Nil) + RangeBoundOrdering(ordering, current, bound) + + case (RangeFrame, _) => + sys.error("Non-Zero range offsets are not supported for windows " + + "with multiple order expressions.") + } + } + + /** + * Collection containing an entry for each window frame to process. Each entry contains a frame's + * [[WindowExpression]]s and factory function for the WindowFrameFunction. + */ + protected lazy val windowFrameExpressionFactoryPairs = { + type FrameKey = (String, FrameType, Expression, Expression) + type ExpressionBuffer = mutable.Buffer[Expression] + val framedFunctions = mutable.Map.empty[FrameKey, (ExpressionBuffer, ExpressionBuffer)] + + // Add a function and its function to the map for a given frame. + def collect(tpe: String, fr: SpecifiedWindowFrame, e: Expression, fn: Expression): Unit = { + val key = (tpe, fr.frameType, fr.lower, fr.upper) + val (es, fns) = framedFunctions.getOrElseUpdate( + key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression])) + es += e + fns += fn + } + + // Collect all valid window functions and group them by their frame. + windowExpression.foreach { x => + x.foreach { + case e @ WindowExpression(function, spec) => + val frame = spec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] + function match { + case AggregateExpression(f, _, _, _) => collect("AGGREGATE", frame, e, f) + case f: AggregateWindowFunction => collect("AGGREGATE", frame, e, f) + case f: OffsetWindowFunction => collect("OFFSET", frame, e, f) + case f: PythonUDF => collect("AGGREGATE", frame, e, f) + case f => sys.error(s"Unsupported window function: $f") + } + case _ => + } + } + + // Map the groups to a (unbound) expression and frame factory pair. + var numExpressions = 0 + val timeZone = conf.sessionLocalTimeZone + framedFunctions.toSeq.map { + case (key, (expressions, functionSeq)) => + val ordinal = numExpressions + val functions = functionSeq.toArray + + // Construct an aggregate processor if we need one. + // Currently we don't allow mixing of Pandas UDF and SQL aggregation functions + // in a single Window physical node. Therefore, we can assume no SQL aggregation + // functions if Pandas UDF exists. In the future, we might mix Pandas UDF and SQL + // aggregation function in a single physical node. + def processor = if (functions.exists(_.isInstanceOf[PythonUDF])) { + null + } else { + AggregateProcessor( + functions, + ordinal, + child.output, + (expressions, schema) => + newMutableProjection(expressions, schema, subexpressionEliminationEnabled)) + } + + // Create the factory + val factory = key match { + // Offset Frame + case ("OFFSET", _, IntegerLiteral(offset), _) => + target: InternalRow => + new OffsetWindowFunctionFrame( + target, + ordinal, + // OFFSET frame functions are guaranteed be OffsetWindowFunctions. + functions.map(_.asInstanceOf[OffsetWindowFunction]), + child.output, + (expressions, schema) => + newMutableProjection(expressions, schema, subexpressionEliminationEnabled), + offset) + + // Entire Partition Frame. + case ("AGGREGATE", _, UnboundedPreceding, UnboundedFollowing) => + target: InternalRow => { + new UnboundedWindowFunctionFrame(target, processor) + } + + // Growing Frame. + case ("AGGREGATE", frameType, UnboundedPreceding, upper) => + target: InternalRow => { + new UnboundedPrecedingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, upper, timeZone)) + } + + // Shrinking Frame. + case ("AGGREGATE", frameType, lower, UnboundedFollowing) => + target: InternalRow => { + new UnboundedFollowingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, lower, timeZone)) + } + + // Moving Frame. + case ("AGGREGATE", frameType, lower, upper) => + target: InternalRow => { + new SlidingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, lower, timeZone), + createBoundOrdering(frameType, upper, timeZone)) + } + } + + // Keep track of the number of expressions. This is a side-effect in a map... + numExpressions += expressions.size + + // Create the Frame Expression - Factory pair. + (expressions, factory) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala index 156002ef58fb..a5601899ea2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray * Before use a frame must be prepared by passing it all the rows in the current partition. After * preparation the update method can be called to fill the output rows. */ -private[window] abstract class WindowFunctionFrame { +abstract class WindowFunctionFrame { /** * Prepare the frame for calculating the results for a partition. * @@ -42,6 +42,20 @@ private[window] abstract class WindowFunctionFrame { * Write the current results to the target row. */ def write(index: Int, current: InternalRow): Unit + + /** + * The current lower window bound in the row array (inclusive). + * + * This should be called after the current row is updated via [[write]] + */ + def currentLowerBound(): Int + + /** + * The current row index of the upper window bound in the row array (exclusive) + * + * This should be called after the current row is updated via [[write]] + */ + def currentUpperBound(): Int } object WindowFunctionFrame { @@ -62,7 +76,7 @@ object WindowFunctionFrame { * @param newMutableProjection function used to create the projection. * @param offset by which rows get moved within a partition. */ -private[window] final class OffsetWindowFunctionFrame( +final class OffsetWindowFunctionFrame( target: InternalRow, ordinal: Int, expressions: Array[OffsetWindowFunction], @@ -137,6 +151,10 @@ private[window] final class OffsetWindowFunctionFrame( } inputIndex += 1 } + + override def currentLowerBound(): Int = throw new UnsupportedOperationException() + + override def currentUpperBound(): Int = throw new UnsupportedOperationException() } /** @@ -148,7 +166,7 @@ private[window] final class OffsetWindowFunctionFrame( * @param lbound comparator used to identify the lower bound of an output row. * @param ubound comparator used to identify the upper bound of an output row. */ -private[window] final class SlidingWindowFunctionFrame( +final class SlidingWindowFunctionFrame( target: InternalRow, processor: AggregateProcessor, lbound: BoundOrdering, @@ -170,24 +188,24 @@ private[window] final class SlidingWindowFunctionFrame( private[this] val buffer = new util.ArrayDeque[InternalRow]() /** - * Index of the first input row with a value greater than the upper bound of the current - * output row. + * Index of the first input row with a value equal to or greater than the lower bound of the + * current output row. */ - private[this] var inputHighIndex = 0 + private[this] var lowerBound = 0 /** - * Index of the first input row with a value equal to or greater than the lower bound of the - * current output row. + * Index of the first input row with a value greater than the upper bound of the current + * output row. */ - private[this] var inputLowIndex = 0 + private[this] var upperBound = 0 /** Prepare the frame for calculating a new partition. Reset all variables. */ override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { input = rows inputIterator = input.generateIterator() nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) - inputHighIndex = 0 - inputLowIndex = 0 + lowerBound = 0 + upperBound = 0 buffer.clear() } @@ -197,27 +215,27 @@ private[window] final class SlidingWindowFunctionFrame( // Drop all rows from the buffer for which the input row value is smaller than // the output row lower bound. - while (!buffer.isEmpty && lbound.compare(buffer.peek(), inputLowIndex, current, index) < 0) { + while (!buffer.isEmpty && lbound.compare(buffer.peek(), lowerBound, current, index) < 0) { buffer.remove() - inputLowIndex += 1 + lowerBound += 1 bufferUpdated = true } // Add all rows to the buffer for which the input row value is equal to or less than // the output row upper bound. - while (nextRow != null && ubound.compare(nextRow, inputHighIndex, current, index) <= 0) { - if (lbound.compare(nextRow, inputLowIndex, current, index) < 0) { - inputLowIndex += 1 + while (nextRow != null && ubound.compare(nextRow, upperBound, current, index) <= 0) { + if (lbound.compare(nextRow, lowerBound, current, index) < 0) { + lowerBound += 1 } else { buffer.add(nextRow.copy()) bufferUpdated = true } nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) - inputHighIndex += 1 + upperBound += 1 } // Only recalculate and update when the buffer changes. - if (bufferUpdated) { + if (processor != null && bufferUpdated) { processor.initialize(input.length) val iter = buffer.iterator() while (iter.hasNext) { @@ -226,6 +244,10 @@ private[window] final class SlidingWindowFunctionFrame( processor.evaluate(target) } } + + override def currentLowerBound(): Int = lowerBound + + override def currentUpperBound(): Int = upperBound } /** @@ -239,27 +261,39 @@ private[window] final class SlidingWindowFunctionFrame( * @param target to write results to. * @param processor to calculate the row values with. */ -private[window] final class UnboundedWindowFunctionFrame( +final class UnboundedWindowFunctionFrame( target: InternalRow, processor: AggregateProcessor) extends WindowFunctionFrame { + val lowerBound: Int = 0 + var upperBound: Int = 0 + /** Prepare the frame for calculating a new partition. Process all rows eagerly. */ override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { - processor.initialize(rows.length) - - val iterator = rows.generateIterator() - while (iterator.hasNext) { - processor.update(iterator.next()) + if (processor != null) { + processor.initialize(rows.length) + val iterator = rows.generateIterator() + while (iterator.hasNext) { + processor.update(iterator.next()) + } } + + upperBound = rows.length } /** Write the frame columns for the current row to the given target row. */ override def write(index: Int, current: InternalRow): Unit = { // Unfortunately we cannot assume that evaluation is deterministic. So we need to re-evaluate // for each row. - processor.evaluate(target) + if (processor != null) { + processor.evaluate(target) + } } + + override def currentLowerBound(): Int = lowerBound + + override def currentUpperBound(): Int = upperBound } /** @@ -276,7 +310,7 @@ private[window] final class UnboundedWindowFunctionFrame( * @param processor to calculate the row values with. * @param ubound comparator used to identify the upper bound of an output row. */ -private[window] final class UnboundedPrecedingWindowFunctionFrame( +final class UnboundedPrecedingWindowFunctionFrame( target: InternalRow, processor: AggregateProcessor, ubound: BoundOrdering) @@ -308,7 +342,9 @@ private[window] final class UnboundedPrecedingWindowFunctionFrame( nextRow = inputIterator.next() } - processor.initialize(input.length) + if (processor != null) { + processor.initialize(input.length) + } } /** Write the frame columns for the current row to the given target row. */ @@ -318,17 +354,23 @@ private[window] final class UnboundedPrecedingWindowFunctionFrame( // Add all rows to the aggregates for which the input row value is equal to or less than // the output row upper bound. while (nextRow != null && ubound.compare(nextRow, inputIndex, current, index) <= 0) { - processor.update(nextRow) + if (processor != null) { + processor.update(nextRow) + } nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) inputIndex += 1 bufferUpdated = true } // Only recalculate and update when the buffer changes. - if (bufferUpdated) { + if (processor != null && bufferUpdated) { processor.evaluate(target) } } + + override def currentLowerBound(): Int = 0 + + override def currentUpperBound(): Int = inputIndex } /** @@ -347,7 +389,7 @@ private[window] final class UnboundedPrecedingWindowFunctionFrame( * @param processor to calculate the row values with. * @param lbound comparator used to identify the lower bound of an output row. */ -private[window] final class UnboundedFollowingWindowFunctionFrame( +final class UnboundedFollowingWindowFunctionFrame( target: InternalRow, processor: AggregateProcessor, lbound: BoundOrdering) @@ -384,7 +426,7 @@ private[window] final class UnboundedFollowingWindowFunctionFrame( } // Only recalculate and update when the buffer changes. - if (bufferUpdated) { + if (processor != null && bufferUpdated) { processor.initialize(input.length) if (nextRow != null) { processor.update(nextRow) @@ -395,4 +437,8 @@ private[window] final class UnboundedFollowingWindowFunctionFrame( processor.evaluate(target) } } + + override def currentLowerBound(): Int = inputIndex + + override def currentUpperBound(): Int = input.length } From d72571e51d8b41e2287750759e120547afeeb7d7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 18 Dec 2018 13:50:55 +0800 Subject: [PATCH 0157/1072] [SPARK-26246][SQL] Inferring TimestampType from JSON ## What changes were proposed in this pull request? The `JsonInferSchema` class is extended to support `TimestampType` inferring from string fields in JSON input: - If the `prefersDecimal` option is set to `true`, it tries to infer decimal type from the string field. - If decimal type inference fails or `prefersDecimal` is disabled, `JsonInferSchema` tries to infer `TimestampType`. - If timestamp type inference fails, `StringType` is returned as the inferred type. ## How was this patch tested? Added new test suite - `JsonInferSchemaSuite` to check date and timestamp types inferring from JSON using `JsonInferSchema` directly. A few tests were added `JsonSuite` to check type merging and roundtrip tests. This changes was tested by `JsonSuite`, `JsonExpressionsSuite` and `JsonFunctionsSuite` as well. Closes #23201 from MaxGekk/json-infer-time. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/json/JsonInferSchema.scala | 22 +++- .../catalyst/json/JsonInferSchemaSuite.scala | 102 ++++++++++++++++++ .../datasources/json/JsonSuite.scala | 52 +++++++++ 3 files changed, 171 insertions(+), 5 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index 263e05de3207..d1bc00c08c1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -28,7 +28,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.JacksonUtils.nextUntil -import org.apache.spark.sql.catalyst.util.{DropMalformedMode, FailFastMode, ParseMode, PermissiveMode} +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -37,6 +37,12 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { private val decimalParser = ExprUtils.getDecimalParser(options.locale) + @transient + private lazy val timestampFormatter = TimestampFormatter( + options.timestampFormat, + options.timeZone, + options.locale) + /** * Infer the type of a collection of json records in three stages: * 1. Infer the type of each record @@ -115,13 +121,19 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { // record fields' types have been combined. NullType - case VALUE_STRING if options.prefersDecimal => + case VALUE_STRING => + val field = parser.getText val decimalTry = allCatch opt { - val bigDecimal = decimalParser(parser.getText) + val bigDecimal = decimalParser(field) DecimalType(bigDecimal.precision, bigDecimal.scale) } - decimalTry.getOrElse(StringType) - case VALUE_STRING => StringType + if (options.prefersDecimal && decimalTry.isDefined) { + decimalTry.get + } else if ((allCatch opt timestampFormatter.parse(field)).isDefined) { + TimestampType + } else { + StringType + } case START_OBJECT => val builder = Array.newBuilder[StructField] 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 new file mode 100644 index 000000000000..9307f9b47b80 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.json + +import com.fasterxml.jackson.core.JsonFactory + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { + + def checkType(options: Map[String, String], json: String, dt: DataType): Unit = { + val jsonOptions = new JSONOptions(options, "UTC", "") + val inferSchema = new JsonInferSchema(jsonOptions) + val factory = new JsonFactory() + jsonOptions.setJacksonOptions(factory) + val parser = CreateJacksonParser.string(factory, json) + parser.nextToken() + val expectedType = StructType(Seq(StructField("a", dt, true))) + + assert(inferSchema.inferField(parser) === expectedType) + } + + def checkTimestampType(pattern: String, json: String): Unit = { + checkType(Map("timestampFormat" -> pattern), json, TimestampType) + } + + test("inferring timestamp type") { + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + 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"}""") + } + } + } + + test("prefer decimals over timestamps") { + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + checkType( + options = Map( + "prefersDecimal" -> "true", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS" + ), + json = """{"a": "20181202.210400123"}""", + dt = DecimalType(17, 9) + ) + } + } + } + + test("skip decimal type inferring") { + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + checkType( + options = Map( + "prefersDecimal" -> "false", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS" + ), + json = """{"a": "20181202.210400123"}""", + dt = TimestampType + ) + } + } + } + + test("fallback to string type") { + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + checkType( + options = Map("timestampFormat" -> "yyyy,MM,dd.HHmmssSSS"), + json = """{"a": "20181202.210400123"}""", + dt = StringType + ) + } + } + } +} 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 786335b42e3c..8f575a371c98 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 @@ -38,6 +38,7 @@ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.StructType.fromDDL import org.apache.spark.util.Utils class TestFileFilter extends PathFilter { @@ -2589,4 +2590,55 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Row(null, Array(0, 1, 2), "abc", """{"a":"-","b":[0, 1, 2],"c":"abc"}""") :: Row(0.1, null, "def", """{"a":0.1,"b":{},"c":"def"}""") :: Nil) } + + test("inferring timestamp type") { + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + def schemaOf(jsons: String*): StructType = spark.read.json(jsons.toDS).schema + + assert(schemaOf( + """{"a":"2018-12-17T10:11:12.123-01:00"}""", + """{"a":"2018-12-16T22:23:24.123-02:00"}""") === fromDDL("a timestamp")) + + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":1}""") + === fromDDL("a string")) + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":"123"}""") + === fromDDL("a string")) + + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":null}""") + === fromDDL("a timestamp")) + assert(schemaOf("""{"a":null}""", """{"a":"2018-12-17T10:11:12.123-01:00"}""") + === fromDDL("a timestamp")) + } + } + } + + test("roundtrip for timestamp type inferring") { + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val customSchema = new StructType().add("date", TimestampType) + withTempDir { dir => + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.json" + val timestampsWithFormat = spark.read + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .json(datesRecords) + assert(timestampsWithFormat.schema === customSchema) + + timestampsWithFormat.write + .format("json") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .save(timestampsWithFormatPath) + + val readBack = spark.read + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .json(timestampsWithFormatPath) + + assert(readBack.schema === customSchema) + checkAnswer(readBack, timestampsWithFormat) + } + } + } + } } From 218341c5db62bf5363c4a16440fa742970f1e919 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 18 Dec 2018 20:52:02 +0800 Subject: [PATCH 0158/1072] [SPARK-26081][SQL][FOLLOW-UP] Use foreach instead of misuse of map (for Unit) ## What changes were proposed in this pull request? This PR proposes to use foreach instead of misuse of map (for Unit). This could cause some weird errors potentially and it's not a good practice anyway. See also SPARK-16694 ## How was this patch tested? N/A Closes #23341 from HyukjinKwon/followup-SPARK-26081. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../spark/sql/execution/datasources/csv/CSVFileFormat.scala | 2 +- .../spark/sql/execution/datasources/json/JsonFileFormat.scala | 2 +- .../spark/sql/execution/datasources/text/TextFileFormat.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index f7d8a9e1042d..f4f139d18005 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -189,5 +189,5 @@ private[csv] class CsvOutputWriter( gen.write(row) } - override def close(): Unit = univocityGenerator.map(_.close()) + override def close(): Unit = univocityGenerator.foreach(_.close()) } 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 3042133ee43a..40f55e706801 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 @@ -190,5 +190,5 @@ private[json] class JsonOutputWriter( gen.writeLineEnding() } - override def close(): Unit = jacksonGenerator.map(_.close()) + override def close(): Unit = jacksonGenerator.foreach(_.close()) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index 01948ab25d63..0607f7b3c0d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -153,7 +153,7 @@ class TextOutputWriter( private var outputStream: Option[OutputStream] = None override def write(row: InternalRow): Unit = { - val os = outputStream.getOrElse{ + val os = outputStream.getOrElse { val newStream = CodecStreams.createOutputStream(context, new Path(path)) outputStream = Some(newStream) newStream @@ -167,6 +167,6 @@ class TextOutputWriter( } override def close(): Unit = { - outputStream.map(_.close()) + outputStream.foreach(_.close()) } } From 4d693ac904d89b3afeba107eb0480120daf78174 Mon Sep 17 00:00:00 2001 From: Stan Zhai Date: Tue, 18 Dec 2018 07:02:09 -0600 Subject: [PATCH 0159/1072] [SPARK-24680][DEPLOY] Support spark.executorEnv.JAVA_HOME in Standalone mode ## What changes were proposed in this pull request? spark.executorEnv.JAVA_HOME does not take effect when a Worker starting an Executor process in Standalone mode. This PR fixed this. ## How was this patch tested? Manual tests. Closes #21663 from stanzhai/fix-executor-env-java-home. Lead-authored-by: Stan Zhai Co-authored-by: Stan Zhai Signed-off-by: Sean Owen --- .../spark/launcher/AbstractCommandBuilder.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) 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 ce24400f557c..56edceb17bfb 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -91,14 +91,18 @@ abstract List buildCommand(Map env) */ List buildJavaCommand(String extraClassPath) throws IOException { List cmd = new ArrayList<>(); - String envJavaHome; - if (javaHome != null) { - cmd.add(join(File.separator, javaHome, "bin", "java")); - } else if ((envJavaHome = System.getenv("JAVA_HOME")) != null) { - cmd.add(join(File.separator, envJavaHome, "bin", "java")); - } else { - cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); + String[] candidateJavaHomes = new String[] { + javaHome, + childEnv.get("JAVA_HOME"), + System.getenv("JAVA_HOME"), + System.getProperty("java.home") + }; + for (String javaHome : candidateJavaHomes) { + if (javaHome != null) { + cmd.add(join(File.separator, javaHome, "bin", "java")); + break; + } } // Load extra JAVA_OPTS from conf/java-opts, if it exists. From 3c0bb6bc45e64fd82052d7857f2a06c34f0c1793 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 19 Dec 2018 00:01:53 +0800 Subject: [PATCH 0160/1072] [SPARK-26384][SQL] Propagate SQL configs for CSV schema inferring ## What changes were proposed in this pull request? Currently, SQL configs are not propagated to executors while schema inferring in CSV datasource. For example, changing of `spark.sql.legacy.timeParser.enabled` does not impact on inferring timestamp types. In the PR, I propose to fix the issue by wrapping schema inferring action using `SQLExecution.withSQLConfPropagated`. ## How was this patch tested? Added logging to `TimestampFormatter`: ```patch -object TimestampFormatter { +object TimestampFormatter extends Logging { def apply(format: String, timeZone: TimeZone, locale: Locale): TimestampFormatter = { if (SQLConf.get.legacyTimeParserEnabled) { + logError("LegacyFallbackTimestampFormatter is being used") new LegacyFallbackTimestampFormatter(format, timeZone, locale) } else { + logError("Iso8601TimestampFormatter is being used") new Iso8601TimestampFormatter(format, timeZone, locale) } } ``` and run the command in `spark-shell`: ```shell $ ./bin/spark-shell --conf spark.sql.legacy.timeParser.enabled=true ``` ```scala scala> Seq("2010|10|10").toDF.repartition(1).write.mode("overwrite").text("/tmp/foo") scala> spark.read.option("inferSchema", "true").option("header", "false").option("timestampFormat", "yyyy|MM|dd").csv("/tmp/foo").printSchema() 18/12/18 10:47:27 ERROR TimestampFormatter: LegacyFallbackTimestampFormatter is being used root |-- _c0: timestamp (nullable = true) ``` Closes #23345 from MaxGekk/csv-schema-infer-propagate-configs. Authored-by: Maxim Gekk Signed-off-by: Hyukjin Kwon --- .../sql/execution/datasources/csv/CSVDataSource.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 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 b46dfb94c133..375cec597166 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 @@ -35,6 +35,7 @@ import org.apache.spark.rdd.{BinaryFileRDD, RDD} import org.apache.spark.sql.{Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVInferSchema, CSVOptions, UnivocityParser} +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.types.StructType @@ -135,7 +136,9 @@ object TextInputCSVDataSource extends CSVDataSource { val parser = new CsvParser(parsedOptions.asParserSettings) linesWithoutHeader.map(parser.parseLine) } - new CSVInferSchema(parsedOptions).infer(tokenRDD, header) + SQLExecution.withSQLConfPropagated(csv.sparkSession) { + new CSVInferSchema(parsedOptions).infer(tokenRDD, header) + } case _ => // If the first line could not be read, just return the empty schema. StructType(Nil) @@ -208,7 +211,9 @@ object MultiLineCSVDataSource extends CSVDataSource { encoding = parsedOptions.charset) } val sampled = CSVUtils.sample(tokenRDD, parsedOptions) - new CSVInferSchema(parsedOptions).infer(sampled, header) + SQLExecution.withSQLConfPropagated(sparkSession) { + new CSVInferSchema(parsedOptions).infer(sampled, header) + } case None => // If the first row could not be read, just return the empty schema. StructType(Nil) From befca983d2da4f7828aa7a7cd7345d17c4f291dd Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 18 Dec 2018 10:09:56 -0800 Subject: [PATCH 0161/1072] [SPARK-26382][CORE] prefix comparator should handle -0.0 ## What changes were proposed in this pull request? This is kind of a followup of https://github.com/apache/spark/pull/23239 The `UnsafeProject` will normalize special float/double values(NaN and -0.0), so the sorter doesn't have to handle it. However, for consistency and future-proof, this PR proposes to normalize `-0.0` in the prefix comparator, so that it's same with the normal ordering. Note that prefix comparator handles NaN as well. This is not a bug fix, but a safe guard. ## How was this patch tested? existing tests Closes #23334 from cloud-fan/sort. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../unsafe/sort/PrefixComparators.java | 2 ++ .../unsafe/sort/PrefixComparatorsSuite.scala | 17 +++++++++++++++-- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java index 0910db22af00..bef1bdadb27a 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java @@ -69,6 +69,8 @@ public static final class DoublePrefixComparator { * details see http://stereopsis.com/radix.html. */ public static long computePrefix(double value) { + // normalize -0.0 to 0.0, as they should be equal + value = value == -0.0 ? 0.0 : value; // Java's doubleToLongBits already canonicalizes all NaN values to the smallest possible // positive NaN, so there's nothing special we need to do for NaNs. long bits = Double.doubleToLongBits(value); diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala index 73546ef1b7a6..38cb37c52459 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala @@ -125,6 +125,7 @@ class PrefixComparatorsSuite extends SparkFunSuite with PropertyChecks { val nan2Prefix = PrefixComparators.DoublePrefixComparator.computePrefix(nan2) assert(nan1Prefix === nan2Prefix) val doubleMaxPrefix = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue) + // NaN is greater than the max double value. assert(PrefixComparators.DOUBLE.compare(nan1Prefix, doubleMaxPrefix) === 1) } @@ -134,22 +135,34 @@ class PrefixComparatorsSuite extends SparkFunSuite with PropertyChecks { assert(java.lang.Double.doubleToRawLongBits(negativeNan) < 0) val prefix = PrefixComparators.DoublePrefixComparator.computePrefix(negativeNan) val doubleMaxPrefix = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue) + // -NaN is greater than the max double value. assert(PrefixComparators.DOUBLE.compare(prefix, doubleMaxPrefix) === 1) } test("double prefix comparator handles other special values properly") { - val nullValue = 0L + // See `SortPrefix.nullValue` for how we deal with nulls for float/double type + val smallestNullPrefix = 0L + val largestNullPrefix = -1L val nan = PrefixComparators.DoublePrefixComparator.computePrefix(Double.NaN) val posInf = PrefixComparators.DoublePrefixComparator.computePrefix(Double.PositiveInfinity) val negInf = PrefixComparators.DoublePrefixComparator.computePrefix(Double.NegativeInfinity) val minValue = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MinValue) val maxValue = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue) val zero = PrefixComparators.DoublePrefixComparator.computePrefix(0.0) + val minusZero = PrefixComparators.DoublePrefixComparator.computePrefix(-0.0) + + // null is greater than everything including NaN, when we need to treat it as the largest value. + assert(PrefixComparators.DOUBLE.compare(largestNullPrefix, nan) === 1) + // NaN is greater than the positive infinity. assert(PrefixComparators.DOUBLE.compare(nan, posInf) === 1) assert(PrefixComparators.DOUBLE.compare(posInf, maxValue) === 1) assert(PrefixComparators.DOUBLE.compare(maxValue, zero) === 1) assert(PrefixComparators.DOUBLE.compare(zero, minValue) === 1) assert(PrefixComparators.DOUBLE.compare(minValue, negInf) === 1) - assert(PrefixComparators.DOUBLE.compare(negInf, nullValue) === 1) + // null is smaller than everything including negative infinity, when we need to treat it as + // the smallest value. + assert(PrefixComparators.DOUBLE.compare(negInf, smallestNullPrefix) === 1) + // 0.0 should be equal to -0.0. + assert(PrefixComparators.DOUBLE.compare(zero, minusZero) === 0) } } From 428eb2ad0ad8a141427120b13de3287962258c2d Mon Sep 17 00:00:00 2001 From: Jackey Lee Date: Tue, 18 Dec 2018 12:15:36 -0600 Subject: [PATCH 0162/1072] [SPARK-26394][CORE] Fix annotation error for Utils.timeStringAsMs ## What changes were proposed in this pull request? Change microseconds to milliseconds in annotation of Utils.timeStringAsMs. Closes #23346 from stczwd/stczwd. Authored-by: Jackey Lee Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index b4ea1ee95021..143abd3bbea8 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1037,7 +1037,7 @@ private[spark] object Utils extends Logging { } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ def timeStringAsMs(str: String): Long = { From 4b3fe3a9ccc8a4a8eb0d037d19cb07a8a288e37a Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 18 Dec 2018 13:30:09 -0800 Subject: [PATCH 0163/1072] [SPARK-25815][K8S] Support kerberos in client mode, keytab-based token renewal. This change hooks up the k8s backed to the updated HadoopDelegationTokenManager, so that delegation tokens are also available in client mode, and keytab-based token renewal is enabled. The change re-works the k8s feature steps related to kerberos so that the driver does all the credential management and provides all the needed information to executors - so nothing needs to be added to executor pods. This also makes cluster mode behave a lot more similarly to client mode, since no driver-related config steps are run in the latter case. The main two things that don't need to happen in executors anymore are: - adding the Hadoop config to the executor pods: this is not needed since the Spark driver will serialize the Hadoop config and send it to executors when running tasks. - mounting the kerberos config file in the executor pods: this is not needed once you remove the above. The Hadoop conf sent by the driver with the tasks is already resolved (i.e. has all the kerberos names properly defined), so executors do not need access to the kerberos realm information anymore. The change also avoids creating delegation tokens unnecessarily. This means that they'll only be created if a secret with tokens was not provided, and if a keytab is not provided. In either of those cases, the driver code will handle delegation tokens: in cluster mode by creating a secret and stashing them, in client mode by using existing mechanisms to send DTs to executors. One last feature: the change also allows defining a keytab with a "local:" URI. This is supported in client mode (although that's the same as not saying "local:"), and in k8s cluster mode. This allows the keytab to be mounted onto the image from a pre-existing secret, for example. Finally, the new code always sets SPARK_USER in the driver and executor pods. This is in line with how other resource managers behave: the submitting user reflects which user will access Hadoop services in the app. (With kerberos, that's overridden by the logged in user.) That user is unrelated to the OS user the app is running as inside the containers. Tested: - client and cluster mode with kinit - cluster mode with keytab - cluster mode with local: keytab - YARN cluster with keytab (to make sure it isn't broken) Closes #22911 from vanzin/SPARK-25815. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/deploy/SparkSubmit.scala | 29 +- .../HadoopDelegationTokenManager.scala | 8 +- .../scala/org/apache/spark/util/Utils.scala | 8 + .../apache/spark/deploy/k8s/Constants.scala | 9 +- .../spark/deploy/k8s/KubernetesConf.scala | 4 - .../apache/spark/deploy/k8s/SparkPod.scala | 25 +- .../k8s/features/BasicDriverFeatureStep.scala | 4 + .../features/BasicExecutorFeatureStep.scala | 4 + .../HadoopConfDriverFeatureStep.scala | 124 +++++++ .../HadoopConfExecutorFeatureStep.scala | 40 --- .../HadoopSparkUserExecutorFeatureStep.scala | 35 -- .../KerberosConfDriverFeatureStep.scala | 315 ++++++++++-------- .../KerberosConfExecutorFeatureStep.scala | 46 --- .../hadooputils/HadoopBootstrapUtil.scala | 283 ---------------- .../hadooputils/KerberosConfigSpec.scala | 33 -- .../k8s/submit/KubernetesDriverBuilder.scala | 1 + .../KubernetesClusterSchedulerBackend.scala | 7 +- .../k8s/KubernetesExecutorBuilder.scala | 5 +- .../BasicDriverFeatureStepSuite.scala | 3 +- .../BasicExecutorFeatureStepSuite.scala | 9 +- .../HadoopConfDriverFeatureStepSuite.scala | 71 ++++ .../KerberosConfDriverFeatureStepSuite.scala | 171 ++++++++++ .../KubernetesFeaturesTestUtils.scala | 6 + .../org/apache/spark/deploy/yarn/Client.scala | 24 +- .../spark/deploy/yarn/ClientSuite.scala | 6 +- 25 files changed, 649 insertions(+), 621 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala 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 d4055cb6c585..763bd0a70a03 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy import java.io._ import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} -import java.net.URL +import java.net.{URI, URL} import java.security.PrivilegedExceptionAction import java.text.ParseException import java.util.UUID @@ -334,19 +334,20 @@ private[spark] class SparkSubmit extends Logging { val hadoopConf = conf.getOrElse(SparkHadoopUtil.newConfiguration(sparkConf)) val targetDir = Utils.createTempDir() - // assure a keytab is available from any place in a JVM - if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient || isKubernetesCluster) { - if (args.principal != null) { - if (args.keytab != null) { - require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") - // Add keytab and principal configurations in sysProps to make them available - // for later use; e.g. in spark sql, the isolated class loader used to talk - // to HiveMetastore will use these settings. They will be set as Java system - // properties and then loaded by SparkConf - sparkConf.set(KEYTAB, args.keytab) - sparkConf.set(PRINCIPAL, args.principal) - UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) - } + // Kerberos is not supported in standalone mode, and keytab support is not yet available + // in Mesos cluster mode. + if (clusterManager != STANDALONE + && !isMesosCluster + && args.principal != null + && args.keytab != null) { + // If client mode, make sure the keytab is just a local path. + if (deployMode == CLIENT && Utils.isLocalUri(args.keytab)) { + args.keytab = new URI(args.keytab).getPath() + } + + if (!Utils.isLocalUri(args.keytab)) { + require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") + UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index 126a6ab80136..f7e3ddecee09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.security import java.io.File +import java.net.URI import java.security.PrivilegedExceptionAction import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import java.util.concurrent.atomic.AtomicReference @@ -71,11 +72,13 @@ private[spark] class HadoopDelegationTokenManager( private val providerEnabledConfig = "spark.security.credentials.%s.enabled" private val principal = sparkConf.get(PRINCIPAL).orNull - private val keytab = sparkConf.get(KEYTAB).orNull + + // The keytab can be a local: URI for cluster mode, so translate it to a regular path. If it is + // needed later on, the code will check that it exists. + private val keytab = sparkConf.get(KEYTAB).map { uri => new URI(uri).getPath() }.orNull require((principal == null) == (keytab == null), "Both principal and keytab must be defined, or neither.") - require(keytab == null || new File(keytab).isFile(), s"Cannot find keytab at $keytab.") private val delegationTokenProviders = loadProviders() logDebug("Using the following builtin delegation token providers: " + @@ -264,6 +267,7 @@ private[spark] class HadoopDelegationTokenManager( private def doLogin(): UserGroupInformation = { logInfo(s"Attempting to login to KDC using principal: $principal") + require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.") val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) logInfo("Successfully logged into KDC.") ugi 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 143abd3bbea8..f322e92c6c8c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -92,6 +92,9 @@ private[spark] object Utils extends Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @volatile private var localRootDirs: Array[String] = null + /** Scheme used for files that are locally available on worker nodes in the cluster. */ + val LOCAL_SCHEME = "local" + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -2829,6 +2832,11 @@ private[spark] object Utils extends Logging { def isClientMode(conf: SparkConf): Boolean = { "client".equals(conf.get(SparkLauncher.DEPLOY_MODE, "client")) } + + /** Returns whether the URI is a "local:" URI. */ + def isLocalUri(uri: String): Boolean = { + uri.startsWith(s"$LOCAL_SCHEME:") + } } private[util] object CallerContext extends Logging { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 85917b88e912..76041e7de518 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -87,25 +87,22 @@ private[spark] object Constants { val NON_JVM_MEMORY_OVERHEAD_FACTOR = 0.4d // Hadoop Configuration - val HADOOP_FILE_VOLUME = "hadoop-properties" + val HADOOP_CONF_VOLUME = "hadoop-properties" val KRB_FILE_VOLUME = "krb5-file" val HADOOP_CONF_DIR_PATH = "/opt/hadoop/conf" val KRB_FILE_DIR_PATH = "/etc" val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" val HADOOP_CONFIG_MAP_NAME = "spark.kubernetes.executor.hadoopConfigMapName" - val KRB5_CONFIG_MAP_NAME = - "spark.kubernetes.executor.krb5ConfigMapName" // Kerberos Configuration - val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = "delegation-tokens" val KERBEROS_DT_SECRET_NAME = "spark.kubernetes.kerberos.dt-secret-name" val KERBEROS_DT_SECRET_KEY = "spark.kubernetes.kerberos.dt-secret-key" - val KERBEROS_SPARK_USER_NAME = - "spark.kubernetes.kerberos.spark-user-name" val KERBEROS_SECRET_KEY = "hadoop-tokens" + val KERBEROS_KEYTAB_VOLUME = "kerberos-keytab" + val KERBEROS_KEYTAB_MOUNT_POINT = "/mnt/secrets/kerberos-keytab" // Hadoop credentials secrets for the Spark app. val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index a06c21b47f15..6febad981af5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -42,10 +42,6 @@ private[spark] abstract class KubernetesConf(val sparkConf: SparkConf) { def appName: String = get("spark.app.name", "spark") - def hadoopConfigMapName: String = s"$resourceNamePrefix-hadoop-config" - - def krbConfigMapName: String = s"$resourceNamePrefix-krb5-file" - def namespace: String = get(KUBERNETES_NAMESPACE) def imagePullPolicy: String = get(CONTAINER_IMAGE_PULL_POLICY) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala index 345dd117fd35..fd1196368a7f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala @@ -18,7 +18,30 @@ package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} -private[spark] case class SparkPod(pod: Pod, container: Container) +private[spark] case class SparkPod(pod: Pod, container: Container) { + + /** + * Convenience method to apply a series of chained transformations to a pod. + * + * Use it like: + * + * original.modify { case pod => + * // update pod and return new one + * }.modify { case pod => + * // more changes that create a new pod + * }.modify { + * case pod if someCondition => // new pod + * } + * + * This makes it cleaner to apply multiple transformations, avoiding having to create + * a bunch of awkwardly-named local variables. Since the argument is a partial function, + * it can do matching without needing to exhaust all the possibilities. If the function + * is not applied, then the original pod will be kept. + */ + def transform(fn: PartialFunction[SparkPod, SparkPod]): SparkPod = fn.lift(this).getOrElse(this) + +} + private[spark] object SparkPod { def initialPod(): SparkPod = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index d8cf3653d322..8362c14fb289 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -110,6 +110,10 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) .withContainerPort(driverUIPort) .withProtocol("TCP") .endPort() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(Utils.getCurrentUserName()) + .endEnv() .addAllToEnv(driverCustomEnvs.asJava) .addNewEnv() .withName(ENV_DRIVER_BIND_ADDRESS) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 4bcf4c9446aa..c8bf7cdb4224 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -163,6 +163,10 @@ private[spark] class BasicExecutorFeatureStep( .addToLimits("memory", executorMemoryQuantity) .addToRequests("cpu", executorCpuQuantity) .endResources() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(Utils.getCurrentUserName()) + .endEnv() .addAllToEnv(executorEnv.asJava) .withPorts(requiredPorts.asJava) .addToArgs("executor") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala new file mode 100644 index 000000000000..d602ed5481e6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +/** + * Mounts the Hadoop configuration - either a pre-defined config map, or a local configuration + * directory - on the driver pod. + */ +private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf) + extends KubernetesFeatureConfigStep { + + private val confDir = Option(conf.sparkConf.getenv(ENV_HADOOP_CONF_DIR)) + private val existingConfMap = conf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) + + KubernetesUtils.requireNandDefined( + confDir, + existingConfMap, + "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + + "as the creation of an additional ConfigMap, when one is already specified is extraneous") + + private lazy val confFiles: Seq[File] = { + val dir = new File(confDir.get) + if (dir.isDirectory) { + dir.listFiles.filter(_.isFile).toSeq + } else { + Nil + } + } + + private def newConfigMapName: String = s"${conf.resourceNamePrefix}-hadoop-config" + + private def hasHadoopConf: Boolean = confDir.isDefined || existingConfMap.isDefined + + override def configurePod(original: SparkPod): SparkPod = { + original.transform { case pod if hasHadoopConf => + val confVolume = if (confDir.isDefined) { + val keyPaths = confFiles.map { file => + new KeyToPathBuilder() + .withKey(file.getName()) + .withPath(file.getName()) + .build() + } + new VolumeBuilder() + .withName(HADOOP_CONF_VOLUME) + .withNewConfigMap() + .withName(newConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .build() + } else { + new VolumeBuilder() + .withName(HADOOP_CONF_VOLUME) + .withNewConfigMap() + .withName(existingConfMap.get) + .endConfigMap() + .build() + } + + val podWithConf = new PodBuilder(pod.pod) + .editSpec() + .addNewVolumeLike(confVolume) + .endVolume() + .endSpec() + .build() + + val containerWithMount = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(HADOOP_CONF_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .build() + + SparkPod(podWithConf, containerWithMount) + } + } + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + if (confDir.isDefined) { + val fileMap = confFiles.map { file => + (file.getName(), Files.toString(file, StandardCharsets.UTF_8)) + }.toMap.asJava + + Seq(new ConfigMapBuilder() + .withNewMetadata() + .withName(newConfigMapName) + .endMetadata() + .addToData(fileMap) + .build()) + } else { + Nil + } + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala deleted file mode 100644 index da332881ae1a..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging - -/** - * This step is responsible for bootstraping the container with ConfigMaps - * containing Hadoop config files mounted as volumes and an ENV variable - * pointed to the mounted file directory. - */ -private[spark] class HadoopConfExecutorFeatureStep(conf: KubernetesExecutorConf) - extends KubernetesFeatureConfigStep with Logging { - - override def configurePod(pod: SparkPod): SparkPod = { - val hadoopConfDirCMapName = conf.getOption(HADOOP_CONFIG_MAP_NAME) - if (hadoopConfDirCMapName.isDefined) { - HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod) - } else { - pod - } - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala deleted file mode 100644 index c038e75491ca..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala +++ /dev/null @@ -1,35 +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.deploy.k8s.features - -import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil - -/** - * This step is responsible for setting ENV_SPARK_USER when HADOOP_FILES are detected - * however, this step would not be run if Kerberos is enabled, as Kerberos sets SPARK_USER - */ -private[spark] class HadoopSparkUserExecutorFeatureStep(conf: KubernetesExecutorConf) - extends KubernetesFeatureConfigStep { - - override def configurePod(pod: SparkPod): SparkPod = { - conf.getOption(KERBEROS_SPARK_USER_NAME).map { user => - HadoopBootstrapUtil.bootstrapSparkUserPod(user, pod) - }.getOrElse(pod) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index c6d5a866fa7b..721d7e97b21f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -16,31 +16,40 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.{HasMetadata, Secret, SecretBuilder} +import java.io.File +import java.nio.charset.StandardCharsets + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ import org.apache.commons.codec.binary.Base64 -import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.{KubernetesDriverConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.hadooputils._ import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils /** - * Runs the necessary Hadoop-based logic based on Kerberos configs and the presence of the - * HADOOP_CONF_DIR. This runs various bootstrap methods defined in HadoopBootstrapUtil. + * Provide kerberos / service credentials to the Spark driver. + * + * There are three use cases, in order of precedence: + * + * - keytab: if a kerberos keytab is defined, it is provided to the driver, and the driver will + * manage the kerberos login and the creation of delegation tokens. + * - existing tokens: if a secret containing delegation tokens is provided, it will be mounted + * on the driver pod, and the driver will handle distribution of those tokens to executors. + * - tgt only: if Hadoop security is enabled, the local TGT will be used to create delegation + * tokens which will be provided to the driver. The driver will handle distribution of the + * tokens to executors. */ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDriverConf) - extends KubernetesFeatureConfigStep { - - private val hadoopConfDir = Option(kubernetesConf.sparkConf.getenv(ENV_HADOOP_CONF_DIR)) - private val hadoopConfigMapName = kubernetesConf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) - KubernetesUtils.requireNandDefined( - hadoopConfDir, - hadoopConfigMapName, - "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + - "as the creation of an additional ConfigMap, when one is already specified is extraneous") + extends KubernetesFeatureConfigStep with Logging { private val principal = kubernetesConf.get(org.apache.spark.internal.config.PRINCIPAL) private val keytab = kubernetesConf.get(org.apache.spark.internal.config.KEYTAB) @@ -49,15 +58,6 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri private val krb5File = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) private val krb5CMap = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) private val hadoopConf = SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf) - private val tokenManager = new HadoopDelegationTokenManager(kubernetesConf.sparkConf, hadoopConf) - private val isKerberosEnabled = - (hadoopConfDir.isDefined && UserGroupInformation.isSecurityEnabled) || - (hadoopConfigMapName.isDefined && (krb5File.isDefined || krb5CMap.isDefined)) - require(keytab.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Keytab") - - require(existingSecretName.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Secret") KubernetesUtils.requireNandDefined( krb5File, @@ -79,128 +79,183 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri "If a secret storing a Kerberos Delegation Token is specified you must also" + " specify the item-key where the data is stored") - private val hadoopConfigurationFiles = hadoopConfDir.map { hConfDir => - HadoopBootstrapUtil.getHadoopConfFiles(hConfDir) + if (!hasKerberosConf) { + logInfo("You have not specified a krb5.conf file locally or via a ConfigMap. " + + "Make sure that you have the krb5.conf locally on the driver image.") } - private val newHadoopConfigMapName = - if (hadoopConfigMapName.isEmpty) { - Some(kubernetesConf.hadoopConfigMapName) - } else { - None - } - // Either use pre-existing secret or login to create new Secret with DT stored within - private val kerberosConfSpec: Option[KerberosConfigSpec] = (for { - secretName <- existingSecretName - secretItemKey <- existingSecretItemKey - } yield { - KerberosConfigSpec( - dtSecret = None, - dtSecretName = secretName, - dtSecretItemKey = secretItemKey, - jobUserName = UserGroupInformation.getCurrentUser.getShortUserName) - }).orElse( - if (isKerberosEnabled) { - Some(buildKerberosSpec()) + // Create delegation tokens if needed. This is a lazy val so that it's not populated + // unnecessarily. But it needs to be accessible to different methods in this class, + // since it's not clear based solely on available configuration options that delegation + // tokens are needed when other credentials are not available. + private lazy val delegationTokens: Array[Byte] = { + if (keytab.isEmpty && existingSecretName.isEmpty) { + val tokenManager = new HadoopDelegationTokenManager(kubernetesConf.sparkConf, + SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf)) + val creds = UserGroupInformation.getCurrentUser().getCredentials() + tokenManager.obtainDelegationTokens(creds) + // If no tokens and no secrets are stored in the credentials, make sure nothing is returned, + // to avoid creating an unnecessary secret. + if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { + SparkHadoopUtil.get.serialize(creds) + } else { + null + } } else { - None + null } - ) + } - override def configurePod(pod: SparkPod): SparkPod = { - if (!isKerberosEnabled) { - return pod - } + private def needKeytabUpload: Boolean = keytab.exists(!Utils.isLocalUri(_)) - val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - hadoopConfDir, - newHadoopConfigMapName, - hadoopConfigMapName, - pod) - kerberosConfSpec.map { hSpec => - HadoopBootstrapUtil.bootstrapKerberosPod( - hSpec.dtSecretName, - hSpec.dtSecretItemKey, - hSpec.jobUserName, - krb5File, - Some(kubernetesConf.krbConfigMapName), - krb5CMap, - hadoopBasedSparkPod) - }.getOrElse( - HadoopBootstrapUtil.bootstrapSparkUserPod( - UserGroupInformation.getCurrentUser.getShortUserName, - hadoopBasedSparkPod)) - } + private def dtSecretName: String = s"${kubernetesConf.resourceNamePrefix}-delegation-tokens" - override def getAdditionalPodSystemProperties(): Map[String, String] = { - if (!isKerberosEnabled) { - return Map.empty - } + private def ktSecretName: String = s"${kubernetesConf.resourceNamePrefix}-kerberos-keytab" - val resolvedConfValues = kerberosConfSpec.map { hSpec => - Map(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, - KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, - KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.krbConfigMapName)) - }.getOrElse( - Map(KERBEROS_SPARK_USER_NAME -> - UserGroupInformation.getCurrentUser.getShortUserName)) - Map(HADOOP_CONFIG_MAP_NAME -> - hadoopConfigMapName.getOrElse(kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues - } + private def hasKerberosConf: Boolean = krb5CMap.isDefined | krb5File.isDefined - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - if (!isKerberosEnabled) { - return Seq.empty - } + private def newConfigMapName: String = s"${kubernetesConf.resourceNamePrefix}-krb5-file" - val hadoopConfConfigMap = for { - hName <- newHadoopConfigMapName - hFiles <- hadoopConfigurationFiles - } yield { - HadoopBootstrapUtil.buildHadoopConfigMap(hName, hFiles) - } + override def configurePod(original: SparkPod): SparkPod = { + original.transform { case pod if hasKerberosConf => + val configMapVolume = if (krb5CMap.isDefined) { + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(krb5CMap.get) + .endConfigMap() + .build() + } else { + val krb5Conf = new File(krb5File.get) + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(newConfigMapName) + .withItems(new KeyToPathBuilder() + .withKey(krb5Conf.getName()) + .withPath(krb5Conf.getName()) + .build()) + .endConfigMap() + .build() + } - val krb5ConfigMap = krb5File.map { fileLocation => - HadoopBootstrapUtil.buildkrb5ConfigMap( - kubernetesConf.krbConfigMapName, - fileLocation) - } + val podWithVolume = new PodBuilder(pod.pod) + .editSpec() + .addNewVolumeLike(configMapVolume) + .endVolume() + .endSpec() + .build() + + val containerWithMount = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(KRB_FILE_VOLUME) + .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") + .withSubPath("krb5.conf") + .endVolumeMount() + .build() + + SparkPod(podWithVolume, containerWithMount) + }.transform { + case pod if needKeytabUpload => + // If keytab is defined and is a submission-local file (not local: URI), then create a + // secret for it. The keytab data will be stored in this secret below. + val podWitKeytab = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(KERBEROS_KEYTAB_VOLUME) + .withNewSecret() + .withSecretName(ktSecretName) + .endSecret() + .endVolume() + .endSpec() + .build() + + val containerWithKeytab = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(KERBEROS_KEYTAB_VOLUME) + .withMountPath(KERBEROS_KEYTAB_MOUNT_POINT) + .endVolumeMount() + .build() + + SparkPod(podWitKeytab, containerWithKeytab) + + case pod if existingSecretName.isDefined | delegationTokens != null => + val secretName = existingSecretName.getOrElse(dtSecretName) + val itemKey = existingSecretItemKey.getOrElse(KERBEROS_SECRET_KEY) + + val podWithTokens = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secretName) + .endSecret() + .endVolume() + .endSpec() + .build() - val kerberosDTSecret = kerberosConfSpec.flatMap(_.dtSecret) + val containerWithTokens = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$itemKey") + .endEnv() + .build() - hadoopConfConfigMap.toSeq ++ - krb5ConfigMap.toSeq ++ - kerberosDTSecret.toSeq + SparkPod(podWithTokens, containerWithTokens) + } } - private def buildKerberosSpec(): KerberosConfigSpec = { - // The JobUserUGI will be taken fom the Local Ticket Cache or via keytab+principal - // The login happens in the SparkSubmit so login logic is not necessary to include - val jobUserUGI = UserGroupInformation.getCurrentUser - val creds = jobUserUGI.getCredentials - tokenManager.obtainDelegationTokens(creds) - val tokenData = SparkHadoopUtil.get.serialize(creds) - require(tokenData.nonEmpty, "Did not obtain any delegation tokens") - val newSecretName = - s"${kubernetesConf.resourceNamePrefix}-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME" - val secretDT = - new SecretBuilder() - .withNewMetadata() - .withName(newSecretName) - .endMetadata() - .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(tokenData)) - .build() - KerberosConfigSpec( - dtSecret = Some(secretDT), - dtSecretName = newSecretName, - dtSecretItemKey = KERBEROS_SECRET_KEY, - jobUserName = jobUserUGI.getShortUserName) + override def getAdditionalPodSystemProperties(): Map[String, String] = { + // If a submission-local keytab is provided, update the Spark config so that it knows the + // path of the keytab in the driver container. + if (needKeytabUpload) { + val ktName = new File(keytab.get).getName() + Map(KEYTAB.key -> s"$KERBEROS_KEYTAB_MOUNT_POINT/$ktName") + } else { + Map.empty + } } - private case class KerberosConfigSpec( - dtSecret: Option[Secret], - dtSecretName: String, - dtSecretItemKey: String, - jobUserName: String) + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + Seq[HasMetadata]() ++ { + krb5File.map { path => + val file = new File(path) + new ConfigMapBuilder() + .withNewMetadata() + .withName(newConfigMapName) + .endMetadata() + .addToData( + Map(file.getName() -> Files.toString(file, StandardCharsets.UTF_8)).asJava) + .build() + } + } ++ { + // If a submission-local keytab is provided, stash it in a secret. + if (needKeytabUpload) { + val kt = new File(keytab.get) + Seq(new SecretBuilder() + .withNewMetadata() + .withName(ktSecretName) + .endMetadata() + .addToData(kt.getName(), Base64.encodeBase64String(Files.toByteArray(kt))) + .build()) + } else { + Nil + } + } ++ { + if (delegationTokens != null) { + Seq(new SecretBuilder() + .withNewMetadata() + .withName(dtSecretName) + .endMetadata() + .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(delegationTokens)) + .build()) + } else { + Nil + } + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala deleted file mode 100644 index 907271b1cb48..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala +++ /dev/null @@ -1,46 +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.deploy.k8s.features - -import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging - -/** - * This step is responsible for mounting the DT secret for the executors - */ -private[spark] class KerberosConfExecutorFeatureStep(conf: KubernetesExecutorConf) - extends KubernetesFeatureConfigStep with Logging { - - override def configurePod(pod: SparkPod): SparkPod = { - val maybeKrb5CMap = conf.getOption(KRB5_CONFIG_MAP_NAME) - if (maybeKrb5CMap.isDefined) { - logInfo(s"Mounting Resources for Kerberos") - HadoopBootstrapUtil.bootstrapKerberosPod( - conf.get(KERBEROS_DT_SECRET_NAME), - conf.get(KERBEROS_DT_SECRET_KEY), - conf.get(KERBEROS_SPARK_USER_NAME), - None, - None, - maybeKrb5CMap, - pod) - } else { - pod - } - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala deleted file mode 100644 index 5bee766caf2b..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ /dev/null @@ -1,283 +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.deploy.k8s.features.hadooputils - -import java.io.File -import java.nio.charset.StandardCharsets - -import scala.collection.JavaConverters._ - -import com.google.common.io.Files -import io.fabric8.kubernetes.api.model._ - -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.SparkPod -import org.apache.spark.internal.Logging - -private[spark] object HadoopBootstrapUtil extends Logging { - - /** - * Mounting the DT secret for both the Driver and the executors - * - * @param dtSecretName Name of the secret that stores the Delegation Token - * @param dtSecretItemKey Name of the Item Key storing the Delegation Token - * @param userName Name of the SparkUser to set SPARK_USER - * @param fileLocation Optional Location of the krb5 file - * @param newKrb5ConfName Optional location of the ConfigMap for Krb5 - * @param existingKrb5ConfName Optional name of ConfigMap for Krb5 - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapKerberosPod( - dtSecretName: String, - dtSecretItemKey: String, - userName: String, - fileLocation: Option[String], - newKrb5ConfName: Option[String], - existingKrb5ConfName: Option[String], - pod: SparkPod): SparkPod = { - - val preConfigMapVolume = existingKrb5ConfName.map { kconf => - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(kconf) - .endConfigMap() - .build() - } - - val createConfigMapVolume = for { - fLocation <- fileLocation - krb5ConfName <- newKrb5ConfName - } yield { - val krb5File = new File(fLocation) - val fileStringPath = krb5File.toPath.getFileName.toString - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(krb5ConfName) - .withItems(new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build()) - .endConfigMap() - .build() - } - - // Breaking up Volume creation for clarity - val configMapVolume = preConfigMapVolume.orElse(createConfigMapVolume) - if (configMapVolume.isEmpty) { - logInfo("You have not specified a krb5.conf file locally or via a ConfigMap. " + - "Make sure that you have the krb5.conf locally on the Driver and Executor images") - } - - val kerberizedPodWithDTSecret = new PodBuilder(pod.pod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(dtSecretName) - .endSecret() - .endVolume() - .endSpec() - .build() - - // Optionally add the krb5.conf ConfigMap - val kerberizedPod = configMapVolume.map { cmVolume => - new PodBuilder(kerberizedPodWithDTSecret) - .editSpec() - .addNewVolumeLike(cmVolume) - .endVolume() - .endSpec() - .build() - }.getOrElse(kerberizedPodWithDTSecret) - - val kerberizedContainerWithMounts = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$dtSecretItemKey") - .endEnv() - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(userName) - .endEnv() - .build() - - // Optionally add the krb5.conf Volume Mount - val kerberizedContainer = - if (configMapVolume.isDefined) { - new ContainerBuilder(kerberizedContainerWithMounts) - .addNewVolumeMount() - .withName(KRB_FILE_VOLUME) - .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") - .withSubPath("krb5.conf") - .endVolumeMount() - .build() - } else { - kerberizedContainerWithMounts - } - - SparkPod(kerberizedPod, kerberizedContainer) - } - - /** - * setting ENV_SPARK_USER when HADOOP_FILES are detected - * - * @param sparkUserName Name of the SPARK_USER - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod): SparkPod = { - val envModifiedContainer = new ContainerBuilder(pod.container) - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(sparkUserName) - .endEnv() - .build() - SparkPod(pod.pod, envModifiedContainer) - } - - /** - * Grabbing files in the HADOOP_CONF_DIR - * - * @param path location of HADOOP_CONF_DIR - * @return a list of File object - */ - def getHadoopConfFiles(path: String): Seq[File] = { - val dir = new File(path) - if (dir.isDirectory) { - dir.listFiles.filter(_.isFile).toSeq - } else { - Seq.empty[File] - } - } - - /** - * Bootstraping the container with ConfigMaps that store - * Hadoop configuration files - * - * @param hadoopConfDir directory location of HADOOP_CONF_DIR env - * @param newHadoopConfigMapName name of the new configMap for HADOOP_CONF_DIR - * @param existingHadoopConfigMapName name of the pre-defined configMap for HADOOP_CONF_DIR - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapHadoopConfDir( - hadoopConfDir: Option[String], - newHadoopConfigMapName: Option[String], - existingHadoopConfigMapName: Option[String], - pod: SparkPod): SparkPod = { - val preConfigMapVolume = existingHadoopConfigMapName.map { hConf => - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hConf) - .endConfigMap() - .build() } - - val createConfigMapVolume = for { - dirLocation <- hadoopConfDir - hConfName <- newHadoopConfigMapName - } yield { - val hadoopConfigFiles = getHadoopConfFiles(dirLocation) - val keyPaths = hadoopConfigFiles.map { file => - val fileStringPath = file.toPath.getFileName.toString - new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build() - } - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hConfName) - .withItems(keyPaths.asJava) - .endConfigMap() - .build() - } - - // Breaking up Volume Creation for clarity - val configMapVolume = preConfigMapVolume.getOrElse(createConfigMapVolume.get) - - val hadoopSupportedPod = new PodBuilder(pod.pod) - .editSpec() - .addNewVolumeLike(configMapVolume) - .endVolume() - .endSpec() - .build() - - val hadoopSupportedContainer = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(HADOOP_FILE_VOLUME) - .withMountPath(HADOOP_CONF_DIR_PATH) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_CONF_DIR) - .withValue(HADOOP_CONF_DIR_PATH) - .endEnv() - .build() - SparkPod(hadoopSupportedPod, hadoopSupportedContainer) - } - - /** - * Builds ConfigMap given the file location of the - * krb5.conf file - * - * @param configMapName name of configMap for krb5 - * @param fileLocation location of krb5 file - * @return a ConfigMap - */ - def buildkrb5ConfigMap( - configMapName: String, - fileLocation: String): ConfigMap = { - val file = new File(fileLocation) - new ConfigMapBuilder() - .withNewMetadata() - .withName(configMapName) - .endMetadata() - .addToData(Map(file.toPath.getFileName.toString -> - Files.toString(file, StandardCharsets.UTF_8)).asJava) - .build() - } - - /** - * Builds ConfigMap given the ConfigMap name - * and a list of Hadoop Conf files - * - * @param hadoopConfigMapName name of hadoopConfigMap - * @param hadoopConfFiles list of hadoopFiles - * @return a ConfigMap - */ - def buildHadoopConfigMap( - hadoopConfigMapName: String, - hadoopConfFiles: Seq[File]): ConfigMap = { - new ConfigMapBuilder() - .withNewMetadata() - .withName(hadoopConfigMapName) - .endMetadata() - .addToData(hadoopConfFiles.map { file => - (file.toPath.getFileName.toString, - Files.toString(file, StandardCharsets.UTF_8)) - }.toMap.asJava) - .build() - } - -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala deleted file mode 100644 index 7f7ef216cf48..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadooputils - -import io.fabric8.kubernetes.api.model.Secret - -/** - * Represents a given configuration of the Kerberos Configuration logic - *

    - * - The secret containing a DT, either previously specified or built on the fly - * - The name of the secret where the DT will be stored - * - The data item-key on the secret which correlates with where the current DT data is stored - * - The Job User's username - */ -private[spark] case class KerberosConfigSpec( - dtSecret: Option[Secret], - dtSecretName: String, - dtSecretItemKey: String, - jobUserName: String) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index d2c0ced9fa2f..57e4060bc85b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -46,6 +46,7 @@ private[spark] class KubernetesDriverBuilder { new LocalDirsFeatureStep(conf), new MountVolumesFeatureStep(conf), new DriverCommandFeatureStep(conf), + new HadoopConfDriverFeatureStep(conf), new KerberosConfDriverFeatureStep(conf), new PodTemplateConfigMapStep(conf)) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 03f5da2bb0bc..cd298971e02a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -25,6 +25,7 @@ import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.rpc.{RpcAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} @@ -143,7 +144,11 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { - new KubernetesDriverEndpoint(rpcEnv, properties) + new KubernetesDriverEndpoint(sc.env.rpcEnv, properties) + } + + override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { + Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration)) } private class KubernetesDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index 0b74966fe868..48aa2c56d4d6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -44,10 +44,7 @@ private[spark] class KubernetesExecutorBuilder { new MountSecretsFeatureStep(conf), new EnvSecretsFeatureStep(conf), new LocalDirsFeatureStep(conf), - new MountVolumesFeatureStep(conf), - new HadoopConfExecutorFeatureStep(conf), - new KerberosConfExecutorFeatureStep(conf), - new HadoopSparkUserExecutorFeatureStep(conf)) + new MountVolumesFeatureStep(conf)) features.foldLeft(initialPod) { case (pod, feature) => feature.configurePod(pod) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index e4951bc1e69e..5ceb9d6d6fcd 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config._ import org.apache.spark.ui.SparkUI +import org.apache.spark.util.Utils class BasicDriverFeatureStepSuite extends SparkFunSuite { @@ -73,7 +74,6 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val foundPortNames = configuredPod.container.getPorts.asScala.toSet assert(expectedPortNames === foundPortNames) - assert(configuredPod.container.getEnv.size === 3) val envs = configuredPod.container .getEnv .asScala @@ -82,6 +82,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { DRIVER_ENVS.foreach { case (k, v) => assert(envs(v) === v) } + assert(envs(ENV_SPARK_USER) === Utils.getCurrentUserName()) assert(configuredPod.pod.getSpec().getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 05989d9be7ad..c2efab01e424 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -200,7 +200,8 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { ENV_EXECUTOR_MEMORY -> "1g", ENV_APPLICATION_ID -> KubernetesTestConf.APP_ID, ENV_SPARK_CONF_DIR -> SPARK_CONF_DIR_INTERNAL, - ENV_EXECUTOR_POD_IP -> null) ++ additionalEnvVars + ENV_EXECUTOR_POD_IP -> null, + ENV_SPARK_USER -> Utils.getCurrentUserName()) val extraJavaOptsStart = additionalEnvVars.keys.count(_.startsWith(ENV_JAVA_OPT_PREFIX)) val extraJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf) @@ -208,9 +209,11 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { s"$ENV_JAVA_OPT_PREFIX${ind + extraJavaOptsStart}" -> opt }.toMap - val mapEnvs = executorPod.container.getEnv.asScala.map { + val containerEnvs = executorPod.container.getEnv.asScala.map { x => (x.getName, x.getValue) }.toMap - assert((defaultEnvs ++ extraJavaOptsEnvs) === mapEnvs) + + val expectedEnvs = defaultEnvs ++ additionalEnvVars ++ extraJavaOptsEnvs + assert(containerEnvs === expectedEnvs) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala new file mode 100644 index 000000000000..e1c01dbdc735 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.ConfigMap + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.util.{SparkConfWithEnv, Utils} + +class HadoopConfDriverFeatureStepSuite extends SparkFunSuite { + + import KubernetesFeaturesTestUtils._ + import SecretVolumeUtils._ + + test("mount hadoop config map if defined") { + val sparkConf = new SparkConf(false) + .set(Config.KUBERNETES_HADOOP_CONF_CONFIG_MAP, "testConfigMap") + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) + val step = new HadoopConfDriverFeatureStep(conf) + checkPod(step.configurePod(SparkPod.initialPod())) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("create hadoop config map if config dir is defined") { + val confDir = Utils.createTempDir() + val confFiles = Set("core-site.xml", "hdfs-site.xml") + + confFiles.foreach { f => + Files.write("some data", new File(confDir, f), UTF_8) + } + + val sparkConf = new SparkConfWithEnv(Map(ENV_HADOOP_CONF_DIR -> confDir.getAbsolutePath())) + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) + + val step = new HadoopConfDriverFeatureStep(conf) + checkPod(step.configurePod(SparkPod.initialPod())) + + val hadoopConfMap = filter[ConfigMap](step.getAdditionalKubernetesResources()).head + assert(hadoopConfMap.getData().keySet().asScala === confFiles) + } + + private def checkPod(pod: SparkPod): Unit = { + assert(podHasVolume(pod.pod, HADOOP_CONF_VOLUME)) + assert(containerHasVolume(pod.container, HADOOP_CONF_VOLUME, HADOOP_CONF_DIR_PATH)) + assert(containerHasEnvVar(pod.container, ENV_HADOOP_CONF_DIR)) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala new file mode 100644 index 000000000000..41ca3a94ce7a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 +import java.security.PrivilegedExceptionAction + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.{ConfigMap, Secret} +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils + +class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { + + import KubernetesFeaturesTestUtils._ + import SecretVolumeUtils._ + + private val tmpDir = Utils.createTempDir() + + test("mount krb5 config map if defined") { + val configMap = "testConfigMap" + val step = createStep( + new SparkConf(false).set(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP, configMap)) + + checkPodForKrbConf(step.configurePod(SparkPod.initialPod()), configMap) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(filter[ConfigMap](step.getAdditionalKubernetesResources()).isEmpty) + } + + test("create krb5.conf config map if local config provided") { + val krbConf = File.createTempFile("krb5", ".conf", tmpDir) + Files.write("some data", krbConf, UTF_8) + + val sparkConf = new SparkConf(false) + .set(KUBERNETES_KERBEROS_KRB5_FILE, krbConf.getAbsolutePath()) + val step = createStep(sparkConf) + + val confMap = filter[ConfigMap](step.getAdditionalKubernetesResources()).head + assert(confMap.getData().keySet().asScala === Set(krbConf.getName())) + + checkPodForKrbConf(step.configurePod(SparkPod.initialPod()), confMap.getMetadata().getName()) + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + + test("create keytab secret if client keytab file used") { + val keytab = File.createTempFile("keytab", ".bin", tmpDir) + Files.write("some data", keytab, UTF_8) + + val sparkConf = new SparkConf(false) + .set(KEYTAB, keytab.getAbsolutePath()) + .set(PRINCIPAL, "alice") + val step = createStep(sparkConf) + + val pod = step.configurePod(SparkPod.initialPod()) + assert(podHasVolume(pod.pod, KERBEROS_KEYTAB_VOLUME)) + assert(containerHasVolume(pod.container, KERBEROS_KEYTAB_VOLUME, KERBEROS_KEYTAB_MOUNT_POINT)) + + assert(step.getAdditionalPodSystemProperties().keys === Set(KEYTAB.key)) + + val secret = filter[Secret](step.getAdditionalKubernetesResources()).head + assert(secret.getData().keySet().asScala === Set(keytab.getName())) + } + + test("do nothing if container-local keytab used") { + val sparkConf = new SparkConf(false) + .set(KEYTAB, "local:/my.keytab") + .set(PRINCIPAL, "alice") + val step = createStep(sparkConf) + + val initial = SparkPod.initialPod() + assert(step.configurePod(initial) === initial) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("mount delegation tokens if provided") { + val dtSecret = "tokenSecret" + val sparkConf = new SparkConf(false) + .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, dtSecret) + .set(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY, "dtokens") + val step = createStep(sparkConf) + + checkPodForTokens(step.configurePod(SparkPod.initialPod()), dtSecret) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("create delegation tokens if needed") { + // Since HadoopDelegationTokenManager does not create any tokens without proper configs and + // services, start with a test user that already has some tokens that will just be piped + // through to the driver. + val testUser = UserGroupInformation.createUserForTesting("k8s", Array()) + testUser.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val creds = testUser.getCredentials() + creds.addSecretKey(new Text("K8S_TEST_KEY"), Array[Byte](0x4, 0x2)) + testUser.addCredentials(creds) + + val tokens = SparkHadoopUtil.get.serialize(creds) + + val step = createStep(new SparkConf(false)) + + val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()).head + assert(dtSecret.getData().get(KERBEROS_SECRET_KEY) === Base64.encodeBase64String(tokens)) + + checkPodForTokens(step.configurePod(SparkPod.initialPod()), + dtSecret.getMetadata().getName()) + + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + }) + } + + test("do nothing if no config and no tokens") { + val step = createStep(new SparkConf(false)) + val initial = SparkPod.initialPod() + assert(step.configurePod(initial) === initial) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + private def checkPodForKrbConf(pod: SparkPod, confMapName: String): Unit = { + val podVolume = pod.pod.getSpec().getVolumes().asScala.find(_.getName() == KRB_FILE_VOLUME) + assert(podVolume.isDefined) + assert(containerHasVolume(pod.container, KRB_FILE_VOLUME, KRB_FILE_DIR_PATH + "/krb5.conf")) + assert(podVolume.get.getConfigMap().getName() === confMapName) + } + + private def checkPodForTokens(pod: SparkPod, dtSecretName: String): Unit = { + val podVolume = pod.pod.getSpec().getVolumes().asScala + .find(_.getName() == SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + assert(podVolume.isDefined) + assert(containerHasVolume(pod.container, SPARK_APP_HADOOP_SECRET_VOLUME_NAME, + SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)) + assert(containerHasEnvVar(pod.container, ENV_HADOOP_TOKEN_FILE_LOCATION)) + assert(podVolume.get.getSecret().getSecretName() === dtSecretName) + } + + private def createStep(conf: SparkConf): KerberosConfDriverFeatureStep = { + val kconf = KubernetesTestConf.createDriverConf(sparkConf = conf) + new KerberosConfDriverFeatureStep(kconf) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala index f90380e30e52..076b681be239 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.k8s.features import scala.collection.JavaConverters._ +import scala.reflect.ClassTag import io.fabric8.kubernetes.api.model.{Container, HasMetadata, PodBuilder, SecretBuilder} import org.mockito.Matchers @@ -63,4 +64,9 @@ object KubernetesFeaturesTestUtils { def containerHasEnvVar(container: Container, envVarName: String): Boolean = { container.getEnv.asScala.exists(envVar => envVar.getName == envVarName) } + + def filter[T: ClassTag](list: Seq[HasMetadata]): Seq[T] = { + val desired = implicitly[ClassTag[T]].runtimeClass + list.filter(_.getClass() == desired).map(_.asInstanceOf[T]).toSeq + } } 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 6240f7b68d2c..184fb6a8ad13 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 @@ -116,6 +116,8 @@ private[spark] class Client( } } + require(keytab == null || !Utils.isLocalUri(keytab), "Keytab should reference a local file.") + private val launcherBackend = new LauncherBackend() { override protected def conf: SparkConf = sparkConf @@ -472,7 +474,7 @@ private[spark] class Client( appMasterOnly: Boolean = false): (Boolean, String) = { val trimmedPath = path.trim() val localURI = Utils.resolveURI(trimmedPath) - if (localURI.getScheme != LOCAL_SCHEME) { + if (localURI.getScheme != Utils.LOCAL_SCHEME) { if (addDistributedUri(localURI)) { val localPath = getQualifiedLocalPath(localURI, hadoopConf) val linkname = targetDir.map(_ + "/").getOrElse("") + @@ -515,7 +517,7 @@ private[spark] class Client( val sparkArchive = sparkConf.get(SPARK_ARCHIVE) if (sparkArchive.isDefined) { val archive = sparkArchive.get - require(!isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.") + require(!Utils.isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.") distribute(Utils.resolveURI(archive).toString, resType = LocalResourceType.ARCHIVE, destName = Some(LOCALIZED_LIB_DIR)) @@ -525,7 +527,7 @@ private[spark] class Client( // Break the list of jars to upload, and resolve globs. val localJars = new ArrayBuffer[String]() jars.foreach { jar => - if (!isLocalUri(jar)) { + if (!Utils.isLocalUri(jar)) { val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf) val pathFs = FileSystem.get(path.toUri(), hadoopConf) pathFs.globStatus(path).filter(_.isFile()).foreach { entry => @@ -814,7 +816,7 @@ private[spark] class Client( } (pySparkArchives ++ pyArchives).foreach { path => val uri = Utils.resolveURI(path) - if (uri.getScheme != LOCAL_SCHEME) { + if (uri.getScheme != Utils.LOCAL_SCHEME) { pythonPath += buildPath(Environment.PWD.$$(), new Path(uri).getName()) } else { pythonPath += uri.getPath() @@ -1183,9 +1185,6 @@ private object Client extends Logging { // Alias for the user jar val APP_JAR_NAME: String = "__app__.jar" - // URI scheme that identifies local resources - val LOCAL_SCHEME = "local" - // Staging directory for any temporary jars or files val SPARK_STAGING: String = ".sparkStaging" @@ -1307,7 +1306,7 @@ private object Client extends Logging { addClasspathEntry(buildPath(Environment.PWD.$$(), LOCALIZED_LIB_DIR, "*"), env) if (sparkConf.get(SPARK_ARCHIVE).isEmpty) { sparkConf.get(SPARK_JARS).foreach { jars => - jars.filter(isLocalUri).foreach { jar => + jars.filter(Utils.isLocalUri).foreach { jar => val uri = new URI(jar) addClasspathEntry(getClusterPath(sparkConf, uri.getPath()), env) } @@ -1340,7 +1339,7 @@ private object Client extends Logging { private def getMainJarUri(mainJar: Option[String]): Option[URI] = { mainJar.flatMap { path => val uri = Utils.resolveURI(path) - if (uri.getScheme == LOCAL_SCHEME) Some(uri) else None + if (uri.getScheme == Utils.LOCAL_SCHEME) Some(uri) else None }.orElse(Some(new URI(APP_JAR_NAME))) } @@ -1368,7 +1367,7 @@ private object Client extends Logging { uri: URI, fileName: String, env: HashMap[String, String]): Unit = { - if (uri != null && uri.getScheme == LOCAL_SCHEME) { + if (uri != null && uri.getScheme == Utils.LOCAL_SCHEME) { addClasspathEntry(getClusterPath(conf, uri.getPath), env) } else if (fileName != null) { addClasspathEntry(buildPath(Environment.PWD.$$(), fileName), env) @@ -1489,11 +1488,6 @@ private object Client extends Logging { components.mkString(Path.SEPARATOR) } - /** Returns whether the URI is a "local:" URI. */ - def isLocalUri(uri: String): Boolean = { - uri.startsWith(s"$LOCAL_SCHEME:") - } - def createAppReport(report: ApplicationReport): YarnAppReport = { val diags = report.getDiagnostics() val diagsOpt = if (diags != null && diags.nonEmpty) Some(diags) else None 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 b3286e8fd824..a6f57fcdb246 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 @@ -100,7 +100,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val cp = env("CLASSPATH").split(":|;|") s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => val uri = new URI(entry) - if (LOCAL_SCHEME.equals(uri.getScheme())) { + if (Utils.LOCAL_SCHEME.equals(uri.getScheme())) { cp should contain (uri.getPath()) } else { cp should not contain (uri.getPath()) @@ -136,7 +136,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val expected = ADDED.split(",") .map(p => { val uri = new URI(p) - if (LOCAL_SCHEME == uri.getScheme()) { + if (Utils.LOCAL_SCHEME == uri.getScheme()) { p } else { Option(uri.getFragment()).getOrElse(new File(p).getName()) @@ -249,7 +249,7 @@ class ClientSuite extends SparkFunSuite with Matchers { any(classOf[MutableHashMap[URI, Path]]), anyBoolean(), any()) classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) - sparkConf.set(SPARK_ARCHIVE, LOCAL_SCHEME + ":" + archive.getPath()) + sparkConf.set(SPARK_ARCHIVE, Utils.LOCAL_SCHEME + ":" + archive.getPath()) intercept[IllegalArgumentException] { client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil) } From 834b8609793525a5a486013732d8c98e1c6e6504 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Tue, 18 Dec 2018 23:21:52 -0800 Subject: [PATCH 0164/1072] [SPARK-26366][SQL] ReplaceExceptWithFilter should consider NULL as False ## What changes were proposed in this pull request? In `ReplaceExceptWithFilter` we do not consider properly the case in which the condition returns NULL. Indeed, in that case, since negating NULL still returns NULL, so it is not true the assumption that negating the condition returns all the rows which didn't satisfy it, rows returning NULL may not be returned. This happens when constraints inferred by `InferFiltersFromConstraints` are not enough, as it happens with `OR` conditions. The rule had also problems with non-deterministic conditions: in such a scenario, this rule would change the probability of the output. The PR fixes these problem by: - returning False for the condition when it is Null (in this way we do return all the rows which didn't satisfy it); - avoiding any transformation when the condition is non-deterministic. ## How was this patch tested? added UTs Closes #23315 from mgaido91/SPARK-26366. Authored-by: Marco Gaido Signed-off-by: gatorsmile --- .../optimizer/ReplaceExceptWithFilter.scala | 32 ++++++++------ .../optimizer/ReplaceOperatorSuite.scala | 44 ++++++++++++++----- .../org/apache/spark/sql/DatasetSuite.scala | 11 +++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 38 ++++++++++++++++ 4 files changed, 101 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala index efd3944eba7f..4996d24dfd29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.catalyst.rules.Rule * Note: * Before flipping the filter condition of the right node, we should: * 1. Combine all it's [[Filter]]. - * 2. Apply InferFiltersFromConstraints rule (to take into account of NULL values in the condition). + * 2. Update the attribute references to the left node; + * 3. Add a Coalesce(condition, False) (to take into account of NULL values in the condition). */ object ReplaceExceptWithFilter extends Rule[LogicalPlan] { @@ -47,23 +48,28 @@ object ReplaceExceptWithFilter extends Rule[LogicalPlan] { plan.transform { case e @ Except(left, right, false) if isEligible(left, right) => - val newCondition = transformCondition(left, skipProject(right)) - newCondition.map { c => - Distinct(Filter(Not(c), left)) - }.getOrElse { + val filterCondition = combineFilters(skipProject(right)).asInstanceOf[Filter].condition + if (filterCondition.deterministic) { + transformCondition(left, filterCondition).map { c => + Distinct(Filter(Not(c), left)) + }.getOrElse { + e + } + } else { e } } } - private def transformCondition(left: LogicalPlan, right: LogicalPlan): Option[Expression] = { - val filterCondition = - InferFiltersFromConstraints(combineFilters(right)).asInstanceOf[Filter].condition - - val attributeNameMap: Map[String, Attribute] = left.output.map(x => (x.name, x)).toMap - - if (filterCondition.references.forall(r => attributeNameMap.contains(r.name))) { - Some(filterCondition.transform { case a: AttributeReference => attributeNameMap(a.name) }) + private def transformCondition(plan: LogicalPlan, condition: Expression): Option[Expression] = { + val attributeNameMap: Map[String, Attribute] = plan.output.map(x => (x.name, x)).toMap + if (condition.references.forall(r => attributeNameMap.contains(r.name))) { + val rewrittenCondition = condition.transform { + case a: AttributeReference => attributeNameMap(a.name) + } + // We need to consider as False when the condition is NULL, otherwise we do not return those + // rows containing NULL which are instead filtered in the Except right plan + Some(Coalesce(Seq(rewrittenCondition, Literal.FalseLiteral))) } else { None } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala index 3b1b2d588ef6..c8e15c7da763 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.Row 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, Not} +import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, If, Literal, Not} import org.apache.spark.sql.catalyst.expressions.aggregate.First import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi, PlanTest} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types.BooleanType class ReplaceOperatorSuite extends PlanTest { @@ -65,8 +66,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Filter(attributeB === 2, Filter(attributeA === 1, table1)))).analyze comparePlans(optimized, correctAnswer) @@ -84,8 +84,8 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), table1)).analyze + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), + table1)).analyze comparePlans(optimized, correctAnswer) } @@ -104,8 +104,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Project(Seq(attributeA, attributeB), table1))).analyze comparePlans(optimized, correctAnswer) @@ -125,8 +124,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Filter(attributeB === 2, Filter(attributeA === 1, table1)))).analyze comparePlans(optimized, correctAnswer) @@ -146,8 +144,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA === 1 && attributeB === 2)), + Filter(Not(Coalesce(Seq(attributeA === 1 && attributeB === 2, Literal.FalseLiteral))), Project(Seq(attributeA, attributeB), Filter(attributeB < 1, Filter(attributeA >= 2, table1))))).analyze @@ -229,4 +226,29 @@ class ReplaceOperatorSuite extends PlanTest { comparePlans(optimized, query) } + + test("SPARK-26366: ReplaceExceptWithFilter should handle properly NULL") { + val basePlan = LocalRelation(Seq('a.int, 'b.int)) + val otherPlan = basePlan.where('a.in(1, 2) || 'b.in()) + val except = Except(basePlan, otherPlan, false) + val result = OptimizeIn(Optimize.execute(except.analyze)) + val correctAnswer = Aggregate(basePlan.output, basePlan.output, + Filter(!Coalesce(Seq( + 'a.in(1, 2) || If('b.isNotNull, Literal.FalseLiteral, Literal(null, BooleanType)), + Literal.FalseLiteral)), + basePlan)).analyze + comparePlans(result, correctAnswer) + } + + test("SPARK-26366: ReplaceExceptWithFilter should not transform non-detrministic") { + val basePlan = LocalRelation(Seq('a.int, 'b.int)) + val otherPlan = basePlan.where('a > rand(1L)) + val except = Except(basePlan, otherPlan, false) + val result = Optimize.execute(except.analyze) + val condition = basePlan.output.zip(otherPlan.output).map { case (a1, a2) => + a1 <=> a2 }.reduce( _ && _) + val correctAnswer = Aggregate(basePlan.output, otherPlan.output, + Join(basePlan, otherPlan, LeftAnti, Option(condition))).analyze + comparePlans(result, correctAnswer) + } } 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 525c7cef3956..c90b15814a53 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 @@ -1656,6 +1656,17 @@ class DatasetSuite extends QueryTest with SharedSQLContext { checkAnswer(df.groupBy(col("a")).agg(first(col("b"))), Seq(Row("0", BigDecimal.valueOf(0.1111)), Row("1", BigDecimal.valueOf(1.1111)))) } + + test("SPARK-26366: return nulls which are not filtered in except") { + val inputDF = sqlContext.createDataFrame( + sparkContext.parallelize(Seq(Row("0", "a"), Row("1", null))), + StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = true)))) + + val exceptDF = inputDF.filter(col("a").isin("0") or col("b") > "c") + checkAnswer(inputDF.except(exceptDF), Seq(Row("1", null))) + } } case class TestDataUnion(x: Int, y: Int, z: Int) 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 4cc8a4539199..37a8815350a5 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 @@ -2899,6 +2899,44 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-26366: verify ReplaceExceptWithFilter") { + Seq(true, false).foreach { enabled => + withSQLConf(SQLConf.REPLACE_EXCEPT_WITH_FILTER.key -> enabled.toString) { + val df = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(0, 3, 5), + Row(0, 3, null), + Row(null, 3, 5), + Row(0, null, 5), + Row(0, null, null), + Row(null, null, 5), + Row(null, 3, null), + Row(null, null, null))), + StructType(Seq(StructField("c1", IntegerType), + StructField("c2", IntegerType), + StructField("c3", IntegerType)))) + val where = "c2 >= 3 OR c1 >= 0" + val whereNullSafe = + """ + |(c2 IS NOT NULL AND c2 >= 3) + |OR (c1 IS NOT NULL AND c1 >= 0) + """.stripMargin + + val df_a = df.filter(where) + val df_b = df.filter(whereNullSafe) + checkAnswer(df.except(df_a), df.except(df_b)) + + val whereWithIn = "c2 >= 3 OR c1 in (2)" + val whereWithInNullSafe = + """ + |(c2 IS NOT NULL AND c2 >= 3) + """.stripMargin + val dfIn_a = df.filter(whereWithIn) + val dfIn_b = df.filter(whereWithInNullSafe) + checkAnswer(df.except(dfIn_a), df.except(dfIn_b)) + } + } + } } case class Foo(bar: Option[String]) From 08f74ada3656af401099aa79471ef8a1155a3f07 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 19 Dec 2018 09:41:30 -0800 Subject: [PATCH 0165/1072] [SPARK-26390][SQL] ColumnPruning rule should only do column pruning ## What changes were proposed in this pull request? This is a small clean up. By design catalyst rules should be orthogonal: each rule should have its own responsibility. However, the `ColumnPruning` rule does not only do column pruning, but also remove no-op project and window. This PR updates the `RemoveRedundantProject` rule to remove no-op window as well, and clean up the `ColumnPruning` rule to only do column pruning. ## How was this patch tested? existing tests Closes #23343 from cloud-fan/column-pruning. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/Optimizer.scala | 23 +++++++++---------- .../optimizer/ColumnPruningSuite.scala | 7 +++--- .../optimizer/CombiningLimitsSuite.scala | 5 ++-- .../optimizer/JoinOptimizationSuite.scala | 1 + .../RemoveRedundantAliasAndProjectSuite.scala | 2 +- .../optimizer/RewriteSubquerySuite.scala | 2 +- .../optimizer/TransposeWindowSuite.scala | 2 +- 7 files changed, 21 insertions(+), 21 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 3eb6bca6ec97..44d554311490 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 @@ -93,7 +93,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) RewriteCorrelatedScalarSubquery, EliminateSerialization, RemoveRedundantAliases, - RemoveRedundantProject, + RemoveNoopOperators, SimplifyExtractValueOps, CombineConcats) ++ extendedOperatorOptimizationRules @@ -177,7 +177,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) RewritePredicateSubquery, ColumnPruning, CollapseProject, - RemoveRedundantProject) :+ + RemoveNoopOperators) :+ Batch("UpdateAttributeReferences", Once, UpdateNullabilityInAttributeReferences) } @@ -403,11 +403,15 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { } /** - * Remove projections from the query plan that do not make any modifications. + * Remove no-op operators from the query plan that do not make any modifications. */ -object RemoveRedundantProject extends Rule[LogicalPlan] { +object RemoveNoopOperators extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p @ Project(_, child) if p.output == child.output => child + // Eliminate no-op Projects + case p @ Project(_, child) if child.sameOutput(p) => child + + // Eliminate no-op Window + case w: Window if w.windowExpressions.isEmpty => w.child } } @@ -602,17 +606,12 @@ object ColumnPruning extends Rule[LogicalPlan] { p.copy(child = w.copy( windowExpressions = w.windowExpressions.filter(p.references.contains))) - // Eliminate no-op Window - case w: Window if w.windowExpressions.isEmpty => w.child - - // Eliminate no-op Projects - case p @ Project(_, child) if child.sameOutput(p) => child - // Can't prune the columns on LeafNode case p @ Project(_, _: LeafNode) => p // for all other logical plans that inherits the output from it's children - case p @ Project(_, child) => + // Project over project is handled by the first case, skip it here. + case p @ Project(_, child) if !child.isInstanceOf[Project] => val required = child.references ++ p.references if (!child.inputSet.subsetOf(required)) { val newChildren = child.children.map(c => prunedChild(c, required)) 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 8d7c9bf220bc..57195d5fda7c 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 @@ -34,6 +34,7 @@ class ColumnPruningSuite extends PlanTest { val batches = Batch("Column pruning", FixedPoint(100), PushDownPredicate, ColumnPruning, + RemoveNoopOperators, CollapseProject) :: Nil } @@ -340,10 +341,8 @@ class ColumnPruningSuite extends PlanTest { test("Column pruning on Union") { val input1 = LocalRelation('a.int, 'b.string, 'c.double) val input2 = LocalRelation('c.int, 'd.string, 'e.double) - val query = Project('b :: Nil, - Union(input1 :: input2 :: Nil)).analyze - val expected = Project('b :: Nil, - Union(Project('b :: Nil, input1) :: Project('d :: Nil, input2) :: Nil)).analyze + val query = Project('b :: Nil, Union(input1 :: input2 :: Nil)).analyze + val expected = Union(Project('b :: Nil, input1) :: Project('d :: Nil, input2) :: Nil).analyze comparePlans(Optimize.execute(query), expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index ef4b848924f0..b190dd5a7c22 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -27,8 +27,9 @@ class CombiningLimitsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("Filter Pushdown", FixedPoint(100), - ColumnPruning) :: + Batch("Column Pruning", FixedPoint(100), + ColumnPruning, + RemoveNoopOperators) :: Batch("Combine Limit", FixedPoint(10), CombineLimits) :: Batch("Constant Folding", FixedPoint(10), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala index e9438b2eee55..6fe5e619d03a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala @@ -39,6 +39,7 @@ class JoinOptimizationSuite extends PlanTest { ReorderJoin, PushPredicateThroughJoin, ColumnPruning, + RemoveNoopOperators, CollapseProject) :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala index 1973b5abb462..3802dbf5d6e0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala @@ -33,7 +33,7 @@ class RemoveRedundantAliasAndProjectSuite extends PlanTest with PredicateHelper FixedPoint(50), PushProjectionThroughUnion, RemoveRedundantAliases, - RemoveRedundantProject) :: Nil + RemoveNoopOperators) :: Nil } test("all expressions in project list are aliased child output") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala index 6b3739c372c3..f00d22e6e96a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala @@ -34,7 +34,7 @@ class RewriteSubquerySuite extends PlanTest { RewritePredicateSubquery, ColumnPruning, CollapseProject, - RemoveRedundantProject) :: Nil + RemoveNoopOperators) :: Nil } test("Column pruning after rewriting predicate subquery") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala index 58b3d1c98f3c..4acd57832d2f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor class TransposeWindowSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("CollapseProject", FixedPoint(100), CollapseProject, RemoveRedundantProject) :: + Batch("CollapseProject", FixedPoint(100), CollapseProject, RemoveNoopOperators) :: Batch("FlipWindow", Once, CollapseWindow, TransposeWindow) :: Nil } From 61c443acd23c74ebcb20fd32e5e0ed6c1722b5dc Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 20 Dec 2018 10:41:45 +0800 Subject: [PATCH 0166/1072] [SPARK-26262][SQL] Runs SQLQueryTestSuite on mixed config sets: WHOLESTAGE_CODEGEN_ENABLED and CODEGEN_FACTORY_MODE ## What changes were proposed in this pull request? For better test coverage, this pr proposed to use the 4 mixed config sets of `WHOLESTAGE_CODEGEN_ENABLED` and `CODEGEN_FACTORY_MODE` when running `SQLQueryTestSuite`: 1. WHOLESTAGE_CODEGEN_ENABLED=true, CODEGEN_FACTORY_MODE=CODEGEN_ONLY 2. WHOLESTAGE_CODEGEN_ENABLED=false, CODEGEN_FACTORY_MODE=CODEGEN_ONLY 3. WHOLESTAGE_CODEGEN_ENABLED=true, CODEGEN_FACTORY_MODE=NO_CODEGEN 4. WHOLESTAGE_CODEGEN_ENABLED=false, CODEGEN_FACTORY_MODE=NO_CODEGEN This pr also moved some existing tests into `ExplainSuite` because explain output results are different between codegen and interpreter modes. ## How was this patch tested? Existing tests. Closes #23213 from maropu/InterpreterModeTest. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../resources/sql-tests/inputs/group-by.sql | 5 - .../sql-tests/inputs/inline-table.sql | 3 - .../resources/sql-tests/inputs/operators.sql | 21 -- .../inputs/sql-compatibility-functions.sql | 5 - .../sql-tests/inputs/string-functions.sql | 27 --- .../inputs/table-valued-functions.sql | 6 - .../sql-tests/results/group-by.sql.out | 30 +-- .../sql-tests/results/inline-table.sql.out | 32 +-- .../sql-tests/results/operators.sql.out | 204 +++++++----------- .../sql-compatibility-functions.sql.out | 61 ++---- .../results/string-functions.sql.out | 131 +++-------- .../results/table-valued-functions.sql.out | 41 +--- .../org/apache/spark/sql/ExplainSuite.scala | 133 +++++++++++- .../apache/spark/sql/SQLQueryTestSuite.scala | 51 ++--- 14 files changed, 281 insertions(+), 469 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index ec263ea70bd4..7e81ff1aba37 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -141,8 +141,3 @@ SELECT every("true"); SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; - --- simple explain of queries having every/some/any agregates. Optimized --- plan should show the rewritten aggregate expression. -EXPLAIN EXTENDED SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k; - diff --git a/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql index 41d316444ed6..b3ec956cd178 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql @@ -49,6 +49,3 @@ select * from values ("one", count(1)), ("two", 2) as data(a, b); -- string to timestamp select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b); - --- cross-join inline tables -EXPLAIN EXTENDED SELECT * FROM VALUES ('one', 1), ('three', null) CROSS JOIN VALUES ('one', 1), ('three', null); diff --git a/sql/core/src/test/resources/sql-tests/inputs/operators.sql b/sql/core/src/test/resources/sql-tests/inputs/operators.sql index 37f9cd44da7f..ba14789d48db 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/operators.sql @@ -29,27 +29,6 @@ select 2 * 5; select 5 % 3; select pmod(-7, 3); --- check operator precedence. --- We follow Oracle operator precedence in the table below that lists the levels of precedence --- among SQL operators from high to low: ------------------------------------------------------------------------------------------- --- Operator Operation ------------------------------------------------------------------------------------------- --- +, - identity, negation --- *, / multiplication, division --- +, -, || addition, subtraction, concatenation --- =, !=, <, >, <=, >=, IS NULL, LIKE, BETWEEN, IN comparison --- NOT exponentiation, logical negation --- AND conjunction --- OR disjunction ------------------------------------------------------------------------------------------- -explain select 'a' || 1 + 2; -explain select 1 - 2 || 'b'; -explain select 2 * 4 + 3 || 'b'; -explain select 3 + 1 || 'a' || 4 / 2; -explain select 1 == 1 OR 'a' || 'b' == 'ab'; -explain select 'a' || 'c' == 'ac' AND 2 == 3; - -- math functions select cot(1); select cot(null); diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql index f1461032065a..1ae49c8bfc76 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql @@ -12,11 +12,6 @@ SELECT nullif(1, 2.1d), nullif(1, 1.0d); SELECT nvl(1, 2.1d), nvl(null, 2.1d); SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d); --- explain for these functions; use range to avoid constant folding -explain extended -select ifnull(id, 'x'), nullif(id, 'x'), nvl(id, 'x'), nvl2(id, 'x', 'y') -from range(2); - -- SPARK-16730 cast alias functions for Hive compatibility SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1); SELECT float(1), double(1), decimal(1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql index 2effb43183d7..fbc231627e36 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql @@ -5,10 +5,6 @@ select format_string(); -- A pipe operator for string concatenation select 'a' || 'b' || 'c'; --- Check if catalyst combine nested `Concat`s -EXPLAIN EXTENDED SELECT (col1 || col2 || col3 || col4) col -FROM (SELECT id col1, id col2, id col3, id col4 FROM range(10)); - -- replace function select replace('abc', 'b', '123'); select replace('abc', 'b'); @@ -25,29 +21,6 @@ select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a'); select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null); select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a'); --- turn off concatBinaryAsString -set spark.sql.function.concatBinaryAsString=false; - --- Check if catalyst combine nested `Concat`s if concatBinaryAsString=false -EXPLAIN SELECT ((col1 || col2) || (col3 || col4)) col -FROM ( - SELECT - string(id) col1, - string(id + 1) col2, - encode(string(id + 2), 'utf-8') col3, - encode(string(id + 3), 'utf-8') col4 - FROM range(10) -); - -EXPLAIN SELECT (col1 || (col3 || col4)) col -FROM ( - SELECT - string(id) col1, - encode(string(id + 2), 'utf-8') col3, - encode(string(id + 3), 'utf-8') col4 - FROM range(10) -); - -- split function SELECT split('aa1cc2ee3', '[1-9]+'); SELECT split('aa1cc2ee3', '[1-9]+', 2); diff --git a/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql index 72cd8ca9d872..6f14c8ca8782 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql @@ -21,9 +21,3 @@ select * from range(1, null); -- range call with a mixed-case function name select * from RaNgE(2); - --- Explain -EXPLAIN select * from RaNgE(2); - --- cross-join table valued functions -EXPLAIN EXTENDED SELECT * FROM range(3) CROSS JOIN range(3); diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 9a8d025331b6..daf47c4d0a39 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 47 +-- Number of queries: 46 -- !query 0 @@ -459,31 +459,3 @@ struct --- !query 46 output -== Parsed Logical Plan == -'Aggregate ['k], ['k, unresolvedalias('every('v), None), unresolvedalias('some('v), None), unresolvedalias('any('v), None)] -+- 'UnresolvedRelation `test_agg` - -== Analyzed Logical Plan == -k: int, every(v): boolean, some(v): boolean, any(v): boolean -Aggregate [k#x], [k#x, every(v#x) AS every(v)#x, some(v#x) AS some(v)#x, any(v#x) AS any(v)#x] -+- SubqueryAlias `test_agg` - +- Project [k#x, v#x] - +- SubqueryAlias `test_agg` - +- LocalRelation [k#x, v#x] - -== Optimized Logical Plan == -Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS some(v)#x, max(v#x) AS any(v)#x] -+- LocalRelation [k#x, v#x] - -== Physical Plan == -*HashAggregate(keys=[k#x], functions=[min(v#x), max(v#x)], output=[k#x, every(v)#x, some(v)#x, any(v)#x]) -+- Exchange hashpartitioning(k#x, 200) - +- *HashAggregate(keys=[k#x], functions=[partial_min(v#x), partial_max(v#x)], output=[k#x, min#x, max#x]) - +- LocalTableScan [k#x, v#x] 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 c065ce501292..4e80f0bda551 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 18 +-- Number of queries: 17 -- !query 0 @@ -151,33 +151,3 @@ select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991- struct> -- !query 16 output 1991-12-06 00:00:00 [1991-12-06 01:00:00.0,1991-12-06 12:00:00.0] - - --- !query 17 -EXPLAIN EXTENDED SELECT * FROM VALUES ('one', 1), ('three', null) CROSS JOIN VALUES ('one', 1), ('three', null) --- !query 17 schema -struct --- !query 17 output -== Parsed Logical Plan == -'Project [*] -+- 'Join Cross - :- 'UnresolvedInlineTable [col1, col2], [List(one, 1), List(three, null)] - +- 'UnresolvedInlineTable [col1, col2], [List(one, 1), List(three, null)] - -== Analyzed Logical Plan == -col1: string, col2: int, col1: string, col2: int -Project [col1#x, col2#x, col1#x, col2#x] -+- Join Cross - :- LocalRelation [col1#x, col2#x] - +- LocalRelation [col1#x, col2#x] - -== Optimized Logical Plan == -Join Cross -:- LocalRelation [col1#x, col2#x] -+- LocalRelation [col1#x, col2#x] - -== Physical Plan == -BroadcastNestedLoopJoin BuildRight, Cross -:- LocalTableScan [col1#x, col2#x] -+- BroadcastExchange IdentityBroadcastMode - +- LocalTableScan [col1#x, col2#x] 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 570b281353f3..e0cbd575bc34 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 55 +-- Number of queries: 49 -- !query 0 @@ -195,260 +195,200 @@ struct -- !query 24 -explain select 'a' || 1 + 2 +select cot(1) -- !query 24 schema -struct +struct -- !query 24 output -== Physical Plan == -*Project [null AS (CAST(concat(a, CAST(1 AS STRING)) AS DOUBLE) + CAST(2 AS DOUBLE))#x] -+- *Scan OneRowRelation[] +0.6420926159343306 -- !query 25 -explain select 1 - 2 || 'b' +select cot(null) -- !query 25 schema -struct +struct -- !query 25 output -== Physical Plan == -*Project [-1b AS concat(CAST((1 - 2) AS STRING), b)#x] -+- *Scan OneRowRelation[] +NULL -- !query 26 -explain select 2 * 4 + 3 || 'b' +select cot(0) -- !query 26 schema -struct +struct -- !query 26 output -== Physical Plan == -*Project [11b AS concat(CAST(((2 * 4) + 3) AS STRING), b)#x] -+- *Scan OneRowRelation[] +Infinity -- !query 27 -explain select 3 + 1 || 'a' || 4 / 2 +select cot(-1) -- !query 27 schema -struct +struct -- !query 27 output -== Physical Plan == -*Project [4a2.0 AS concat(concat(CAST((3 + 1) AS STRING), a), CAST((CAST(4 AS DOUBLE) / CAST(2 AS DOUBLE)) AS STRING))#x] -+- *Scan OneRowRelation[] +-0.6420926159343306 -- !query 28 -explain select 1 == 1 OR 'a' || 'b' == 'ab' +select ceiling(0) -- !query 28 schema -struct +struct -- !query 28 output -== Physical Plan == -*Project [true AS ((1 = 1) OR (concat(a, b) = ab))#x] -+- *Scan OneRowRelation[] +0 -- !query 29 -explain select 'a' || 'c' == 'ac' AND 2 == 3 +select ceiling(1) -- !query 29 schema -struct +struct -- !query 29 output -== Physical Plan == -*Project [false AS ((concat(a, c) = ac) AND (2 = 3))#x] -+- *Scan OneRowRelation[] +1 -- !query 30 -select cot(1) +select ceil(1234567890123456) -- !query 30 schema -struct +struct -- !query 30 output -0.6420926159343306 +1234567890123456 -- !query 31 -select cot(null) +select ceiling(1234567890123456) -- !query 31 schema -struct +struct -- !query 31 output -NULL +1234567890123456 -- !query 32 -select cot(0) +select ceil(0.01) -- !query 32 schema -struct +struct -- !query 32 output -Infinity +1 -- !query 33 -select cot(-1) +select ceiling(-0.10) -- !query 33 schema -struct +struct -- !query 33 output --0.6420926159343306 +0 -- !query 34 -select ceiling(0) +select floor(0) -- !query 34 schema -struct +struct -- !query 34 output 0 -- !query 35 -select ceiling(1) +select floor(1) -- !query 35 schema -struct +struct -- !query 35 output 1 -- !query 36 -select ceil(1234567890123456) +select floor(1234567890123456) -- !query 36 schema -struct +struct -- !query 36 output 1234567890123456 -- !query 37 -select ceiling(1234567890123456) --- !query 37 schema -struct --- !query 37 output -1234567890123456 - - --- !query 38 -select ceil(0.01) --- !query 38 schema -struct --- !query 38 output -1 - - --- !query 39 -select ceiling(-0.10) --- !query 39 schema -struct --- !query 39 output -0 - - --- !query 40 -select floor(0) --- !query 40 schema -struct --- !query 40 output -0 - - --- !query 41 -select floor(1) --- !query 41 schema -struct --- !query 41 output -1 - - --- !query 42 -select floor(1234567890123456) --- !query 42 schema -struct --- !query 42 output -1234567890123456 - - --- !query 43 select floor(0.01) --- !query 43 schema +-- !query 37 schema struct --- !query 43 output +-- !query 37 output 0 --- !query 44 +-- !query 38 select floor(-0.10) --- !query 44 schema +-- !query 38 schema struct --- !query 44 output +-- !query 38 output -1 --- !query 45 +-- !query 39 select 1 > 0.00001 --- !query 45 schema +-- !query 39 schema struct<(CAST(1 AS BIGINT) > 0):boolean> --- !query 45 output +-- !query 39 output true --- !query 46 +-- !query 40 select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null) --- !query 46 schema +-- !query 40 schema struct<(7 % 2):int,(7 % 0):int,(0 % 2):int,(7 % CAST(NULL AS INT)):int,(CAST(NULL AS INT) % 2):int,(CAST(NULL AS DOUBLE) % CAST(NULL AS DOUBLE)):double> --- !query 46 output +-- !query 40 output 1 NULL 0 NULL NULL NULL --- !query 47 +-- !query 41 select BIT_LENGTH('abc') --- !query 47 schema +-- !query 41 schema struct --- !query 47 output +-- !query 41 output 24 --- !query 48 +-- !query 42 select CHAR_LENGTH('abc') --- !query 48 schema +-- !query 42 schema struct --- !query 48 output +-- !query 42 output 3 --- !query 49 +-- !query 43 select CHARACTER_LENGTH('abc') --- !query 49 schema +-- !query 43 schema struct --- !query 49 output +-- !query 43 output 3 --- !query 50 +-- !query 44 select OCTET_LENGTH('abc') --- !query 50 schema +-- !query 44 schema struct --- !query 50 output +-- !query 44 output 3 --- !query 51 +-- !query 45 select abs(-3.13), abs('-2.19') --- !query 51 schema +-- !query 45 schema struct --- !query 51 output +-- !query 45 output 3.13 2.19 --- !query 52 +-- !query 46 select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11) --- !query 52 schema +-- !query 46 schema struct<(+ CAST(-1.11 AS DOUBLE)):double,(+ -1.11):decimal(3,2),(- CAST(-1.11 AS DOUBLE)):double,(- -1.11):decimal(3,2)> --- !query 52 output +-- !query 46 output -1.11 -1.11 1.11 1.11 --- !query 53 +-- !query 47 select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null) --- !query 53 schema +-- !query 47 schema struct --- !query 53 output +-- !query 47 output 1 0 NULL NULL NULL NULL --- !query 54 +-- !query 48 select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)) --- !query 54 schema +-- !query 48 schema struct --- !query 54 output +-- !query 48 output NULL 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 e035505f15d2..69a8e958000d 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 15 +-- Number of queries: 14 -- !query 0 @@ -67,74 +67,49 @@ struct -- !query 8 -explain extended -select ifnull(id, 'x'), nullif(id, 'x'), nvl(id, 'x'), nvl2(id, 'x', 'y') -from range(2) --- !query 8 schema -struct --- !query 8 output -== Parsed Logical Plan == -'Project [unresolvedalias('ifnull('id, x), None), unresolvedalias('nullif('id, x), None), unresolvedalias('nvl('id, x), None), unresolvedalias('nvl2('id, x, y), None)] -+- 'UnresolvedTableValuedFunction range, [2] - -== Analyzed Logical Plan == -ifnull(`id`, 'x'): string, nullif(`id`, 'x'): bigint, nvl(`id`, 'x'): string, nvl2(`id`, 'x', 'y'): string -Project [ifnull(id#xL, x) AS ifnull(`id`, 'x')#x, nullif(id#xL, x) AS nullif(`id`, 'x')#xL, nvl(id#xL, x) AS nvl(`id`, 'x')#x, nvl2(id#xL, x, y) AS nvl2(`id`, 'x', 'y')#x] -+- Range (0, 2, step=1, splits=None) - -== Optimized Logical Plan == -Project [coalesce(cast(id#xL as string), x) AS ifnull(`id`, 'x')#x, id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, x AS nvl2(`id`, 'x', 'y')#x] -+- Range (0, 2, step=1, splits=None) - -== Physical Plan == -*Project [coalesce(cast(id#xL as string), x) AS ifnull(`id`, 'x')#x, id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, x AS nvl2(`id`, 'x', 'y')#x] -+- *Range (0, 2, step=1, splits=2) - - --- !query 9 SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1) --- !query 9 schema +-- !query 8 schema struct --- !query 9 output +-- !query 8 output true 1 1 1 1 --- !query 10 +-- !query 9 SELECT float(1), double(1), decimal(1) --- !query 10 schema +-- !query 9 schema struct --- !query 10 output +-- !query 9 output 1.0 1.0 1 --- !query 11 +-- !query 10 SELECT date("2014-04-04"), timestamp(date("2014-04-04")) --- !query 11 schema +-- !query 10 schema struct --- !query 11 output +-- !query 10 output 2014-04-04 2014-04-04 00:00:00 --- !query 12 +-- !query 11 SELECT string(1, 2) --- !query 12 schema +-- !query 11 schema struct<> --- !query 12 output +-- !query 11 output org.apache.spark.sql.AnalysisException Function string accepts only one argument; line 1 pos 7 --- !query 13 +-- !query 12 CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st) --- !query 13 schema +-- !query 12 schema struct<> --- !query 13 output +-- !query 12 output --- !query 14 +-- !query 13 SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value") --- !query 14 schema +-- !query 13 schema struct --- !query 14 output +-- !query 13 output gamma 1 diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index e8f2e0a81455..25d93b206314 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 17 +-- Number of queries: 13 -- !query 0 @@ -29,151 +29,80 @@ abc -- !query 3 -EXPLAIN EXTENDED SELECT (col1 || col2 || col3 || col4) col -FROM (SELECT id col1, id col2, id col3, id col4 FROM range(10)) --- !query 3 schema -struct --- !query 3 output -== Parsed Logical Plan == -'Project [concat(concat(concat('col1, 'col2), 'col3), 'col4) AS col#x] -+- 'SubqueryAlias `__auto_generated_subquery_name` - +- 'Project ['id AS col1#x, 'id AS col2#x, 'id AS col3#x, 'id AS col4#x] - +- 'UnresolvedTableValuedFunction range, [10] - -== Analyzed Logical Plan == -col: string -Project [concat(concat(concat(cast(col1#xL as string), cast(col2#xL as string)), cast(col3#xL as string)), cast(col4#xL as string)) AS col#x] -+- SubqueryAlias `__auto_generated_subquery_name` - +- Project [id#xL AS col1#xL, id#xL AS col2#xL, id#xL AS col3#xL, id#xL AS col4#xL] - +- Range (0, 10, step=1, splits=None) - -== Optimized Logical Plan == -Project [concat(cast(id#xL as string), cast(id#xL as string), cast(id#xL as string), cast(id#xL as string)) AS col#x] -+- Range (0, 10, step=1, splits=None) - -== Physical Plan == -*Project [concat(cast(id#xL as string), cast(id#xL as string), cast(id#xL as string), cast(id#xL as string)) AS col#x] -+- *Range (0, 10, step=1, splits=2) - - --- !query 4 select replace('abc', 'b', '123') --- !query 4 schema +-- !query 3 schema struct --- !query 4 output +-- !query 3 output a123c --- !query 5 +-- !query 4 select replace('abc', 'b') --- !query 5 schema +-- !query 4 schema struct --- !query 5 output +-- !query 4 output ac --- !query 6 +-- !query 5 select length(uuid()), (uuid() <> uuid()) --- !query 6 schema +-- !query 5 schema struct --- !query 6 output +-- !query 5 output 36 true --- !query 7 +-- !query 6 select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) --- !query 7 schema +-- !query 6 schema struct --- !query 7 output +-- !query 6 output 4 NULL NULL --- !query 8 +-- !query 7 select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) --- !query 8 schema +-- !query 7 schema struct --- !query 8 output +-- !query 7 output ab abcd ab NULL --- !query 9 +-- !query 8 select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a') --- !query 9 schema +-- !query 8 schema struct --- !query 9 output +-- !query 8 output NULL NULL --- !query 10 +-- !query 9 select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) --- !query 10 schema +-- !query 9 schema struct --- !query 10 output +-- !query 9 output cd abcd cd NULL --- !query 11 +-- !query 10 select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a') --- !query 11 schema +-- !query 10 schema struct --- !query 11 output +-- !query 10 output NULL NULL --- !query 12 -set spark.sql.function.concatBinaryAsString=false --- !query 12 schema -struct --- !query 12 output -spark.sql.function.concatBinaryAsString false - - --- !query 13 -EXPLAIN SELECT ((col1 || col2) || (col3 || col4)) col -FROM ( - SELECT - string(id) col1, - string(id + 1) col2, - encode(string(id + 2), 'utf-8') col3, - encode(string(id + 3), 'utf-8') col4 - FROM range(10) -) --- !query 13 schema -struct --- !query 13 output -== Physical Plan == -*Project [concat(cast(id#xL as string), cast((id#xL + 1) as string), cast(encode(cast((id#xL + 2) as string), utf-8) as string), cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x] -+- *Range (0, 10, step=1, splits=2) - - --- !query 14 -EXPLAIN SELECT (col1 || (col3 || col4)) col -FROM ( - SELECT - string(id) col1, - encode(string(id + 2), 'utf-8') col3, - encode(string(id + 3), 'utf-8') col4 - FROM range(10) -) --- !query 14 schema -struct --- !query 14 output -== Physical Plan == -*Project [concat(cast(id#xL as string), cast(encode(cast((id#xL + 2) as string), utf-8) as string), cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x] -+- *Range (0, 10, step=1, splits=2) - - --- !query 15 +-- !query 11 SELECT split('aa1cc2ee3', '[1-9]+') --- !query 15 schema +-- !query 11 schema struct> --- !query 15 output +-- !query 11 output ["aa","cc","ee",""] --- !query 16 +-- !query 12 SELECT split('aa1cc2ee3', '[1-9]+', 2) --- !query 16 schema +-- !query 12 schema struct> --- !query 16 output +-- !query 12 output ["aa","cc2ee3"] diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out index 94af9181225d..fdbea0ee9072 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 10 +-- Number of queries: 8 -- !query 0 @@ -99,42 +99,3 @@ struct -- !query 7 output 0 1 - - --- !query 8 -EXPLAIN select * from RaNgE(2) --- !query 8 schema -struct --- !query 8 output -== Physical Plan == -*Range (0, 2, step=1, splits=2) - - --- !query 9 -EXPLAIN EXTENDED SELECT * FROM range(3) CROSS JOIN range(3) --- !query 9 schema -struct --- !query 9 output -== Parsed Logical Plan == -'Project [*] -+- 'Join Cross - :- 'UnresolvedTableValuedFunction range, [3] - +- 'UnresolvedTableValuedFunction range, [3] - -== Analyzed Logical Plan == -id: bigint, id: bigint -Project [id#xL, id#xL] -+- Join Cross - :- Range (0, 3, step=1, splits=None) - +- Range (0, 3, step=1, splits=None) - -== Optimized Logical Plan == -Join Cross -:- Range (0, 3, step=1, splits=None) -+- Range (0, 3, step=1, splits=None) - -== Physical Plan == -BroadcastNestedLoopJoin BuildRight, Cross -:- *Range (0, 3, step=1, splits=2) -+- BroadcastExchange IdentityBroadcastMode - +- *Range (0, 3, step=1, splits=2) 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 56d300e30a58..ce475922eb5e 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType @@ -29,10 +30,11 @@ class ExplainSuite extends QueryTest with SharedSQLContext { private def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { val output = new java.io.ByteArrayOutputStream() Console.withOut(output) { - df.explain(extended = false) + df.explain(extended = true) } + val normalizedOutput = output.toString.replaceAll("#\\d+", "#x") for (key <- keywords) { - assert(output.toString.contains(key)) + assert(normalizedOutput.contains(key)) } } @@ -53,6 +55,133 @@ class ExplainSuite extends QueryTest with SharedSQLContext { checkKeywordsExistsInExplain(df, keywords = "InMemoryRelation", "StorageLevel(disk, memory, deserialized, 1 replicas)") } + + test("optimized plan should show the rewritten aggregate expression") { + withTempView("test_agg") { + sql( + """ + |CREATE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + | (1, true), (1, false), + | (2, true), + | (3, false), (3, null), + | (4, null), (4, null), + | (5, null), (5, true), (5, false) AS test_agg(k, v) + """.stripMargin) + + // simple explain of queries having every/some/any aggregates. Optimized + // plan should show the rewritten aggregate expression. + val df = sql("SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k") + checkKeywordsExistsInExplain(df, + "Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS some(v)#x, " + + "max(v#x) AS any(v)#x]") + } + } + + test("explain inline tables cross-joins") { + val df = sql( + """ + |SELECT * FROM VALUES ('one', 1), ('three', null) + | CROSS JOIN VALUES ('one', 1), ('three', null) + """.stripMargin) + checkKeywordsExistsInExplain(df, + "Join Cross", + ":- LocalRelation [col1#x, col2#x]", + "+- LocalRelation [col1#x, col2#x]") + } + + test("explain table valued functions") { + checkKeywordsExistsInExplain(sql("select * from RaNgE(2)"), "Range (0, 2, step=1, splits=None)") + checkKeywordsExistsInExplain(sql("SELECT * FROM range(3) CROSS JOIN range(3)"), + "Join Cross", + ":- Range (0, 3, step=1, splits=None)", + "+- Range (0, 3, step=1, splits=None)") + } + + test("explain string functions") { + // Check if catalyst combine nested `Concat`s + val df1 = sql( + """ + |SELECT (col1 || col2 || col3 || col4) col + | FROM (SELECT id col1, id col2, id col3, id col4 FROM range(10)) + """.stripMargin) + checkKeywordsExistsInExplain(df1, + "Project [concat(cast(id#xL as string), cast(id#xL as string), cast(id#xL as string)" + + ", cast(id#xL as string)) AS col#x]") + + // Check if catalyst combine nested `Concat`s if concatBinaryAsString=false + withSQLConf(SQLConf.CONCAT_BINARY_AS_STRING.key -> "false") { + val df2 = sql( + """ + |SELECT ((col1 || col2) || (col3 || col4)) col + |FROM ( + | SELECT + | string(id) col1, + | string(id + 1) col2, + | encode(string(id + 2), 'utf-8') col3, + | encode(string(id + 3), 'utf-8') col4 + | FROM range(10) + |) + """.stripMargin) + checkKeywordsExistsInExplain(df2, + "Project [concat(cast(id#xL as string), cast((id#xL + 1) as string), " + + "cast(encode(cast((id#xL + 2) as string), utf-8) as string), " + + "cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x]") + + val df3 = sql( + """ + |SELECT (col1 || (col3 || col4)) col + |FROM ( + | SELECT + | string(id) col1, + | encode(string(id + 2), 'utf-8') col3, + | encode(string(id + 3), 'utf-8') col4 + | FROM range(10) + |) + """.stripMargin) + checkKeywordsExistsInExplain(df3, + "Project [concat(cast(id#xL as string), " + + "cast(encode(cast((id#xL + 2) as string), utf-8) as string), " + + "cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x]") + } + } + + test("check operator precedence") { + // We follow Oracle operator precedence in the table below that lists the levels + // of precedence among SQL operators from high to low: + // --------------------------------------------------------------------------------------- + // Operator Operation + // --------------------------------------------------------------------------------------- + // +, - identity, negation + // *, / multiplication, division + // +, -, || addition, subtraction, concatenation + // =, !=, <, >, <=, >=, IS NULL, LIKE, BETWEEN, IN comparison + // NOT exponentiation, logical negation + // AND conjunction + // OR disjunction + // --------------------------------------------------------------------------------------- + checkKeywordsExistsInExplain(sql("select 'a' || 1 + 2"), + "Project [null AS (CAST(concat(a, CAST(1 AS STRING)) AS DOUBLE) + CAST(2 AS DOUBLE))#x]") + checkKeywordsExistsInExplain(sql("select 1 - 2 || 'b'"), + "Project [-1b AS concat(CAST((1 - 2) AS STRING), b)#x]") + checkKeywordsExistsInExplain(sql("select 2 * 4 + 3 || 'b'"), + "Project [11b AS concat(CAST(((2 * 4) + 3) AS STRING), b)#x]") + checkKeywordsExistsInExplain(sql("select 3 + 1 || 'a' || 4 / 2"), + "Project [4a2.0 AS concat(concat(CAST((3 + 1) AS STRING), a), " + + "CAST((CAST(4 AS DOUBLE) / CAST(2 AS DOUBLE)) AS STRING))#x]") + checkKeywordsExistsInExplain(sql("select 1 == 1 OR 'a' || 'b' == 'ab'"), + "Project [true AS ((1 = 1) OR (concat(a, b) = ab))#x]") + checkKeywordsExistsInExplain(sql("select 'a' || 'c' == 'ac' AND 2 == 3"), + "Project [false AS ((concat(a, c) = ac) AND (2 = 3))#x]") + } + + test("explain for these functions; use range to avoid constant folding") { + val df = sql("select ifnull(id, 'x'), nullif(id, 'x'), nvl(id, 'x'), nvl2(id, 'x', 'y') " + + "from range(2)") + checkKeywordsExistsInExplain(df, + "Project [coalesce(cast(id#xL as string), x) AS ifnull(`id`, 'x')#x, " + + "id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, " + + "x AS nvl2(`id`, 'x', 'y')#x]") + } } case class ExplainSingleData(id: Int) 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 cf4585bf7ac6..b2515226d9a1 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 @@ -137,28 +137,39 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } } + // For better test coverage, runs the tests on mixed config sets: WHOLESTAGE_CODEGEN_ENABLED + // and CODEGEN_FACTORY_MODE. + private lazy val codegenConfigSets = Array( + ("true", "CODEGEN_ONLY"), + ("false", "CODEGEN_ONLY"), + ("false", "NO_CODEGEN") + ).map { case (wholeStageCodegenEnabled, codegenFactoryMode) => + Array(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> wholeStageCodegenEnabled, + SQLConf.CODEGEN_FACTORY_MODE.key -> codegenFactoryMode) + } + /** Run a test case. */ private def runTest(testCase: TestCase): Unit = { val input = fileToString(new File(testCase.inputFile)) val (comments, code) = input.split("\n").partition(_.startsWith("--")) - // Runs all the tests on both codegen-only and interpreter modes - val codegenConfigSets = Array(CODEGEN_ONLY, NO_CODEGEN).map { - case codegenFactoryMode => - Array(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenFactoryMode.toString) - } - val configSets = { - val configLines = comments.filter(_.startsWith("--SET")).map(_.substring(5)) - val configs = configLines.map(_.split(",").map { confAndValue => - val (conf, value) = confAndValue.span(_ != '=') - conf.trim -> value.substring(1).trim - }) - // When we are regenerating the golden files, we don't need to set any config as they - // all need to return the same result - if (regenerateGoldenFiles) { - Array.empty[Array[(String, String)]] - } else { + // List of SQL queries to run + // note: this is not a robust way to split queries using semicolon, but works for now. + val queries = code.mkString("\n").split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq + + // When we are regenerating the golden files, we don't need to set any config as they + // all need to return the same result + if (regenerateGoldenFiles) { + runQueries(queries, testCase.resultFile, None) + } else { + val configSets = { + val configLines = comments.filter(_.startsWith("--SET")).map(_.substring(5)) + val configs = configLines.map(_.split(",").map { confAndValue => + val (conf, value) = confAndValue.span(_ != '=') + conf.trim -> value.substring(1).trim + }) + if (configs.nonEmpty) { codegenConfigSets.flatMap { codegenConfig => configs.map { config => @@ -169,15 +180,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { codegenConfigSets } } - } - // List of SQL queries to run - // note: this is not a robust way to split queries using semicolon, but works for now. - val queries = code.mkString("\n").split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq - - if (configSets.isEmpty) { - runQueries(queries, testCase.resultFile, None) - } else { configSets.foreach { configSet => try { runQueries(queries, testCase.resultFile, Some(configSet)) From 5ad03607d1487e7ab3e3b6d00eef9c4028ed4975 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 20 Dec 2018 10:47:24 +0800 Subject: [PATCH 0167/1072] [SPARK-25271][SQL] Hive ctas commands should use data source if it is convertible ## What changes were proposed in this pull request? In Spark 2.3.0 and previous versions, Hive CTAS command will convert to use data source to write data into the table when the table is convertible. This behavior is controlled by the configs like HiveUtils.CONVERT_METASTORE_ORC and HiveUtils.CONVERT_METASTORE_PARQUET. In 2.3.1, we drop this optimization by mistake in the PR [SPARK-22977](https://github.com/apache/spark/pull/20521/files#r217254430). Since that Hive CTAS command only uses Hive Serde to write data. This patch adds this optimization back to Hive CTAS command. This patch adds OptimizedCreateHiveTableAsSelectCommand which uses data source to write data. ## How was this patch tested? Added test. Closes #22514 from viirya/SPARK-25271-2. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../spark/sql/execution/command/ddl.scala | 8 ++ .../datasources/DataSourceStrategy.scala | 12 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 43 +++++- .../spark/sql/hive/HiveStrategies.scala | 62 +++----- .../org/apache/spark/sql/hive/HiveUtils.scala | 8 ++ .../CreateHiveTableAsSelectCommand.scala | 134 +++++++++++++----- .../spark/sql/hive/HiveParquetSuite.scala | 14 ++ .../sql/hive/execution/SQLQuerySuite.scala | 40 ++++++ 8 files changed, 230 insertions(+), 91 deletions(-) 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 e1faecedd20e..096481f68275 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 @@ -820,6 +820,14 @@ object DDLUtils { table.provider.isDefined && table.provider.get.toLowerCase(Locale.ROOT) != HIVE_PROVIDER } + def readHiveTable(table: CatalogTable): HiveTableRelation = { + HiveTableRelation( + table, + // Hive table columns are always nullable. + table.dataSchema.asNullable.toAttributes, + table.partitionSchema.asNullable.toAttributes) + } + /** * Throws a standard error for actions that require partitionProvider = hive. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index b304e2da6e1c..b5cf8c9515bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -244,27 +244,19 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] }) } - private def readHiveTable(table: CatalogTable): LogicalPlan = { - HiveTableRelation( - table, - // Hive table columns are always nullable. - table.dataSchema.asNullable.toAttributes, - table.partitionSchema.asNullable.toAttributes) - } - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case i @ InsertIntoTable(UnresolvedCatalogRelation(tableMeta), _, _, _, _) if DDLUtils.isDatasourceTable(tableMeta) => i.copy(table = readDataSourceTable(tableMeta)) case i @ InsertIntoTable(UnresolvedCatalogRelation(tableMeta), _, _, _, _) => - i.copy(table = readHiveTable(tableMeta)) + i.copy(table = DDLUtils.readHiveTable(tableMeta)) case UnresolvedCatalogRelation(tableMeta) if DDLUtils.isDatasourceTable(tableMeta) => readDataSourceTable(tableMeta) case UnresolvedCatalogRelation(tableMeta) => - readHiveTable(tableMeta) + DDLUtils.readHiveTable(tableMeta) } } 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 5823548a8063..03f4b8d83e35 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import java.util.Locale + import scala.util.control.NonFatal import com.google.common.util.concurrent.Striped @@ -29,6 +31,8 @@ import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode._ import org.apache.spark.sql.types._ @@ -113,7 +117,44 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } } - def convertToLogicalRelation( + // Return true for Apache ORC and Hive ORC-related configuration names. + // Note that Spark doesn't support configurations like `hive.merge.orcfile.stripe.level`. + private def isOrcProperty(key: String) = + key.startsWith("orc.") || key.contains(".orc.") + + private def isParquetProperty(key: String) = + key.startsWith("parquet.") || key.contains(".parquet.") + + def convert(relation: HiveTableRelation): LogicalRelation = { + val serde = relation.tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + + // 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) ++ + 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) ++ + relation.tableMeta.storage.properties + if (SQLConf.get.getConf(SQLConf.ORC_IMPLEMENTATION) == "native") { + convertToLogicalRelation( + relation, + options, + classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat], + "orc") + } else { + convertToLogicalRelation( + relation, + options, + classOf[org.apache.spark.sql.hive.orc.OrcFileFormat], + "orc") + } + } + } + + private def convertToLogicalRelation( relation: HiveTableRelation, options: Map[String, String], fileFormatClass: Class[_ <: FileFormat], 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 07ee10540431..8a5ab188a949 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -31,8 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTab import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} -import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} +import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -181,49 +180,17 @@ case class RelationConversions( conf: SQLConf, sessionCatalog: HiveSessionCatalog) extends Rule[LogicalPlan] { private def isConvertible(relation: HiveTableRelation): Boolean = { - val serde = relation.tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) - serde.contains("parquet") && conf.getConf(HiveUtils.CONVERT_METASTORE_PARQUET) || - serde.contains("orc") && conf.getConf(HiveUtils.CONVERT_METASTORE_ORC) + isConvertible(relation.tableMeta) } - // Return true for Apache ORC and Hive ORC-related configuration names. - // Note that Spark doesn't support configurations like `hive.merge.orcfile.stripe.level`. - private def isOrcProperty(key: String) = - key.startsWith("orc.") || key.contains(".orc.") - - private def isParquetProperty(key: String) = - key.startsWith("parquet.") || key.contains(".parquet.") - - private def convert(relation: HiveTableRelation): LogicalRelation = { - val serde = relation.tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) - - // 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) ++ - relation.tableMeta.storage.properties + (ParquetOptions.MERGE_SCHEMA -> - conf.getConf(HiveUtils.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING).toString) - sessionCatalog.metastoreCatalog - .convertToLogicalRelation(relation, options, classOf[ParquetFileFormat], "parquet") - } else { - val options = relation.tableMeta.properties.filterKeys(isOrcProperty) ++ - relation.tableMeta.storage.properties - if (conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native") { - sessionCatalog.metastoreCatalog.convertToLogicalRelation( - relation, - options, - classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat], - "orc") - } else { - sessionCatalog.metastoreCatalog.convertToLogicalRelation( - relation, - options, - classOf[org.apache.spark.sql.hive.orc.OrcFileFormat], - "orc") - } - } + private def isConvertible(tableMeta: CatalogTable): Boolean = { + val serde = tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + serde.contains("parquet") && SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_PARQUET) || + serde.contains("orc") && SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_ORC) } + private val metastoreCatalog = sessionCatalog.metastoreCatalog + override def apply(plan: LogicalPlan): LogicalPlan = { plan resolveOperators { // Write path @@ -231,12 +198,21 @@ case class RelationConversions( // Inserting into partitioned table is not supported in Parquet/Orc data source (yet). if query.resolved && DDLUtils.isHiveTable(r.tableMeta) && !r.isPartitioned && isConvertible(r) => - InsertIntoTable(convert(r), partition, query, overwrite, ifPartitionNotExists) + InsertIntoTable(metastoreCatalog.convert(r), partition, + query, overwrite, ifPartitionNotExists) // Read path case relation: HiveTableRelation if DDLUtils.isHiveTable(relation.tableMeta) && isConvertible(relation) => - convert(relation) + metastoreCatalog.convert(relation) + + // CTAS + case CreateTable(tableDesc, mode, Some(query)) + if DDLUtils.isHiveTable(tableDesc) && tableDesc.partitionColumnNames.isEmpty && + isConvertible(tableDesc) && SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_CTAS) => + DDLUtils.checkDataColNames(tableDesc) + OptimizedCreateHiveTableAsSelectCommand( + tableDesc, query, query.output.map(_.name), mode) } } } 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 66067704195d..b60d4c71f594 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 @@ -110,6 +110,14 @@ private[spark] object HiveUtils extends Logging { .booleanConf .createWithDefault(true) + val CONVERT_METASTORE_CTAS = buildConf("spark.sql.hive.convertMetastoreCtas") + .doc("When set to true, Spark will try to use built-in data source writer " + + "instead of Hive serde in CTAS. This flag is effective only if " + + "`spark.sql.hive.convertMetastoreParquet` or `spark.sql.hive.convertMetastoreOrc` is " + + "enabled respectively for Parquet and ORC formats") + .booleanConf + .createWithDefault(true) + val HIVE_METASTORE_SHARED_PREFIXES = buildConf("spark.sql.hive.metastore.sharedPrefixes") .doc("A comma separated list of class prefixes that should be loaded using the classloader " + "that is shared between Spark SQL and a specific version of Hive. An example of classes " + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index fd1e931ee0c7..608f21e72625 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -20,32 +20,26 @@ package org.apache.spark.sql.hive.execution import scala.util.control.NonFatal import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} -import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.DataWritingCommand +import org.apache.spark.sql.execution.command.{DataWritingCommand, DDLUtils} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InsertIntoHadoopFsRelationCommand, LogicalRelation} +import org.apache.spark.sql.hive.HiveSessionCatalog +trait CreateHiveTableAsSelectBase extends DataWritingCommand { + val tableDesc: CatalogTable + val query: LogicalPlan + val outputColumnNames: Seq[String] + val mode: SaveMode -/** - * Create table and insert the query result into it. - * - * @param tableDesc the Table Describe, which may contain serde, storage handler etc. - * @param query the query whose result will be insert into the new relation - * @param mode SaveMode - */ -case class CreateHiveTableAsSelectCommand( - tableDesc: CatalogTable, - query: LogicalPlan, - outputColumnNames: Seq[String], - mode: SaveMode) - extends DataWritingCommand { - - private val tableIdentifier = tableDesc.identifier + protected val tableIdentifier = tableDesc.identifier override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - if (catalog.tableExists(tableIdentifier)) { + val tableExists = catalog.tableExists(tableIdentifier) + + if (tableExists) { assert(mode != SaveMode.Overwrite, s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") @@ -57,15 +51,8 @@ case class CreateHiveTableAsSelectCommand( return Seq.empty } - // For CTAS, there is no static partition values to insert. - val partition = tableDesc.partitionColumnNames.map(_ -> None).toMap - InsertIntoHiveTable( - tableDesc, - partition, - query, - overwrite = false, - ifPartitionNotExists = false, - outputColumnNames = outputColumnNames).run(sparkSession, child) + val command = getWritingCommand(catalog, tableDesc, tableExists = true) + command.run(sparkSession, child) } else { // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data @@ -77,15 +64,8 @@ case class CreateHiveTableAsSelectCommand( try { // Read back the metadata of the table which was created just now. val createdTableMeta = catalog.getTableMetadata(tableDesc.identifier) - // For CTAS, there is no static partition values to insert. - val partition = createdTableMeta.partitionColumnNames.map(_ -> None).toMap - InsertIntoHiveTable( - createdTableMeta, - partition, - query, - overwrite = true, - ifPartitionNotExists = false, - outputColumnNames = outputColumnNames).run(sparkSession, child) + val command = getWritingCommand(catalog, createdTableMeta, tableExists = false) + command.run(sparkSession, child) } catch { case NonFatal(e) => // drop the created table. @@ -97,9 +77,89 @@ case class CreateHiveTableAsSelectCommand( Seq.empty[Row] } + // Returns `DataWritingCommand` which actually writes data into the table. + def getWritingCommand( + catalog: SessionCatalog, + tableDesc: CatalogTable, + tableExists: Boolean): DataWritingCommand + override def argString: String = { s"[Database:${tableDesc.database}, " + s"TableName: ${tableDesc.identifier.table}, " + s"InsertIntoHiveTable]" } } + +/** + * Create table and insert the query result into it. + * + * @param tableDesc the table description, which may contain serde, storage handler etc. + * @param query the query whose result will be insert into the new relation + * @param mode SaveMode + */ +case class CreateHiveTableAsSelectCommand( + tableDesc: CatalogTable, + query: LogicalPlan, + outputColumnNames: Seq[String], + mode: SaveMode) + extends CreateHiveTableAsSelectBase { + + override def getWritingCommand( + catalog: SessionCatalog, + tableDesc: CatalogTable, + tableExists: Boolean): DataWritingCommand = { + // For CTAS, there is no static partition values to insert. + val partition = tableDesc.partitionColumnNames.map(_ -> None).toMap + InsertIntoHiveTable( + tableDesc, + partition, + query, + overwrite = if (tableExists) false else true, + ifPartitionNotExists = false, + outputColumnNames = outputColumnNames) + } +} + +/** + * Create table and insert the query result into it. This creates Hive table but inserts + * the query result into it by using data source. + * + * @param tableDesc the table description, which may contain serde, storage handler etc. + * @param query the query whose result will be insert into the new relation + * @param mode SaveMode + */ +case class OptimizedCreateHiveTableAsSelectCommand( + tableDesc: CatalogTable, + query: LogicalPlan, + outputColumnNames: Seq[String], + mode: SaveMode) + extends CreateHiveTableAsSelectBase { + + override def getWritingCommand( + catalog: SessionCatalog, + tableDesc: CatalogTable, + tableExists: Boolean): DataWritingCommand = { + val metastoreCatalog = catalog.asInstanceOf[HiveSessionCatalog].metastoreCatalog + val hiveTable = DDLUtils.readHiveTable(tableDesc) + + val hadoopRelation = metastoreCatalog.convert(hiveTable) match { + case LogicalRelation(t: HadoopFsRelation, _, _, _) => t + case _ => throw new AnalysisException(s"$tableIdentifier should be converted to " + + "HadoopFsRelation.") + } + + InsertIntoHadoopFsRelationCommand( + hadoopRelation.location.rootPaths.head, + Map.empty, // We don't support to convert partitioned table. + false, + Seq.empty, // We don't support to convert partitioned table. + hadoopRelation.bucketSpec, + hadoopRelation.fileFormat, + hadoopRelation.options, + query, + if (tableExists) mode else SaveMode.Overwrite, + Some(tableDesc), + Some(hadoopRelation.location), + query.output.map(_.name)) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index e5c9df05d567..470c6a342b4d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -92,4 +92,18 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton } } } + + test("SPARK-25271: write empty map into hive parquet table") { + import testImplicits._ + + Seq(Map(1 -> "a"), Map.empty[Int, String]).toDF("m").createOrReplaceTempView("p") + withTempView("p") { + val targetTable = "targetTable" + withTable(targetTable) { + sql(s"CREATE TABLE $targetTable STORED AS PARQUET AS SELECT m FROM p") + checkAnswer(sql(s"SELECT m FROM $targetTable"), + Row(Map(1 -> "a")) :: Row(Map.empty[Int, String]) :: Nil) + } + } + } } 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 fab2a27cdef1..6acf44606cbb 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 @@ -2276,6 +2276,46 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-25271: Hive ctas commands should use data source if it is convertible") { + withTempView("p") { + Seq(1, 2, 3).toDF("id").createOrReplaceTempView("p") + + Seq("orc", "parquet").foreach { format => + Seq(true, false).foreach { isConverted => + withSQLConf( + HiveUtils.CONVERT_METASTORE_ORC.key -> s"$isConverted", + HiveUtils.CONVERT_METASTORE_PARQUET.key -> s"$isConverted") { + Seq(true, false).foreach { isConvertedCtas => + withSQLConf(HiveUtils.CONVERT_METASTORE_CTAS.key -> s"$isConvertedCtas") { + + val targetTable = "targetTable" + withTable(targetTable) { + val df = sql(s"CREATE TABLE $targetTable STORED AS $format AS SELECT id FROM p") + checkAnswer(sql(s"SELECT id FROM $targetTable"), + Row(1) :: Row(2) :: Row(3) :: Nil) + + val ctasDSCommand = df.queryExecution.analyzed.collect { + case _: OptimizedCreateHiveTableAsSelectCommand => true + }.headOption + val ctasCommand = df.queryExecution.analyzed.collect { + case _: CreateHiveTableAsSelectCommand => true + }.headOption + + if (isConverted && isConvertedCtas) { + assert(ctasDSCommand.nonEmpty) + assert(ctasCommand.isEmpty) + } else { + assert(ctasDSCommand.isEmpty) + assert(ctasCommand.nonEmpty) + } + } + } + } + } + } + } + } + } test("SPARK-26181 hasMinMaxStats method of ColumnStatsMap is not correct") { withSQLConf(SQLConf.CBO_ENABLED.key -> "true") { From 04d8e3a33c6bb08b2891ca52613cd5ccd24a69dd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E4=BA=AE?= Date: Thu, 20 Dec 2018 13:22:12 +0800 Subject: [PATCH 0168/1072] [SPARK-26318][SQL] Deprecate Row.merge MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Deprecate Row.merge ## How was this patch tested? N/A Closes #23271 from KyleLi1985/master. Authored-by: 李亮 Signed-off-by: Hyukjin Kwon --- sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index e12bf9616e2d..4f5af9ac80b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -57,6 +57,7 @@ object Row { /** * Merge multiple rows into a single row, one after another. */ + @deprecated("This method is deprecated and will be removed in future versions.", "3.0.0") def merge(rows: Row*): Row = { // TODO: Improve the performance of this if used in performance critical part. new GenericRow(rows.flatMap(_.toSeq).toArray) From 98c0ca78610ccf62784081353584717c62285485 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Thu, 20 Dec 2018 14:17:44 +0800 Subject: [PATCH 0169/1072] [SPARK-26308][SQL] Avoid cast of decimals for ScalaUDF ## What changes were proposed in this pull request? Currently, when we infer the schema for scala/java decimals, we return as data type the `SYSTEM_DEFAULT` implementation, ie. the decimal type with precision 38 and scale 18. But this is not right, as we know nothing about the right precision and scale and these values can be not enough to store the data. This problem arises in particular with UDF, where we cast all the input of type `DecimalType` to a `DecimalType(38, 18)`: in case this is not enough, null is returned as input for the UDF. The PR defines a custom handling for casting to the expected data types for ScalaUDF: the decimal precision and scale is picked from the input, so no casting to different and maybe wrong percision and scale happens. ## How was this patch tested? added UTs Closes #23308 from mgaido91/SPARK-26308. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/TypeCoercion.scala | 31 ++++++++++++++++++ .../sql/catalyst/expressions/ScalaUDF.scala | 2 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 32 ++++++++++++++++++- 3 files changed, 63 insertions(+), 2 deletions(-) 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 133fa119b7aa..1706b3eece6d 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 @@ -879,6 +879,37 @@ object TypeCoercion { } } e.withNewChildren(children) + + case udf: ScalaUDF if udf.inputTypes.nonEmpty => + val children = udf.children.zip(udf.inputTypes).map { case (in, expected) => + implicitCast(in, udfInputToCastType(in.dataType, expected)).getOrElse(in) + } + udf.withNewChildren(children) + } + + private def udfInputToCastType(input: DataType, expectedType: DataType): DataType = { + (input, expectedType) match { + // SPARK-26308: avoid casting to an arbitrary precision and scale for decimals. Please note + // that precision and scale cannot be inferred properly for a ScalaUDF because, when it is + // created, it is not bound to any column. So here the precision and scale of the input + // column is used. + case (in: DecimalType, _: DecimalType) => in + case (ArrayType(dtIn, _), ArrayType(dtExp, nullableExp)) => + ArrayType(udfInputToCastType(dtIn, dtExp), nullableExp) + case (MapType(keyDtIn, valueDtIn, _), MapType(keyDtExp, valueDtExp, nullableExp)) => + MapType(udfInputToCastType(keyDtIn, keyDtExp), + udfInputToCastType(valueDtIn, valueDtExp), + nullableExp) + case (StructType(fieldsIn), StructType(fieldsExp)) => + val fieldTypes = + fieldsIn.map(_.dataType).zip(fieldsExp.map(_.dataType)).map { case (dtIn, dtExp) => + udfInputToCastType(dtIn, dtExp) + } + StructType(fieldsExp.zip(fieldTypes).map { case (field, newDt) => + field.copy(dataType = newDt) + }) + case (_, other) => other + } } /** 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 fae90caebf96..a23aaa3a0b3e 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 @@ -52,7 +52,7 @@ case class ScalaUDF( udfName: Option[String] = None, nullable: Boolean = true, udfDeterministic: Boolean = true) - extends Expression with ImplicitCastInputTypes with NonSQLExpression with UserDefinedExpression { + extends Expression with NonSQLExpression with UserDefinedExpression { // The constructor for SPARK 2.1 and 2.2 def this( 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 20dcefa7e3ca..a26d306cff6b 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.math.BigDecimal + import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.QueryExecution @@ -26,7 +28,7 @@ import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationComm import org.apache.spark.sql.functions.{lit, udf} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ -import org.apache.spark.sql.types.{DataTypes, DoubleType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.util.QueryExecutionListener @@ -420,4 +422,32 @@ class UDFSuite extends QueryTest with SharedSQLContext { checkAnswer(df, Seq(Row("null1x"), Row(null), Row("N3null"))) } } + + test("SPARK-26308: udf with decimal") { + val df1 = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(new BigDecimal("2011000000000002456556")))), + StructType(Seq(StructField("col1", DecimalType(30, 0))))) + val udf1 = org.apache.spark.sql.functions.udf((value: BigDecimal) => { + if (value == null) null else value.toBigInteger.toString + }) + checkAnswer(df1.select(udf1(df1.col("col1"))), Seq(Row("2011000000000002456556"))) + } + + test("SPARK-26308: udf with complex types of decimal") { + val df1 = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(Array(new BigDecimal("2011000000000002456556"))))), + StructType(Seq(StructField("col1", ArrayType(DecimalType(30, 0)))))) + val udf1 = org.apache.spark.sql.functions.udf((arr: Seq[BigDecimal]) => { + arr.map(value => if (value == null) null else value.toBigInteger.toString) + }) + checkAnswer(df1.select(udf1($"col1")), Seq(Row(Array("2011000000000002456556")))) + + val df2 = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(Map("a" -> new BigDecimal("2011000000000002456556"))))), + StructType(Seq(StructField("col1", MapType(StringType, DecimalType(30, 0)))))) + val udf2 = org.apache.spark.sql.functions.udf((map: Map[String, BigDecimal]) => { + map.mapValues(value => if (value == null) null else value.toBigInteger.toString) + }) + checkAnswer(df2.select(udf2($"col1")), Seq(Row(Map("a" -> "2011000000000002456556")))) + } } From 7c8f4756c34a0b00931c2987c827a18d989e6c08 Mon Sep 17 00:00:00 2001 From: zhoukang Date: Thu, 20 Dec 2018 08:26:25 -0600 Subject: [PATCH 0170/1072] [SPARK-24687][CORE] Avoid job hanging when generate task binary causes fatal error ## What changes were proposed in this pull request? When NoClassDefFoundError thrown,it will cause job hang. `Exception in thread "dag-scheduler-event-loop" java.lang.NoClassDefFoundError: Lcom/xxx/data/recommend/aggregator/queue/QueueName; at java.lang.Class.getDeclaredFields0(Native Method) at java.lang.Class.privateGetDeclaredFields(Class.java:2436) at java.lang.Class.getDeclaredField(Class.java:1946) at java.io.ObjectStreamClass.getDeclaredSUID(ObjectStreamClass.java:1659) at java.io.ObjectStreamClass.access$700(ObjectStreamClass.java:72) at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:480) at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:468) at java.security.AccessController.doPrivileged(Native Method) at java.io.ObjectStreamClass.(ObjectStreamClass.java:468) at java.io.ObjectStreamClass.lookup(ObjectStreamClass.java:365) at java.io.ObjectOutputStream.writeClass(ObjectOutputStream.java:1212) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1119) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1377) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1173) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1377)` It is caused by NoClassDefFoundError will not catch up during task seriazation. `var taskBinary: Broadcast[Array[Byte]] = null try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). val taskBinaryBytes: Array[Byte] = stage match { case stage: ShuffleMapStage => JavaUtils.bufferToArray( closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) case stage: ResultStage => JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) } taskBinary = sc.broadcast(taskBinaryBytes) } catch { // In the case of a failure during serialization, abort the stage. case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString, Some(e)) runningStages -= stage // Abort execution return case NonFatal(e) => abortStage(stage, s"Task serialization failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage return }` image below shows that stage 33 blocked and never be scheduled. 2018-06-28 4 28 42 2018-06-28 4 28 49 ## How was this patch tested? UT Closes #21664 from caneGuy/zhoukang/fix-noclassdeferror. Authored-by: zhoukang Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 06966e77db81..6f4c326442e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1170,9 +1170,11 @@ private[spark] class DAGScheduler( // Abort execution return - case NonFatal(e) => + case e: Throwable => abortStage(stage, s"Task serialization failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage + + // Abort execution return } From a888d202ab719ccb13b328aa445341d1d6b06881 Mon Sep 17 00:00:00 2001 From: Jorge Machado Date: Thu, 20 Dec 2018 08:29:51 -0600 Subject: [PATCH 0171/1072] [SPARK-26324][DOCS] Add Spark docs for Running in Mesos with SSL ## What changes were proposed in this pull request? Added docs for running spark jobs with Mesos on SSL Closes #23342 from jomach/master. Lead-authored-by: Jorge Machado Co-authored-by: Jorge Machado Co-authored-by: Jorge Machado Co-authored-by: Jorge Machado Signed-off-by: Sean Owen --- docs/running-on-mesos.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 968d668e2c93..a07773c1c71e 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -108,6 +108,19 @@ Please note that if you specify multiple ways to obtain the credentials then the An equivalent order applies for the secret. Essentially we prefer the configuration to be specified directly rather than indirectly by files, and we prefer that configuration settings are used over environment variables. +### Deploy to a Mesos running on Secure Sockets + +If you want to deploy a Spark Application into a Mesos cluster that is running in a secure mode there are some environment variables that need to be set. + +- `LIBPROCESS_SSL_ENABLED=true` enables SSL communication +- `LIBPROCESS_SSL_VERIFY_CERT=false` verifies the ssl certificate +- `LIBPROCESS_SSL_KEY_FILE=pathToKeyFile.key` path to key +- `LIBPROCESS_SSL_CERT_FILE=pathToCRTFile.crt` the certificate file to be used + +All options can be found at http://mesos.apache.org/documentation/latest/ssl/ + +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 From 6692bacf3e74e7a17d8e676e8a06ab198f85d328 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 20 Dec 2018 10:05:56 -0800 Subject: [PATCH 0172/1072] [SPARK-26409][SQL][TESTS] SQLConf should be serializable in test sessions ## What changes were proposed in this pull request? `SQLConf` is supposed to be serializable. However, currently it is not serializable in `WithTestConf`. `WithTestConf` uses the method `overrideConfs` in closure, while the classes which implements it (`TestHiveSessionStateBuilder` and `TestSQLSessionStateBuilder`) are not serializable. This PR is to use a local variable to fix it. ## How was this patch tested? Add unit test. Closes #23352 from gengliangwang/serializableSQLConf. Authored-by: Gengliang Wang Signed-off-by: gatorsmile --- .../apache/spark/sql/internal/BaseSessionStateBuilder.scala | 3 ++- .../test/scala/org/apache/spark/sql/SerializationSuite.scala | 5 +++++ 2 files changed, 7 insertions(+), 1 deletion(-) 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 ac07e1f6bb4f..319c2649592f 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 @@ -309,13 +309,14 @@ private[sql] trait WithTestConf { self: BaseSessionStateBuilder => def overrideConfs: Map[String, String] override protected lazy val conf: SQLConf = { + val overrideConfigurations = overrideConfs val conf = parentState.map(_.conf.clone()).getOrElse { new SQLConf { clear() override def clear(): Unit = { super.clear() // Make sure we start with the default test configs even after clear - overrideConfs.foreach { case (key, value) => setConfString(key, value) } + overrideConfigurations.foreach { case (key, value) => setConfString(key, value) } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala index cd6b2647e0be..1a1c956aed3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala @@ -27,4 +27,9 @@ class SerializationSuite extends SparkFunSuite with SharedSQLContext { val spark = SparkSession.builder.getOrCreate() new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sqlContext) } + + test("[SPARK-26409] SQLConf should be serializable") { + val spark = SparkSession.builder.getOrCreate() + new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sessionState.conf) + } } From 3d6b44d9ea92dc1eabb8f211176861e51240bf93 Mon Sep 17 00:00:00 2001 From: Ngone51 Date: Thu, 20 Dec 2018 10:25:52 -0800 Subject: [PATCH 0173/1072] [SPARK-26392][YARN] Cancel pending allocate requests by taking locality preference into account ## What changes were proposed in this pull request? Right now, we cancel pending allocate requests by its sending order. I thing we can take locality preference into account when do this to perfom least impact on task locality preference. ## How was this patch tested? N.A. Closes #23344 from Ngone51/dev-cancel-pending-allocate-requests-by-taking-locality-preference-into-account. Authored-by: Ngone51 Signed-off-by: Marcelo Vanzin --- .../spark/deploy/yarn/YarnAllocator.scala | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) 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 d37d0d66d8ae..54b1ec266113 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 @@ -294,6 +294,15 @@ private[yarn] class YarnAllocator( s"pending: $numPendingAllocate, running: ${runningExecutors.size}, " + s"executorsStarting: ${numExecutorsStarting.get}") + // Split the pending container request into three groups: locality matched list, locality + // unmatched list and non-locality list. Take the locality matched container request into + // consideration of container placement, treat as allocated containers. + // For locality unmatched and locality free container requests, cancel these container + // requests, since required locality preference has been changed, recalculating using + // container placement strategy. + val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality( + hostToLocalTaskCounts, pendingAllocate) + if (missing > 0) { if (log.isInfoEnabled()) { var requestContainerMessage = s"Will request $missing executor container(s), each with " + @@ -306,15 +315,6 @@ private[yarn] class YarnAllocator( logInfo(requestContainerMessage) } - // Split the pending container request into three groups: locality matched list, locality - // unmatched list and non-locality list. Take the locality matched container request into - // consideration of container placement, treat as allocated containers. - // For locality unmatched and locality free container requests, cancel these container - // requests, since required locality preference has been changed, recalculating using - // container placement strategy. - val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality( - hostToLocalTaskCounts, pendingAllocate) - // cancel "stale" requests for locations that are no longer needed staleRequests.foreach { stale => amClient.removeContainerRequest(stale) @@ -374,14 +374,9 @@ private[yarn] class YarnAllocator( val numToCancel = math.min(numPendingAllocate, -missing) logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new desired " + s"total $targetNumExecutors executors.") - - val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource) - if (!matchingRequests.isEmpty) { - matchingRequests.iterator().next().asScala - .take(numToCancel).foreach(amClient.removeContainerRequest) - } else { - logWarning("Expected to find pending requests, but found none.") - } + // cancel pending allocate requests by taking locality preference into account + val cancelRequests = (staleRequests ++ anyHostRequests ++ localRequests).take(numToCancel) + cancelRequests.foreach(amClient.removeContainerRequest) } } From aa0d4ca8bab08a467645080a5b8a28bf6dd8a042 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 20 Dec 2018 11:22:49 -0800 Subject: [PATCH 0174/1072] [SPARK-25970][ML] Add Instrumentation to PrefixSpan ## What changes were proposed in this pull request? Add Instrumentation to PrefixSpan ## How was this patch tested? existing tests Closes #22971 from zhengruifeng/log_PrefixSpan. Authored-by: zhengruifeng Signed-off-by: Xiangrui Meng --- .../src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala index 2a3413553a6a..b0006a8d4a58 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.fpm import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.param._ import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.fpm.{PrefixSpan => mllibPrefixSpan} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.col @@ -135,7 +136,10 @@ final class PrefixSpan(@Since("2.4.0") override val uid: String) extends Params * - `freq: Long` */ @Since("2.4.0") - def findFrequentSequentialPatterns(dataset: Dataset[_]): DataFrame = { + def findFrequentSequentialPatterns(dataset: Dataset[_]): DataFrame = instrumented { instr => + instr.logDataset(dataset) + instr.logParams(this, params: _*) + val sequenceColParam = $(sequenceCol) val inputType = dataset.schema(sequenceColParam).dataType require(inputType.isInstanceOf[ArrayType] && From 98ecda3e8ef9db5e21b5b9605df09d1653094b9c Mon Sep 17 00:00:00 2001 From: liuxian Date: Fri, 21 Dec 2018 13:01:14 +0800 Subject: [PATCH 0175/1072] [MINOR][SQL] Locality does not need to be implemented ## What changes were proposed in this pull request? `HadoopFileWholeTextReader` and `HadoopFileLinesReader` will be eventually called in `FileSourceScanExec`. In fact, locality has been implemented in `FileScanRDD`, even if we implement it in `HadoopFileWholeTextReader ` and `HadoopFileLinesReader`, it would be useless. So I think these `TODO` can be removed. ## How was this patch tested? N/A Closes #23339 from 10110346/noneededtodo. Authored-by: liuxian Signed-off-by: Wenchen Fan --- .../spark/sql/execution/datasources/HadoopFileLinesReader.scala | 2 +- .../sql/execution/datasources/HadoopFileWholeTextReader.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 00a78f7343c5..57082b40e113 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 @@ -51,7 +51,7 @@ class HadoopFileLinesReader( new Path(new URI(file.filePath)), file.start, file.length, - // TODO: Implement Locality + // The locality is decided by `getPreferredLocations` in `FileScanRDD`. Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) 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 c61a89e6e8c3..f5724f7c5955 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 @@ -40,7 +40,7 @@ class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration) Array(new Path(new URI(file.filePath))), Array(file.start), Array(file.length), - // TODO: Implement Locality + // The locality is decided by `getPreferredLocations` in `FileScanRDD`. Array.empty[String]) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) From 305e9b5ad22b428501fd42d3730d73d2e09ad4c5 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 21 Dec 2018 16:09:30 +0800 Subject: [PATCH 0176/1072] [SPARK-26422][R] Support to disable Hive support in SparkR even for Hadoop versions unsupported by Hive fork ## What changes were proposed in this pull request? Currently, even if I explicitly disable Hive support in SparkR session as below: ```r sparkSession <- sparkR.session("local[4]", "SparkR", Sys.getenv("SPARK_HOME"), enableHiveSupport = FALSE) ``` produces when the Hadoop version is not supported by our Hive fork: ``` java.lang.reflect.InvocationTargetException ... Caused by: java.lang.IllegalArgumentException: Unrecognized Hadoop major version number: 3.1.1.3.1.0.0-78 at org.apache.hadoop.hive.shims.ShimLoader.getMajorVersion(ShimLoader.java:174) at org.apache.hadoop.hive.shims.ShimLoader.loadShims(ShimLoader.java:139) at org.apache.hadoop.hive.shims.ShimLoader.getHadoopShims(ShimLoader.java:100) at org.apache.hadoop.hive.conf.HiveConf$ConfVars.(HiveConf.java:368) ... 43 more Error in handleErrors(returnStatus, conn) : java.lang.ExceptionInInitializerError at org.apache.hadoop.hive.conf.HiveConf.(HiveConf.java:105) at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:348) at org.apache.spark.util.Utils$.classForName(Utils.scala:193) at org.apache.spark.sql.SparkSession$.hiveClassesArePresent(SparkSession.scala:1116) at org.apache.spark.sql.api.r.SQLUtils$.getOrCreateSparkSession(SQLUtils.scala:52) at org.apache.spark.sql.api.r.SQLUtils.getOrCreateSparkSession(SQLUtils.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ``` The root cause is that: ``` SparkSession.hiveClassesArePresent ``` check if the class is loadable or not to check if that's in classpath but `org.apache.hadoop.hive.conf.HiveConf` has a check for Hadoop version as static logic which is executed right away. This throws an `IllegalArgumentException` and that's not caught: https://github.com/apache/spark/blob/36edbac1c8337a4719f90e4abd58d38738b2e1fb/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala#L1113-L1121 So, currently, if users have a Hive built-in Spark with unsupported Hadoop version by our fork (namely 3+), there's no way to use SparkR even though it could work. This PR just propose to change the order of bool comparison so that we can don't execute `SparkSession.hiveClassesArePresent` when: 1. `enableHiveSupport` is explicitly disabled 2. `spark.sql.catalogImplementation` is `in-memory` so that we **only** check `SparkSession.hiveClassesArePresent` when Hive support is explicitly enabled by short circuiting. ## How was this patch tested? It's difficult to write a test since we don't run tests against Hadoop 3 yet. See https://github.com/apache/spark/pull/21588. Manually tested. Closes #23356 from HyukjinKwon/SPARK-26422. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/sql/api/r/SQLUtils.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index becb05cf72ab..e98cab8b56d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -49,9 +49,17 @@ private[sql] object SQLUtils extends Logging { sparkConfigMap: JMap[Object, Object], enableHiveSupport: Boolean): SparkSession = { val spark = - if (SparkSession.hiveClassesArePresent && enableHiveSupport && + if (enableHiveSupport && jsc.sc.conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase(Locale.ROOT) == - "hive") { + "hive" && + // Note that the order of conditions here are on purpose. + // `SparkSession.hiveClassesArePresent` checks if Hive's `HiveConf` is loadable or not; + // however, `HiveConf` itself has some static logic to check if Hadoop version is + // supported or not, which throws an `IllegalArgumentException` if unsupported. + // If this is checked first, there's no way to disable Hive support in the case above. + // So, we intentionally check if Hive classes are loadable or not only when + // Hive support is explicitly enabled by short-circuiting. See also SPARK-26422. + SparkSession.hiveClassesArePresent) { SparkSession.builder().sparkContext(withHiveExternalCatalog(jsc.sc)).getOrCreate() } else { if (enableHiveSupport) { From 8e76d6621aaddb8b73443b14ea2c6eebe9089893 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 21 Dec 2018 10:41:25 -0800 Subject: [PATCH 0177/1072] [SPARK-26267][SS] Retry when detecting incorrect offsets from Kafka ## What changes were proposed in this pull request? Due to [KAFKA-7703](https://issues.apache.org/jira/browse/KAFKA-7703), Kafka may return an earliest offset when we are request a latest offset. This will cause Spark to reprocess data. As per suggestion in KAFKA-7703, we put a position call between poll and seekToEnd to block the fetch request triggered by `poll` before calling `seekToEnd`. In addition, to avoid other unknown issues, we also use the previous known offsets to audit the latest offsets returned by Kafka. If we find some incorrect offsets (a latest offset is less than an offset in `knownOffsets`), we will retry at most `maxOffsetFetchAttempts` times. ## How was this patch tested? Jenkins Closes #23324 from zsxwing/SPARK-26267. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu --- .../kafka010/KafkaContinuousReadSupport.scala | 4 +- .../kafka010/KafkaMicroBatchReadSupport.scala | 19 ++++- .../kafka010/KafkaOffsetRangeCalculator.scala | 2 + .../sql/kafka010/KafkaOffsetReader.scala | 80 +++++++++++++++++-- .../spark/sql/kafka010/KafkaSource.scala | 5 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 48 +++++++++++ 6 files changed, 145 insertions(+), 13 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala index 1753a28fba2f..02dfb9ca2b95 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala @@ -60,7 +60,7 @@ class KafkaContinuousReadSupport( override def initialOffset(): Offset = { val offsets = initialOffsets match { case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) } logInfo(s"Initial offsets: $offsets") @@ -107,7 +107,7 @@ class KafkaContinuousReadSupport( override def needsReconfiguration(config: ScanConfig): Boolean = { val knownPartitions = config.asInstanceOf[KafkaContinuousScanConfig].knownPartitions - offsetReader.fetchLatestOffsets().keySet != knownPartitions + offsetReader.fetchLatestOffsets(None).keySet != knownPartitions } override def toString(): String = s"KafkaSource[$offsetReader]" diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala index bb4de674c3c7..b4f042e93a5d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala @@ -84,7 +84,7 @@ private[kafka010] class KafkaMicroBatchReadSupport( override def latestOffset(start: Offset): Offset = { val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets - val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets() + val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets)) endPartitionOffsets = KafkaSourceOffset(maxOffsetsPerTrigger.map { maxOffsets => rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets) }.getOrElse { @@ -133,10 +133,21 @@ private[kafka010] class KafkaMicroBatchReadSupport( }.toSeq logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) + val fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets + val untilOffsets = endPartitionOffsets + untilOffsets.foreach { case (tp, untilOffset) => + fromOffsets.get(tp).foreach { fromOffset => + if (untilOffset < fromOffset) { + reportDataLoss(s"Partition $tp's offset was changed from " + + s"$fromOffset to $untilOffset, some data may have been missed") + } + } + } + // Calculate offset ranges val offsetRanges = rangeCalculator.getRanges( - fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets, - untilOffsets = endPartitionOffsets, + fromOffsets = fromOffsets, + untilOffsets = untilOffsets, executorLocations = getSortedExecutorList()) // Reuse Kafka consumers only when all the offset ranges have distinct TopicPartitions, @@ -186,7 +197,7 @@ private[kafka010] class KafkaMicroBatchReadSupport( case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaOffsetReader.fetchEarliestOffsets()) case LatestOffsetRangeLimit => - KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets()) + KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss) } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala index fb209c724afb..600879492405 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala @@ -37,6 +37,8 @@ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int * the read tasks of the skewed partitions to multiple Spark tasks. * The number of Spark tasks will be *approximately* `numPartitions`. It can be less or more * depending on rounding errors or Kafka partitions that didn't receive any new data. + * + * Empty ranges (`KafkaOffsetRange.size <= 0`) will be dropped. */ def getRanges( fromOffsets: PartitionOffsetMap, 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 82066697cb95..fc443d22bf5a 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 @@ -21,6 +21,7 @@ import java.{util => ju} import java.util.concurrent.{Executors, ThreadFactory} import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration import scala.util.control.NonFatal @@ -137,6 +138,12 @@ private[kafka010] class KafkaOffsetReader( // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() + + // Call `position` to wait until the potential offset request triggered by `poll(0)` is + // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by + // `poll(0)` may reset offsets that should have been set by another request. + partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) + consumer.pause(partitions) assert(partitions.asScala == partitionOffsets.keySet, "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + @@ -192,19 +199,82 @@ private[kafka010] class KafkaOffsetReader( /** * Fetch the latest offsets for the topic partitions that are indicated * in the [[ConsumerStrategy]]. + * + * Kafka may return earliest offsets when we are requesting latest offsets if `poll` is called + * right before `seekToEnd` (KAFKA-7703). As a workaround, we will call `position` right after + * `poll` to wait until the potential offset request triggered by `poll(0)` is done. + * + * In addition, to avoid other unknown issues, we also use the given `knownOffsets` to audit the + * latest offsets returned by Kafka. If we find some incorrect offsets (a latest offset is less + * than an offset in `knownOffsets`), we will retry at most `maxOffsetFetchAttempts` times. When + * a topic is recreated, the latest offsets may be less than offsets in `knownOffsets`. We cannot + * distinguish this with KAFKA-7703, so we just return whatever we get from Kafka after retrying. */ - def fetchLatestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { + def fetchLatestOffsets( + knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap = runUninterruptibly { withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() + + // Call `position` to wait until the potential offset request triggered by `poll(0)` is + // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by + // `poll(0)` may reset offsets that should have been set by another request. + partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) + consumer.pause(partitions) logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") - consumer.seekToEnd(partitions) - val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got latest offsets for partition : $partitionOffsets") - partitionOffsets + if (knownOffsets.isEmpty) { + consumer.seekToEnd(partitions) + partitions.asScala.map(p => p -> consumer.position(p)).toMap + } else { + var partitionOffsets: PartitionOffsetMap = Map.empty + + /** + * Compare `knownOffsets` and `partitionOffsets`. Returns all partitions that have incorrect + * latest offset (offset in `knownOffsets` is great than the one in `partitionOffsets`). + */ + def findIncorrectOffsets(): Seq[(TopicPartition, Long, Long)] = { + var incorrectOffsets = ArrayBuffer[(TopicPartition, Long, Long)]() + partitionOffsets.foreach { case (tp, offset) => + knownOffsets.foreach(_.get(tp).foreach { knownOffset => + if (knownOffset > offset) { + val incorrectOffset = (tp, knownOffset, offset) + incorrectOffsets += incorrectOffset + } + }) + } + incorrectOffsets + } + + // Retry to fetch latest offsets when detecting incorrect offsets. We don't use + // `withRetriesWithoutInterrupt` to retry because: + // + // - `withRetriesWithoutInterrupt` will reset the consumer for each attempt but a fresh + // consumer has a much bigger chance to hit KAFKA-7703. + // - Avoid calling `consumer.poll(0)` which may cause KAFKA-7703. + var incorrectOffsets: Seq[(TopicPartition, Long, Long)] = Nil + var attempt = 0 + do { + consumer.seekToEnd(partitions) + partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + attempt += 1 + + incorrectOffsets = findIncorrectOffsets() + if (incorrectOffsets.nonEmpty) { + logWarning("Found incorrect offsets in some partitions " + + s"(partition, previous offset, fetched offset): $incorrectOffsets") + if (attempt < maxOffsetFetchAttempts) { + logWarning("Retrying to fetch latest offsets because of incorrect offsets") + Thread.sleep(offsetFetchAttemptIntervalMs) + } + } + } while (incorrectOffsets.nonEmpty && attempt < maxOffsetFetchAttempts) + + logDebug(s"Got latest offsets for partition : $partitionOffsets") + partitionOffsets + } } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 66ec7e0cd084..d65b3cea632c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -130,7 +130,7 @@ private[kafka010] class KafkaSource( metadataLog.get(0).getOrElse { val offsets = startingOffsets match { case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaReader.fetchSpecificOffsets(p, reportDataLoss) } metadataLog.add(0, offsets) @@ -148,7 +148,8 @@ private[kafka010] class KafkaSource( // Make sure initialPartitionOffsets is initialized initialPartitionOffsets - val latest = kafkaReader.fetchLatestOffsets() + val latest = kafkaReader.fetchLatestOffsets( + currentPartitionOffsets.orElse(Some(initialPartitionOffsets))) val offsets = maxOffsetsPerTrigger match { case None => latest 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 5ee76990b54f..61cbb3285a4f 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 @@ -329,6 +329,54 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { ) } + test("subscribe topic by pattern with topic recreation between batches") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-good" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, Array("1", "3")) + testUtils.createTopic(topic2, partitions = 1) + testUtils.sendMessages(topic2, Array("2", "4")) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("kafka.default.api.timeout.ms", "3000") + .option("startingOffsets", "earliest") + .option("subscribePattern", s"$topicPrefix-.*") + + val ds = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + .map(kv => kv._2.toInt) + + testStream(ds)( + StartStream(), + AssertOnQuery { q => + q.processAllAvailable() + true + }, + CheckAnswer(1, 2, 3, 4), + // Restart the stream in this test to make the test stable. When recreating a topic when a + // consumer is alive, it may not be able to see the recreated topic even if a fresh consumer + // has seen it. + StopStream, + // Recreate `topic2` and wait until it's available + WithOffsetSync(new TopicPartition(topic2, 0), expectedOffset = 1) { () => + testUtils.deleteTopic(topic2) + testUtils.createTopic(topic2) + testUtils.sendMessages(topic2, Array("6")) + }, + StartStream(), + ExpectFailure[IllegalStateException](e => { + // The offset of `topic2` should be changed from 2 to 1 + assert(e.getMessage.contains("was changed from 2 to 1")) + }) + ) + } + test("ensure that initial offset are written with an extra byte in the beginning (SPARK-19517)") { withTempDir { metadataPath => val topic = "kafka-initial-offset-current" From d6a5f859848bbd237e19075dd26e1547fb3af417 Mon Sep 17 00:00:00 2001 From: wuyi Date: Fri, 21 Dec 2018 13:21:58 -0600 Subject: [PATCH 0178/1072] [SPARK-26269][YARN] Yarnallocator should have same blacklist behaviour with yarn to maxmize use of cluster resource ## What changes were proposed in this pull request? As I mentioned in jira [SPARK-26269](https://issues.apache.org/jira/browse/SPARK-26269), in order to maxmize the use of cluster resource, this pr try to make `YarnAllocator` have the same blacklist behaviour with YARN. ## How was this patch tested? Added. Closes #23223 from Ngone51/dev-YarnAllocator-should-have-same-blacklist-behaviour-with-YARN. Lead-authored-by: wuyi Co-authored-by: Ngone51 Signed-off-by: Thomas Graves --- .../spark/deploy/yarn/YarnAllocator.scala | 32 ++++++-- .../yarn/YarnAllocatorBlacklistTracker.scala | 4 +- .../YarnAllocatorBlacklistTrackerSuite.scala | 2 +- .../deploy/yarn/YarnAllocatorSuite.scala | 75 ++++++++++++++++++- 4 files changed, 101 insertions(+), 12 deletions(-) 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 54b1ec266113..a3feca5dfd22 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 @@ -607,13 +607,23 @@ private[yarn] class YarnAllocator( val message = "Container killed by YARN for exceeding physical memory limits. " + s"$diag Consider boosting ${EXECUTOR_MEMORY_OVERHEAD.key}." (true, message) - case _ => - // all the failures which not covered above, like: - // disk failure, kill by app master or resource manager, ... - allocatorBlacklistTracker.handleResourceAllocationFailure(hostOpt) - (true, "Container marked as failed: " + containerId + onHostStr + - ". Exit status: " + completedContainer.getExitStatus + - ". Diagnostics: " + completedContainer.getDiagnostics) + case other_exit_status => + // SPARK-26269: follow YARN's blacklisting behaviour(see https://github + // .com/apache/hadoop/blob/228156cfd1b474988bc4fedfbf7edddc87db41e3/had + // oop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/ap + // ache/hadoop/yarn/util/Apps.java#L273 for details) + if (NOT_APP_AND_SYSTEM_FAULT_EXIT_STATUS.contains(other_exit_status)) { + (false, s"Container marked as failed: $containerId$onHostStr" + + s". Exit status: ${completedContainer.getExitStatus}" + + s". Diagnostics: ${completedContainer.getDiagnostics}.") + } else { + // completed container from a bad node + allocatorBlacklistTracker.handleResourceAllocationFailure(hostOpt) + (true, s"Container from a bad node: $containerId$onHostStr" + + s". Exit status: ${completedContainer.getExitStatus}" + + s". Diagnostics: ${completedContainer.getDiagnostics}.") + } + } if (exitCausedByApp) { @@ -739,4 +749,12 @@ private object YarnAllocator { val MEM_REGEX = "[0-9.]+ [KMG]B" val VMEM_EXCEEDED_EXIT_CODE = -103 val PMEM_EXCEEDED_EXIT_CODE = -104 + + val NOT_APP_AND_SYSTEM_FAULT_EXIT_STATUS = Set( + ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + ContainerExitStatus.KILLED_BY_APPMASTER, + ContainerExitStatus.KILLED_AFTER_APP_COMPLETION, + ContainerExitStatus.ABORTED, + ContainerExitStatus.DISKS_FAILED + ) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala index ceac7cda5f8b..268976b62950 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala @@ -120,7 +120,9 @@ private[spark] class YarnAllocatorBlacklistTracker( if (removals.nonEmpty) { logInfo(s"removing nodes from YARN application master's blacklist: $removals") } - amClient.updateBlacklist(additions.asJava, removals.asJava) + if (additions.nonEmpty || removals.nonEmpty) { + amClient.updateBlacklist(additions.asJava, removals.asJava) + } currentBlacklistedYarnNodes = nodesToBlacklist } 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 aeac68e6ed33..201910731e93 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 @@ -87,7 +87,7 @@ class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers // expired blacklisted nodes (simulating a resource request) yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host1", "host2")) // no change is communicated to YARN regarding the blacklisting - verify(amClientMock).updateBlacklist(Collections.emptyList(), Collections.emptyList()) + verify(amClientMock, times(0)).updateBlacklist(Collections.emptyList(), Collections.emptyList()) } test("combining scheduler and allocation blacklist") { 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 b61e7df4420e..53a538dc1de2 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 @@ -17,6 +17,8 @@ package org.apache.spark.deploy.yarn +import java.util.Collections + import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration @@ -114,13 +116,29 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter clock) } - def createContainer(host: String, resource: Resource = containerResource): Container = { - val containerId = ContainerId.newContainerId(appAttemptId, containerNum) + def createContainer( + host: String, + containerNumber: Int = containerNum, + resource: Resource = containerResource): Container = { + val containerId: ContainerId = ContainerId.newContainerId(appAttemptId, containerNum) containerNum += 1 val nodeId = NodeId.newInstance(host, 1000) Container.newInstance(containerId, nodeId, "", resource, RM_REQUEST_PRIORITY, null) } + def createContainers(hosts: Seq[String], containerIds: Seq[Int]): Seq[Container] = { + hosts.zip(containerIds).map{case (host, id) => createContainer(host, id)} + } + + def createContainerStatus( + containerId: ContainerId, + exitStatus: Int, + containerState: ContainerState = ContainerState.COMPLETE, + diagnostics: String = "diagnostics"): ContainerStatus = { + ContainerStatus.newInstance(containerId, containerState, diagnostics, exitStatus) + } + + test("single container allocated") { // request a single container and receive it val handler = createAllocator(1) @@ -148,7 +166,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter Map(YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "gpu" -> "2G")) handler.updateResourceRequests() - val container = createContainer("host1", handler.resource) + val container = createContainer("host1", resource = handler.resource) handler.handleAllocatedContainers(Array(container)) // get amount of memory and vcores from resource, so effectively skipping their validation @@ -417,4 +435,55 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter clock.advance(50 * 1000L) handler.getNumExecutorsFailed should be (0) } + + test("SPARK-26269: YarnAllocator should have same blacklist behaviour with YARN") { + val rmClientSpy = spy(rmClient) + val maxExecutors = 11 + + val handler = createAllocator( + maxExecutors, + rmClientSpy, + Map( + "spark.yarn.blacklist.executor.launch.blacklisting.enabled" -> "true", + "spark.blacklist.application.maxFailedExecutorsPerNode" -> "0")) + handler.updateResourceRequests() + + val hosts = (0 until maxExecutors).map(i => s"host$i") + val ids = 0 to maxExecutors + val containers = createContainers(hosts, ids) + + val nonBlacklistedStatuses = Seq( + ContainerExitStatus.SUCCESS, + ContainerExitStatus.PREEMPTED, + ContainerExitStatus.KILLED_EXCEEDED_VMEM, + ContainerExitStatus.KILLED_EXCEEDED_PMEM, + ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + ContainerExitStatus.KILLED_BY_APPMASTER, + ContainerExitStatus.KILLED_AFTER_APP_COMPLETION, + ContainerExitStatus.ABORTED, + ContainerExitStatus.DISKS_FAILED) + + val nonBlacklistedContainerStatuses = nonBlacklistedStatuses.zipWithIndex.map { + case (exitStatus, idx) => createContainerStatus(containers(idx).getId, exitStatus) + } + + val BLACKLISTED_EXIT_CODE = 1 + val blacklistedStatuses = Seq(ContainerExitStatus.INVALID, BLACKLISTED_EXIT_CODE) + + val blacklistedContainerStatuses = blacklistedStatuses.zip(9 until maxExecutors).map { + case (exitStatus, idx) => createContainerStatus(containers(idx).getId, exitStatus) + } + + handler.handleAllocatedContainers(containers.slice(0, 9)) + handler.processCompletedContainers(nonBlacklistedContainerStatuses) + verify(rmClientSpy, never()) + .updateBlacklist(hosts.slice(0, 9).asJava, Collections.emptyList()) + + handler.handleAllocatedContainers(containers.slice(9, 11)) + handler.processCompletedContainers(blacklistedContainerStatuses) + verify(rmClientSpy) + .updateBlacklist(hosts.slice(9, 10).asJava, Collections.emptyList()) + verify(rmClientSpy) + .updateBlacklist(hosts.slice(10, 11).asJava, Collections.emptyList()) + } } From 8dd29fe36b781d115213b1d6a8446ad04e9239bb Mon Sep 17 00:00:00 2001 From: pgandhi Date: Fri, 21 Dec 2018 11:28:22 -0800 Subject: [PATCH 0179/1072] [SPARK-25642][YARN] Adding two new metrics to record the number of registered connections as well as the number of active connections to YARN Shuffle Service Recently, the ability to expose the metrics for YARN Shuffle Service was added as part of [SPARK-18364](https://github.com/apache/spark/pull/22485). We need to add some metrics to be able to determine the number of active connections as well as open connections to the external shuffle service to benchmark network and connection issues on large cluster environments. Added two more shuffle server metrics for Spark Yarn shuffle service: numRegisteredConnections which indicate the number of registered connections to the shuffle service and numActiveConnections which indicate the number of active connections to the shuffle service at any given point in time. If these metrics are outputted to a file, we get something like this: 1533674653489 default.shuffleService: Hostname=server1.abc.com, openBlockRequestLatencyMillis_count=729, openBlockRequestLatencyMillis_rate15=0.7110833548897356, openBlockRequestLatencyMillis_rate5=1.657808981793011, openBlockRequestLatencyMillis_rate1=2.2404486061620474, openBlockRequestLatencyMillis_rateMean=0.9242558551196706, numRegisteredConnections=35, blockTransferRateBytes_count=2635880512, blockTransferRateBytes_rate15=2578547.6094160094, blockTransferRateBytes_rate5=6048721.726302424, blockTransferRateBytes_rate1=8548922.518223226, blockTransferRateBytes_rateMean=3341878.633637769, registeredExecutorsSize=5, registerExecutorRequestLatencyMillis_count=5, registerExecutorRequestLatencyMillis_rate15=0.0027973949328659836, registerExecutorRequestLatencyMillis_rate5=0.0021278007987206426, registerExecutorRequestLatencyMillis_rate1=2.8270296777387467E-6, registerExecutorRequestLatencyMillis_rateMean=0.006339206380043053, numActiveConnections=35 Closes #22498 from pgandhi999/SPARK-18364. Authored-by: pgandhi Signed-off-by: Marcelo Vanzin --- .../spark/network/TransportContext.java | 9 ++++++- .../server/TransportChannelHandler.java | 18 +++++++++++++- .../spark/network/server/TransportServer.java | 5 ++++ .../shuffle/ExternalShuffleBlockHandler.java | 24 +++++++++++++++++-- .../network/yarn/YarnShuffleService.java | 21 +++++++++------- .../yarn/YarnShuffleServiceMetrics.java | 5 ++++ .../spark/deploy/ExternalShuffleService.scala | 2 ++ .../yarn/YarnShuffleServiceMetricsSuite.scala | 3 ++- 8 files changed, 73 insertions(+), 14 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java index 480b52652de5..1a3f3f2a6f24 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.List; +import com.codahale.metrics.Counter; import io.netty.channel.Channel; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; @@ -66,6 +67,8 @@ public class TransportContext { private final RpcHandler rpcHandler; private final boolean closeIdleConnections; private final boolean isClientOnly; + // Number of registered connections to the shuffle service + private Counter registeredConnections = new Counter(); /** * Force to create MessageEncoder and MessageDecoder so that we can make sure they will be created @@ -221,7 +224,7 @@ private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, rpcHandler, conf.maxChunksBeingTransferred()); return new TransportChannelHandler(client, responseHandler, requestHandler, - conf.connectionTimeoutMs(), closeIdleConnections); + conf.connectionTimeoutMs(), closeIdleConnections, this); } /** @@ -234,4 +237,8 @@ private ChunkFetchRequestHandler createChunkFetchHandler(TransportChannelHandler } public TransportConf getConf() { return conf; } + + public Counter getRegisteredConnections() { + return registeredConnections; + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java index c824a7b0d474..ca81099c4d5c 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -21,6 +21,7 @@ import io.netty.channel.SimpleChannelInboundHandler; import io.netty.handler.timeout.IdleState; import io.netty.handler.timeout.IdleStateEvent; +import org.apache.spark.network.TransportContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,18 +58,21 @@ public class TransportChannelHandler extends SimpleChannelInboundHandler allMetrics; // Time latency for open block request in ms private final Timer openBlockRequestLatencyMillis = new Timer(); @@ -181,14 +183,20 @@ private class ShuffleMetrics implements MetricSet { private final Timer registerExecutorRequestLatencyMillis = new Timer(); // Block transfer rate in byte per second private final Meter blockTransferRateBytes = new Meter(); + // Number of active connections to the shuffle service + private Counter activeConnections = new Counter(); + // Number of registered connections to the shuffle service + private Counter registeredConnections = new Counter(); - private ShuffleMetrics() { + public ShuffleMetrics() { allMetrics = new HashMap<>(); allMetrics.put("openBlockRequestLatencyMillis", openBlockRequestLatencyMillis); allMetrics.put("registerExecutorRequestLatencyMillis", registerExecutorRequestLatencyMillis); allMetrics.put("blockTransferRateBytes", blockTransferRateBytes); allMetrics.put("registeredExecutorsSize", (Gauge) () -> blockManager.getRegisteredExecutorsSize()); + allMetrics.put("numActiveConnections", activeConnections); + allMetrics.put("numRegisteredConnections", registeredConnections); } @Override @@ -244,4 +252,16 @@ public ManagedBuffer next() { } } + @Override + public void channelActive(TransportClient client) { + metrics.activeConnections.inc(); + super.channelActive(client); + } + + @Override + public void channelInactive(TransportClient client) { + metrics.activeConnections.dec(); + super.channelInactive(client); + } + } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 72ae1a129523..7e8d3b2bc3ba 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -170,15 +170,6 @@ protected void serviceInit(Configuration conf) throws Exception { TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); - // register metrics on the block handler into the Node Manager's metrics system. - YarnShuffleServiceMetrics serviceMetrics = - new YarnShuffleServiceMetrics(blockHandler.getAllMetrics()); - - MetricsSystemImpl metricsSystem = (MetricsSystemImpl) DefaultMetricsSystem.instance(); - metricsSystem.register( - "sparkShuffleService", "Metrics on the Spark Shuffle Service", serviceMetrics); - logger.info("Registered metrics with Hadoop's DefaultMetricsSystem"); - // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests List bootstraps = Lists.newArrayList(); @@ -199,6 +190,18 @@ protected void serviceInit(Configuration conf) throws Exception { port = shuffleServer.getPort(); boundPort = port; String authEnabledString = authEnabled ? "enabled" : "not enabled"; + + // register metrics on the block handler into the Node Manager's metrics system. + blockHandler.getAllMetrics().getMetrics().put("numRegisteredConnections", + shuffleServer.getRegisteredConnections()); + YarnShuffleServiceMetrics serviceMetrics = + new YarnShuffleServiceMetrics(blockHandler.getAllMetrics()); + + MetricsSystemImpl metricsSystem = (MetricsSystemImpl) DefaultMetricsSystem.instance(); + metricsSystem.register( + "sparkShuffleService", "Metrics on the Spark Shuffle Service", serviceMetrics); + logger.info("Registered metrics with Hadoop's DefaultMetricsSystem"); + logger.info("Started YARN shuffle service for Spark on port {}. " + "Authentication is {}. Registered executor file is {}", port, authEnabledString, registeredExecutorFile); diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java index 3e4d479b862b..501237407e9b 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java @@ -107,6 +107,11 @@ public static void collectMetric( throw new IllegalStateException( "Not supported class type of metric[" + name + "] for value " + gaugeValue); } + } else if (metric instanceof Counter) { + Counter c = (Counter) metric; + long counterValue = c.getCount(); + metricsRecordBuilder.addGauge(new ShuffleServiceMetricsInfo(name, "Number of " + + "connections to shuffle service " + name), counterValue); } } diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index f6b3c37f0fe7..03e3abb3ce56 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -84,6 +84,8 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana server = transportContext.createServer(port, bootstraps.asJava) shuffleServiceSource.registerMetricSet(server.getAllMetrics) + blockHandler.getAllMetrics.getMetrics.put("numRegisteredConnections", + server.getRegisteredConnections) shuffleServiceSource.registerMetricSet(blockHandler.getAllMetrics) masterMetricsSystem.registerSource(shuffleServiceSource) masterMetricsSystem.start() 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 40b92282a3b8..952fd0b70bb7 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 @@ -38,7 +38,8 @@ class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers { test("metrics named as expected") { val allMetrics = Set( "openBlockRequestLatencyMillis", "registerExecutorRequestLatencyMillis", - "blockTransferRateBytes", "registeredExecutorsSize") + "blockTransferRateBytes", "registeredExecutorsSize", "numActiveConnections", + "numRegisteredConnections") metrics.getMetrics.keySet().asScala should be (allMetrics) } From bba506f8f454c7a8fa82e93a1728e02428fe0d35 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 22 Dec 2018 10:16:27 +0800 Subject: [PATCH 0180/1072] [SPARK-26216][SQL][FOLLOWUP] use abstract class instead of trait for UserDefinedFunction ## What changes were proposed in this pull request? A followup of https://github.com/apache/spark/pull/23178 , to keep binary compability by using abstract class. ## How was this patch tested? Manual test. I created a simple app with Spark 2.4 ``` object TryUDF { def main(args: Array[String]): Unit = { val spark = SparkSession.builder().appName("test").master("local[*]").getOrCreate() import spark.implicits._ val f1 = udf((i: Int) => i + 1) println(f1.deterministic) spark.range(10).select(f1.asNonNullable().apply($"id")).show() spark.stop() } } ``` When I run it with current master, it fails with ``` java.lang.IncompatibleClassChangeError: Found interface org.apache.spark.sql.expressions.UserDefinedFunction, but class was expected ``` When I run it with this PR, it works Closes #23351 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- docs/sql-migration-guide-upgrade.md | 2 -- project/MimaExcludes.scala | 28 ++++++++++++++++++- .../sql/expressions/UserDefinedFunction.scala | 2 +- 3 files changed, 28 insertions(+), 4 deletions(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 115fc6516fb4..1bd3b5ad0e1a 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -33,8 +33,6 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. - - Spark applications which are built with Spark version 2.4 and prior, and call methods of `UserDefinedFunction`, need to be re-compiled with Spark 3.0, as they are not binary compatible with Spark 3.0. - - Since Spark 3.0, CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpuse with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7bb70a29195d..89fc53ce3972 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -241,7 +241,33 @@ object MimaExcludes { // [SPARK-26216][SQL] Do not use case class as public API (UserDefinedFunction) ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.expressions.UserDefinedFunction$"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.sql.expressions.UserDefinedFunction") + ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.sql.expressions.UserDefinedFunction"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.inputTypes"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullableTypes_="), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.dataType"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.f"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.this"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNonNullable"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNonNullable"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullable"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullable"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNondeterministic"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNondeterministic"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.deterministic"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.deterministic"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.apply"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.apply"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.withName"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.withName"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productElement"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productArity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$2"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.canEqual"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$1"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productIterator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productPrefix"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$3") ) // Exclude rules for 2.4.x 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 f88e0e0f299d..901472d8e036 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 @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.DataType * @since 1.3.0 */ @Stable -sealed trait UserDefinedFunction { +sealed abstract class UserDefinedFunction { /** * Returns true when the UDF can return a nullable value. From 81addaa6b7b6f16e477f8dbb26a5d5e9541131b0 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 22 Dec 2018 00:41:21 -0800 Subject: [PATCH 0181/1072] [SPARK-26427][BUILD] Upgrade Apache ORC to 1.5.4 ## What changes were proposed in this pull request? This PR aims to update Apache ORC dependency to the latest version 1.5.4 released at Dec. 20. ([Release Notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12318320&version=12344187])) ``` [ORC-237] OrcFile.mergeFiles Specified block size is less than configured minimum value [ORC-409] Changes for extending MemoryManagerImpl [ORC-410] Fix a locale-dependent test in TestCsvReader [ORC-416] Avoid opening data reader when there is no stripe [ORC-417] Use dynamic Apache Maven mirror link [ORC-419] Ensure to call `close` at RecordReaderImpl constructor exception [ORC-432] openjdk 8 has a bug that prevents surefire from working [ORC-435] Ability to read stripes that are greater than 2GB [ORC-437] Make acid schema checks case insensitive [ORC-411] Update build to work with Java 10. [ORC-418] Fix broken docker build script ``` ## How was this patch tested? Build and pass Jenkins. Closes #23364 from dongjoon-hyun/SPARK-26427. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 6 +++--- dev/deps/spark-deps-hadoop-3.1 | 6 +++--- pom.xml | 6 +++++- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 71423af0789c..1af29fcaff2a 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -155,9 +155,9 @@ objenesis-2.5.1.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.3-nohive.jar -orc-mapreduce-1.5.3-nohive.jar -orc-shims-1.5.3.jar +orc-core-1.5.4-nohive.jar +orc-mapreduce-1.5.4-nohive.jar +orc-shims-1.5.4.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 93eafef04533..05f180b17a58 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -172,9 +172,9 @@ okhttp-2.7.5.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.3-nohive.jar -orc-mapreduce-1.5.3-nohive.jar -orc-shims-1.5.3.jar +orc-core-1.5.4-nohive.jar +orc-mapreduce-1.5.4-nohive.jar +orc-shims-1.5.4.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index 310d7de95512..de9421419edc 100644 --- a/pom.xml +++ b/pom.xml @@ -132,7 +132,7 @@ 2.1.0 10.12.1.1 1.10.0 - 1.5.3 + 1.5.4 nohive 1.6.0 9.4.12.v20180830 @@ -1740,6 +1740,10 @@ ${orc.classifier} ${orc.deps.scope} + + javax.xml.bind + jaxb-api + org.apache.hadoop hadoop-common From ceff0c8450a4f2e31ec52dfc4d101f67c67853c5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 22 Dec 2018 00:43:59 -0800 Subject: [PATCH 0182/1072] [SPARK-26428][SS][TEST] Minimize deprecated `ProcessingTime` usage ## What changes were proposed in this pull request? Use of `ProcessingTime` class was deprecated in favor of `Trigger.ProcessingTime` in Spark 2.2. And, [SPARK-21464](https://issues.apache.org/jira/browse/SPARK-21464) minimized it at 2.2.1. Recently, it grows again in test suites. This PR aims to clean up newly introduced deprecation warnings for Spark 3.0. ## How was this patch tested? Pass the Jenkins with existing tests and manually check the warnings. Closes #23367 from dongjoon-hyun/SPARK-26428. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../kafka010/KafkaMicroBatchSourceSuite.scala | 16 ++++++++-------- .../sql/streaming/FileStreamSourceSuite.scala | 2 +- .../apache/spark/sql/streaming/StreamSuite.scala | 4 ++-- .../streaming/StreamingQueryListenerSuite.scala | 6 +++--- .../sql/streaming/StreamingQuerySuite.scala | 4 ++-- 5 files changed, 16 insertions(+), 16 deletions(-) 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 61cbb3285a4f..d4eb52654005 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 @@ -42,7 +42,7 @@ import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.sql.sources.v2.DataSourceOptions -import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest} +import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSQLContext @@ -236,7 +236,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { } testStream(mapped)( - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, // 1 from smallest, 1 from middle, 8 from biggest CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107), @@ -247,7 +247,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { 11, 108, 109, 110, 111, 112, 113, 114, 115, 116 ), StopStream, - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, // smallest now empty, 1 more from middle, 9 more from biggest CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, @@ -282,7 +282,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { val mapped = kafka.map(kv => kv._2.toInt + 1) testStream(mapped)( - StartStream(trigger = ProcessingTime(1)), + StartStream(trigger = Trigger.ProcessingTime(1)), makeSureGetOffsetCalled, AddKafkaData(Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), @@ -605,7 +605,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { } testStream(kafka)( - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, // 5 from smaller topic, 5 from bigger one CheckLastBatch((0 to 4) ++ (100 to 104): _*), @@ -618,7 +618,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { // smaller topic empty, 5 from bigger one CheckLastBatch(110 to 114: _*), StopStream, - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, // smallest now empty, 5 from bigger one CheckLastBatch(115 to 119: _*), @@ -727,7 +727,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { // The message values are the same as their offsets to make the test easy to follow testUtils.withTranscationalProducer { producer => testStream(mapped)( - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, CheckAnswer(), WithOffsetSync(topicPartition, expectedOffset = 5) { () => @@ -850,7 +850,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { // The message values are the same as their offsets to make the test easy to follow testUtils.withTranscationalProducer { producer => testStream(mapped)( - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, CheckNewAnswer(), WithOffsetSync(topicPartition, expectedOffset = 5) { () => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index d4bd9c7987f2..de664cafed3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -1360,7 +1360,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { options = srcOptions) val clock = new StreamManualClock() testStream(fileStream)( - StartStream(trigger = ProcessingTime(10), triggerClock = clock), + StartStream(trigger = Trigger.ProcessingTime(10), triggerClock = clock), AssertOnQuery { _ => // Block until the first batch finishes. eventually(timeout(streamingTimeout)) { 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 f55ddb5419d2..55fdcee83f11 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 @@ -312,7 +312,7 @@ class StreamSuite extends StreamTest { val inputData = MemoryStream[Int] testStream(inputData.toDS())( - StartStream(ProcessingTime("10 seconds"), new StreamManualClock), + StartStream(Trigger.ProcessingTime("10 seconds"), new StreamManualClock), /* -- batch 0 ----------------------- */ // Add some data in batch 0 @@ -353,7 +353,7 @@ class StreamSuite extends StreamTest { /* Stop then restart the Stream */ StopStream, - StartStream(ProcessingTime("10 seconds"), new StreamManualClock(60 * 1000)), + StartStream(Trigger.ProcessingTime("10 seconds"), new StreamManualClock(60 * 1000)), /* -- batch 1 no rerun ----------------- */ // batch 1 would not re-run because the latest batch id logged in commit log is 1 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 fe77a1b4469c..d00f2e3bf4d1 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 @@ -82,7 +82,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { testStream(df, OutputMode.Append)( // Start event generated when query started - StartStream(ProcessingTime(100), triggerClock = clock), + StartStream(Trigger.ProcessingTime(100), triggerClock = clock), AssertOnQuery { query => assert(listener.startEvent !== null) assert(listener.startEvent.id === query.id) @@ -124,7 +124,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { }, // Termination event generated with exception message when stopped with error - StartStream(ProcessingTime(100), triggerClock = clock), + StartStream(Trigger.ProcessingTime(100), triggerClock = clock), AssertStreamExecThreadToWaitForClock(), AddData(inputData, 0), AdvanceManualClock(100), // process bad data @@ -306,7 +306,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } val clock = new StreamManualClock() val actions = mutable.ArrayBuffer[StreamAction]() - actions += StartStream(trigger = ProcessingTime(10), triggerClock = clock) + actions += StartStream(trigger = Trigger.ProcessingTime(10), triggerClock = clock) for (_ <- 1 to 100) { actions += AdvanceManualClock(10) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index c170641372d6..29b816486a1f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -257,7 +257,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi var lastProgressBeforeStop: StreamingQueryProgress = null testStream(mapped, OutputMode.Complete)( - StartStream(ProcessingTime(1000), triggerClock = clock), + StartStream(Trigger.ProcessingTime(1000), triggerClock = clock), AssertStreamExecThreadIsWaitingForTime(1000), AssertOnQuery(_.status.isDataAvailable === false), AssertOnQuery(_.status.isTriggerActive === false), @@ -370,7 +370,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi AssertOnQuery(_.status.message === "Stopped"), // Test status and progress after query terminated with error - StartStream(ProcessingTime(1000), triggerClock = clock), + StartStream(Trigger.ProcessingTime(1000), triggerClock = clock), AdvanceManualClock(1000), // ensure initial trigger completes before AddData AddData(inputData, 0), AdvanceManualClock(1000), // allow another trigger From c7bfb4cf832d5b20527df6e19855b6a7436988a9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 22 Dec 2018 00:46:36 -0800 Subject: [PATCH 0183/1072] [SPARK-26430][BUILD][TEST-MAVEN] Upgrade Surefire plugin to 3.0.0-M2 ## What changes were proposed in this pull request? This PR aims to upgrade Maven Surefile plugin for JDK11 support. 3.0.0-M2 is [released Dec. 9th.](https://issues.apache.org/jira/projects/SUREFIRE/versions/12344396) ``` [SUREFIRE-1568] Versions 2.21 and higher doesn't work with junit-platform for Java 9 module [SUREFIRE-1605] NoClassDefFoundError (RunNotifier) with JDK 11 [SUREFIRE-1600] Surefire Project using surefire:2.12.4 is not fully able to work with JDK 10+ on internal build system. Therefore surefire-shadefire should go with Surefire:3.0.0-M2. [SUREFIRE-1593] 3.0.0-M1 produces invalid code sources on Windows [SUREFIRE-1602] Surefire fails loading class ForkedBooter when using a sub-directory pom file and a local maven repo [SUREFIRE-1606] maven-shared-utils must not be on provider's classpath [SUREFIRE-1531] Option to switch-off Java 9 modules [SUREFIRE-1590] Deploy multiple versions of Report XSD [SUREFIRE-1591] Java 1.7 feature Diamonds replaced Generics [SUREFIRE-1594] Java 1.7 feature try-catch - multiple exceptions in one catch [SUREFIRE-1595] Java 1.7 feature System.lineSeparator() [SUREFIRE-1597] ModularClasspathForkConfiguration with debug logs (args file and its path on file system) [SUREFIRE-1596] Unnecessary check JAVA_RECENT == JAVA_1_7 in unit tests [SUREFIRE-1598] Fixed typo in assertion statement in integration test Surefire855AllowFailsafeUseArtifactFileIT [SUREFIRE-1607] Roadmap on Project Site ``` ## How was this patch tested? Pass the Jenkins. Closes #23370 from dongjoon-hyun/SPARK-26430. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index de9421419edc..321de209a56a 100644 --- a/pom.xml +++ b/pom.xml @@ -2103,7 +2103,7 @@ org.apache.maven.plugins maven-surefire-plugin - 3.0.0-M1 + 3.0.0-M2 From 0a02d5c36fc5035abcfb930e1a229d65c6cf683f Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Sat, 22 Dec 2018 09:03:02 -0600 Subject: [PATCH 0184/1072] =?UTF-8?q?[SPARK-26285][CORE]=20accumulator=20m?= =?UTF-8?q?etrics=20sources=20for=20LongAccumulator=20and=20Doub=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …leAccumulator ## What changes were proposed in this pull request? This PR implements metric sources for LongAccumulator and DoubleAccumulator, such that a user can register these accumulators easily and have their values be reported by the driver's metric namespace. ## How was this patch tested? Unit tests, and manual tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23242 from abellina/SPARK-26285_accumulator_source. Lead-authored-by: Alessandro Bellina Co-authored-by: Alessandro Bellina Co-authored-by: Alessandro Bellina Signed-off-by: Thomas Graves --- .../metrics/source/AccumulatorSource.scala | 89 ++++++++++++++++++ .../source/AccumulatorSourceSuite.scala | 91 +++++++++++++++++++ .../examples/AccumulatorMetricsTest.scala | 77 ++++++++++++++++ 3 files changed, 257 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/metrics/source/AccumulatorSource.scala create mode 100644 core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala diff --git a/core/src/main/scala/org/apache/spark/metrics/source/AccumulatorSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/AccumulatorSource.scala new file mode 100644 index 000000000000..45a4d224d45f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/source/AccumulatorSource.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics.source + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.util.{AccumulatorV2, DoubleAccumulator, LongAccumulator} + +/** + * AccumulatorSource is a Spark metric Source that reports the current value + * of the accumulator as a gauge. + * + * It is restricted to the LongAccumulator and the DoubleAccumulator, as those + * are the current built-in numerical accumulators with Spark, and excludes + * the CollectionAccumulator, as that is a List of values (hard to report, + * to a metrics system) + */ +private[spark] class AccumulatorSource extends Source { + private val registry = new MetricRegistry + protected def register[T](accumulators: Map[String, AccumulatorV2[_, T]]): Unit = { + accumulators.foreach { + case (name, accumulator) => + val gauge = new Gauge[T] { + override def getValue: T = accumulator.value + } + registry.register(MetricRegistry.name(name), gauge) + } + } + + override def sourceName: String = "AccumulatorSource" + override def metricRegistry: MetricRegistry = registry +} + +@Experimental +class LongAccumulatorSource extends AccumulatorSource + +@Experimental +class DoubleAccumulatorSource extends AccumulatorSource + +/** + * :: Experimental :: + * Metrics source specifically for LongAccumulators. Accumulators + * are only valid on the driver side, so these metrics are reported + * only by the driver. + * Register LongAccumulators using: + * LongAccumulatorSource.register(sc, {"name" -> longAccumulator}) + */ +@Experimental +object LongAccumulatorSource { + def register(sc: SparkContext, accumulators: Map[String, LongAccumulator]): Unit = { + val source = new LongAccumulatorSource + source.register(accumulators) + sc.env.metricsSystem.registerSource(source) + } +} + +/** + * :: Experimental :: + * Metrics source specifically for DoubleAccumulators. Accumulators + * are only valid on the driver side, so these metrics are reported + * only by the driver. + * Register DoubleAccumulators using: + * DoubleAccumulatorSource.register(sc, {"name" -> doubleAccumulator}) + */ +@Experimental +object DoubleAccumulatorSource { + def register(sc: SparkContext, accumulators: Map[String, DoubleAccumulator]): Unit = { + val source = new DoubleAccumulatorSource + source.register(accumulators) + sc.env.metricsSystem.registerSource(source) + } +} diff --git a/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala b/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala new file mode 100644 index 000000000000..6a6c07cb068c --- /dev/null +++ b/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics.source + +import com.codahale.metrics.MetricRegistry +import org.mockito.ArgumentCaptor +import org.mockito.Mockito.{mock, never, spy, times, verify, when} + +import org.apache.spark.{SparkContext, SparkEnv, SparkFunSuite} +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.util.{DoubleAccumulator, LongAccumulator} + +class AccumulatorSourceSuite extends SparkFunSuite { + test("that that accumulators register against the metric system's register") { + val acc1 = new LongAccumulator() + val acc2 = new LongAccumulator() + val mockContext = mock(classOf[SparkContext]) + val mockEnvironment = mock(classOf[SparkEnv]) + val mockMetricSystem = mock(classOf[MetricsSystem]) + when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) + when(mockContext.env) thenReturn (mockEnvironment) + val accs = Map("my-accumulator-1" -> acc1, + "my-accumulator-2" -> acc2) + LongAccumulatorSource.register(mockContext, accs) + val captor = new ArgumentCaptor[AccumulatorSource]() + verify(mockMetricSystem, times(1)).registerSource(captor.capture()) + val source = captor.getValue() + val gauges = source.metricRegistry.getGauges() + assert (gauges.size == 2) + assert (gauges.firstKey == "my-accumulator-1") + assert (gauges.lastKey == "my-accumulator-2") + } + + test("the accumulators value property is checked when the gauge's value is requested") { + val acc1 = new LongAccumulator() + acc1.add(123) + val acc2 = new LongAccumulator() + acc2.add(456) + val mockContext = mock(classOf[SparkContext]) + val mockEnvironment = mock(classOf[SparkEnv]) + val mockMetricSystem = mock(classOf[MetricsSystem]) + when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) + when(mockContext.env) thenReturn (mockEnvironment) + val accs = Map("my-accumulator-1" -> acc1, + "my-accumulator-2" -> acc2) + LongAccumulatorSource.register(mockContext, accs) + val captor = new ArgumentCaptor[AccumulatorSource]() + verify(mockMetricSystem, times(1)).registerSource(captor.capture()) + val source = captor.getValue() + val gauges = source.metricRegistry.getGauges() + assert(gauges.get("my-accumulator-1").getValue() == 123) + assert(gauges.get("my-accumulator-2").getValue() == 456) + } + + test("the double accumulators value propety is checked when the gauge's value is requested") { + val acc1 = new DoubleAccumulator() + acc1.add(123.123) + val acc2 = new DoubleAccumulator() + acc2.add(456.456) + val mockContext = mock(classOf[SparkContext]) + val mockEnvironment = mock(classOf[SparkEnv]) + val mockMetricSystem = mock(classOf[MetricsSystem]) + when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) + when(mockContext.env) thenReturn (mockEnvironment) + val accs = Map( + "my-accumulator-1" -> acc1, + "my-accumulator-2" -> acc2) + DoubleAccumulatorSource.register(mockContext, accs) + val captor = new ArgumentCaptor[AccumulatorSource]() + verify(mockMetricSystem, times(1)).registerSource(captor.capture()) + val source = captor.getValue() + val gauges = source.metricRegistry.getGauges() + assert(gauges.get("my-accumulator-1").getValue() == 123.123) + assert(gauges.get("my-accumulator-2").getValue() == 456.456) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala b/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala new file mode 100644 index 000000000000..5d9a9a73f12e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples + +import org.apache.spark.metrics.source.{DoubleAccumulatorSource, LongAccumulatorSource} +import org.apache.spark.sql.SparkSession + +/** + * Usage: AccumulatorMetricsTest [numElem] + * + * This example shows how to register accumulators against the accumulator source. + * A simple RDD is created, and during the map, the accumulators are incremented. + * + * The only argument, numElem, sets the number elements in the collection to parallize. + * + * The result is output to stdout in the driver with the values of the accumulators. + * For the long accumulator, it should equal numElem the double accumulator should be + * roughly 1.1 x numElem (within double precision.) This example also sets up a + * ConsoleSink (metrics) instance, and so registered codahale metrics (like the + * accumulator source) are reported to stdout as well. + */ +object AccumulatorMetricsTest { + def main(args: Array[String]) { + + val spark = SparkSession + .builder() + .config("spark.metrics.conf.*.sink.console.class", + "org.apache.spark.metrics.sink.ConsoleSink") + .getOrCreate() + + val sc = spark.sparkContext + + val acc = sc.longAccumulator("my-long-metric") + // register the accumulator, the metric system will report as + // [spark.metrics.namespace].[execId|driver].AccumulatorSource.my-long-metric + LongAccumulatorSource.register(sc, List(("my-long-metric" -> acc)).toMap) + + val acc2 = sc.doubleAccumulator("my-double-metric") + // register the accumulator, the metric system will report as + // [spark.metrics.namespace].[execId|driver].AccumulatorSource.my-double-metric + DoubleAccumulatorSource.register(sc, List(("my-double-metric" -> acc2)).toMap) + + val num = if (args.length > 0) args(0).toInt else 1000000 + + val startTime = System.nanoTime + + val accumulatorTest = sc.parallelize(1 to num).foreach(_ => { + acc.add(1) + acc2.add(1.1) + }) + + // Print a footer with test time and accumulator values + println("Test took %.0f milliseconds".format((System.nanoTime - startTime) / 1E6)) + println("Accumulator values:") + println("*** Long accumulator (my-long-metric): " + acc.value) + println("*** Double accumulator (my-double-metric): " + acc2.value) + + spark.stop() + } +} +// scalastyle:on println From 90a810352e94c0b74c19324301e51e8f5bbe98dd Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Sat, 22 Dec 2018 10:32:32 -0600 Subject: [PATCH 0185/1072] [SPARK-25245][DOCS][SS] Explain regarding limiting modification on "spark.sql.shuffle.partitions" for structured streaming ## What changes were proposed in this pull request? This patch adds explanation of `why "spark.sql.shuffle.partitions" keeps unchanged in structured streaming`, which couple of users already wondered and some of them even thought it as a bug. This patch would help other end users to know about such behavior before they find by theirselves and being wondered. ## How was this patch tested? No need to test because this is a simple addition on guide doc with markdown editor. Closes #22238 from HeartSaVioR/SPARK-25245. Lead-authored-by: Jungtaek Lim Co-authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Sean Owen --- docs/structured-streaming-programming-guide.md | 10 ++++++++++ .../scala/org/apache/spark/sql/internal/SQLConf.scala | 8 ++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 32d61dcdb459..e76b53dbb4dc 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -3113,6 +3113,16 @@ See [Input Sources](#input-sources) and [Output Sinks](#output-sinks) sections f # Additional Information +**Notes** + +- Several configurations are not modifiable after the query has run. To change them, discard the checkpoint and start a new query. These configurations include: + - `spark.sql.shuffle.partitions` + - This is due to the physical partitioning of state: state is partitioned via applying hash function to key, hence the number of partitions for state should be unchanged. + - If you want to run fewer tasks for stateful operations, `coalesce` would help with avoiding unnecessary repartitioning. + - After `coalesce`, the number of (reduced) tasks will be kept unless another shuffle happens. + - `spark.sql.streaming.stateStore.providerClass`: To read the previous state of the query properly, the class of state store provider should be unchanged. + - `spark.sql.streaming.multipleWatermarkPolicy`: Modification of this would lead inconsistent watermark value when query contains multiple watermarks, hence the policy should be unchanged. + **Further Reading** - See and run the 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 86e068bf632b..fe445e001935 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 @@ -263,7 +263,9 @@ object SQLConf { .createWithDefault(true) val SHUFFLE_PARTITIONS = buildConf("spark.sql.shuffle.partitions") - .doc("The default number of partitions to use when shuffling data for joins or aggregations.") + .doc("The default number of partitions to use when shuffling data for joins or aggregations. " + + "Note: For structured streaming, this configuration cannot be changed between query " + + "restarts from the same checkpoint location.") .intConf .createWithDefault(200) @@ -882,7 +884,9 @@ object SQLConf { .internal() .doc( "The class used to manage state data in stateful streaming queries. This class must " + - "be a subclass of StateStoreProvider, and must have a zero-arg constructor.") + "be a subclass of StateStoreProvider, and must have a zero-arg constructor. " + + "Note: For structured streaming, this configuration cannot be changed between query " + + "restarts from the same checkpoint location.") .stringConf .createWithDefault( "org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider") From a5a24d92bdf6e6a8e33bdc8833bedba033576b4c Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Sat, 22 Dec 2018 10:35:14 -0800 Subject: [PATCH 0186/1072] [SPARK-26402][SQL] Accessing nested fields with different cases in case insensitive mode ## What changes were proposed in this pull request? GetStructField with different optional names should be semantically equal. We will use this as building block to compare the nested fields used in the plans to be optimized by catalyst optimizer. This PR also fixes a bug below that accessing nested fields with different cases in case insensitive mode will result `AnalysisException`. ``` sql("create table t (s struct) using json") sql("select s.I from t group by s.i") ``` which is currently failing ``` org.apache.spark.sql.AnalysisException: expression 'default.t.`s`' is neither present in the group by, nor is it an aggregate function ``` as cloud-fan pointed out. ## How was this patch tested? New tests are added. Closes #23353 from dbtsai/nestedEqual. Lead-authored-by: DB Tsai Co-authored-by: DB Tsai Signed-off-by: Dongjoon Hyun --- .../catalyst/expressions/Canonicalize.scala | 4 ++- .../expressions/CanonicalizeSuite.scala | 29 ++++++++++++++++++ .../BinaryComparisonSimplificationSuite.scala | 30 +++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 19 ++++++++++++ 4 files changed, 81 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala index fe6db8b344d3..4d218b936b3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala @@ -26,6 +26,7 @@ package org.apache.spark.sql.catalyst.expressions * * The following rules are applied: * - Names and nullability hints for [[org.apache.spark.sql.types.DataType]]s are stripped. + * - Names for [[GetStructField]] are stripped. * - Commutative and associative operations ([[Add]] and [[Multiply]]) have their children ordered * by `hashCode`. * - [[EqualTo]] and [[EqualNullSafe]] are reordered by `hashCode`. @@ -37,10 +38,11 @@ object Canonicalize { expressionReorder(ignoreNamesTypes(e)) } - /** Remove names and nullability from types. */ + /** Remove names and nullability from types, and names from `GetStructField`. */ private[expressions] def ignoreNamesTypes(e: Expression): Expression = e match { case a: AttributeReference => AttributeReference("none", a.dataType.asNullable)(exprId = a.exprId) + case GetStructField(child, ordinal, Some(_)) => GetStructField(child, ordinal, None) case _ => e } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala index 28e6940f3cca..9802a6e5891b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class CanonicalizeSuite extends SparkFunSuite { @@ -50,4 +51,32 @@ class CanonicalizeSuite extends SparkFunSuite { assert(range.where(arrays1).sameResult(range.where(arrays2))) assert(!range.where(arrays1).sameResult(range.where(arrays3))) } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + val expId = NamedExpression.newExprId + val qualifier = Seq.empty[String] + val structType = StructType( + StructField("a", StructType(StructField("b", IntegerType, false) :: Nil), false) :: Nil) + + // GetStructField with different names are semantically equal + val fieldA1 = GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")) + val fieldA2 = GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")) + assert(fieldA1.semanticEquals(fieldA2)) + + val fieldB1 = GetStructField( + GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")), + 0, Some("b1")) + val fieldB2 = GetStructField( + GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")), + 0, Some("b2")) + assert(fieldB1.semanticEquals(fieldB2)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala index a313681eeb8f..5794691a365a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLite import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper { @@ -92,4 +93,33 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper val correctAnswer = nonNullableRelation.analyze comparePlans(actual, correctAnswer) } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + val expId = NamedExpression.newExprId + val qualifier = Seq.empty[String] + val structType = StructType( + StructField("a", StructType(StructField("b", IntegerType, false) :: Nil), false) :: Nil) + + val fieldA1 = GetStructField( + GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")), + 0, Some("b1")) + val fieldA2 = GetStructField( + GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")), + 0, Some("b2")) + + // GetStructField with different names are semantically equal; thus, `EqualTo(fieldA1, fieldA2)` + // will be optimized to `TrueLiteral` by `SimplifyBinaryComparison`. + val originalQuery = nonNullableRelation + .where(EqualTo(fieldA1, fieldA2)) + .analyze + + val optimized = Optimize.execute(originalQuery) + val correctAnswer = nonNullableRelation.analyze + + comparePlans(optimized, correctAnswer) + } } 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 37a8815350a5..656da9fa0180 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 @@ -2937,6 +2937,25 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val msg = intercept[AnalysisException] { + withTable("t") { + sql("create table t (s struct) using json") + checkAnswer(sql("select s.I from t group by s.i"), Nil) + } + }.message + assert(msg.contains("No such struct field I in i")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTable("t") { + sql("create table t (s struct) using json") + checkAnswer(sql("select s.I from t group by s.i"), Nil) + } + } + } } case class Foo(bar: Option[String]) From 1008ab0801c192e8f261001eaaf58a6c9f6e747a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 24 Dec 2018 10:47:47 +0800 Subject: [PATCH 0187/1072] [SPARK-26178][SPARK-26243][SQL][FOLLOWUP] Replacing SimpleDateFormat by DateTimeFormatter in comments ## What changes were proposed in this pull request? The PRs #23150 and #23196 switched JSON and CSV datasources on new formatter for dates/timestamps which is based on `DateTimeFormatter`. In this PR, I replaced `SimpleDateFormat` by `DateTimeFormatter` to reflect the changes. Closes #23374 from MaxGekk/java-time-docs. Authored-by: Maxim Gekk Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/readwriter.py | 28 +++++++++++-------- python/pyspark/sql/streaming.py | 14 ++++++---- .../apache/spark/sql/DataFrameReader.scala | 12 ++++---- .../apache/spark/sql/DataFrameWriter.scala | 12 ++++---- .../sql/streaming/DataStreamReader.scala | 12 ++++---- 5 files changed, 42 insertions(+), 36 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 7b10512a4329..3da052391a95 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -226,11 +226,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, it uses the value specified in ``spark.sql.columnNameOfCorruptRecord``. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.text.SimpleDateFormat``. This + follow the formats at ``java.time.format.DateTimeFormatter``. This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. - :param timestampFormat: sets the string that indicates a timestamp format. Custom date - formats follow the formats at ``java.text.SimpleDateFormat``. + :param timestampFormat: sets the string that indicates a timestamp format. + Custom date formats follow the formats at + ``java.time.format.DateTimeFormatter``. This applies to timestamp type. If None is set, it uses the default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. :param multiLine: parse one record, which may span multiple lines, per file. If None is @@ -406,11 +407,12 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param negativeInf: sets the string representation of a negative infinity value. If None is set, it uses the default value, ``Inf``. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.text.SimpleDateFormat``. This + follow the formats at ``java.time.format.DateTimeFormatter``. This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. - :param timestampFormat: sets the string that indicates a timestamp format. Custom date - formats follow the formats at ``java.text.SimpleDateFormat``. + :param timestampFormat: sets the string that indicates a timestamp format. + Custom date formats follow the formats at + ``java.time.format.DateTimeFormatter``. This applies to timestamp type. If None is set, it uses the default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. :param maxColumns: defines a hard limit of how many columns a record can have. If None is @@ -803,11 +805,12 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.text.SimpleDateFormat``. This + follow the formats at ``java.time.format.DateTimeFormatter``. This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. - :param timestampFormat: sets the string that indicates a timestamp format. Custom date - formats follow the formats at ``java.text.SimpleDateFormat``. + :param timestampFormat: sets the string that indicates a timestamp format. + Custom date formats follow the formats at + ``java.time.format.DateTimeFormatter``. This applies to timestamp type. If None is set, it uses the default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. :param encoding: specifies encoding (charset) of saved json files. If None is set, @@ -904,11 +907,12 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No :param nullValue: sets the string representation of a null value. If None is set, it uses the default value, empty string. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.text.SimpleDateFormat``. This + follow the formats at ``java.time.format.DateTimeFormatter``. This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. - :param timestampFormat: sets the string that indicates a timestamp format. Custom date - formats follow the formats at ``java.text.SimpleDateFormat``. + :param timestampFormat: sets the string that indicates a timestamp format. + Custom date formats follow the formats at + ``java.time.format.DateTimeFormatter``. This applies to timestamp type. If None is set, it uses the default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. :param ignoreLeadingWhiteSpace: a flag indicating whether or not leading whitespaces from diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index fc23b9d99c34..b981fdc4edc7 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -456,11 +456,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, it uses the value specified in ``spark.sql.columnNameOfCorruptRecord``. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.text.SimpleDateFormat``. This + follow the formats at ``java.time.format.DateTimeFormatter``. This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. - :param timestampFormat: sets the string that indicates a timestamp format. Custom date - formats follow the formats at ``java.text.SimpleDateFormat``. + :param timestampFormat: sets the string that indicates a timestamp format. + Custom date formats follow the formats at + ``java.time.format.DateTimeFormatter``. This applies to timestamp type. If None is set, it uses the default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. :param multiLine: parse one record, which may span multiple lines, per file. If None is @@ -630,11 +631,12 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param negativeInf: sets the string representation of a negative infinity value. If None is set, it uses the default value, ``Inf``. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.text.SimpleDateFormat``. This + follow the formats at ``java.time.format.DateTimeFormatter``. This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. - :param timestampFormat: sets the string that indicates a timestamp format. Custom date - formats follow the formats at ``java.text.SimpleDateFormat``. + :param timestampFormat: sets the string that indicates a timestamp format. + Custom date formats follow the formats at + ``java.time.format.DateTimeFormatter``. This applies to timestamp type. If None is set, it uses the default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. :param maxColumns: defines a hard limit of how many columns a record can have. If None is 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 9751528654ff..ce8e4c8f5b82 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 @@ -375,11 +375,11 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.

  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. This applies to timestamp type. *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • *
  • `encoding` (by default it is not set): allows to forcibly set one of standard basic @@ -585,11 +585,11 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity * value.
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. This applies to timestamp type. *
  • `maxColumns` (default `20480`): defines a hard limit of how many columns * a record can have.
  • *
  • `maxCharsPerColumn` (default `-1`): defines the maximum number of characters allowed 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 b9c4076994e9..981b3a8fd4ac 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 @@ -530,11 +530,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. This applies to timestamp type. *
  • `encoding` (by default it is not set): specifies encoding (charset) of saved json * files. If it is not set, the UTF-8 charset will be used.
  • *
  • `lineSep` (default `\n`): defines the line separator that should be used for writing.
  • @@ -649,11 +649,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`). *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. This applies to timestamp type. *
  • `ignoreLeadingWhiteSpace` (default `true`): a flag indicating whether or not leading * whitespaces from values being written should be skipped.
  • *
  • `ignoreTrailingWhiteSpace` (default `true`): a flag indicating defines whether or not 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 914fa90ae7e1..98589da9552c 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 @@ -286,11 +286,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. This applies to timestamp type. *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator @@ -347,11 +347,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
  • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity * value.
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. This applies to timestamp type. *
  • `maxColumns` (default `20480`): defines a hard limit of how many columns * a record can have.
  • *
  • `maxCharsPerColumn` (default `-1`): defines the maximum number of characters allowed From 0523f5e378e69f406104fabaf3ebe913de976bdb Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 23 Dec 2018 21:09:44 -0800 Subject: [PATCH 0188/1072] [SPARK-14023][CORE][SQL] Don't reference 'field' in StructField errors for clarity in exceptions ## What changes were proposed in this pull request? Variation of https://github.com/apache/spark/pull/20500 I cheated by not referencing fields or columns at all as this exception propagates in contexts where both would be applicable. ## How was this patch tested? Existing tests Closes #23373 from srowen/SPARK-14023.2. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/types/StructType.scala | 17 +++++++---------- .../spark/sql/types/StructTypeSuite.scala | 8 ++++---- 2 files changed, 11 insertions(+), 14 deletions(-) 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 6e8bbde7787a..e01d7c59cac5 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 @@ -28,7 +28,6 @@ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, truncatedString} -import org.apache.spark.util.Utils /** * A [[StructType]] object can be constructed by @@ -57,7 +56,7 @@ import org.apache.spark.util.Utils * * // If this struct does not have a field called "d", it throws an exception. * struct("d") - * // java.lang.IllegalArgumentException: Field "d" does not exist. + * // java.lang.IllegalArgumentException: d does not exist. * // ... * * // Extract multiple StructFields. Field names are provided in a set. @@ -69,7 +68,7 @@ import org.apache.spark.util.Utils * // Any names without matching fields will throw an exception. * // For the case shown below, an exception is thrown due to "d". * struct(Set("b", "c", "d")) - * // java.lang.IllegalArgumentException: Field "d" does not exist. + * // java.lang.IllegalArgumentException: d does not exist. * // ... * }}} * @@ -272,22 +271,21 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru def apply(name: String): StructField = { nameToField.getOrElse(name, throw new IllegalArgumentException( - s"""Field "$name" does not exist. - |Available fields: ${fieldNames.mkString(", ")}""".stripMargin)) + s"$name does not exist. Available: ${fieldNames.mkString(", ")}")) } /** * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the * original order of fields. * - * @throws IllegalArgumentException if a field cannot be found for any of the given names + * @throws IllegalArgumentException if at least one given field name does not exist */ def apply(names: Set[String]): StructType = { val nonExistFields = names -- fieldNamesSet if (nonExistFields.nonEmpty) { throw new IllegalArgumentException( - s"""Nonexistent field(s): ${nonExistFields.mkString(", ")}. - |Available fields: ${fieldNames.mkString(", ")}""".stripMargin) + s"${nonExistFields.mkString(", ")} do(es) not exist. " + + s"Available: ${fieldNames.mkString(", ")}") } // Preserve the original order of fields. StructType(fields.filter(f => names.contains(f.name))) @@ -301,8 +299,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru def fieldIndex(name: String): Int = { nameToIndex.getOrElse(name, throw new IllegalArgumentException( - s"""Field "$name" does not exist. - |Available fields: ${fieldNames.mkString(", ")}""".stripMargin)) + s"$name does not exist. Available: ${fieldNames.mkString(", ")}")) } private[sql] def getFieldIndex(name: String): Option[Int] = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index 53a78c94aa6f..b4ce26be24de 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -22,21 +22,21 @@ import org.apache.spark.sql.types.StructType.fromDDL class StructTypeSuite extends SparkFunSuite { - val s = StructType.fromDDL("a INT, b STRING") + private val s = StructType.fromDDL("a INT, b STRING") test("lookup a single missing field should output existing fields") { val e = intercept[IllegalArgumentException](s("c")).getMessage - assert(e.contains("Available fields: a, b")) + assert(e.contains("Available: a, b")) } test("lookup a set of missing fields should output existing fields") { val e = intercept[IllegalArgumentException](s(Set("a", "c"))).getMessage - assert(e.contains("Available fields: a, b")) + assert(e.contains("Available: a, b")) } test("lookup fieldIndex for missing field should output existing fields") { val e = intercept[IllegalArgumentException](s.fieldIndex("c")).getMessage - assert(e.contains("Available fields: a, b")) + assert(e.contains("Available: a, b")) } test("SPARK-24849: toDDL - simple struct") { From 827383a97c11a61661440ff86ce0c3382a2a23b2 Mon Sep 17 00:00:00 2001 From: wangyanlin01 Date: Tue, 25 Dec 2018 15:53:42 +0800 Subject: [PATCH 0189/1072] [SPARK-26426][SQL] fix ExpresionInfo assert error in windows operation system. ## What changes were proposed in this pull request? fix ExpresionInfo assert error in windows operation system, when running unit tests. ## How was this patch tested? unit tests Closes #23363 from yanlin-Lynn/unit-test-windows. Authored-by: wangyanlin01 Signed-off-by: Hyukjin Kwon --- .../apache/spark/sql/catalyst/expressions/ExpressionInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java index ab13ac9cc548..d5a1b77c0ec8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java @@ -79,7 +79,7 @@ public ExpressionInfo( assert name != null; assert arguments != null; assert examples != null; - assert examples.isEmpty() || examples.startsWith("\n Examples:"); + assert examples.isEmpty() || examples.startsWith(System.lineSeparator() + " Examples:"); assert note != null; assert since != null; From 7c7fccfeb5bc079fede41eb64f57ab6b1b4b9018 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 27 Dec 2018 11:09:50 +0800 Subject: [PATCH 0190/1072] [SPARK-26424][SQL] Use java.time API in date/timestamp expressions ## What changes were proposed in this pull request? In the PR, I propose to switch the `DateFormatClass`, `ToUnixTimestamp`, `FromUnixTime`, `UnixTime` on java.time API for parsing/formatting dates and timestamps. The API has been already implemented by the `Timestamp`/`DateFormatter` classes. One of benefit is those classes support parsing timestamps with microsecond precision. Old behaviour can be switched on via SQL config: `spark.sql.legacy.timeParser.enabled` (`false` by default). ## How was this patch tested? It was tested by existing test suites - `DateFunctionsSuite`, `DateExpressionsSuite`, `JsonSuite`, `CsvSuite`, `SQLQueryTestSuite` as well as PySpark tests. Closes #23358 from MaxGekk/new-time-cast. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- R/pkg/R/functions.R | 8 +- docs/sql-migration-guide-upgrade.md | 1 + python/pyspark/sql/functions.py | 6 +- .../sql/catalyst/csv/CSVInferSchema.scala | 3 +- .../expressions/datetimeExpressions.scala | 82 +++++++++++-------- .../sql/catalyst/json/JsonInferSchema.scala | 3 +- .../sql/catalyst/util/DateFormatter.scala | 8 +- .../util/DateTimeFormatterHelper.scala | 21 +++-- .../sql/catalyst/util/DateTimeUtils.scala | 10 --- .../catalyst/util/TimestampFormatter.scala | 22 ++++- .../catalyst/csv/UnivocityParserSuite.scala | 2 +- .../spark/sql/util/DateFormatterSuite.scala | 7 ++ .../sql/util/TimestampFormatterSuite.scala | 12 +++ .../org/apache/spark/sql/functions.scala | 10 +-- .../apache/spark/sql/DateFunctionsSuite.scala | 2 +- 15 files changed, 122 insertions(+), 75 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index f568a931ae1f..5b3cc0940d9c 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -1723,7 +1723,7 @@ setMethod("radians", #' @details #' \code{to_date}: Converts the column into a DateType. You may optionally specify #' a format according to the rules in: -#' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. +#' \url{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}. #' If the string cannot be parsed according to the specified format (or default), #' the value of the column will be null. #' By default, it follows casting rules to a DateType if the format is omitted @@ -1819,7 +1819,7 @@ setMethod("to_csv", signature(x = "Column"), #' @details #' \code{to_timestamp}: Converts the column into a TimestampType. You may optionally specify #' a format according to the rules in: -#' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. +#' \url{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}. #' If the string cannot be parsed according to the specified format (or default), #' the value of the column will be null. #' By default, it follows casting rules to a TimestampType if the format is omitted @@ -2240,7 +2240,7 @@ setMethod("n", signature(x = "Column"), #' \code{date_format}: Converts a date/timestamp/string to a value of string in the format #' specified by the date format given by the second argument. A pattern could be for instance #' \code{dd.MM.yyyy} and could return a string like '18.03.1993'. All -#' pattern letters of \code{java.text.SimpleDateFormat} can be used. +#' pattern letters of \code{java.time.format.DateTimeFormatter} can be used. #' Note: Use when ever possible specialized functions like \code{year}. These benefit from a #' specialized implementation. #' @@ -2666,7 +2666,7 @@ setMethod("format_string", signature(format = "character", x = "Column"), #' \code{from_unixtime}: 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 JVM #' in the given format. -#' See \href{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}{ +#' See \href{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}{ #' Customizing Formats} for available options. #' #' @rdname column_datetime_functions diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 1bd3b5ad0e1a..c4d2157de8b6 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -39,6 +39,7 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. + - Since Spark 3.0, the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions use java.time API for parsing and formatting dates/timestamps from/to strings by using ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html) based on Proleptic Gregorian calendar. In Spark version 2.4 and earlier, java.text.SimpleDateFormat and java.util.GregorianCalendar (hybrid calendar that supports both the Julian and Gregorian calendar systems, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html) is used for the same purpuse. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and 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`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index d188de39e21c..d2a771e9bb8e 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -874,7 +874,7 @@ def date_format(date, format): format given by the second argument. A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All - pattern letters of the Java class `java.text.SimpleDateFormat` can be used. + pattern letters of the Java class `java.time.format.DateTimeFormatter` can be used. .. note:: Use when ever possible specialized functions like `year`. These benefit from a specialized implementation. @@ -1094,7 +1094,7 @@ def to_date(col, format=None): """Converts a :class:`Column` of :class:`pyspark.sql.types.StringType` or :class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType` using the optionally specified format. Specify formats according to - `SimpleDateFormats `_. + `DateTimeFormatter `_. # noqa By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format is omitted (equivalent to ``col.cast("date")``). @@ -1119,7 +1119,7 @@ def to_timestamp(col, format=None): """Converts a :class:`Column` of :class:`pyspark.sql.types.StringType` or :class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType` using the optionally specified format. Specify formats according to - `SimpleDateFormats `_. + `DateTimeFormatter `_. # noqa By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format is omitted (equivalent to ``col.cast("timestamp")``). 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 35ade136cc60..4dd41042856d 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 @@ -27,8 +27,7 @@ import org.apache.spark.sql.types._ class CSVInferSchema(val options: CSVOptions) extends Serializable { - @transient - private lazy val timestampParser = TimestampFormatter( + private val timestampParser = TimestampFormatter( options.timestampFormat, options.timeZone, options.locale) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 45e17ae235a9..73af0a3c5c2e 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 @@ -18,8 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import java.text.DateFormat -import java.util.{Calendar, TimeZone} +import java.util.{Calendar, Locale, TimeZone} import scala.util.control.NonFatal @@ -28,7 +27,8 @@ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -562,16 +562,17 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti copy(timeZoneId = Option(timeZoneId)) override protected def nullSafeEval(timestamp: Any, format: Any): Any = { - val df = DateTimeUtils.newDateFormat(format.toString, timeZone) - UTF8String.fromString(df.format(new java.util.Date(timestamp.asInstanceOf[Long] / 1000))) + val df = TimestampFormatter(format.toString, timeZone, Locale.US) + UTF8String.fromString(df.format(timestamp.asInstanceOf[Long])) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val tf = TimestampFormatter.getClass.getName.stripSuffix("$") val tz = ctx.addReferenceObj("timeZone", timeZone) + val locale = ctx.addReferenceObj("locale", Locale.US) defineCodeGen(ctx, ev, (timestamp, format) => { - s"""UTF8String.fromString($dtu.newDateFormat($format.toString(), $tz) - .format(new java.util.Date($timestamp / 1000)))""" + s"""UTF8String.fromString($tf.apply($format.toString(), $tz, $locale) + .format($timestamp))""" }) } @@ -612,9 +613,10 @@ case class ToUnixTimestamp( } /** - * Converts time string with given pattern. - * (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) - * to Unix time stamp (in seconds), returns null if fail. + * Converts time string with given pattern to Unix time stamp (in seconds), returns null if fail. + * See [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html] + * if SQL config spark.sql.legacy.timeParser.enabled is set to true otherwise + * [https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html]. * Note that hive Language Manual says it returns 0 if fail, but in fact it returns null. * If the second parameter is missing, use "yyyy-MM-dd HH:mm:ss". * If no parameters provided, the first parameter will be current_timestamp. @@ -663,9 +665,9 @@ abstract class UnixTime override def nullable: Boolean = true private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] - private lazy val formatter: DateFormat = + private lazy val formatter: TimestampFormatter = try { - DateTimeUtils.newDateFormat(constFormat.toString, timeZone) + TimestampFormatter(constFormat.toString, timeZone, Locale.US) } catch { case NonFatal(_) => null } @@ -677,16 +679,16 @@ abstract class UnixTime } else { left.dataType match { case DateType => - DateTimeUtils.daysToMillis(t.asInstanceOf[Int], timeZone) / 1000L + DateTimeUtils.daysToMillis(t.asInstanceOf[Int], timeZone) / MILLIS_PER_SECOND case TimestampType => - t.asInstanceOf[Long] / 1000000L + t.asInstanceOf[Long] / MICROS_PER_SECOND case StringType if right.foldable => if (constFormat == null || formatter == null) { null } else { try { formatter.parse( - t.asInstanceOf[UTF8String].toString).getTime / 1000L + t.asInstanceOf[UTF8String].toString) / MICROS_PER_SECOND } catch { case NonFatal(_) => null } @@ -698,8 +700,8 @@ abstract class UnixTime } else { val formatString = f.asInstanceOf[UTF8String].toString try { - DateTimeUtils.newDateFormat(formatString, timeZone).parse( - t.asInstanceOf[UTF8String].toString).getTime / 1000L + TimestampFormatter(formatString, timeZone, Locale.US).parse( + t.asInstanceOf[UTF8String].toString) / MICROS_PER_SECOND } catch { case NonFatal(_) => null } @@ -712,7 +714,7 @@ abstract class UnixTime val javaType = CodeGenerator.javaType(dataType) left.dataType match { case StringType if right.foldable => - val df = classOf[DateFormat].getName + val df = classOf[TimestampFormatter].getName if (formatter == null) { ExprCode.forNullValue(dataType) } else { @@ -724,24 +726,35 @@ abstract class UnixTime $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { try { - ${ev.value} = $formatterName.parse(${eval1.value}.toString()).getTime() / 1000L; + ${ev.value} = $formatterName.parse(${eval1.value}.toString()) / 1000000L; + } catch (java.lang.IllegalArgumentException e) { + ${ev.isNull} = true; } catch (java.text.ParseException e) { ${ev.isNull} = true; + } catch (java.time.format.DateTimeParseException e) { + ${ev.isNull} = true; + } catch (java.time.DateTimeException e) { + ${ev.isNull} = true; } }""") } case StringType => val tz = ctx.addReferenceObj("timeZone", timeZone) - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val locale = ctx.addReferenceObj("locale", Locale.US) + val dtu = TimestampFormatter.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (string, format) => { s""" try { - ${ev.value} = $dtu.newDateFormat($format.toString(), $tz) - .parse($string.toString()).getTime() / 1000L; + ${ev.value} = $dtu.apply($format.toString(), $tz, $locale) + .parse($string.toString()) / 1000000L; } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; } catch (java.text.ParseException e) { ${ev.isNull} = true; + } catch (java.time.format.DateTimeParseException e) { + ${ev.isNull} = true; + } catch (java.time.DateTimeException e) { + ${ev.isNull} = true; } """ }) @@ -806,9 +819,9 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ copy(timeZoneId = Option(timeZoneId)) private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] - private lazy val formatter: DateFormat = + private lazy val formatter: TimestampFormatter = try { - DateTimeUtils.newDateFormat(constFormat.toString, timeZone) + TimestampFormatter(constFormat.toString, timeZone, Locale.US) } catch { case NonFatal(_) => null } @@ -823,8 +836,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ null } else { try { - UTF8String.fromString(formatter.format( - new java.util.Date(time.asInstanceOf[Long] * 1000L))) + UTF8String.fromString(formatter.format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { case NonFatal(_) => null } @@ -835,8 +847,8 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ null } else { try { - UTF8String.fromString(DateTimeUtils.newDateFormat(f.toString, timeZone) - .format(new java.util.Date(time.asInstanceOf[Long] * 1000L))) + UTF8String.fromString(TimestampFormatter(f.toString, timeZone, Locale.US) + .format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { case NonFatal(_) => null } @@ -846,7 +858,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val df = classOf[DateFormat].getName + val df = classOf[TimestampFormatter].getName if (format.foldable) { if (formatter == null) { ExprCode.forNullValue(StringType) @@ -859,8 +871,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { try { - ${ev.value} = UTF8String.fromString($formatterName.format( - new java.util.Date(${t.value} * 1000L))); + ${ev.value} = UTF8String.fromString($formatterName.format(${t.value} * 1000000L)); } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; } @@ -868,12 +879,13 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ } } else { val tz = ctx.addReferenceObj("timeZone", timeZone) - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val locale = ctx.addReferenceObj("locale", Locale.US) + val tf = TimestampFormatter.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (seconds, f) => { s""" try { - ${ev.value} = UTF8String.fromString($dtu.newDateFormat($f.toString(), $tz).format( - new java.util.Date($seconds * 1000L))); + ${ev.value} = UTF8String.fromString($tf.apply($f.toString(), $tz, $locale). + format($seconds * 1000000L)); } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; }""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index d1bc00c08c1c..3203e626ea40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -37,8 +37,7 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { private val decimalParser = ExprUtils.getDecimalParser(options.locale) - @transient - private lazy val timestampFormatter = TimestampFormatter( + private val timestampFormatter = TimestampFormatter( options.timestampFormat, options.timeZone, options.locale) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 9e8d51cc65f0..b4c99674fc1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -26,7 +26,7 @@ import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.sql.internal.SQLConf -sealed trait DateFormatter { +sealed trait DateFormatter extends Serializable { def parse(s: String): Int // returns days since epoch def format(days: Int): String } @@ -35,7 +35,8 @@ class Iso8601DateFormatter( pattern: String, locale: Locale) extends DateFormatter with DateTimeFormatterHelper { - private val formatter = buildFormatter(pattern, locale) + @transient + private lazy val formatter = buildFormatter(pattern, locale) private val UTC = ZoneId.of("UTC") private def toInstant(s: String): Instant = { @@ -56,7 +57,8 @@ class Iso8601DateFormatter( } class LegacyDateFormatter(pattern: String, locale: Locale) extends DateFormatter { - private val format = FastDateFormat.getInstance(pattern, locale) + @transient + private lazy val format = FastDateFormat.getInstance(pattern, locale) override def parse(s: String): Int = { val milliseconds = format.parse(s).getTime 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 b85101d38d9e..91cc57e0bb01 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 @@ -17,27 +17,36 @@ package org.apache.spark.sql.catalyst.util -import java.time.{Instant, LocalDateTime, ZonedDateTime, ZoneId} -import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder} -import java.time.temporal.{ChronoField, TemporalAccessor} +import java.time._ +import java.time.chrono.IsoChronology +import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverStyle} +import java.time.temporal.{ChronoField, TemporalAccessor, TemporalQueries} import java.util.Locale trait DateTimeFormatterHelper { protected def buildFormatter(pattern: String, locale: Locale): DateTimeFormatter = { new DateTimeFormatterBuilder() + .parseCaseInsensitive() .appendPattern(pattern) - .parseDefaulting(ChronoField.YEAR_OF_ERA, 1970) + .parseDefaulting(ChronoField.ERA, 1) .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) - .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) .toFormatter(locale) + .withChronology(IsoChronology.INSTANCE) + .withResolverStyle(ResolverStyle.STRICT) } protected def toInstantWithZoneId(temporalAccessor: TemporalAccessor, zoneId: ZoneId): Instant = { - val localDateTime = LocalDateTime.from(temporalAccessor) + val localTime = if (temporalAccessor.query(TemporalQueries.localTime) == null) { + LocalTime.ofNanoOfDay(0) + } else { + LocalTime.from(temporalAccessor) + } + val localDate = LocalDate.from(temporalAccessor) + val localDateTime = LocalDateTime.of(localDate, localTime) val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) Instant.from(zonedDateTime) } 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 c6dfdbf2505b..3e5e1fbc2b36 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 @@ -111,16 +111,6 @@ object DateTimeUtils { computedTimeZones.computeIfAbsent(timeZoneId, computeTimeZone) } - def newDateFormat(formatString: String, timeZone: TimeZone): DateFormat = { - val sdf = new SimpleDateFormat(formatString, Locale.US) - sdf.setTimeZone(timeZone) - // Enable strict parsing, if the input date/format is invalid, it will throw an exception. - // e.g. to parse invalid date '2016-13-12', or '2016-01-12' with invalid format 'yyyy-aa-dd', - // an exception will be throwed. - sdf.setLenient(false) - sdf - } - // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisUtc: Long): SQLDate = { millisToDays(millisUtc, defaultTimeZone()) 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 eb1303303463..b67b2d7cc3c5 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 @@ -17,7 +17,9 @@ package org.apache.spark.sql.catalyst.util +import java.text.ParseException import java.time._ +import java.time.format.DateTimeParseException import java.time.temporal.TemporalQueries import java.util.{Locale, TimeZone} @@ -27,7 +29,19 @@ import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.sql.internal.SQLConf -sealed trait TimestampFormatter { +sealed trait TimestampFormatter extends Serializable { + /** + * Parses a timestamp in a string and converts it to microseconds. + * + * @param s - string with timestamp to parse + * @return microseconds since epoch. + * @throws ParseException can be thrown by legacy parser + * @throws DateTimeParseException can be thrown by new parser + * @throws DateTimeException unable to obtain local date or time + */ + @throws(classOf[ParseException]) + @throws(classOf[DateTimeParseException]) + @throws(classOf[DateTimeException]) def parse(s: String): Long // returns microseconds since epoch def format(us: Long): String } @@ -36,7 +50,8 @@ class Iso8601TimestampFormatter( pattern: String, timeZone: TimeZone, locale: Locale) extends TimestampFormatter with DateTimeFormatterHelper { - private val formatter = buildFormatter(pattern, locale) + @transient + private lazy val formatter = buildFormatter(pattern, locale) private def toInstant(s: String): Instant = { val temporalAccessor = formatter.parse(s) @@ -68,7 +83,8 @@ class LegacyTimestampFormatter( pattern: String, timeZone: TimeZone, locale: Locale) extends TimestampFormatter { - private val format = FastDateFormat.getInstance(pattern, timeZone, locale) + @transient + private lazy val format = FastDateFormat.getInstance(pattern, timeZone, locale) protected def toMillis(s: String): Long = format.parse(s).getTime diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index 2d0b0d3033a9..4ae61bc61255 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -112,7 +112,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { assert(parser.makeConverter("_1", BooleanType).apply("true") == true) var timestampsOptions = - new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy hh:mm"), false, "GMT") + new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy HH:mm"), false, "GMT") parser = new UnivocityParser(StructType(Seq.empty), timestampsOptions) val customTimestamp = "31/01/2015 00:00" var format = FastDateFormat.getInstance( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala index 019615b81101..2dc55e0e1f63 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.util +import java.time.LocalDate import java.util.Locale import org.apache.spark.SparkFunSuite @@ -89,4 +90,10 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { } } } + + test("parsing date without explicit day") { + val formatter = DateFormatter("yyyy MMM", Locale.US) + val daysSinceEpoch = formatter.parse("2018 Dec") + assert(daysSinceEpoch === LocalDate.of(2018, 12, 1).toEpochDay) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index c110ffa01f73..edccbb2a7f5d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql.util +import java.time.{LocalDateTime, ZoneOffset} import java.util.{Locale, TimeZone} +import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -106,4 +108,14 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { } } } + + test(" case insensitive parsing of am and pm") { + val formatter = TimestampFormatter( + "yyyy MMM dd hh:mm:ss a", + TimeZone.getTimeZone("UTC"), + Locale.US) + val micros = formatter.parse("2009 Mar 20 11:30:01 am") + assert(micros === TimeUnit.SECONDS.toMicros( + LocalDateTime.of(2009, 3, 20, 11, 30, 1).toEpochSecond(ZoneOffset.UTC))) + } } 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 33186f778d86..645452553e6a 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 @@ -2578,7 +2578,7 @@ object functions { * Converts a date/timestamp/string to a value of string in the format specified by the date * format given by the second argument. * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns * * @param dateExpr A date, timestamp or string. If a string, the data must be in a format that * can be cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` @@ -2811,7 +2811,7 @@ object functions { * representing the timestamp of that moment in the current system time zone in the given * format. * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns * * @param ut A number of a type that is castable to a long, such as string or integer. Can be * negative for timestamps before the unix epoch @@ -2855,7 +2855,7 @@ object functions { /** * Converts time string with given pattern to Unix timestamp (in seconds). * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns * * @param s A date, timestamp or string. If a string, the data must be in a format that can be * cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` @@ -2883,7 +2883,7 @@ object functions { /** * Converts time string with the given pattern to timestamp. * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns * * @param s A date, timestamp or string. If a string, the data must be in a format that can be * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` @@ -2908,7 +2908,7 @@ object functions { /** * Converts the column into a `DateType` with a specified format * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns * * @param e A date, timestamp or string. If a string, the data must be in a format that can be * cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` 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 c4ec7150c407..62bb72dd6ea2 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 @@ -405,7 +405,7 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { Row(Date.valueOf("2014-12-31")))) checkAnswer( df.select(to_date(col("s"), "yyyy-MM-dd")), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) + Seq(Row(null), Row(Date.valueOf("2014-12-31")), Row(null))) // now switch format checkAnswer( From f89cdec8b9a9fcc95ba7458869b4ba9d038560f9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 27 Dec 2018 16:03:14 +0800 Subject: [PATCH 0191/1072] [SPARK-26435][SQL] Support creating partitioned table using Hive CTAS by specifying partition column names ## What changes were proposed in this pull request? Spark SQL doesn't support creating partitioned table using Hive CTAS in SQL syntax. However it is supported by using DataFrameWriter API. ```scala val df = Seq(("a", 1)).toDF("part", "id") df.write.format("hive").partitionBy("part").saveAsTable("t") ``` Hive begins to support this syntax in newer version: https://issues.apache.org/jira/browse/HIVE-20241: ``` CREATE TABLE t PARTITIONED BY (part) AS SELECT 1 as id, "a" as part ``` This patch adds this support to SQL syntax. ## How was this patch tested? Added tests. Closes #23376 from viirya/hive-ctas-partitioned-table. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 +- .../spark/sql/execution/SparkSqlParser.scala | 33 ++++++++----- .../sql/hive/execution/HiveDDLSuite.scala | 48 ++++++++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- 4 files changed, 71 insertions(+), 17 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 5e732edb17ba..b39681d886c5 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 @@ -88,7 +88,8 @@ statement (AS? query)? #createTable | createTableHeader ('(' columns=colTypeList ')')? ((COMMENT comment=STRING) | - (PARTITIONED BY '(' partitionColumns=colTypeList ')') | + (PARTITIONED BY '(' partitionColumns=colTypeList ')' | + PARTITIONED BY partitionColumnNames=identifierList) | bucketSpec | skewSpec | rowFormat | 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 364efea52830..8deb55b00a9d 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 @@ -1196,33 +1196,40 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { selectQuery match { case Some(q) => - // Hive does not allow to use a CTAS statement to create a partitioned table. - if (tableDesc.partitionColumnNames.nonEmpty) { - val errorMessage = "A Create Table As Select (CTAS) statement is not allowed to " + - "create a partitioned table using Hive's file formats. " + - "Please use the syntax of \"CREATE TABLE tableName USING dataSource " + - "OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a " + - "CTAS statement." - operationNotAllowed(errorMessage, ctx) - } - // Don't allow explicit specification of schema for CTAS. - if (schema.nonEmpty) { + if (dataCols.nonEmpty) { operationNotAllowed( "Schema may not be specified in a Create Table As Select (CTAS) statement", ctx) } + // When creating partitioned table with CTAS statement, we can't specify data type for the + // partition columns. + if (partitionCols.nonEmpty) { + val errorMessage = "Create Partitioned Table As Select cannot specify data type for " + + "the partition columns of the target table." + operationNotAllowed(errorMessage, ctx) + } + + // Hive CTAS supports dynamic partition by specifying partition column names. + val partitionColumnNames = + Option(ctx.partitionColumnNames) + .map(visitIdentifierList(_).toArray) + .getOrElse(Array.empty[String]) + + val tableDescWithPartitionColNames = + tableDesc.copy(partitionColumnNames = partitionColumnNames) + val hasStorageProperties = (ctx.createFileFormat.size != 0) || (ctx.rowFormat.size != 0) if (conf.convertCTAS && !hasStorageProperties) { // At here, both rowStorage.serdeProperties and fileStorage.serdeProperties // are empty Maps. - val newTableDesc = tableDesc.copy( + val newTableDesc = tableDescWithPartitionColNames.copy( storage = CatalogStorageFormat.empty.copy(locationUri = locUri), provider = Some(conf.defaultDataSourceName)) CreateTable(newTableDesc, mode, Some(q)) } else { - CreateTable(tableDesc, mode, Some(q)) + CreateTable(tableDescWithPartitionColNames, mode, Some(q)) } case None => CreateTable(tableDesc, mode, None) } 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 fd38944a5dd2..6abdc4054cb0 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 @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.execution import java.io.File import java.net.URI -import java.util.Date import scala.language.existentials @@ -33,6 +32,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} 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.execution.command.{DDLSuite, DDLUtils} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.HiveExternalCatalog @@ -2370,4 +2370,50 @@ class HiveDDLSuite )) } } + + test("Hive CTAS can't create partitioned table by specifying schema") { + val err1 = intercept[ParseException] { + spark.sql( + s""" + |CREATE TABLE t (a int) + |PARTITIONED BY (b string) + |STORED AS parquet + |AS SELECT 1 as a, "a" as b + """.stripMargin) + }.getMessage + assert(err1.contains("Schema may not be specified in a Create Table As Select " + + "(CTAS) statement")) + + val err2 = intercept[ParseException] { + spark.sql( + s""" + |CREATE TABLE t + |PARTITIONED BY (b string) + |STORED AS parquet + |AS SELECT 1 as a, "a" as b + """.stripMargin) + }.getMessage + assert(err2.contains("Create Partitioned Table As Select cannot specify data type for " + + "the partition columns of the target table")) + } + + test("Hive CTAS with dynamic partition") { + Seq("orc", "parquet").foreach { format => + withTable("t") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + spark.sql( + s""" + |CREATE TABLE t + |PARTITIONED BY (b) + |STORED AS $format + |AS SELECT 1 as a, "a" as b + """.stripMargin) + checkAnswer(spark.table("t"), Row(1, "a")) + + assert(spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + .partitionColumnNames === Seq("b")) + } + } + } + } } 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 6acf44606cbb..70efad103d13 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 @@ -692,8 +692,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |AS SELECT key, value FROM mytable1 """.stripMargin) }.getMessage - assert(e.contains("A Create Table As Select (CTAS) statement is not allowed to " + - "create a partitioned table using Hive's file formats")) + assert(e.contains("Create Partitioned Table As Select cannot specify data type for " + + "the partition columns of the target table")) } } } From a1c1dd3484a4dcd7c38fe256e69dbaaaf10d1a92 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 27 Dec 2018 11:13:16 +0100 Subject: [PATCH 0192/1072] [SPARK-26191][SQL] Control truncation of Spark plans via maxFields parameter ## What changes were proposed in this pull request? In the PR, I propose to add `maxFields` parameter to all functions involved in creation of textual representation of spark plans such as `simpleString` and `verboseString`. New parameter restricts number of fields converted to truncated strings. Any elements beyond the limit will be dropped and replaced by a `"... N more fields"` placeholder. The threshold is bumped up to `Int.MaxValue` for `toFile()`. ## How was this patch tested? Added a test to `QueryExecutionSuite` which checks `maxFields` impacts on number of truncated fields in `LocalRelation`. Closes #23159 from MaxGekk/to-file-max-fields. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Herman van Hovell --- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 8 ++- .../sql/catalyst/analysis/TypeCoercion.scala | 4 +- .../catalyst/encoders/ExpressionEncoder.scala | 8 ++- .../sql/catalyst/expressions/Expression.scala | 6 +- .../expressions/codegen/javaCode.scala | 2 +- .../sql/catalyst/expressions/generators.scala | 3 +- .../expressions/higherOrderFunctions.scala | 4 +- .../spark/sql/catalyst/expressions/misc.scala | 5 +- .../expressions/namedExpressions.scala | 4 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +- .../catalyst/plans/logical/LogicalPlan.scala | 4 +- .../plans/logical/basicLogicalOperators.scala | 8 +-- .../spark/sql/catalyst/trees/TreeNode.scala | 56 +++++++++++-------- .../spark/sql/catalyst/util/package.scala | 10 ++-- .../apache/spark/sql/types/StructType.scala | 6 +- .../aggregate/PercentileSuite.scala | 2 +- .../org/apache/spark/sql/util/UtilSuite.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 12 ++-- .../spark/sql/execution/ExistingRDD.scala | 6 +- .../spark/sql/execution/QueryExecution.scala | 21 ++++--- .../spark/sql/execution/SparkPlanInfo.scala | 6 +- .../sql/execution/WholeStageCodegenExec.scala | 28 ++++++++-- .../aggregate/HashAggregateExec.scala | 12 ++-- .../aggregate/ObjectHashAggregateExec.scala | 12 ++-- .../aggregate/SortAggregateExec.scala | 12 ++-- .../execution/basicPhysicalOperators.scala | 4 +- .../execution/columnar/InMemoryRelation.scala | 4 +- .../datasources/LogicalRelation.scala | 4 +- .../SaveIntoDataSourceCommand.scala | 2 +- .../spark/sql/execution/datasources/ddl.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 8 ++- .../datasources/v2/DataSourceV2ScanExec.scala | 4 +- .../v2/DataSourceV2StreamingScanExec.scala | 2 +- .../v2/DataSourceV2StringFormat.scala | 6 +- .../spark/sql/execution/debug/package.scala | 3 +- .../apache/spark/sql/execution/limit.scala | 6 +- .../streaming/MicroBatchExecution.scala | 6 +- .../continuous/ContinuousExecution.scala | 7 ++- .../sql/execution/streaming/memory.scala | 5 +- .../apache/spark/sql/execution/subquery.scala | 2 +- .../sql/execution/QueryExecutionSuite.scala | 13 +++++ .../CreateHiveTableAsSelectCommand.scala | 2 +- 43 files changed, 203 insertions(+), 126 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 777053168a05..198645d875c4 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 @@ -979,7 +979,7 @@ class Analyzer( a.mapExpressions(resolveExpressionTopDown(_, appendColumns)) case q: LogicalPlan => - logTrace(s"Attempting to resolve ${q.simpleString}") + logTrace(s"Attempting to resolve ${q.simpleString(SQLConf.get.maxToStringFields)}") q.mapExpressions(resolveExpressionTopDown(_, q)) } @@ -1777,7 +1777,7 @@ class Analyzer( case p if p.expressions.exists(hasGenerator) => throw new AnalysisException("Generators are not supported outside the SELECT clause, but " + - "got: " + p.simpleString) + "got: " + p.simpleString(SQLConf.get.maxToStringFields)) } } 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 88d41e882440..c28a97839fe4 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 @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -303,7 +304,7 @@ trait CheckAnalysis extends PredicateHelper { val missingAttributes = o.missingInput.mkString(",") val input = o.inputSet.mkString(",") val msgForMissingAttributes = s"Resolved attribute(s) $missingAttributes missing " + - s"from $input in operator ${operator.simpleString}." + s"from $input in operator ${operator.simpleString(SQLConf.get.maxToStringFields)}." val resolver = plan.conf.resolver val attrsWithSameName = o.missingInput.filter { missing => @@ -368,7 +369,7 @@ trait CheckAnalysis extends PredicateHelper { s"""nondeterministic expressions are only allowed in |Project, Filter, Aggregate or Window, found: | ${o.expressions.map(_.sql).mkString(",")} - |in operator ${operator.simpleString} + |in operator ${operator.simpleString(SQLConf.get.maxToStringFields)} """.stripMargin) case _: UnresolvedHint => @@ -380,7 +381,8 @@ trait CheckAnalysis extends PredicateHelper { } extendedCheckRules.foreach(_(plan)) plan.foreachUp { - case o if !o.resolved => failAnalysis(s"unresolved operator ${o.simpleString}") + case o if !o.resolved => + failAnalysis(s"unresolved operator ${o.simpleString(SQLConf.get.maxToStringFields)}") case _ => } 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 1706b3eece6d..b19aa50ba215 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 @@ -1069,8 +1069,8 @@ trait TypeCoercionRule extends Rule[LogicalPlan] with Logging { // Leave the same if the dataTypes match. case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => - logDebug( - s"Promoting $a from ${a.dataType} to ${newType.dataType} in ${q.simpleString}") + logDebug(s"Promoting $a from ${a.dataType} to ${newType.dataType} in " + + s" ${q.simpleString(SQLConf.get.maxToStringFields)}") newType } } 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 fbf0bd68b958..da5c1fd0feb0 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 @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, InitializeJavaBean, Invoke, NewInstance} import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LocalRelation} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ObjectType, StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -323,8 +324,8 @@ case class ExpressionEncoder[T]( extractProjection(inputRow) } catch { case e: Exception => - throw new RuntimeException( - s"Error while encoding: $e\n${serializer.map(_.simpleString).mkString("\n")}", e) + throw new RuntimeException(s"Error while encoding: $e\n" + + s"${serializer.map(_.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")}", e) } /** @@ -336,7 +337,8 @@ case class ExpressionEncoder[T]( constructProjection(row).get(0, ObjectType(clsTag.runtimeClass)).asInstanceOf[T] } catch { case e: Exception => - throw new RuntimeException(s"Error while decoding: $e\n${deserializer.simpleString}", e) + throw new RuntimeException(s"Error while decoding: $e\n" + + s"${deserializer.simpleString(SQLConf.get.maxToStringFields)}", e) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index c89c2272be75..d5d119543da7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -259,12 +259,12 @@ abstract class Expression extends TreeNode[Expression] { // Marks this as final, Expression.verboseString should never be called, and thus shouldn't be // overridden by concrete classes. - final override def verboseString: String = simpleString + final override def verboseString(maxFields: Int): String = simpleString(maxFields) - override def simpleString: String = toString + override def simpleString(maxFields: Int): String = toString override def toString: String = prettyName + truncatedString( - flatArguments.toSeq, "(", ", ", ")") + flatArguments.toSeq, "(", ", ", ")", SQLConf.get.maxToStringFields) /** * Returns SQL representation of this expression. For expressions extending [[NonSQLExpression]], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala index 17d4a0dc4e88..17fff64a1b7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala @@ -197,7 +197,7 @@ trait Block extends TreeNode[Block] with JavaCode { case _ => code"$this\n$other" } - override def verboseString: String = toString + override def verboseString(maxFields: Int): String = toString } object Block { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 9c74fdf6c9a1..6b6da1c8b414 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -101,7 +102,7 @@ case class UserDefinedGenerator( inputRow = new InterpretedProjection(children) convertToScala = { val inputSchema = StructType(children.map { e => - StructField(e.simpleString, e.dataType, nullable = true) + StructField(e.simpleString(SQLConf.get.maxToStringFields), e.dataType, nullable = true) }) CatalystTypeConverters.createToScalaConverter(inputSchema) }.asInstanceOf[InternalRow => Row] 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 7141b6e99638..e6cc11d1ad28 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 @@ -76,7 +76,9 @@ case class NamedLambdaVariable( override def toString: String = s"lambda $name#${exprId.id}$typeSuffix" - override def simpleString: String = s"lambda $name#${exprId.id}: ${dataType.simpleString}" + override def simpleString(maxFields: Int): String = { + s"lambda $name#${exprId.id}: ${dataType.simpleString(maxFields)}" + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 0cdeda9b1051..1f1decc45a3f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.RandomUUIDGenerator +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -40,7 +41,7 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { input } - private val outputPrefix = s"Result of ${child.simpleString} is " + private val outputPrefix = s"Result of ${child.simpleString(SQLConf.get.maxToStringFields)} is " override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val outputPrefixField = ctx.addReferenceObj("outputPrefix", outputPrefix) @@ -72,7 +73,7 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa override def prettyName: String = "assert_true" - private val errMsg = s"'${child.simpleString}' is not true!" + private val errMsg = s"'${child.simpleString(SQLConf.get.maxToStringFields)}' is not true!" override def eval(input: InternalRow) : Any = { val v = child.eval(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 02b48f9e30f2..131459bf27bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -330,7 +330,9 @@ case class AttributeReference( // Since the expression id is not in the first constructor it is missing from the default // tree string. - override def simpleString: String = s"$name#${exprId.id}: ${dataType.simpleString}" + override def simpleString(maxFields: Int): String = { + s"$name#${exprId.id}: ${dataType.simpleString(maxFields)}" + } override def sql: String = { val qualifierPrefix = if (qualifier.nonEmpty) qualifier.mkString(".") + "." else "" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index ca0cea6ba7de..125181fb213f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -172,9 +172,9 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT */ protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else "" - override def simpleString: String = statePrefix + super.simpleString + override def simpleString(maxFields: Int): String = statePrefix + super.simpleString(maxFields) - override def verboseString: String = simpleString + override def verboseString(maxFields: Int): String = simpleString(maxFields) /** * All the subqueries of current plan. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 3ad2ee692361..51e0f4b4c84d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -36,8 +36,8 @@ abstract class LogicalPlan /** Returns true if this subtree has data from a streaming data source. */ def isStreaming: Boolean = children.exists(_.isStreaming == true) - override def verboseStringWithSuffix: String = { - super.verboseString + statsCache.map(", " + _.toString).getOrElse("") + override def verboseStringWithSuffix(maxFields: Int): String = { + super.verboseString(maxFields) + statsCache.map(", " + _.toString).getOrElse("") } /** 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 a26ec4eed864..d8b3a4af4f7b 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 @@ -468,7 +468,7 @@ case class View( override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { s"View (${desc.identifier}, ${output.mkString("[", ",", "]")})" } } @@ -484,8 +484,8 @@ case class View( case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { override def output: Seq[Attribute] = child.output - override def simpleString: String = { - val cteAliases = truncatedString(cteRelations.map(_._1), "[", ", ", "]") + override def simpleString(maxFields: Int): String = { + val cteAliases = truncatedString(cteRelations.map(_._1), "[", ", ", "]", maxFields) s"CTE $cteAliases" } @@ -557,7 +557,7 @@ case class Range( override def newInstance(): Range = copy(output = output.map(_.newInstance())) - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { s"Range ($start, $end, step=$step, splits=$numSlices)" } 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 2e9f9f53e94a..21e59bbd283e 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 @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel @@ -433,17 +434,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { private lazy val allChildren: Set[TreeNode[_]] = (children ++ innerChildren).toSet[TreeNode[_]] /** Returns a string representing the arguments to this node, minus any children */ - def argString: String = stringArgs.flatMap { + def argString(maxFields: Int): String = stringArgs.flatMap { case tn: TreeNode[_] if allChildren.contains(tn) => Nil case Some(tn: TreeNode[_]) if allChildren.contains(tn) => Nil - case Some(tn: TreeNode[_]) => tn.simpleString :: Nil - case tn: TreeNode[_] => tn.simpleString :: Nil + case Some(tn: TreeNode[_]) => tn.simpleString(maxFields) :: Nil + case tn: TreeNode[_] => tn.simpleString(maxFields) :: Nil case seq: Seq[Any] if seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil case iter: Iterable[_] if iter.isEmpty => Nil - case seq: Seq[_] => truncatedString(seq, "[", ", ", "]") :: Nil - case set: Set[_] => truncatedString(set.toSeq, "{", ", ", "}") :: Nil + case seq: Seq[_] => truncatedString(seq, "[", ", ", "]", maxFields) :: Nil + case set: Set[_] => truncatedString(set.toSeq, "{", ", ", "}", maxFields) :: Nil case array: Array[_] if array.isEmpty => Nil - case array: Array[_] => truncatedString(array, "[", ", ", "]") :: Nil + case array: Array[_] => truncatedString(array, "[", ", ", "]", maxFields) :: Nil case null => Nil case None => Nil case Some(null) => Nil @@ -456,24 +457,33 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case other => other :: Nil }.mkString(", ") - /** ONE line description of this node. */ - def simpleString: String = s"$nodeName $argString".trim + /** + * ONE line description of this node. + * @param maxFields Maximum number of fields that will be converted to strings. + * Any elements beyond the limit will be dropped. + */ + def simpleString(maxFields: Int): String = { + s"$nodeName ${argString(maxFields)}".trim + } /** ONE line description of this node with more information */ - def verboseString: String + def verboseString(maxFields: Int): String /** ONE line description of this node with some suffix information */ - def verboseStringWithSuffix: String = verboseString + def verboseStringWithSuffix(maxFields: Int): String = verboseString(maxFields) override def toString: String = treeString /** Returns a string representation of the nodes in this tree */ def treeString: String = treeString(verbose = true) - def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { + def treeString( + verbose: Boolean, + addSuffix: Boolean = false, + maxFields: Int = SQLConf.get.maxToStringFields): String = { val writer = new StringBuilderWriter() try { - treeString(writer, verbose, addSuffix) + treeString(writer, verbose, addSuffix, maxFields) writer.toString } finally { writer.close() @@ -483,8 +493,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString( writer: Writer, verbose: Boolean, - addSuffix: Boolean): Unit = { - generateTreeString(0, Nil, writer, verbose, "", addSuffix) + addSuffix: Boolean, + maxFields: Int): Unit = { + generateTreeString(0, Nil, writer, verbose, "", addSuffix, maxFields) } /** @@ -550,7 +561,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { writer: Writer, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): Unit = { + addSuffix: Boolean = false, + maxFields: Int): Unit = { if (depth > 0) { lastChildren.init.foreach { isLast => @@ -560,9 +572,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } val str = if (verbose) { - if (addSuffix) verboseStringWithSuffix else verboseString + if (addSuffix) verboseStringWithSuffix(maxFields) else verboseString(maxFields) } else { - simpleString + simpleString(maxFields) } writer.write(prefix) writer.write(str) @@ -571,17 +583,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { if (innerChildren.nonEmpty) { innerChildren.init.foreach(_.generateTreeString( depth + 2, lastChildren :+ children.isEmpty :+ false, writer, verbose, - addSuffix = addSuffix)) + addSuffix = addSuffix, maxFields = maxFields)) innerChildren.last.generateTreeString( depth + 2, lastChildren :+ children.isEmpty :+ true, writer, verbose, - addSuffix = addSuffix) + addSuffix = addSuffix, maxFields = maxFields) } if (children.nonEmpty) { children.init.foreach(_.generateTreeString( - depth + 1, lastChildren :+ false, writer, verbose, prefix, addSuffix)) + depth + 1, lastChildren :+ false, writer, verbose, prefix, addSuffix, maxFields)) children.last.generateTreeString( - depth + 1, lastChildren :+ true, writer, verbose, prefix, addSuffix) + depth + 1, lastChildren :+ true, writer, verbose, prefix, addSuffix, maxFields) } } @@ -664,7 +676,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { t.forall(_.isInstanceOf[Partitioning]) || t.forall(_.isInstanceOf[DataType]) => JArray(t.map(parseToJson).toList) case t: Seq[_] if t.length > 0 && t.head.isInstanceOf[String] => - JString(truncatedString(t, "[", ", ", "]")) + JString(truncatedString(t, "[", ", ", "]", SQLConf.get.maxToStringFields)) case t: Seq[_] => JNull case m: Map[_, _] => JNull // if it's a scala object, we can simply keep the full class path. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 277584b20dcd..7f5860e12cfd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -184,14 +184,14 @@ package object util extends Logging { start: String, sep: String, end: String, - maxNumFields: Int = SQLConf.get.maxToStringFields): String = { - if (seq.length > maxNumFields) { + maxFields: Int): String = { + if (seq.length > maxFields) { if (truncationWarningPrinted.compareAndSet(false, true)) { logWarning( "Truncated the string representation of a plan since it was too large. This " + s"behavior can be adjusted by setting '${SQLConf.MAX_TO_STRING_FIELDS.key}'.") } - val numFields = math.max(0, maxNumFields - 1) + val numFields = math.max(0, maxFields - 1) seq.take(numFields).mkString( start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) } else { @@ -200,7 +200,9 @@ package object util extends Logging { } /** Shorthand for calling truncatedString() without start or end strings. */ - def truncatedString[T](seq: Seq[T], sep: String): String = truncatedString(seq, "", sep, "") + def truncatedString[T](seq: Seq[T], sep: String, maxFields: Int): String = { + truncatedString(seq, "", sep, "", maxFields) + } /* FIX ME implicit class debugLogging(a: Any) { 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 e01d7c59cac5..d563276a5711 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 @@ -28,6 +28,7 @@ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, truncatedString} +import org.apache.spark.sql.internal.SQLConf /** * A [[StructType]] object can be constructed by @@ -343,7 +344,10 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override def simpleString: String = { val fieldTypes = fields.view.map(field => s"${field.name}:${field.dataType.simpleString}") - truncatedString(fieldTypes, "struct<", ",", ">") + truncatedString( + fieldTypes, + "struct<", ",", ">", + SQLConf.get.maxToStringFields) } override def catalogString: String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala index 63c7b4297802..0e0c8e167a0a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala @@ -215,7 +215,7 @@ class PercentileSuite extends SparkFunSuite { val percentile2 = new Percentile(child, percentage) assertEqual(percentile2.checkInputDataTypes(), TypeCheckFailure(s"Percentage(s) must be between 0.0 and 1.0, " + - s"but got ${percentage.simpleString}")) + s"but got ${percentage.simpleString(100)}")) } val nonFoldablePercentage = Seq(NonFoldableLiteral(0.5), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala index 9c162026942f..d95de71e897a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala @@ -26,6 +26,6 @@ class UtilSuite extends SparkFunSuite { assert(truncatedString(Seq(1, 2), "[", ", ", "]", 2) == "[1, 2]") assert(truncatedString(Seq(1, 2, 3), "[", ", ", "]", 2) == "[1, ... 2 more fields]") assert(truncatedString(Seq(1, 2, 3), "[", ", ", "]", -5) == "[, ... 3 more fields]") - assert(truncatedString(Seq(1, 2, 3), ", ") == "1, 2, 3") + assert(truncatedString(Seq(1, 2, 3), ", ", 10) == "1, 2, 3") } } 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 322ffffca564..1d7dd73706c4 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 @@ -52,19 +52,19 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport { // Metadata that describes more details of this scan. protected def metadata: Map[String, String] - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { val metadataEntries = metadata.toSeq.sorted.map { case (key, value) => key + ": " + StringUtils.abbreviate(redact(value), 100) } - val metadataStr = truncatedString(metadataEntries, " ", ", ", "") - s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]")}$metadataStr" + val metadataStr = truncatedString(metadataEntries, " ", ", ", "", maxFields) + s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]", maxFields)}$metadataStr" } - override def verboseString: String = redact(super.verboseString) + override def verboseString(maxFields: Int): String = redact(super.verboseString(maxFields)) - override def treeString(verbose: Boolean, addSuffix: Boolean): String = { - redact(super.treeString(verbose, addSuffix)) + override def treeString(verbose: Boolean, addSuffix: Boolean, maxFields: Int): String = { + redact(super.treeString(verbose, addSuffix, maxFields)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 49fb288fdea6..981ecae80a72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -79,7 +79,7 @@ case class ExternalRDDScanExec[T]( } } - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { s"$nodeName${output.mkString("[", ",", "]")}" } } @@ -156,8 +156,8 @@ case class RDDScanExec( } } - override def simpleString: String = { - s"$nodeName${truncatedString(output, "[", ",", "]")}" + override def simpleString(maxFields: Int): String = { + s"$nodeName${truncatedString(output, "[", ",", "]", maxFields)}" } // Input can be InternalRow, has to be turned into UnsafeRows. 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 eef5a3f899f5..9b8d2e830867 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.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, TimestampType, _} import org.apache.spark.util.Utils @@ -208,27 +209,27 @@ class QueryExecution( } } - private def writePlans(writer: Writer): Unit = { + private def writePlans(writer: Writer, maxFields: Int): Unit = { val (verbose, addSuffix) = (true, false) writer.write("== Parsed Logical Plan ==\n") - writeOrError(writer)(logical.treeString(_, verbose, addSuffix)) + writeOrError(writer)(logical.treeString(_, verbose, addSuffix, maxFields)) writer.write("\n== Analyzed Logical Plan ==\n") val analyzedOutput = stringOrError(truncatedString( - analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ")) + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields)) writer.write(analyzedOutput) writer.write("\n") - writeOrError(writer)(analyzed.treeString(_, verbose, addSuffix)) + writeOrError(writer)(analyzed.treeString(_, verbose, addSuffix, maxFields)) writer.write("\n== Optimized Logical Plan ==\n") - writeOrError(writer)(optimizedPlan.treeString(_, verbose, addSuffix)) + writeOrError(writer)(optimizedPlan.treeString(_, verbose, addSuffix, maxFields)) writer.write("\n== Physical Plan ==\n") - writeOrError(writer)(executedPlan.treeString(_, verbose, addSuffix)) + writeOrError(writer)(executedPlan.treeString(_, verbose, addSuffix, maxFields)) } override def toString: String = withRedaction { val writer = new StringBuilderWriter() try { - writePlans(writer) + writePlans(writer, SQLConf.get.maxToStringFields) writer.toString } finally { writer.close() @@ -280,14 +281,16 @@ class QueryExecution( /** * Dumps debug information about query execution into the specified file. + * + * @param maxFields maximim number of fields converted to string representation. */ - def toFile(path: String): Unit = { + def toFile(path: String, maxFields: Int = Int.MaxValue): Unit = { val filePath = new Path(path) val fs = filePath.getFileSystem(sparkSession.sessionState.newHadoopConf()) val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) try { - writePlans(writer) + writePlans(writer, maxFields) writer.write("\n== Whole Stage Codegen ==\n") org.apache.spark.sql.execution.debug.writeCodegen(writer, executedPlan) } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 59ffd1638111..f554ff0aa775 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo +import org.apache.spark.sql.internal.SQLConf /** * :: DeveloperApi :: @@ -62,7 +63,10 @@ private[execution] object SparkPlanInfo { case fileScan: FileSourceScanExec => fileScan.metadata case _ => Map[String, String]() } - new SparkPlanInfo(plan.nodeName, plan.simpleString, children.map(fromSparkPlan), + new SparkPlanInfo( + plan.nodeName, + plan.simpleString(SQLConf.get.maxToStringFields), + children.map(fromSparkPlan), metadata, metrics) } } 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 fbda0d87a175..f4927dedabe5 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 @@ -87,7 +87,7 @@ trait CodegenSupport extends SparkPlan { this.parent = parent ctx.freshNamePrefix = variablePrefix s""" - |${ctx.registerComment(s"PRODUCE: ${this.simpleString}")} + |${ctx.registerComment(s"PRODUCE: ${this.simpleString(SQLConf.get.maxToStringFields)}")} |${doProduce(ctx)} """.stripMargin } @@ -188,7 +188,7 @@ trait CodegenSupport extends SparkPlan { parent.doConsume(ctx, inputVars, rowVar) } s""" - |${ctx.registerComment(s"CONSUME: ${parent.simpleString}")} + |${ctx.registerComment(s"CONSUME: ${parent.simpleString(SQLConf.get.maxToStringFields)}")} |$evaluated |$consumeFunc """.stripMargin @@ -496,8 +496,16 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with InputRDDCod writer: Writer, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): Unit = { - child.generateTreeString(depth, lastChildren, writer, verbose, prefix = "", addSuffix = false) + addSuffix: Boolean = false, + maxFields: Int): Unit = { + child.generateTreeString( + depth, + lastChildren, + writer, + verbose, + prefix = "", + addSuffix = false, + maxFields) } override def needCopyResult: Boolean = false @@ -772,8 +780,16 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) writer: Writer, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): Unit = { - child.generateTreeString(depth, lastChildren, writer, verbose, s"*($codegenStageId) ", false) + addSuffix: Boolean = false, + maxFields: Int): Unit = { + child.generateTreeString( + depth, + lastChildren, + writer, + verbose, + s"*($codegenStageId) ", + false, + maxFields) } override def needStopCheck: Boolean = true 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 4827f838fc51..2355d305c38e 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 @@ -922,18 +922,18 @@ case class HashAggregateExec( """ } - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Int): String = toString(verbose = false, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Int): String = { val allAggregateExpressions = aggregateExpressions testFallbackStartsAt match { case None => - val keyString = truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = truncatedString(output, "[", ", ", "]") + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"HashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { 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 7145bb03028d..bd52c6321647 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 @@ -137,15 +137,15 @@ case class ObjectHashAggregateExec( } } - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Int): String = toString(verbose = false, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Int): String = { val allAggregateExpressions = aggregateExpressions - val keyString = truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = truncatedString(output, "[", ", ", "]") + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"ObjectHashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { 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 d732b905dcdd..7ab6ecc08a7b 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 @@ -107,16 +107,16 @@ case class SortAggregateExec( } } - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Int): String = toString(verbose = false, maxFields) - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Int): String = { val allAggregateExpressions = aggregateExpressions - val keyString = truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = truncatedString(output, "[", ", ", "]") + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"SortAggregate(key=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 09effe087e19..2570b36b3166 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -586,7 +586,9 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) } } - override def simpleString: String = s"Range ($start, $end, step=$step, splits=$numSlices)" + override def simpleString(maxFields: Int): String = { + s"Range ($start, $end, step=$step, splits=$numSlices)" + } } /** 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 73eb65f84489..4109d9994dd8 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 @@ -209,6 +209,6 @@ case class InMemoryRelation( override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache) - override def simpleString: String = - s"InMemoryRelation [${truncatedString(output, ", ")}], ${cacheBuilder.storageLevel}" + override def simpleString(maxFields: Int): String = + s"InMemoryRelation [${truncatedString(output, ", ", maxFields)}], ${cacheBuilder.storageLevel}" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 1023572d19e2..db3604fe92cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -63,7 +63,9 @@ case class LogicalRelation( case _ => // Do nothing. } - override def simpleString: String = s"Relation[${truncatedString(output, ",")}] $relation" + override def simpleString(maxFields: Int): String = { + s"Relation[${truncatedString(output, ",", maxFields)}] $relation" + } } object LogicalRelation { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala index 00b1b5dedb59..f29e7869fb27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala @@ -48,7 +48,7 @@ case class SaveIntoDataSourceCommand( Seq.empty[Row] } - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { val redacted = SQLConf.get.redactOptions(options) s"SaveIntoDataSourceCommand ${dataSource}, ${redacted}, ${mode}" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index fdc5e85f3c2e..042320edea4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -68,7 +68,7 @@ case class CreateTempViewUsing( s"Temporary view '$tableIdent' should not have specified a database") } - override def argString: String = { + override def argString(maxFields: Int): String = { s"[tableIdent:$tableIdent " + userSpecifiedSchema.map(_ + " ").getOrElse("") + s"replace:$replace " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 0a6b0afe6cfe..7bf2b8bff373 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -52,8 +52,8 @@ case class DataSourceV2Relation( override def name: String = table.name() - override def simpleString: String = { - s"RelationV2${truncatedString(output, "[", ", ", "]")} $name" + override def simpleString(maxFields: Int): String = { + s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" } def newWriteSupport(): BatchWriteSupport = source.createWriteSupport(options, schema) @@ -96,7 +96,9 @@ case class StreamingDataSourceV2Relation( override def isStreaming: Boolean = true - override def simpleString: String = "Streaming RelationV2 " + metadataString + override def simpleString(maxFields: Int): String = { + "Streaming RelationV2 " + metadataString(maxFields) + } override def pushedFilters: Seq[Expression] = Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala index 725bcc3af3ca..53e4e77c65e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala @@ -35,8 +35,8 @@ case class DataSourceV2ScanExec( @transient batch: Batch) extends LeafExecNode with ColumnarBatchScan { - override def simpleString: String = { - s"ScanV2${truncatedString(output, "[", ", ", "]")} $scanDesc" + override def simpleString(maxFields: Int): String = { + s"ScanV2${truncatedString(output, "[", ", ", "]", maxFields)} $scanDesc" } // TODO: unify the equal/hashCode implementation for all data source v2 query plans. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala index c87294090996..be75fe4f596d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala @@ -42,7 +42,7 @@ case class DataSourceV2StreamingScanExec( @transient scanConfig: ScanConfig) extends LeafExecNode with DataSourceV2StringFormat with ColumnarBatchScan { - override def simpleString: String = "ScanV2 " + metadataString + override def simpleString(maxFields: Int): String = "ScanV2 " + metadataString(maxFields) // TODO: unify the equal/hashCode implementation for all data source v2 query plans. override def equals(other: Any): Boolean = other match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala index e829f621b4ea..f11703c8a277 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala @@ -59,7 +59,7 @@ trait DataSourceV2StringFormat { case _ => Utils.getSimpleName(source.getClass) } - def metadataString: String = { + def metadataString(maxFields: Int): String = { val entries = scala.collection.mutable.ArrayBuffer.empty[(String, String)] if (pushedFilters.nonEmpty) { @@ -73,12 +73,12 @@ trait DataSourceV2StringFormat { }.mkString("[", ",", "]") } - val outputStr = truncatedString(output, "[", ", ", "]") + val outputStr = truncatedString(output, "[", ", ", "]", maxFields) val entriesStr = if (entries.nonEmpty) { truncatedString(entries.map { case (key, value) => key + ": " + StringUtils.abbreviate(value, 100) - }, " (", ", ", ")") + }, " (", ", ", ")", maxFields) } else { "" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 3511cefa7c29..ae8197f617a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, Codegen import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.execution.streaming.{StreamExecution, StreamingQueryWrapper} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.util.{AccumulatorV2, LongAccumulator} @@ -216,7 +217,7 @@ package object debug { val columnStats: Array[ColumnMetrics] = Array.fill(child.output.size)(new ColumnMetrics()) def dumpStats(): Unit = { - debugPrint(s"== ${child.simpleString} ==") + debugPrint(s"== ${child.simpleString(SQLConf.get.maxToStringFields)} ==") debugPrint(s"Tuples output: ${tupleCount.value}") child.output.zip(columnStats).foreach { case (attr, metric) => // This is called on driver. All accumulator updates have a fixed value. So it's safe to use diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index bfaf080292bc..56973af8fd64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -198,9 +198,9 @@ case class TakeOrderedAndProjectExec( override def outputPartitioning: Partitioning = SinglePartition - override def simpleString: String = { - val orderByString = truncatedString(sortOrder, "[", ",", "]") - val outputString = truncatedString(output, "[", ",", "]") + override def simpleString(maxFields: Int): String = { + val orderByString = truncatedString(sortOrder, "[", ",", "]", maxFields) + val outputString = truncatedString(output, "[", ",", "]", maxFields) s"TakeOrderedAndProject(limit=$limit, orderBy=$orderByString, output=$outputString)" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 8ad436a4ff57..38ecb0dd12da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamWriterCommitProgress, WriteToDataSourceV2, WriteToDataSourceV2Exec} import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWritSupport, RateControlMicroBatchReadSupport} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} @@ -482,9 +483,10 @@ class MicroBatchExecution( val newBatchesPlan = logicalPlan transform { case StreamingExecutionRelation(source, output) => newData.get(source).map { dataPlan => + val maxFields = SQLConf.get.maxToStringFields assert(output.size == dataPlan.output.size, - s"Invalid batch: ${truncatedString(output, ",")} != " + - s"${truncatedString(dataPlan.output, ",")}") + s"Invalid batch: ${truncatedString(output, ",", maxFields)} != " + + s"${truncatedString(dataPlan.output, ",", maxFields)}") val aliases = output.zip(dataPlan.output).map { case (to, from) => Alias(from, to.name)(exprId = to.exprId, explicitMetadata = Some(from.metadata)) 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 f0859aaaa304..89033b70f143 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 @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2StreamingScanExec, StreamingDataSourceV2Relation} import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingRelationV2, _} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2 import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, StreamingWriteSupportProvider} import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset} @@ -166,10 +167,10 @@ class ContinuousExecution( val readSupport = continuousSources(insertedSourceId) insertedSourceId += 1 val newOutput = readSupport.fullSchema().toAttributes - + val maxFields = SQLConf.get.maxToStringFields assert(output.size == newOutput.size, - s"Invalid reader: ${truncatedString(output, ",")} != " + - s"${truncatedString(newOutput, ",")}") + s"Invalid reader: ${truncatedString(output, ",", maxFields)} != " + + s"${truncatedString(newOutput, ",", maxFields)}") replacements ++= output.zip(newOutput) val loggedOffset = offsets.offsets(0) 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 daee089f3871..13b75ae4a433 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 @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Stati import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} import org.apache.spark.sql.streaming.OutputMode @@ -117,7 +118,9 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } } - override def toString: String = s"MemoryStream[${truncatedString(output, ",")}]" + override def toString: String = { + s"MemoryStream[${truncatedString(output, ",", SQLConf.get.maxToStringFields)}]" + } override def deserializeOffset(json: String): OffsetV2 = LongOffset(json.toLong) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 310ebcdf6768..e180d2228c3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -51,7 +51,7 @@ case class ScalarSubquery( override def dataType: DataType = plan.schema.fields.head.dataType override def children: Seq[Expression] = Nil override def nullable: Boolean = true - override def toString: String = plan.simpleString + override def toString: String = plan.simpleString(SQLConf.get.maxToStringFields) override def withNewPlan(query: SubqueryExec): ScalarSubquery = copy(plan = query) override def semanticEquals(other: Expression): Boolean = other match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 0c47a2040f17..3cc97c995702 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -106,6 +106,19 @@ class QueryExecutionSuite extends SharedSQLContext { } } + test("check maximum fields restriction") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" + val ds = spark.createDataset(Seq(QueryExecutionTestRecord( + 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))) + ds.queryExecution.debug.toFile(path) + val localRelations = Source.fromFile(path).getLines().filter(_.contains("LocalRelation")) + + assert(!localRelations.exists(_.contains("more fields"))) + } + } + test("toString() exception/error handling") { spark.experimental.extraStrategies = Seq( new SparkStrategy { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 608f21e72625..7249eacfbf9a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -83,7 +83,7 @@ trait CreateHiveTableAsSelectBase extends DataWritingCommand { tableDesc: CatalogTable, tableExists: Boolean): DataWritingCommand - override def argString: String = { + override def argString(maxFields: Int): String = { s"[Database:${tableDesc.database}, " + s"TableName: ${tableDesc.identifier.table}, " + s"InsertIntoHiveTable]" From add287f397d41c1725464dff89d4a555ffc9db04 Mon Sep 17 00:00:00 2001 From: Kevin Yu Date: Thu, 27 Dec 2018 22:26:37 +0800 Subject: [PATCH 0193/1072] [SPARK-25892][SQL] Change AttributeReference.withMetadata's return type to AttributeReference ## What changes were proposed in this pull request? Currently the `AttributeReference.withMetadata` method have return type `Attribute`, the rest of with methods in the `AttributeReference` return type are `AttributeReference`, as the [spark-25892](https://issues.apache.org/jira/browse/SPARK-25892?jql=project%20%3D%20SPARK%20AND%20component%20in%20(ML%2C%20PySpark%2C%20SQL)) mentioned. This PR will change `AttributeReference.withMetadata` method's return type from `Attribute` to `AttributeReference`. ## How was this patch tested? Run all `sql/test,` `catalyst/test` and `org.apache.spark.sql.execution.streaming.*` Closes #22918 from kevinyu98/spark-25892. Authored-by: Kevin Yu Signed-off-by: Hyukjin Kwon --- .../spark/sql/catalyst/expressions/namedExpressions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 131459bf27bc..7ebb171f34ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -311,7 +311,7 @@ case class AttributeReference( } } - override def withMetadata(newMetadata: Metadata): Attribute = { + override def withMetadata(newMetadata: Metadata): AttributeReference = { AttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier) } From 68496c1af310aadfb1b226cb05be510252769d43 Mon Sep 17 00:00:00 2001 From: deepyaman Date: Fri, 28 Dec 2018 00:02:41 +0800 Subject: [PATCH 0194/1072] [SPARK-26451][SQL] Change lead/lag argument name from count to offset ## What changes were proposed in this pull request? Change aligns argument name with that in Scala version and documentation. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23357 from deepyaman/patch-1. Authored-by: deepyaman Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index d2a771e9bb8e..3c33e2bed92d 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -798,7 +798,7 @@ def factorial(col): # --------------- Window functions ------------------------ @since(1.4) -def lag(col, count=1, default=None): +def lag(col, offset=1, default=None): """ Window function: returns the value that is `offset` rows before the current row, and `defaultValue` if there is less than `offset` rows before the current row. For example, @@ -807,15 +807,15 @@ def lag(col, count=1, default=None): This is equivalent to the LAG function in SQL. :param col: name of column or expression - :param count: number of row to extend + :param offset: number of row to extend :param default: default value """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.lag(_to_java_column(col), count, default)) + return Column(sc._jvm.functions.lag(_to_java_column(col), offset, default)) @since(1.4) -def lead(col, count=1, default=None): +def lead(col, offset=1, default=None): """ Window function: returns the value that is `offset` rows after the current row, and `defaultValue` if there is less than `offset` rows after the current row. For example, @@ -824,11 +824,11 @@ def lead(col, count=1, default=None): This is equivalent to the LEAD function in SQL. :param col: name of column or expression - :param count: number of row to extend + :param offset: number of row to extend :param default: default value """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.lead(_to_java_column(col), count, default)) + return Column(sc._jvm.functions.lead(_to_java_column(col), offset, default)) @since(1.4) From f2adb610680f9addec51bf470acf64f3849073e9 Mon Sep 17 00:00:00 2001 From: wuqingxin Date: Fri, 28 Dec 2018 00:15:57 -0800 Subject: [PATCH 0195/1072] [SPARK-26446][CORE] Add cachedExecutorIdleTimeout docs at ExecutorAllocationManager ## What changes were proposed in this pull request? Add docs to describe how remove policy act while considering the property `spark.dynamicAllocation.cachedExecutorIdleTimeout` in ExecutorAllocationManager ## How was this patch tested? comment-only PR. Closes #23386 from TopGunViper/SPARK-26446. Authored-by: wuqingxin Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/ExecutorAllocationManager.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index c3e5b96a5588..3f0b71bbe17f 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -57,7 +57,8 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} * a long time to ramp up under heavy workloads. * * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not - * been scheduled to run any tasks, then it is removed. + * been scheduled to run any tasks, then it is removed. Note that an executor caching any data + * blocks will be removed if it has been idle for more than L seconds. * * There is no retry logic in either case because we make the assumption that the cluster manager * will eventually fulfill all requests it receives asynchronously. @@ -81,7 +82,12 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} * This is used only after the initial backlog timeout is exceeded * * spark.dynamicAllocation.executorIdleTimeout (K) - - * If an executor has been idle for this duration, remove it + * If an executor without caching any data blocks has been idle for this duration, remove it + * + * spark.dynamicAllocation.cachedExecutorIdleTimeout (L) - + * If an executor with caching data blocks has been idle for more than this duration, + * the executor will be removed + * */ private[spark] class ExecutorAllocationManager( client: ExecutorAllocationClient, From 5bef4fedfe1916320223b1245bacb58f151cee66 Mon Sep 17 00:00:00 2001 From: seancxmao Date: Fri, 28 Dec 2018 07:40:59 -0600 Subject: [PATCH 0196/1072] [SPARK-26444][WEBUI] Stage color doesn't change with it's status ## What changes were proposed in this pull request? On job page, in event timeline section, stage color doesn't change according to its status. Below are some screenshots. ACTIVE: active COMPLETE: complete FAILED: failed This PR lets stage color change with it's status. The main idea is to make css style class name match the corresponding stage status. ## How was this patch tested? Manually tested locally. ``` // active/complete stage sc.parallelize(1 to 3, 3).map { n => Thread.sleep(10* 1000); n }.count // failed stage sc.parallelize(1 to 3, 3).map { n => Thread.sleep(10* 1000); throw new Exception() }.count ``` Note we need to clear browser cache to let new `timeline-view.css` take effect. Below are screenshots after this PR. ACTIVE: active-after COMPLETE: complete-after FAILED: failed-after Closes #23385 from seancxmao/timeline-stage-color. Authored-by: seancxmao Signed-off-by: Sean Owen --- .../org/apache/spark/ui/static/timeline-view.css | 8 ++++---- .../src/main/scala/org/apache/spark/ui/jobs/JobPage.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css index 3bf3e8bfa1f3..10bceae2fbdd 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css @@ -98,12 +98,12 @@ rect.getting-result-time-proportion { cursor: pointer; } -.vis-timeline .vis-item.stage.succeeded { +.vis-timeline .vis-item.stage.complete { background-color: #A0DFFF; border-color: #3EC0FF; } -.vis-timeline .vis-item.stage.succeeded.vis-selected { +.vis-timeline .vis-item.stage.complete.vis-selected { background-color: #A0DFFF; border-color: #3EC0FF; z-index: auto; @@ -130,12 +130,12 @@ rect.getting-result-time-proportion { stroke: #FF4D6D; } -.vis-timeline .vis-item.stage.running { +.vis-timeline .vis-item.stage.active { background-color: #A2FCC0; border-color: #36F572; } -.vis-timeline .vis-item.stage.running.vis-selected { +.vis-timeline .vis-item.stage.active.vis-selected { background-color: #A2FCC0; border-color: #36F572; z-index: auto; 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 b58a6ca447ed..cd82439223b0 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 @@ -62,7 +62,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP val stageId = stage.stageId val attemptId = stage.attemptId val name = stage.name - val status = stage.status.toString + val status = stage.status.toString.toLowerCase(Locale.ROOT) val submissionTime = stage.submissionTime.get.getTime() val completionTime = stage.completionTime.map(_.getTime()) .getOrElse(System.currentTimeMillis()) From e0054b88a1624ec5196dc206997db065731099ac Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 28 Dec 2018 11:29:06 -0800 Subject: [PATCH 0197/1072] [SPARK-26424][SQL][FOLLOWUP] Fix DateFormatClass/UnixTime codegen ## What changes were proposed in this pull request? This PR fixes the codegen bug introduced by #23358 . - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.11/158/ ``` Line 44, Column 93: A method named "apply" is not declared in any enclosing class nor any supertype, nor through a static import ``` ## How was this patch tested? Manual. `DateExpressionsSuite` should be passed with Scala-2.11. Closes #23394 from dongjoon-hyun/SPARK-26424. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/expressions/datetimeExpressions.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 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 73af0a3c5c2e..8fc0112c0257 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 @@ -571,7 +571,7 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti val tz = ctx.addReferenceObj("timeZone", timeZone) val locale = ctx.addReferenceObj("locale", Locale.US) defineCodeGen(ctx, ev, (timestamp, format) => { - s"""UTF8String.fromString($tf.apply($format.toString(), $tz, $locale) + s"""UTF8String.fromString($tf$$.MODULE$$.apply($format.toString(), $tz, $locale) .format($timestamp))""" }) } @@ -741,11 +741,11 @@ abstract class UnixTime case StringType => val tz = ctx.addReferenceObj("timeZone", timeZone) val locale = ctx.addReferenceObj("locale", Locale.US) - val dtu = TimestampFormatter.getClass.getName.stripSuffix("$") + val tf = TimestampFormatter.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (string, format) => { s""" try { - ${ev.value} = $dtu.apply($format.toString(), $tz, $locale) + ${ev.value} = $tf$$.MODULE$$.apply($format.toString(), $tz, $locale) .parse($string.toString()) / 1000000L; } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; From e63243df8aca9f44255879e931e0c372beef9fc2 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Sat, 29 Dec 2018 12:11:45 -0800 Subject: [PATCH 0198/1072] [SPARK-26496][SS][TEST] Avoid to use Random.nextString in StreamingInnerJoinSuite MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Similar with https://github.com/apache/spark/pull/21446. Looks random string is not quite safe as a directory name. ```scala scala> val prefix = Random.nextString(10); val dir = new File("/tmp", "del_" + prefix + "-" + UUID.randomUUID.toString); dir.mkdirs() prefix: String = 窽텘⒘駖ⵚ駢⡞Ρ닋੎ dir: java.io.File = /tmp/del_窽텘⒘駖ⵚ駢⡞Ρ닋੎-a3f99855-c429-47a0-a108-47bca6905745 res40: Boolean = false // nope, didn't like this one ``` ## How was this patch tested? Unit test was added, and manually. Closes #23405 from HyukjinKwon/SPARK-26496. Authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/streaming/StreamingJoinSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c5cc8df4356a..42fe9f34ee3e 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 @@ -350,7 +350,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with withTempDir { tempDir => val queryId = UUID.randomUUID val opId = 0 - val path = Utils.createDirectory(tempDir.getAbsolutePath, Random.nextString(10)).toString + val path = Utils.createDirectory(tempDir.getAbsolutePath, Random.nextFloat.toString).toString val stateInfo = StatefulOperatorStateInfo(path, queryId, opId, 0L, 5) implicit val sqlContext = spark.sqlContext From e6d3e7d0d8c80adaa51b43d76f1cc83bb9a010b9 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 29 Dec 2018 17:33:43 -0800 Subject: [PATCH 0199/1072] [SPARK-26443][CORE] Use ConfigEntry for hardcoded configs for history category. ## What changes were proposed in this pull request? This pr makes hardcoded "spark.history" configs to use `ConfigEntry` and put them in `History` config object. ## How was this patch tested? Existing tests. Closes #23384 from ueshin/issues/SPARK-26443/hardcoded_history_configs. Authored-by: Takuya UESHIN Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/SparkConf.scala | 4 +- .../deploy/history/FsHistoryProvider.scala | 21 ++++----- .../spark/deploy/history/HistoryServer.scala | 16 ++++--- .../history/HistoryServerArguments.scala | 2 +- .../spark/internal/config/History.scala | 46 ++++++++++++++++++- .../org/apache/spark/SparkConfSuite.scala | 2 +- .../history/FsHistoryProviderSuite.scala | 23 +++++----- .../history/HistoryServerArgumentsSuite.scala | 9 ++-- .../deploy/history/HistoryServerSuite.scala | 13 +++--- 9 files changed, 89 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 8d135d3e083d..0b47da12b5b4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -682,11 +682,11 @@ private[spark] object SparkConf extends Logging { private val configsWithAlternatives = Map[String, Seq[AlternateConfig]]( "spark.executor.userClassPathFirst" -> Seq( AlternateConfig("spark.files.userClassPathFirst", "1.3")), - "spark.history.fs.update.interval" -> Seq( + UPDATE_INTERVAL_S.key -> Seq( AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), AlternateConfig("spark.history.fs.updateInterval", "1.3"), AlternateConfig("spark.history.updateInterval", "1.3")), - "spark.history.fs.cleaner.interval" -> Seq( + CLEANER_INTERVAL_S.key -> Seq( AlternateConfig("spark.history.fs.cleaner.interval.seconds", "1.4")), MAX_LOG_AGE_S.key -> Seq( AlternateConfig("spark.history.fs.cleaner.maxAge.seconds", "1.4")), 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 da6e5f03aabb..709a380dfb63 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 @@ -42,7 +42,7 @@ import org.fusesource.leveldbjni.internal.NativeDB import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.DRIVER_LOG_DFS_DIR +import org.apache.spark.internal.config.{DRIVER_LOG_DFS_DIR, History} import org.apache.spark.internal.config.History._ import org.apache.spark.internal.config.Status._ import org.apache.spark.io.CompressionCodec @@ -91,24 +91,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) import FsHistoryProvider._ // Interval between safemode checks. - private val SAFEMODE_CHECK_INTERVAL_S = conf.getTimeAsSeconds( - "spark.history.fs.safemodeCheck.interval", "5s") + private val SAFEMODE_CHECK_INTERVAL_S = conf.get(History.SAFEMODE_CHECK_INTERVAL_S) // Interval between each check for event log updates - private val UPDATE_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.update.interval", "10s") + private val UPDATE_INTERVAL_S = conf.get(History.UPDATE_INTERVAL_S) // Interval between each cleaner checks for event logs to delete - private val CLEAN_INTERVAL_S = conf.get(CLEANER_INTERVAL_S) + private val CLEAN_INTERVAL_S = conf.get(History.CLEANER_INTERVAL_S) // Number of threads used to replay event logs. - private val NUM_PROCESSING_THREADS = conf.getInt(SPARK_HISTORY_FS_NUM_REPLAY_THREADS, - Math.ceil(Runtime.getRuntime.availableProcessors() / 4f).toInt) + private val NUM_PROCESSING_THREADS = conf.get(History.NUM_REPLAY_THREADS) - private val logDir = conf.get(EVENT_LOG_DIR) + private val logDir = conf.get(History.HISTORY_LOG_DIR) - private val HISTORY_UI_ACLS_ENABLE = conf.getBoolean("spark.history.ui.acls.enable", false) - private val HISTORY_UI_ADMIN_ACLS = conf.get("spark.history.ui.admin.acls", "") - private val HISTORY_UI_ADMIN_ACLS_GROUPS = conf.get("spark.history.ui.admin.acls.groups", "") + private val HISTORY_UI_ACLS_ENABLE = conf.get(History.UI_ACLS_ENABLE) + private val HISTORY_UI_ADMIN_ACLS = conf.get(History.UI_ADMIN_ACLS) + private val HISTORY_UI_ADMIN_ACLS_GROUPS = conf.get(History.UI_ADMIN_ACLS_GROUPS) logInfo(s"History server ui acls " + (if (HISTORY_UI_ACLS_ENABLE) "enabled" else "disabled") + "; users with admin permissions: " + HISTORY_UI_ADMIN_ACLS.toString + "; groups with admin permissions" + HISTORY_UI_ADMIN_ACLS_GROUPS.toString) @@ -1089,7 +1087,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } private[history] object FsHistoryProvider { - private val SPARK_HISTORY_FS_NUM_REPLAY_THREADS = "spark.history.fs.numReplayThreads" private val APPL_START_EVENT_PREFIX = "{\"Event\":\"SparkListenerApplicationStart\"" 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 5856c7057b74..b9303388638f 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 @@ -30,7 +30,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.internal.config.History.HISTORY_SERVER_UI_PORT +import org.apache.spark.internal.config.History import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -56,7 +56,7 @@ class HistoryServer( with Logging with UIRoot with ApplicationCacheOperations { // How many applications to retain - private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50) + private val retainedApplications = conf.get(History.RETAINED_APPLICATIONS) // How many applications the summary ui displays private[history] val maxApplications = conf.get(HISTORY_UI_MAX_APPS); @@ -273,14 +273,14 @@ object HistoryServer extends Logging { initSecurity() val securityManager = createSecurityManager(conf) - val providerName = conf.getOption("spark.history.provider") + val providerName = conf.get(History.PROVIDER) .getOrElse(classOf[FsHistoryProvider].getName()) val provider = Utils.classForName(providerName) .getConstructor(classOf[SparkConf]) .newInstance(conf) .asInstanceOf[ApplicationHistoryProvider] - val port = conf.get(HISTORY_SERVER_UI_PORT) + val port = conf.get(History.HISTORY_SERVER_UI_PORT) val server = new HistoryServer(conf, provider, securityManager, port) server.bind() @@ -319,10 +319,12 @@ object HistoryServer extends Logging { // from a keytab file so that we can access HDFS beyond the kerberos ticket expiration. // As long as it is using Hadoop rpc (hdfs://), a relogin will automatically // occur from the keytab. - if (conf.getBoolean("spark.history.kerberos.enabled", false)) { + if (conf.get(History.KERBEROS_ENABLED)) { // if you have enabled kerberos the following 2 params must be set - val principalName = conf.get("spark.history.kerberos.principal") - val keytabFilename = conf.get("spark.history.kerberos.keytab") + val principalName = conf.get(History.KERBEROS_PRINCIPAL) + .getOrElse(throw new NoSuchElementException(History.KERBEROS_PRINCIPAL.key)) + val keytabFilename = conf.get(History.KERBEROS_KEYTAB) + .getOrElse(throw new NoSuchElementException(History.KERBEROS_KEYTAB.key)) SparkHadoopUtil.get.loginUserFromKeytab(principalName, keytabFilename) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 49f00cb10179..dec89769c030 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -79,7 +79,7 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin | | spark.history.fs.logDirectory Directory where app logs are stored | (default: file:/tmp/spark-events) - | spark.history.fs.updateInterval How often to reload log data from storage + | spark.history.fs.update.interval How often to reload log data from storage | (in seconds, default: 10) |""".stripMargin) // scalastyle:on println 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 b7d8061d26d2..f984dd385344 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 @@ -25,10 +25,18 @@ private[spark] object History { val DEFAULT_LOG_DIR = "file:/tmp/spark-events" - val EVENT_LOG_DIR = ConfigBuilder("spark.history.fs.logDirectory") + val HISTORY_LOG_DIR = ConfigBuilder("spark.history.fs.logDirectory") .stringConf .createWithDefault(DEFAULT_LOG_DIR) + val SAFEMODE_CHECK_INTERVAL_S = ConfigBuilder("spark.history.fs.safemodeCheck.interval") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("5s") + + val UPDATE_INTERVAL_S = ConfigBuilder("spark.history.fs.update.interval") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("10s") + val CLEANER_ENABLED = ConfigBuilder("spark.history.fs.cleaner.enabled") .booleanConf .createWithDefault(false) @@ -79,4 +87,40 @@ private[spark] object History { val MAX_DRIVER_LOG_AGE_S = ConfigBuilder("spark.history.fs.driverlog.cleaner.maxAge") .fallbackConf(MAX_LOG_AGE_S) + + val UI_ACLS_ENABLE = ConfigBuilder("spark.history.ui.acls.enable") + .booleanConf + .createWithDefault(false) + + val UI_ADMIN_ACLS = ConfigBuilder("spark.history.ui.admin.acls") + .stringConf + .createWithDefault("") + + val UI_ADMIN_ACLS_GROUPS = ConfigBuilder("spark.history.ui.admin.acls.groups") + .stringConf + .createWithDefault("") + + val NUM_REPLAY_THREADS = ConfigBuilder("spark.history.fs.numReplayThreads") + .intConf + .createWithDefaultFunction(() => Math.ceil(Runtime.getRuntime.availableProcessors() / 4f).toInt) + + val RETAINED_APPLICATIONS = ConfigBuilder("spark.history.retainedApplications") + .intConf + .createWithDefault(50) + + val PROVIDER = ConfigBuilder("spark.history.provider") + .stringConf + .createOptional + + val KERBEROS_ENABLED = ConfigBuilder("spark.history.kerberos.enabled") + .booleanConf + .createWithDefault(false) + + val KERBEROS_PRINCIPAL = ConfigBuilder("spark.history.kerberos.principal") + .stringConf + .createOptional + + val KERBEROS_KEYTAB = ConfigBuilder("spark.history.kerberos.keytab") + .stringConf + .createOptional } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 7cb03deae139..e14a5dcb5ef8 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -232,7 +232,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst test("deprecated configs") { val conf = new SparkConf() - val newName = "spark.history.fs.update.interval" + val newName = UPDATE_INTERVAL_S.key assert(!conf.contains(newName)) 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 c1ae27aa940f..6d2e329094ae 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 @@ -294,7 +294,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val maxAge = TimeUnit.SECONDS.toMillis(10) val clock = new ManualClock(maxAge / 2) val provider = new FsHistoryProvider( - createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) + createTestConf().set(MAX_LOG_AGE_S.key, s"${maxAge}ms"), clock) val log1 = newLogFile("app1", Some("attempt1"), inProgress = false) writeFile(log1, true, None, @@ -379,7 +379,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val maxAge = TimeUnit.SECONDS.toMillis(40) val clock = new ManualClock(0) val provider = new FsHistoryProvider( - createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) + createTestConf().set(MAX_LOG_AGE_S.key, s"${maxAge}ms"), clock) val log1 = newLogFile("inProgressApp1", None, inProgress = true) writeFile(log1, true, None, @@ -462,8 +462,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val maxAge = TimeUnit.SECONDS.toSeconds(40) val clock = new ManualClock(0) val testConf = new SparkConf() - testConf.set("spark.history.fs.logDirectory", - Utils.createTempDir(namePrefix = "eventLog").getAbsolutePath()) + testConf.set(HISTORY_LOG_DIR, Utils.createTempDir(namePrefix = "eventLog").getAbsolutePath()) testConf.set(DRIVER_LOG_DFS_DIR, testDir.getAbsolutePath()) testConf.set(DRIVER_LOG_CLEANER_ENABLED, true) testConf.set(DRIVER_LOG_CLEANER_INTERVAL, maxAge / 4) @@ -645,9 +644,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test both history ui admin acls and application acls are configured. val conf1 = createTestConf() - .set("spark.history.ui.acls.enable", "true") - .set("spark.history.ui.admin.acls", "user1,user2") - .set("spark.history.ui.admin.acls.groups", "group1") + .set(UI_ACLS_ENABLE, true) + .set(UI_ADMIN_ACLS, "user1,user2") + .set(UI_ADMIN_ACLS_GROUPS, "group1") .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) createAndCheck(conf1, ("spark.admin.acls", "user"), ("spark.admin.acls.groups", "group")) { @@ -667,9 +666,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test only history ui admin acls are configured. val conf2 = createTestConf() - .set("spark.history.ui.acls.enable", "true") - .set("spark.history.ui.admin.acls", "user1,user2") - .set("spark.history.ui.admin.acls.groups", "group1") + .set(UI_ACLS_ENABLE, true) + .set(UI_ADMIN_ACLS, "user1,user2") + .set(UI_ADMIN_ACLS_GROUPS, "group1") .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) createAndCheck(conf2) { securityManager => // Test whether user has permission to access UI. @@ -687,7 +686,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test neither history ui admin acls nor application acls are configured. val conf3 = createTestConf() - .set("spark.history.ui.acls.enable", "true") + .set(UI_ACLS_ENABLE, true) .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) createAndCheck(conf3) { securityManager => // Test whether user has permission to access UI. @@ -1036,7 +1035,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc private def createTestConf(inMemory: Boolean = false): SparkConf = { val conf = new SparkConf() - .set(EVENT_LOG_DIR, testDir.getAbsolutePath()) + .set(HISTORY_LOG_DIR, testDir.getAbsolutePath()) .set(FAST_IN_PROGRESS_PARSING, true) if (!inMemory) { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala index e89733a144cf..6b479873f69f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala @@ -22,21 +22,22 @@ import java.nio.charset.StandardCharsets._ import com.google.common.io.Files import org.apache.spark._ +import org.apache.spark.internal.config.History._ import org.apache.spark.util.Utils class HistoryServerArgumentsSuite extends SparkFunSuite { private val logDir = new File("src/test/resources/spark-events") private val conf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) - .set("spark.history.fs.updateInterval", "1") + .set(HISTORY_LOG_DIR, logDir.getAbsolutePath) + .set(UPDATE_INTERVAL_S, 1L) .set("spark.testing", "true") test("No Arguments Parsing") { val argStrings = Array.empty[String] val hsa = new HistoryServerArguments(conf, argStrings) - assert(conf.get("spark.history.fs.logDirectory") === logDir.getAbsolutePath) - assert(conf.get("spark.history.fs.updateInterval") === "1") + assert(conf.get(HISTORY_LOG_DIR) === logDir.getAbsolutePath) + assert(conf.get(UPDATE_INTERVAL_S) === 1L) assert(conf.get("spark.testing") === "true") } 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 2a2d013bacbd..a9dee67ae938 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 @@ -78,8 +78,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers Utils.deleteRecursively(storeDir) assert(storeDir.mkdir()) val conf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir) - .set("spark.history.fs.update.interval", "0") + .set(HISTORY_LOG_DIR, logDir) + .set(UPDATE_INTERVAL_S.key, "0") .set("spark.testing", "true") .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) .set("spark.eventLog.logStageExecutorMetrics.enabled", "true") @@ -416,11 +416,10 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // allowed refresh rate (1Hz) stop() val myConf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) + .set(HISTORY_LOG_DIR, logDir.getAbsolutePath) .set("spark.eventLog.dir", logDir.getAbsolutePath) - .set("spark.history.fs.update.interval", "1s") + .set(UPDATE_INTERVAL_S.key, "1s") .set("spark.eventLog.enabled", "true") - .set("spark.history.cache.window", "250ms") .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) .remove("spark.testing") val provider = new FsHistoryProvider(myConf) @@ -613,8 +612,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers stop() init( "spark.ui.filters" -> classOf[FakeAuthFilter].getName(), - "spark.history.ui.acls.enable" -> "true", - "spark.history.ui.admin.acls" -> admin) + UI_ACLS_ENABLE.key -> "true", + UI_ADMIN_ACLS.key -> admin) val tests = Seq( (owner, HttpServletResponse.SC_OK), From 240817b7aea14d12f7764e17ab11073d14e8e6aa Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sat, 29 Dec 2018 21:47:49 -0600 Subject: [PATCH 0200/1072] [SPARK-26363][WEBUI] Avoid duplicated KV store lookups in method `taskList` ## What changes were proposed in this pull request? In the method `taskList`(since https://github.com/apache/spark/pull/21688), the executor log value is queried in KV store for every task(method `constructTaskData`). This PR propose to use a hashmap for reducing duplicated KV store lookups in the method. ![image](https://user-images.githubusercontent.com/1097932/49946230-841c7680-ff29-11e8-8b83-d8f7553bfe5e.png) ## How was this patch tested? Manual check Closes #23310 from gengliangwang/removeExecutorLog. Authored-by: Gengliang Wang Signed-off-by: Sean Owen --- .../apache/spark/status/AppStatusStore.scala | 52 ++++++++++--------- 1 file changed, 28 insertions(+), 24 deletions(-) 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 312bcccb1cca..0487f2f07c09 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -20,6 +20,7 @@ package org.apache.spark.status import java.util.{List => JList} import scala.collection.JavaConverters._ +import scala.collection.mutable.HashMap import org.apache.spark.{JobExecutionStatus, SparkConf} import org.apache.spark.status.api.v1 @@ -386,10 +387,9 @@ private[spark] class AppStatusStore( def taskList(stageId: Int, stageAttemptId: Int, maxTasks: Int): Seq[v1.TaskData] = { val stageKey = Array(stageId, stageAttemptId) - store.view(classOf[TaskDataWrapper]).index("stage").first(stageKey).last(stageKey).reverse() - .max(maxTasks).asScala.map { taskDataWrapper => - constructTaskData(taskDataWrapper) - }.toSeq.reverse + val taskDataWrapperIter = store.view(classOf[TaskDataWrapper]).index("stage") + .first(stageKey).last(stageKey).reverse().max(maxTasks).asScala + constructTaskDataList(taskDataWrapperIter).reverse } def taskList( @@ -428,9 +428,8 @@ private[spark] class AppStatusStore( } val ordered = if (ascending) indexed else indexed.reverse() - ordered.skip(offset).max(length).asScala.map { taskDataWrapper => - constructTaskData(taskDataWrapper) - }.toSeq + val taskDataWrapperIter = ordered.skip(offset).max(length).asScala + constructTaskDataList(taskDataWrapperIter) } def executorSummary(stageId: Int, attemptId: Int): Map[String, v1.ExecutorStageSummary] = { @@ -536,24 +535,29 @@ private[spark] class AppStatusStore( store.close() } - def constructTaskData(taskDataWrapper: TaskDataWrapper) : v1.TaskData = { - val taskDataOld: v1.TaskData = taskDataWrapper.toApi - val executorLogs: Option[Map[String, String]] = try { - Some(executorSummary(taskDataOld.executorId).executorLogs) - } catch { - case e: NoSuchElementException => e.getMessage - None - } - new v1.TaskData(taskDataOld.taskId, taskDataOld.index, - taskDataOld.attempt, taskDataOld.launchTime, taskDataOld.resultFetchStart, - taskDataOld.duration, taskDataOld.executorId, taskDataOld.host, taskDataOld.status, - taskDataOld.taskLocality, taskDataOld.speculative, taskDataOld.accumulatorUpdates, - taskDataOld.errorMessage, taskDataOld.taskMetrics, - executorLogs.getOrElse(Map[String, String]()), - AppStatusUtils.schedulerDelay(taskDataOld), - AppStatusUtils.gettingResultTime(taskDataOld)) + def constructTaskDataList(taskDataWrapperIter: Iterable[TaskDataWrapper]): Seq[v1.TaskData] = { + val executorIdToLogs = new HashMap[String, Map[String, String]]() + taskDataWrapperIter.map { taskDataWrapper => + val taskDataOld: v1.TaskData = taskDataWrapper.toApi + val executorLogs = executorIdToLogs.getOrElseUpdate(taskDataOld.executorId, { + try { + executorSummary(taskDataOld.executorId).executorLogs + } catch { + case e: NoSuchElementException => + Map.empty + } + }) + + new v1.TaskData(taskDataOld.taskId, taskDataOld.index, + taskDataOld.attempt, taskDataOld.launchTime, taskDataOld.resultFetchStart, + taskDataOld.duration, taskDataOld.executorId, taskDataOld.host, taskDataOld.status, + taskDataOld.taskLocality, taskDataOld.speculative, taskDataOld.accumulatorUpdates, + taskDataOld.errorMessage, taskDataOld.taskMetrics, + executorLogs, + AppStatusUtils.schedulerDelay(taskDataOld), + AppStatusUtils.gettingResultTime(taskDataOld)) + }.toSeq } - } private[spark] object AppStatusStore { From 0996b7c95a79fb018169ed1da7a8e3e482260838 Mon Sep 17 00:00:00 2001 From: seancxmao Date: Mon, 31 Dec 2018 08:24:18 -0600 Subject: [PATCH 0201/1072] [SPARK-23375][SQL][FOLLOWUP][TEST] Test Sort metrics while Sort is missing ## What changes were proposed in this pull request? #20560/[SPARK-23375](https://issues.apache.org/jira/browse/SPARK-23375) introduced an optimizer rule to eliminate redundant Sort. For a test case named "Sort metrics" in `SQLMetricsSuite`, because range is already sorted, sort is removed by the `RemoveRedundantSorts`, which makes this test case meaningless. This PR modifies the query for testing Sort metrics and checks Sort exists in the plan. ## How was this patch tested? Modify the existing test case. Closes #23258 from seancxmao/sort-metrics. Authored-by: seancxmao Signed-off-by: Sean Owen --- .../execution/metric/SQLMetricsSuite.scala | 18 ++++++-- .../metric/SQLMetricsTestUtils.scala | 43 ++++++++++++++++--- 2 files changed, 51 insertions(+), 10 deletions(-) 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 47265df4831d..7368a6c9e1d6 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 @@ -194,10 +194,20 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } test("Sort metrics") { - // Assume the execution plan is - // WholeStageCodegen(nodeId = 0, Range(nodeId = 2) -> Sort(nodeId = 1)) - val ds = spark.range(10).sort('id) - testSparkPlanMetrics(ds.toDF(), 2, Map.empty) + // Assume the execution plan with node id is + // Sort(nodeId = 0) + // Exchange(nodeId = 1) + // Project(nodeId = 2) + // LocalTableScan(nodeId = 3) + // Because of SPARK-25267, ConvertToLocalRelation is disabled in the test cases of sql/core, + // so Project here is not collapsed into LocalTableScan. + val df = Seq(1, 3, 2).toDF("id").sort('id) + testSparkPlanMetricsWithPredicates(df, 2, Map( + 0L -> (("Sort", Map( + "sort time total (min, med, max)" -> {_.toString.matches(timingMetricPattern)}, + "peak memory total (min, med, max)" -> {_.toString.matches(sizeMetricPattern)}, + "spill size total (min, med, max)" -> {_.toString.matches(sizeMetricPattern)}))) + )) } test("SortMergeJoin metrics") { 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 dcc540fc4f10..2d245d2ba1e3 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 @@ -40,6 +40,18 @@ trait SQLMetricsTestUtils extends SQLTestUtils { protected def statusStore: SQLAppStatusStore = spark.sharedState.statusStore + // Pattern of size SQLMetric value, e.g. "\n96.2 MiB (32.1 MiB, 32.1 MiB, 32.1 MiB)" + protected val sizeMetricPattern = { + val bytes = "([0-9]+(\\.[0-9]+)?) (EiB|PiB|TiB|GiB|MiB|KiB|B)" + s"\\n$bytes \\($bytes, $bytes, $bytes\\)" + } + + // Pattern of timing SQLMetric value, e.g. "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms)" + protected val timingMetricPattern = { + val duration = "([0-9]+(\\.[0-9]+)?) (ms|s|m|h)" + s"\\n$duration \\($duration, $duration, $duration\\)" + } + /** * Get execution metrics for the SQL execution and verify metrics values. * @@ -185,15 +197,34 @@ trait SQLMetricsTestUtils extends SQLTestUtils { df: DataFrame, expectedNumOfJobs: Int, expectedMetrics: Map[Long, (String, Map[String, Any])]): Unit = { - val optActualMetrics = getSparkPlanMetrics(df, expectedNumOfJobs, expectedMetrics.keySet) + val expectedMetricsPredicates = expectedMetrics.mapValues { case (nodeName, nodeMetrics) => + (nodeName, nodeMetrics.mapValues(expectedMetricValue => + (actualMetricValue: Any) => expectedMetricValue.toString === actualMetricValue)) + } + testSparkPlanMetricsWithPredicates(df, expectedNumOfJobs, expectedMetricsPredicates) + } + + /** + * Call `df.collect()` and verify if the collected metrics satisfy the specified predicates. + * @param df `DataFrame` to run + * @param expectedNumOfJobs number of jobs that will run + * @param expectedMetricsPredicates the expected metrics predicates. The format is + * `nodeId -> (operatorName, metric name -> metric predicate)`. + */ + protected def testSparkPlanMetricsWithPredicates( + df: DataFrame, + expectedNumOfJobs: Int, + expectedMetricsPredicates: Map[Long, (String, Map[String, Any => Boolean])]): Unit = { + val optActualMetrics = + getSparkPlanMetrics(df, expectedNumOfJobs, expectedMetricsPredicates.keySet) optActualMetrics.foreach { actualMetrics => - assert(expectedMetrics.keySet === actualMetrics.keySet) - for (nodeId <- expectedMetrics.keySet) { - val (expectedNodeName, expectedMetricsMap) = expectedMetrics(nodeId) + assert(expectedMetricsPredicates.keySet === actualMetrics.keySet) + for ((nodeId, (expectedNodeName, expectedMetricsPredicatesMap)) + <- expectedMetricsPredicates) { val (actualNodeName, actualMetricsMap) = actualMetrics(nodeId) assert(expectedNodeName === actualNodeName) - for (metricName <- expectedMetricsMap.keySet) { - assert(expectedMetricsMap(metricName).toString === actualMetricsMap(metricName)) + for ((metricName, metricPredicate) <- expectedMetricsPredicatesMap) { + assert(metricPredicate(actualMetricsMap(metricName))) } } } From 89c92ccc2046d068aea23ae7973a97c58cfdc966 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 31 Dec 2018 16:39:46 +0100 Subject: [PATCH 0202/1072] [SPARK-26504][SQL] Rope-wise dumping of Spark plans ## What changes were proposed in this pull request? Proposed new class `StringConcat` for converting a sequence of strings to string with one memory allocation in the `toString` method. `StringConcat` replaces `StringBuilderWriter` in methods of dumping of Spark plans and codegen to strings. All `Writer` arguments are replaced by `String => Unit` in methods related to Spark plans stringification. ## How was this patch tested? It was tested by existing suites `QueryExecutionSuite`, `DebuggingSuite` as well as new tests for `StringConcat` in `StringUtilsSuite`. Closes #23406 from MaxGekk/rope-plan. Authored-by: Maxim Gekk Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/plans/QueryPlan.scala | 17 ++++ .../spark/sql/catalyst/trees/TreeNode.scala | 38 ++++----- .../spark/sql/catalyst/util/StringUtils.scala | 32 +++++++ .../sql/catalyst/util/StringUtilsSuite.scala | 13 +++ .../spark/sql/execution/QueryExecution.scala | 85 +++++++++---------- .../sql/execution/WholeStageCodegenExec.scala | 8 +- .../spark/sql/execution/debug/package.scala | 27 +++--- 7 files changed, 133 insertions(+), 87 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 125181fb213f..8f5444ed8a5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.plans +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode} import org.apache.spark.sql.internal.SQLConf @@ -301,4 +302,20 @@ object QueryPlan extends PredicateHelper { Nil } } + + /** + * Converts the query plan to string and appends it via provided function. + */ + def append[T <: QueryPlan[T]]( + plan: => QueryPlan[T], + append: String => Unit, + verbose: Boolean, + addSuffix: Boolean, + maxFields: Int = SQLConf.get.maxToStringFields): Unit = { + try { + plan.treeString(append, verbose, addSuffix, maxFields) + } catch { + case e: AnalysisException => append(e.toString) + } + } } 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 21e59bbd283e..570a019b2af7 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 @@ -17,13 +17,11 @@ package org.apache.spark.sql.catalyst.trees -import java.io.Writer import java.util.UUID import scala.collection.Map import scala.reflect.ClassTag -import org.apache.commons.io.output.StringBuilderWriter import org.apache.commons.lang3.ClassUtils import org.json4s.JsonAST._ import org.json4s.JsonDSL._ @@ -37,6 +35,7 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -481,21 +480,18 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { verbose: Boolean, addSuffix: Boolean = false, maxFields: Int = SQLConf.get.maxToStringFields): String = { - val writer = new StringBuilderWriter() - try { - treeString(writer, verbose, addSuffix, maxFields) - writer.toString - } finally { - writer.close() - } + val concat = new StringConcat() + + treeString(concat.append, verbose, addSuffix, maxFields) + concat.toString } def treeString( - writer: Writer, + append: String => Unit, verbose: Boolean, addSuffix: Boolean, maxFields: Int): Unit = { - generateTreeString(0, Nil, writer, verbose, "", addSuffix, maxFields) + generateTreeString(0, Nil, append, verbose, "", addSuffix, maxFields) } /** @@ -558,7 +554,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - writer: Writer, + append: String => Unit, verbose: Boolean, prefix: String = "", addSuffix: Boolean = false, @@ -566,9 +562,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { if (depth > 0) { lastChildren.init.foreach { isLast => - writer.write(if (isLast) " " else ": ") + append(if (isLast) " " else ": ") } - writer.write(if (lastChildren.last) "+- " else ":- ") + append(if (lastChildren.last) "+- " else ":- ") } val str = if (verbose) { @@ -576,24 +572,24 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } else { simpleString(maxFields) } - writer.write(prefix) - writer.write(str) - writer.write("\n") + append(prefix) + append(str) + append("\n") if (innerChildren.nonEmpty) { innerChildren.init.foreach(_.generateTreeString( - depth + 2, lastChildren :+ children.isEmpty :+ false, writer, verbose, + depth + 2, lastChildren :+ children.isEmpty :+ false, append, verbose, addSuffix = addSuffix, maxFields = maxFields)) innerChildren.last.generateTreeString( - depth + 2, lastChildren :+ children.isEmpty :+ true, writer, verbose, + depth + 2, lastChildren :+ children.isEmpty :+ true, append, verbose, addSuffix = addSuffix, maxFields = maxFields) } if (children.nonEmpty) { children.init.foreach(_.generateTreeString( - depth + 1, lastChildren :+ false, writer, verbose, prefix, addSuffix, maxFields)) + depth + 1, lastChildren :+ false, append, verbose, prefix, addSuffix, maxFields)) children.last.generateTreeString( - depth + 1, lastChildren :+ true, writer, verbose, prefix, addSuffix, maxFields) + depth + 1, lastChildren :+ true, append, verbose, prefix, addSuffix, maxFields) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index bc861a805ce6..643b83b1741a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.util import java.util.regex.{Pattern, PatternSyntaxException} +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.sql.AnalysisException import org.apache.spark.unsafe.types.UTF8String @@ -87,4 +89,34 @@ object StringUtils { } funcNames.toSeq } + + /** + * Concatenation of sequence of strings to final string with cheap append method + * and one memory allocation for the final string. + */ + class StringConcat { + private val strings = new ArrayBuffer[String] + private var length: Int = 0 + + /** + * Appends a string and accumulates its length to allocate a string buffer for all + * appended strings once in the toString method. + */ + def append(s: String): Unit = { + if (s != null) { + strings.append(s) + length += s.length + } + } + + /** + * The method allocates memory for all appended strings, writes them to the memory and + * returns concatenated string. + */ + override def toString: String = { + val result = new java.lang.StringBuilder(length) + strings.foreach(result.append) + result.toString + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala index 78fee5135c3a..616ec12032db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala @@ -43,4 +43,17 @@ class StringUtilsSuite extends SparkFunSuite { assert(filterPattern(names, " a. ") === Seq("a1", "a2")) assert(filterPattern(names, " d* ") === Nil) } + + test("string concatenation") { + def concat(seq: String*): String = { + seq.foldLeft(new StringConcat())((acc, s) => {acc.append(s); acc}).toString + } + + assert(new StringConcat().toString == "") + assert(concat("") == "") + assert(concat(null) == "") + assert(concat("a") == "a") + assert(concat("1", "2") == "12") + assert(concat("abc", "\n", "123") == "abc\n123") + } } 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 9b8d2e830867..7fccbf65d852 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 @@ -17,20 +17,21 @@ package org.apache.spark.sql.execution -import java.io.{BufferedWriter, OutputStreamWriter, Writer} +import java.io.{BufferedWriter, OutputStreamWriter} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} -import org.apache.commons.io.output.StringBuilderWriter import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} @@ -108,10 +109,6 @@ class QueryExecution( ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf)) - protected def stringOrError[A](f: => A): String = - try f.toString catch { case e: AnalysisException => e.toString } - - /** * Returns the result as a hive compatible sequence of strings. This is used in tests and * `SparkSQLDriver` for CLI applications. @@ -197,55 +194,53 @@ class QueryExecution( } def simpleString: String = withRedaction { - s"""== Physical Plan == - |${stringOrError(executedPlan.treeString(verbose = false))} - """.stripMargin.trim - } - - private def writeOrError(writer: Writer)(f: Writer => Unit): Unit = { - try f(writer) - catch { - case e: AnalysisException => writer.write(e.toString) - } + val concat = new StringConcat() + concat.append("== Physical Plan ==\n") + QueryPlan.append(executedPlan, concat.append, verbose = false, addSuffix = false) + concat.append("\n") + concat.toString } - private def writePlans(writer: Writer, maxFields: Int): Unit = { + private def writePlans(append: String => Unit, maxFields: Int): Unit = { val (verbose, addSuffix) = (true, false) - - writer.write("== Parsed Logical Plan ==\n") - writeOrError(writer)(logical.treeString(_, verbose, addSuffix, maxFields)) - writer.write("\n== Analyzed Logical Plan ==\n") - val analyzedOutput = stringOrError(truncatedString( - analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields)) - writer.write(analyzedOutput) - writer.write("\n") - writeOrError(writer)(analyzed.treeString(_, verbose, addSuffix, maxFields)) - writer.write("\n== Optimized Logical Plan ==\n") - writeOrError(writer)(optimizedPlan.treeString(_, verbose, addSuffix, maxFields)) - writer.write("\n== Physical Plan ==\n") - writeOrError(writer)(executedPlan.treeString(_, verbose, addSuffix, maxFields)) + append("== Parsed Logical Plan ==\n") + QueryPlan.append(logical, append, verbose, addSuffix, maxFields) + append("\n== Analyzed Logical Plan ==\n") + val analyzedOutput = try { + truncatedString( + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields) + } catch { + case e: AnalysisException => e.toString + } + append(analyzedOutput) + append("\n") + QueryPlan.append(analyzed, append, verbose, addSuffix, maxFields) + append("\n== Optimized Logical Plan ==\n") + QueryPlan.append(optimizedPlan, append, verbose, addSuffix, maxFields) + append("\n== Physical Plan ==\n") + QueryPlan.append(executedPlan, append, verbose, addSuffix, maxFields) } override def toString: String = withRedaction { - val writer = new StringBuilderWriter() - try { - writePlans(writer, SQLConf.get.maxToStringFields) - writer.toString - } finally { - writer.close() - } + val concat = new StringConcat() + writePlans(concat.append, SQLConf.get.maxToStringFields) + concat.toString } def stringWithStats: String = withRedaction { + val concat = new StringConcat() + val maxFields = SQLConf.get.maxToStringFields + // trigger to compute stats for logical plans optimizedPlan.stats // only show optimized logical plan and physical plan - s"""== Optimized Logical Plan == - |${stringOrError(optimizedPlan.treeString(verbose = true, addSuffix = true))} - |== Physical Plan == - |${stringOrError(executedPlan.treeString(verbose = true))} - """.stripMargin.trim + concat.append("== Optimized Logical Plan ==\n") + QueryPlan.append(optimizedPlan, concat.append, verbose = true, addSuffix = true, maxFields) + concat.append("\n== Physical Plan ==\n") + QueryPlan.append(executedPlan, concat.append, verbose = true, addSuffix = false, maxFields) + concat.append("\n") + concat.toString } /** @@ -282,7 +277,7 @@ class QueryExecution( /** * Dumps debug information about query execution into the specified file. * - * @param maxFields maximim number of fields converted to string representation. + * @param maxFields maximum number of fields converted to string representation. */ def toFile(path: String, maxFields: Int = Int.MaxValue): Unit = { val filePath = new Path(path) @@ -290,9 +285,9 @@ class QueryExecution( val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) try { - writePlans(writer, maxFields) + writePlans(writer.write, maxFields) writer.write("\n== Whole Stage Codegen ==\n") - org.apache.spark.sql.execution.debug.writeCodegen(writer, executedPlan) + org.apache.spark.sql.execution.debug.writeCodegen(writer.write, executedPlan) } finally { writer.close() } 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 f4927dedabe5..3b0a99669ccd 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 @@ -493,7 +493,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with InputRDDCod override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - writer: Writer, + append: String => Unit, verbose: Boolean, prefix: String = "", addSuffix: Boolean = false, @@ -501,7 +501,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with InputRDDCod child.generateTreeString( depth, lastChildren, - writer, + append, verbose, prefix = "", addSuffix = false, @@ -777,7 +777,7 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - writer: Writer, + append: String => Unit, verbose: Boolean, prefix: String = "", addSuffix: Boolean = false, @@ -785,7 +785,7 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) child.generateTreeString( depth, lastChildren, - writer, + append, verbose, s"*($codegenStageId) ", false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index ae8197f617a2..53b74c7c8559 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -17,13 +17,10 @@ package org.apache.spark.sql.execution -import java.io.Writer import java.util.Collections import scala.collection.JavaConverters._ -import org.apache.commons.io.output.StringBuilderWriter - import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ @@ -32,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.execution.streaming.{StreamExecution, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQuery @@ -73,24 +71,19 @@ package object debug { * @return single String containing all WholeStageCodegen subtrees and corresponding codegen */ def codegenString(plan: SparkPlan): String = { - val writer = new StringBuilderWriter() - - try { - writeCodegen(writer, plan) - writer.toString - } finally { - writer.close() - } + val concat = new StringConcat() + writeCodegen(concat.append, plan) + concat.toString } - def writeCodegen(writer: Writer, plan: SparkPlan): Unit = { + def writeCodegen(append: String => Unit, plan: SparkPlan): Unit = { val codegenSeq = codegenStringSeq(plan) - writer.write(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") + append(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") for (((subtree, code), i) <- codegenSeq.zipWithIndex) { - writer.write(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n") - writer.write(subtree) - writer.write("\nGenerated code:\n") - writer.write(s"${code}\n") + append(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n") + append(subtree) + append("\nGenerated code:\n") + append(s"${code}\n") } } From c0b9db120d4c2ad0b5b99b9152549e94ef8f5a2d Mon Sep 17 00:00:00 2001 From: Hirobe Keiichi Date: Mon, 31 Dec 2018 10:15:14 -0600 Subject: [PATCH 0203/1072] [SPARK-26339][SQL] Throws better exception when reading files that start with underscore ## What changes were proposed in this pull request? As the description in SPARK-26339, spark.read behavior is very confusing when reading files that start with underscore, fix this by throwing exception which message is "Path does not exist". ## How was this patch tested? manual tests. Both of codes below throws exception which message is "Path does not exist". ``` spark.read.csv("/home/forcia/work/spark/_test.csv") spark.read.schema("test STRING, number INT").csv("/home/forcia/work/spark/_test.csv") ``` Closes #23288 from KeiichiHirobe/SPARK-26339. Authored-by: Hirobe Keiichi Signed-off-by: Sean Owen --- .../execution/datasources/DataSource.scala | 17 +++++++++++++++- .../src/test/resources/test-data/_cars.csv | 7 +++++++ .../execution/datasources/csv/CSVSuite.scala | 20 +++++++++++++++++++ 3 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/resources/test-data/_cars.csv 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 fefff68c4ba8..517e04317d94 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 @@ -543,7 +543,7 @@ case class DataSource( checkFilesExist: Boolean): Seq[Path] = { val allPaths = caseInsensitiveOptions.get("path") ++ paths val hadoopConf = sparkSession.sessionState.newHadoopConf() - allPaths.flatMap { path => + val allGlobPath = allPaths.flatMap { path => val hdfsPath = new Path(path) val fs = hdfsPath.getFileSystem(hadoopConf) val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) @@ -560,6 +560,21 @@ case class DataSource( } globPath }.toSeq + + val (filteredOut, filteredIn) = allGlobPath.partition { path => + InMemoryFileIndex.shouldFilterOut(path.getName) + } + if (filteredOut.nonEmpty) { + if (filteredIn.isEmpty) { + throw new AnalysisException( + s"All paths were ignored:\n${filteredOut.mkString("\n ")}") + } else { + logDebug( + s"Some paths were ignored:\n${filteredOut.mkString("\n ")}") + } + } + + allGlobPath } } diff --git a/sql/core/src/test/resources/test-data/_cars.csv b/sql/core/src/test/resources/test-data/_cars.csv new file mode 100644 index 000000000000..40ded573ade5 --- /dev/null +++ b/sql/core/src/test/resources/test-data/_cars.csv @@ -0,0 +1,7 @@ + +year,make,model,comment,blank +"2012","Tesla","S","No comment", + +1997,Ford,E350,"Go get one now they are going fast", +2015,Chevy,Volt + 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 d9e5d7af1967..fb1bedfaa32c 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 @@ -53,6 +53,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te private val carsEmptyValueFile = "test-data/cars-empty-value.csv" private val carsBlankColName = "test-data/cars-blank-column-name.csv" private val carsCrlf = "test-data/cars-crlf.csv" + private val carsFilteredOutFile = "test-data/_cars.csv" private val emptyFile = "test-data/empty.csv" private val commentsFile = "test-data/comments.csv" private val disableCommentsFile = "test-data/disable_comments.csv" @@ -346,6 +347,25 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te assert(result.schema.fieldNames.size === 1) } + test("SPARK-26339 Not throw an exception if some of specified paths are filtered in") { + val cars = spark + .read + .option("header", "false") + .csv(testFile(carsFile), testFile(carsFilteredOutFile)) + + verifyCars(cars, withHeader = false, checkTypes = false) + } + + test("SPARK-26339 Throw an exception only if all of the specified paths are filtered out") { + val e = intercept[AnalysisException] { + val cars = spark + .read + .option("header", "false") + .csv(testFile(carsFilteredOutFile)) + }.getMessage + assert(e.contains("All paths were ignored:")) + } + test("DDL test with empty file") { withView("carsTable") { spark.sql( From c0368363f8a81dd739c6c90fb2849b2a3ab4d8e4 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 31 Dec 2018 17:46:06 +0100 Subject: [PATCH 0204/1072] [SPARK-26495][SQL] Simplify the SelectedField extractor. ## What changes were proposed in this pull request? The current `SelectedField` extractor is somewhat complicated and it seems to be handling cases that should be handled automatically: - `GetArrayItem(child: GetStructFieldObject())` - `GetArrayStructFields(child: GetArrayStructFields())` - `GetMap(value: GetStructFieldObject())` This PR removes those cases and simplifies the extractor by passing down the data type instead of a field. ## How was this patch tested? Existing tests. Closes #23397 from hvanhovell/SPARK-26495. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../spark/sql/execution/SelectedField.scala | 103 +++++++----------- 1 file changed, 41 insertions(+), 62 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala index 0e7c593f9fb6..68f797a856a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -51,8 +52,6 @@ import org.apache.spark.sql.types._ * type appropriate to the complex type extractor. In our example, the name of the child expression * is "name" and its data type is a [[org.apache.spark.sql.types.StructType]] with a single string * field named "first". - * - * @param expr the top-level complex type extractor */ private[execution] object SelectedField { def unapply(expr: Expression): Option[StructField] = { @@ -64,71 +63,51 @@ private[execution] object SelectedField { selectField(unaliased, None) } - private def selectField(expr: Expression, fieldOpt: Option[StructField]): Option[StructField] = { + /** + * Convert an expression into the parts of the schema (the field) it accesses. + */ + private def selectField(expr: Expression, dataTypeOpt: Option[DataType]): Option[StructField] = { expr match { - // No children. Returns a StructField with the attribute name or None if fieldOpt is None. - case AttributeReference(name, dataType, nullable, metadata) => - fieldOpt.map(field => - StructField(name, wrapStructType(dataType, field), nullable, metadata)) - // Handles case "expr0.field[n]", where "expr0" is of struct type and "expr0.field" is of - // array type. - case GetArrayItem(x @ GetStructFieldObject(child, field @ StructField(name, - dataType, nullable, metadata)), _) => - val childField = fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), nullable, metadata)).getOrElse(field) - selectField(child, Some(childField)) - // Handles case "expr0.field[n]", where "expr0.field" is of array type. - case GetArrayItem(child, _) => - selectField(child, fieldOpt) - // Handles case "expr0.field.subfield", where "expr0" and "expr0.field" are of array type. - case GetArrayStructFields(child: GetArrayStructFields, - field @ StructField(name, dataType, nullable, metadata), _, _, _) => - val childField = fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), - nullable, metadata)).orElse(Some(field)) - selectField(child, childField) - // Handles case "expr0.field", where "expr0" is of array type. - case GetArrayStructFields(child, - field @ StructField(name, dataType, nullable, metadata), _, _, _) => - val childField = - fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), - nullable, metadata)).orElse(Some(field)) - selectField(child, childField) - // Handles case "expr0.field[key]", where "expr0" is of struct type and "expr0.field" is of - // map type. - case GetMapValue(x @ GetStructFieldObject(child, field @ StructField(name, - dataType, - nullable, metadata)), _) => - val childField = fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), - nullable, metadata)).orElse(Some(field)) - selectField(child, childField) - // Handles case "expr0.field[key]", where "expr0.field" is of map type. + case a: Attribute => + dataTypeOpt.map { dt => + StructField(a.name, dt, a.nullable) + } + case c: GetStructField => + val field = c.childSchema(c.ordinal) + val newField = field.copy(dataType = dataTypeOpt.getOrElse(field.dataType)) + selectField(c.child, Option(struct(newField))) + case GetArrayStructFields(child, field, _, _, containsNull) => + val newFieldDataType = dataTypeOpt match { + case None => + // GetArrayStructFields is the top level extractor. This means its result is + // not pruned and we need to use the element type of the array its producing. + field.dataType + case Some(ArrayType(dataType, _)) => + // GetArrayStructFields is part of a chain of extractors and its result is pruned + // by a parent expression. In this case need to use the parent element type. + dataType + case Some(x) => + // This should not happen. + throw new AnalysisException(s"DataType '$x' is not supported by GetArrayStructFields.") + } + val newField = StructField(field.name, newFieldDataType, field.nullable) + selectField(child, Option(ArrayType(struct(newField), containsNull))) case GetMapValue(child, _) => - selectField(child, fieldOpt) - // Handles case "expr0.field", where expr0 is of struct type. - case GetStructFieldObject(child, - field @ StructField(name, dataType, nullable, metadata)) => - val childField = fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), - nullable, metadata)).orElse(Some(field)) - selectField(child, childField) + // GetMapValue does not select a field from a struct (i.e. prune the struct) so it can't be + // the top-level extractor. However it can be part of an extractor chain. + val MapType(keyType, _, valueContainsNull) = child.dataType + val opt = dataTypeOpt.map(dt => MapType(keyType, dt, valueContainsNull)) + selectField(child, opt) + case GetArrayItem(child, _) => + // GetArrayItem does not select a field from a struct (i.e. prune the struct) so it can't be + // the top-level extractor. However it can be part of an extractor chain. + val ArrayType(_, containsNull) = child.dataType + val opt = dataTypeOpt.map(dt => ArrayType(dt, containsNull)) + selectField(child, opt) case _ => None } } - // Constructs a composition of complex types with a StructType(Array(field)) at its core. Returns - // a StructType for a StructType, an ArrayType for an ArrayType and a MapType for a MapType. - private def wrapStructType(dataType: DataType, field: StructField): DataType = { - dataType match { - case _: StructType => - StructType(Array(field)) - case ArrayType(elementType, containsNull) => - ArrayType(wrapStructType(elementType, field), containsNull) - case MapType(keyType, valueType, valueContainsNull) => - MapType(keyType, wrapStructType(valueType, field), valueContainsNull) - } - } + private def struct(field: StructField): StructType = StructType(Array(field)) } From b1a9b5eff59f64c370cd7388761effdf2152a108 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Mon, 31 Dec 2018 13:35:02 -0800 Subject: [PATCH 0205/1072] [SPARK-26470][CORE] Use ConfigEntry for hardcoded configs for eventLog category ## What changes were proposed in this pull request? The PR makes hardcoded `spark.eventLog` configs to use `ConfigEntry` and put them in the `config` package. ## How was this patch tested? existing tests Closes #23395 from mgaido91/SPARK-26470. Authored-by: Marco Gaido Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/SparkContext.scala | 7 +++---- .../apache/spark/internal/config/package.scala | 9 +++++++++ .../spark/deploy/history/HistoryServerSuite.scala | 9 +++++---- .../scheduler/EventLoggingListenerSuite.scala | 15 ++++++++------- 4 files changed, 25 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 09cc346db0ed..3475859c3ed6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -230,7 +230,7 @@ class SparkContext(config: SparkConf) extends Logging { def deployMode: String = _conf.getOption("spark.submit.deployMode").getOrElse("client") def appName: String = _conf.get("spark.app.name") - private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) + private[spark] def isEventLogEnabled: Boolean = _conf.get(EVENT_LOG_ENABLED) private[spark] def eventLogDir: Option[URI] = _eventLogDir private[spark] def eventLogCodec: Option[String] = _eventLogCodec @@ -396,15 +396,14 @@ class SparkContext(config: SparkConf) extends Logging { _eventLogDir = if (isEventLogEnabled) { - val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) - .stripSuffix("/") + val unresolvedDir = conf.get(EVENT_LOG_DIR).stripSuffix("/") Some(Utils.resolveURI(unresolvedDir)) } else { None } _eventLogCodec = { - val compress = _conf.getBoolean("spark.eventLog.compress", false) + val compress = _conf.get(EVENT_LOG_COMPRESS) if (compress && isEventLogEnabled) { Some(CompressionCodec.getCodecName(_conf)).map(CompressionCodec.getShortName) } else { 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 f1c1c034df49..d8e9c099028f 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 @@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.ByteUnit +import org.apache.spark.scheduler.EventLoggingListener import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils @@ -62,6 +63,14 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val EVENT_LOG_ENABLED = ConfigBuilder("spark.eventLog.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val EVENT_LOG_DIR = ConfigBuilder("spark.eventLog.dir") + .stringConf + .createWithDefault(EventLoggingListener.DEFAULT_LOG_DIR) + private[spark] val EVENT_LOG_COMPRESS = ConfigBuilder("spark.eventLog.compress") .booleanConf 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 a9dee67ae938..96458c55b5f5 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 @@ -45,6 +45,7 @@ import org.scalatest.mockito.MockitoSugar import org.scalatest.selenium.WebBrowser import org.apache.spark._ +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History._ import org.apache.spark.status.api.v1.ApplicationInfo import org.apache.spark.status.api.v1.JobData @@ -82,8 +83,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers .set(UPDATE_INTERVAL_S.key, "0") .set("spark.testing", "true") .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) - .set("spark.eventLog.logStageExecutorMetrics.enabled", "true") - .set("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled", "true") + .set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) + .set(EVENT_LOG_PROCESS_TREE_METRICS, true) conf.setAll(extraConf) provider = new FsHistoryProvider(conf) provider.checkForLogs() @@ -417,9 +418,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers stop() val myConf = new SparkConf() .set(HISTORY_LOG_DIR, logDir.getAbsolutePath) - .set("spark.eventLog.dir", logDir.getAbsolutePath) + .set(EVENT_LOG_DIR, logDir.getAbsolutePath) .set(UPDATE_INTERVAL_S.key, "1s") - .set("spark.eventLog.enabled", "true") + .set(EVENT_LOG_ENABLED, true) .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) .remove("spark.testing") val provider = new FsHistoryProvider(myConf) 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 0c04a93646d7..04987e6ef79e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.io._ import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -122,7 +123,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // Expected IOException, since we haven't enabled log overwrite. intercept[IOException] { testEventLogging() } // Try again, but enable overwriting. - testEventLogging(extraConf = Map("spark.eventLog.overwrite" -> "true")) + testEventLogging(extraConf = Map(EVENT_LOG_OVERWRITE.key -> "true")) } test("Event log name") { @@ -526,15 +527,15 @@ object EventLoggingListenerSuite { /** Get a SparkConf with event logging enabled. */ def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { val conf = new SparkConf - conf.set("spark.eventLog.enabled", "true") - conf.set("spark.eventLog.logBlockUpdates.enabled", "true") - conf.set("spark.eventLog.testing", "true") - conf.set("spark.eventLog.dir", logDir.toString) + conf.set(EVENT_LOG_ENABLED, true) + conf.set(EVENT_LOG_BLOCK_UPDATES, true) + conf.set(EVENT_LOG_TESTING, true) + conf.set(EVENT_LOG_DIR, logDir.toString) compressionCodec.foreach { codec => - conf.set("spark.eventLog.compress", "true") + conf.set(EVENT_LOG_COMPRESS, true) conf.set("spark.io.compression.codec", codec) } - conf.set("spark.eventLog.logStageExecutorMetrics.enabled", "true") + conf.set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) conf } From 993736154b6a46ffd7c3218173a2653a3842bba0 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 1 Jan 2019 09:14:23 +0800 Subject: [PATCH 0206/1072] [MINOR] Fix inconsistency log level among delegation token providers ## What changes were proposed in this pull request? There's some inconsistency for log level while logging error messages in delegation token providers. (DEBUG, INFO, WARNING) Given that failing to obtain token would often crash the query, I guess it would be nice to set higher log level for error log messages. ## How was this patch tested? The patch just changed the log level. Closes #23418 from HeartSaVioR/FIX-inconsistency-log-level-between-delegation-token-providers. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Hyukjin Kwon --- .../HBaseDelegationTokenProvider.scala | 4 +- .../HadoopFSDelegationTokenProvider.scala | 45 +++++++++++-------- .../HiveDelegationTokenProvider.scala | 4 +- .../KafkaDelegationTokenProvider.scala | 2 +- 4 files changed, 31 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala index 5dcde4ec3a8a..6ef68351bc9b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala @@ -50,7 +50,7 @@ private[security] class HBaseDelegationTokenProvider creds.addToken(token.getService, token) } catch { case NonFatal(e) => - logDebug(s"Failed to get token from service $serviceName", e) + logWarning(s"Failed to get token from service $serviceName", e) } None @@ -71,7 +71,7 @@ private[security] class HBaseDelegationTokenProvider confCreate.invoke(null, conf).asInstanceOf[Configuration] } catch { case NonFatal(e) => - logDebug("Fail to invoke HBaseConfiguration", e) + logWarning("Fail to invoke HBaseConfiguration", e) conf } } 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 767b5521e8d7..00200f807d22 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 @@ -19,6 +19,7 @@ package org.apache.spark.deploy.security import scala.collection.JavaConverters._ import scala.util.Try +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem @@ -44,28 +45,34 @@ private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: () => Set[Fil hadoopConf: Configuration, sparkConf: SparkConf, creds: Credentials): Option[Long] = { - val fsToGetTokens = fileSystems() - val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fsToGetTokens, creds) + try { + val fsToGetTokens = fileSystems() + val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fsToGetTokens, creds) - // Get the token renewal interval if it is not set. It will only be called once. - if (tokenRenewalInterval == null) { - tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, fsToGetTokens) - } + // Get the token renewal interval if it is not set. It will only be called once. + if (tokenRenewalInterval == null) { + tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, fsToGetTokens) + } - // Get the time of next renewal. - val nextRenewalDate = tokenRenewalInterval.flatMap { interval => - val nextRenewalDates = fetchCreds.getAllTokens.asScala - .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) - .map { token => - val identifier = token - .decodeIdentifier() - .asInstanceOf[AbstractDelegationTokenIdentifier] - identifier.getIssueDate + interval - } - if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) - } + // Get the time of next renewal. + val nextRenewalDate = tokenRenewalInterval.flatMap { interval => + val nextRenewalDates = fetchCreds.getAllTokens.asScala + .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) + .map { token => + val identifier = token + .decodeIdentifier() + .asInstanceOf[AbstractDelegationTokenIdentifier] + identifier.getIssueDate + interval + } + if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) + } - nextRenewalDate + nextRenewalDate + } catch { + case NonFatal(e) => + logWarning(s"Failed to get token from service $serviceName", e) + None + } } override def delegationTokensRequired( diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala index 7249eb85ac7c..90f705138157 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala @@ -49,7 +49,7 @@ private[spark] class HiveDelegationTokenProvider new HiveConf(hadoopConf, classOf[HiveConf]) } catch { case NonFatal(e) => - logDebug("Fail to create Hive Configuration", e) + logWarning("Fail to create Hive Configuration", e) hadoopConf case e: NoClassDefFoundError => logWarning(classNotFoundErrorStr) @@ -104,7 +104,7 @@ private[spark] class HiveDelegationTokenProvider None } catch { case NonFatal(e) => - logDebug(s"Failed to get token from service $serviceName", e) + logWarning(s"Failed to get token from service $serviceName", e) None case e: NoClassDefFoundError => logWarning(classNotFoundErrorStr) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala index 45995be630cc..f67cb26259fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala @@ -44,7 +44,7 @@ private[security] class KafkaDelegationTokenProvider return Some(nextRenewalDate) } catch { case NonFatal(e) => - logInfo(s"Failed to get token from service $serviceName", e) + logWarning(s"Failed to get token from service $serviceName", e) } None } From f7455618ce6de8d2e70f10722dc112fcc6ee3cee Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 1 Jan 2019 09:29:28 +0800 Subject: [PATCH 0207/1072] Revert "[SPARK-26339][SQL] Throws better exception when reading files that start with underscore" This reverts commit c0b9db120d4c2ad0b5b99b9152549e94ef8f5a2d. --- .../execution/datasources/DataSource.scala | 17 +--------------- .../src/test/resources/test-data/_cars.csv | 7 ------- .../execution/datasources/csv/CSVSuite.scala | 20 ------------------- 3 files changed, 1 insertion(+), 43 deletions(-) delete mode 100644 sql/core/src/test/resources/test-data/_cars.csv 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 517e04317d94..fefff68c4ba8 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 @@ -543,7 +543,7 @@ case class DataSource( checkFilesExist: Boolean): Seq[Path] = { val allPaths = caseInsensitiveOptions.get("path") ++ paths val hadoopConf = sparkSession.sessionState.newHadoopConf() - val allGlobPath = allPaths.flatMap { path => + allPaths.flatMap { path => val hdfsPath = new Path(path) val fs = hdfsPath.getFileSystem(hadoopConf) val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) @@ -560,21 +560,6 @@ case class DataSource( } globPath }.toSeq - - val (filteredOut, filteredIn) = allGlobPath.partition { path => - InMemoryFileIndex.shouldFilterOut(path.getName) - } - if (filteredOut.nonEmpty) { - if (filteredIn.isEmpty) { - throw new AnalysisException( - s"All paths were ignored:\n${filteredOut.mkString("\n ")}") - } else { - logDebug( - s"Some paths were ignored:\n${filteredOut.mkString("\n ")}") - } - } - - allGlobPath } } diff --git a/sql/core/src/test/resources/test-data/_cars.csv b/sql/core/src/test/resources/test-data/_cars.csv deleted file mode 100644 index 40ded573ade5..000000000000 --- a/sql/core/src/test/resources/test-data/_cars.csv +++ /dev/null @@ -1,7 +0,0 @@ - -year,make,model,comment,blank -"2012","Tesla","S","No comment", - -1997,Ford,E350,"Go get one now they are going fast", -2015,Chevy,Volt - 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 fb1bedfaa32c..d9e5d7af1967 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 @@ -53,7 +53,6 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te private val carsEmptyValueFile = "test-data/cars-empty-value.csv" private val carsBlankColName = "test-data/cars-blank-column-name.csv" private val carsCrlf = "test-data/cars-crlf.csv" - private val carsFilteredOutFile = "test-data/_cars.csv" private val emptyFile = "test-data/empty.csv" private val commentsFile = "test-data/comments.csv" private val disableCommentsFile = "test-data/disable_comments.csv" @@ -347,25 +346,6 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te assert(result.schema.fieldNames.size === 1) } - test("SPARK-26339 Not throw an exception if some of specified paths are filtered in") { - val cars = spark - .read - .option("header", "false") - .csv(testFile(carsFile), testFile(carsFilteredOutFile)) - - verifyCars(cars, withHeader = false, checkTypes = false) - } - - test("SPARK-26339 Throw an exception only if all of the specified paths are filtered out") { - val e = intercept[AnalysisException] { - val cars = spark - .read - .option("header", "false") - .csv(testFile(carsFilteredOutFile)) - }.getMessage - assert(e.contains("All paths were ignored:")) - } - test("DDL test with empty file") { withView("carsTable") { spark.sql( From 5f0ddd2d6e2fdebf549207bbc4b13ca709eee3c4 Mon Sep 17 00:00:00 2001 From: Thomas D'Silva Date: Tue, 1 Jan 2019 14:11:14 +0800 Subject: [PATCH 0208/1072] [SPARK-26499][SQL] JdbcUtils.makeGetter does not handle ByteType MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …Type ## What changes were proposed in this pull request? Modifed JdbcUtils.makeGetter to handle ByteType. ## How was this patch tested? Added a new test to JDBCSuite that maps ```TINYINT``` to ```ByteType```. Closes #23400 from twdsilva/tiny_int_support. Authored-by: Thomas D'Silva Signed-off-by: Hyukjin Kwon --- .../datasources/jdbc/JdbcUtils.scala | 4 +++ .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 25 +++++++++++++++++++ 2 files changed, 29 insertions(+) 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 edea549748b4..922bef284c98 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 @@ -438,6 +438,10 @@ object JdbcUtils extends Logging { (rs: ResultSet, row: InternalRow, pos: Int) => row.setShort(pos, rs.getShort(pos + 1)) + case ByteType => + (rs: ResultSet, row: InternalRow, pos: Int) => + row.update(pos, rs.getByte(pos + 1)) + case StringType => (rs: ResultSet, row: InternalRow, pos: Int) => // TODO(davies): use getBytes for better performance, if the encoding is UTF-8 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 71e83767964a..e4641631e607 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 @@ -56,6 +56,20 @@ class JDBCSuite extends QueryTest Some(StringType) } + val testH2DialectTinyInt = new JdbcDialect { + override def canHandle(url: String): Boolean = url.startsWith("jdbc:h2") + override def getCatalystType( + sqlType: Int, + typeName: String, + size: Int, + md: MetadataBuilder): Option[DataType] = { + sqlType match { + case java.sql.Types.TINYINT => Some(ByteType) + case _ => None + } + } + } + before { Utils.classForName("org.h2.Driver") // Extra properties that will be specified for our database. We need these to test @@ -693,6 +707,17 @@ class JDBCSuite extends QueryTest JdbcDialects.unregisterDialect(testH2Dialect) } + test("Map TINYINT to ByteType via JdbcDialects") { + JdbcDialects.registerDialect(testH2DialectTinyInt) + val df = spark.read.jdbc(urlWithUserAndPass, "test.inttypes", new Properties()) + val rows = df.collect() + assert(rows.length === 2) + assert(rows(0).get(2).isInstanceOf[Byte]) + assert(rows(0).getByte(2) === 3) + assert(rows(1).isNullAt(2)) + JdbcDialects.unregisterDialect(testH2DialectTinyInt) + } + test("Default jdbc dialect registration") { assert(JdbcDialects.get("jdbc:mysql://127.0.0.1/db") == MySQLDialect) assert(JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") == PostgresDialect) From 2bf4d97118c20812e26a7ea59826ee470ab42f7c Mon Sep 17 00:00:00 2001 From: zhoukang Date: Tue, 1 Jan 2019 09:13:13 -0600 Subject: [PATCH 0209/1072] [SPARK-24544][SQL] Print actual failure cause when look up function failed ## What changes were proposed in this pull request? When we operate as below: ` 0: jdbc:hive2://xxx/> create function funnel_analysis as 'com.xxx.hive.extend.udf.UapFunnelAnalysis'; ` ` 0: jdbc:hive2://xxx/> select funnel_analysis(1,",",1,''); Error: org.apache.spark.sql.AnalysisException: Undefined function: 'funnel_analysis'. This function is neither a registered temporary function nor a permanent function registered in the database 'xxx'.; line 1 pos 7 (state=,code=0) ` ` 0: jdbc:hive2://xxx/> describe function funnel_analysis; +-----------------------------------------------------------+--+ | function_desc | +-----------------------------------------------------------+--+ | Function: xxx.funnel_analysis | | Class: com.xxx.hive.extend.udf.UapFunnelAnalysis | | Usage: N/A. | +-----------------------------------------------------------+--+ ` We can see describe funtion will get right information,but when we actually use this funtion,we will get an undefined exception. Which is really misleading,the real cause is below: ` No handler for Hive UDF 'com.xxx.xxx.hive.extend.udf.UapFunnelAnalysis': java.lang.IllegalStateException: Should not be called directly; at org.apache.hadoop.hive.ql.udf.generic.GenericUDTF.initialize(GenericUDTF.java:72) at org.apache.spark.sql.hive.HiveGenericUDTF.outputInspector$lzycompute(hiveUDFs.scala:204) at org.apache.spark.sql.hive.HiveGenericUDTF.outputInspector(hiveUDFs.scala:204) at org.apache.spark.sql.hive.HiveGenericUDTF.elementSchema$lzycompute(hiveUDFs.scala:212) at org.apache.spark.sql.hive.HiveGenericUDTF.elementSchema(hiveUDFs.scala:212) ` This patch print the actual failure for quick debugging. ## How was this patch tested? UT Closes #21790 from caneGuy/zhoukang/print-warning1. Authored-by: zhoukang Signed-off-by: Sean Owen --- .../catalyst/analysis/NoSuchItemException.scala | 4 ++-- .../sql/catalyst/catalog/SessionCatalog.scala | 5 +++-- .../sql/catalyst/catalog/SessionCatalogSuite.scala | 14 ++++++++++++++ .../apache/spark/sql/hive/HiveSessionCatalog.scala | 9 ++++++--- 4 files changed, 25 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index f5aae60431c1..8bf6f69f3b17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -40,10 +40,10 @@ class NoSuchPartitionException( class NoSuchPermanentFunctionException(db: String, func: String) extends AnalysisException(s"Function '$func' not found in database '$db'") -class NoSuchFunctionException(db: String, func: String) +class NoSuchFunctionException(db: String, func: String, cause: Option[Throwable] = None) extends AnalysisException( s"Undefined function: '$func'. This function is neither a registered temporary function nor " + - s"a permanent function registered in the database '$db'.") + s"a permanent function registered in the database '$db'.", cause = cause) class NoSuchPartitionsException(db: String, table: String, specs: Seq[TablePartitionSpec]) extends AnalysisException( 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 b6771ec4dffe..1dbe946503e5 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 @@ -1222,9 +1222,10 @@ class SessionCatalog( databaseExists(db) && externalCatalog.functionExists(db, name.funcName) } - protected def failFunctionLookup(name: FunctionIdentifier): Nothing = { + protected[sql] def failFunctionLookup( + name: FunctionIdentifier, cause: Option[Throwable] = None): Nothing = { throw new NoSuchFunctionException( - db = name.database.getOrElse(getCurrentDatabase), func = name.funcName) + db = name.database.getOrElse(getCurrentDatabase), func = name.funcName, cause) } /** 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 19e8c0334689..92f87ea796e8 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 @@ -1448,4 +1448,18 @@ abstract class SessionCatalogSuite extends AnalysisTest { } } } + + test("SPARK-24544: test print actual failure cause when look up function failed") { + withBasicCatalog { catalog => + val cause = intercept[NoSuchFunctionException] { + catalog.failFunctionLookup(FunctionIdentifier("failureFunc"), + Some(new Exception("Actual error"))) + } + + // fullStackTrace will be printed, but `cause.getMessage` has been + // override in `AnalysisException`,so here we get the root cause + // exception message for check. + assert(cause.cause.get.getMessage.contains("Actual error")) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 405c0c8bfe66..7560805bb3b0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DecimalType, DoubleType} +import org.apache.spark.util.Utils private[sql] class HiveSessionCatalog( @@ -141,8 +142,10 @@ private[sql] class HiveSessionCatalog( // let's try to load it as a Hive's built-in function. // Hive is case insensitive. val functionName = funcName.unquotedString.toLowerCase(Locale.ROOT) + logWarning("Encountered a failure during looking up function:" + + s" ${Utils.exceptionString(error)}") if (!hiveFunctions.contains(functionName)) { - failFunctionLookup(funcName) + failFunctionLookup(funcName, Some(error)) } // TODO: Remove this fallback path once we implement the list of fallback functions @@ -150,12 +153,12 @@ private[sql] class HiveSessionCatalog( val functionInfo = { try { Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse( - failFunctionLookup(funcName)) + failFunctionLookup(funcName, Some(error))) } catch { // If HiveFunctionRegistry.getFunctionInfo throws an exception, // we are failing to load a Hive builtin function, which means that // the given function is not a Hive builtin function. - case NonFatal(e) => failFunctionLookup(funcName) + case NonFatal(e) => failFunctionLookup(funcName, Some(e)) } } val className = functionInfo.getFunctionClass.getName From 001d3095385626e329b3853364a4feeb811aac5a Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Tue, 1 Jan 2019 09:18:58 -0600 Subject: [PATCH 0210/1072] [SPARK-25765][ML] Add training cost to BisectingKMeans summary ## What changes were proposed in this pull request? The PR adds the `trainingCost` value to the `BisectingKMeansSummary`, in order to expose the information retrievable by running `computeCost` on the training dataset. This fills the gap with `KMeans` implementation. ## How was this patch tested? improved UTs Closes #22764 from mgaido91/SPARK-25765. Authored-by: Marco Gaido Signed-off-by: Sean Owen --- .../spark/ml/clustering/BisectingKMeans.scala | 13 +++- .../mllib/clustering/BisectingKMeans.scala | 3 +- .../clustering/BisectingKMeansModel.scala | 59 ++++++++++++++++--- .../ml/clustering/BisectingKMeansSuite.scala | 2 + .../clustering/BisectingKMeansSuite.scala | 1 + project/MimaExcludes.scala | 3 + python/pyspark/ml/clustering.py | 12 +++- 7 files changed, 82 insertions(+), 11 deletions(-) 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 49e9f5136813..d846f17e7f54 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 @@ -264,7 +264,12 @@ class BisectingKMeans @Since("2.0.0") ( val parentModel = bkm.run(rdd, Some(instr)) val model = copyValues(new BisectingKMeansModel(uid, parentModel).setParent(this)) val summary = new BisectingKMeansSummary( - model.transform(dataset), $(predictionCol), $(featuresCol), $(k), $(maxIter)) + model.transform(dataset), + $(predictionCol), + $(featuresCol), + $(k), + $(maxIter), + parentModel.trainingCost) instr.logNamedValue("clusterSizes", summary.clusterSizes) instr.logNumFeatures(model.clusterCenters.head.size) model.setSummary(Some(summary)) @@ -294,6 +299,8 @@ object BisectingKMeans extends DefaultParamsReadable[BisectingKMeans] { * @param featuresCol Name for column of features in `predictions`. * @param k Number of clusters. * @param numIter Number of iterations. + * @param trainingCost Sum of the cost to the nearest centroid for all points in the training + * dataset. This is equivalent to sklearn's inertia. */ @Since("2.1.0") @Experimental @@ -302,4 +309,6 @@ class BisectingKMeansSummary private[clustering] ( predictionCol: String, featuresCol: String, k: Int, - numIter: Int) extends ClusteringSummary(predictions, predictionCol, featuresCol, k, numIter) + numIter: Int, + @Since("3.0.0") val trainingCost: Double) + extends ClusteringSummary(predictions, predictionCol, featuresCol, k, numIter) 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 80ab8eb9bc8b..696dff0f319a 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 @@ -242,7 +242,8 @@ class BisectingKMeans private ( norms.unpersist(false) val clusters = activeClusters ++ inactiveClusters val root = buildTree(clusters, dMeasure) - new BisectingKMeansModel(root, this.distanceMeasure) + val totalCost = root.leafNodes.map(_.cost).sum + new BisectingKMeansModel(root, this.distanceMeasure, totalCost) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala index 4c5794fbffc8..b54b8917e060 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -41,11 +41,12 @@ import org.apache.spark.sql.{Row, SparkSession} @Since("1.6.0") class BisectingKMeansModel private[clustering] ( private[clustering] val root: ClusteringTreeNode, - @Since("2.4.0") val distanceMeasure: String + @Since("2.4.0") val distanceMeasure: String, + @Since("3.0.0") val trainingCost: Double ) extends Serializable with Saveable with Logging { @Since("1.6.0") - def this(root: ClusteringTreeNode) = this(root, DistanceMeasure.EUCLIDEAN) + def this(root: ClusteringTreeNode) = this(root, DistanceMeasure.EUCLIDEAN, 0.0) private val distanceMeasureInstance: DistanceMeasure = DistanceMeasure.decodeFromString(distanceMeasure) @@ -109,10 +110,10 @@ class BisectingKMeansModel private[clustering] ( @Since("2.0.0") override def save(sc: SparkContext, path: String): Unit = { - BisectingKMeansModel.SaveLoadV2_0.save(sc, this, path) + BisectingKMeansModel.SaveLoadV3_0.save(sc, this, path) } - override protected def formatVersion: String = "2.0" + override protected def formatVersion: String = "3.0" } @Since("2.0.0") @@ -128,11 +129,15 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { case (SaveLoadV2_0.thisClassName, SaveLoadV2_0.thisFormatVersion) => val model = SaveLoadV2_0.load(sc, path) model + case (SaveLoadV3_0.thisClassName, SaveLoadV3_0.thisFormatVersion) => + val model = SaveLoadV3_0.load(sc, path) + model case _ => throw new Exception( s"BisectingKMeansModel.load did not recognize model with (className, format version):" + s"($loadedClassName, $formatVersion). Supported:\n" + s" (${SaveLoadV1_0.thisClassName}, ${SaveLoadV1_0.thisClassName}\n" + - s" (${SaveLoadV2_0.thisClassName}, ${SaveLoadV2_0.thisClassName})") + s" (${SaveLoadV2_0.thisClassName}, ${SaveLoadV2_0.thisClassName})\n" + + s" (${SaveLoadV3_0.thisClassName}, ${SaveLoadV3_0.thisClassName})") } } @@ -195,7 +200,8 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") val nodes = data.rdd.map(Data.apply).collect().map(d => (d.index, d)).toMap val rootNode = buildTree(rootId, nodes) - new BisectingKMeansModel(rootNode, DistanceMeasure.EUCLIDEAN) + val totalCost = rootNode.leafNodes.map(_.cost).sum + new BisectingKMeansModel(rootNode, DistanceMeasure.EUCLIDEAN, totalCost) } } @@ -231,7 +237,46 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") val nodes = data.rdd.map(Data.apply).collect().map(d => (d.index, d)).toMap val rootNode = buildTree(rootId, nodes) - new BisectingKMeansModel(rootNode, distanceMeasure) + val totalCost = rootNode.leafNodes.map(_.cost).sum + new BisectingKMeansModel(rootNode, distanceMeasure, totalCost) + } + } + + private[clustering] object SaveLoadV3_0 { + private[clustering] val thisFormatVersion = "3.0" + + private[clustering] + val thisClassName = "org.apache.spark.mllib.clustering.BisectingKMeansModel" + + def save(sc: SparkContext, model: BisectingKMeansModel, path: String): Unit = { + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) + ~ ("rootId" -> model.root.index) ~ ("distanceMeasure" -> model.distanceMeasure) + ~ ("trainingCost" -> model.trainingCost))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + val data = getNodes(model.root).map(node => Data(node.index, node.size, + node.centerWithNorm.vector, node.centerWithNorm.norm, node.cost, node.height, + node.children.map(_.index))) + spark.createDataFrame(data).write.parquet(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): BisectingKMeansModel = { + implicit val formats: DefaultFormats = DefaultFormats + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + val rootId = (metadata \ "rootId").extract[Int] + val distanceMeasure = (metadata \ "distanceMeasure").extract[String] + val trainingCost = (metadata \ "trainingCost").extract[Double] + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val rows = spark.read.parquet(Loader.dataPath(path)) + Loader.checkSchema[Data](rows.schema) + val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") + val nodes = data.rdd.map(Data.apply).collect().map(d => (d.index, d)).toMap + val rootNode = buildTree(rootId, nodes) + new BisectingKMeansModel(rootNode, distanceMeasure, trainingCost) } } } 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 1b7780e171e7..461f8b8d211d 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 @@ -134,6 +134,8 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { assert(clusterSizes.sum === numRows) assert(clusterSizes.forall(_ >= 0)) assert(summary.numIter == 20) + assert(summary.trainingCost < 0.1) + assert(model.computeCost(dataset) == summary.trainingCost) model.setSummary(None) assert(!model.hasSummary) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala index 4a4d8b5c89de..10d5f325d68e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala @@ -194,6 +194,7 @@ class BisectingKMeansSuite extends SparkFunSuite with MLlibTestSparkContext { assert(model.k === sameModel.k) assert(model.distanceMeasure === sameModel.distanceMeasure) model.clusterCenters.zip(sameModel.clusterCenters).foreach(c => c._1 === c._2) + assert(model.trainingCost == sameModel.trainingCost) } finally { Utils.deleteRecursively(tempDir) } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 89fc53ce3972..cf8d9f3c24d0 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,9 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( + // [SPARK-25765][ML] Add training cost to BisectingKMeans summary + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel.this"), + // [SPARK-24243][CORE] Expose exceptions from InProcessAppHandle ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.launcher.SparkAppHandle.getError"), diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index d8a6dfb7d3a7..5a776aec1425 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -587,6 +587,8 @@ class BisectingKMeans(JavaEstimator, HasDistanceMeasure, HasFeaturesCol, HasPred 2 >>> summary.clusterSizes [2, 2] + >>> summary.trainingCost + 2.000... >>> transformed = model.transform(df).select("features", "prediction") >>> rows = transformed.collect() >>> rows[0].prediction == rows[1].prediction @@ -700,7 +702,15 @@ class BisectingKMeansSummary(ClusteringSummary): .. versionadded:: 2.1.0 """ - pass + + @property + @since("3.0.0") + def trainingCost(self): + """ + Sum of squared distances to the nearest centroid for all points in the training dataset. + This is equivalent to sklearn's inertia. + """ + return self._call_java("trainingCost") @inherit_doc From 5da55873fa330f4ab21fb05a0a7dbf45bbeb5a54 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 2 Jan 2019 07:59:32 +0800 Subject: [PATCH 0211/1072] [SPARK-26374][TEST][SQL] Enable TimestampFormatter in HadoopFsRelationTest ## What changes were proposed in this pull request? Default timestamp pattern defined in `JSONOptions` doesn't allow saving/loading timestamps with time zones of seconds precision. Because of that, the round trip test failed for timestamps before 1582. In the PR, I propose to extend zone offset section from `XXX` to `XXXXX` which should allow to save/load zone offsets like `-07:52:48`. ## How was this patch tested? It was tested by `JsonHadoopFsRelationSuite` and `TimestampFormatterSuite`. Closes #23417 from MaxGekk/hadoopfsrelationtest-new-formatter. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Hyukjin Kwon --- .../sql/util/TimestampFormatterSuite.scala | 32 ++++++++++--------- .../sql/sources/HadoopFsRelationTest.scala | 6 ++-- 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index edccbb2a7f5d..2ce3eacc30cc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -70,21 +70,23 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { } test("roundtrip micros -> timestamp -> micros using timezones") { - Seq( - -58710115316212000L, - -18926315945345679L, - -9463427405253013L, - -244000001L, - 0L, - 99628200102030L, - 1543749753123456L, - 2177456523456789L, - 11858049903010203L).foreach { micros => - DateTimeTestUtils.outstandingTimezones.foreach { timeZone => - val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone, Locale.US) - val timestamp = formatter.format(micros) - val parsed = formatter.parse(timestamp) - assert(micros === parsed) + Seq("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXXXX").foreach { pattern => + Seq( + -58710115316212000L, + -18926315945345679L, + -9463427405253013L, + -244000001L, + 0L, + 99628200102030L, + 1543749753123456L, + 2177456523456789L, + 11858049903010203L).foreach { micros => + DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + val formatter = TimestampFormatter(pattern, timeZone, Locale.US) + val timestamp = formatter.format(micros) + val parsed = formatter.parse(timestamp) + assert(micros === parsed) + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index f0f62b608785..57b896612bfe 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -126,8 +126,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } else { Seq(false) } - // TODO: Support new parser too, see SPARK-26374. - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> "true") { + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> "false") { for (dataType <- supportedDataTypes) { for (parquetDictionaryEncodingEnabled <- parquetDictionaryEncodingEnabledConfs) { val extraMessage = if (isParquetDataSource) { @@ -138,7 +137,8 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes logInfo(s"Testing $dataType data type$extraMessage") val extraOptions = Map[String, String]( - "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString + "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString, + "timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SSSXXXXX" ) withTempPath { file => From 39a0493387d66a1e5c04f568804ebc83c2a5f644 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 2 Jan 2019 08:01:34 +0800 Subject: [PATCH 0212/1072] [SPARK-26227][R] from_[csv|json] should accept schema_of_[csv|json] in R API MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? **1. Document `from_csv(..., schema_of_csv(...))` support:** ```R csv <- "Amsterdam,2018" df <- sql(paste0("SELECT '", csv, "' as csv")) head(select(df, from_csv(df$csv, schema_of_csv(csv)))) ``` ``` from_csv(csv) 1 Amsterdam, 2018 ``` **2. Allow `from_json(..., schema_of_json(...))`** Before: ```R df2 <- sql("SELECT named_struct('name', 'Bob') as people") df2 <- mutate(df2, people_json = to_json(df2$people)) head(select(df2, from_json(df2$people_json, schema_of_json(head(df2)$people_json)))) ``` ``` Error in (function (classes, fdef, mtable) : unable to find an inherited method for function ‘from_json’ for signature ‘"Column", "Column"’ ``` After: ```R df2 <- sql("SELECT named_struct('name', 'Bob') as people") df2 <- mutate(df2, people_json = to_json(df2$people)) head(select(df2, from_json(df2$people_json, schema_of_json(head(df2)$people_json)))) ``` ``` from_json(people_json) 1 Bob ``` **3. (While I'm here) Allow `structType` as schema for `from_csv` support to match with `from_json`.** Before: ```R csv <- "Amsterdam,2018" df <- sql(paste0("SELECT '", csv, "' as csv")) head(select(df, from_csv(df$csv, structType("city STRING, year INT")))) ``` ``` Error in (function (classes, fdef, mtable) : unable to find an inherited method for function ‘from_csv’ for signature ‘"Column", "structType"’ ``` After: ```R csv <- "Amsterdam,2018" df <- sql(paste0("SELECT '", csv, "' as csv")) head(select(df, from_csv(df$csv, structType("city STRING, year INT")))) ``` ``` from_csv(csv) 1 Amsterdam, 2018 ``` ## How was this patch tested? Manually tested and unittests were added. Closes #23184 from HyukjinKwon/SPARK-26227-1. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- R/pkg/R/functions.R | 60 ++++++++++++------- R/pkg/tests/fulltests/test_sparkSQL.R | 16 ++++- .../org/apache/spark/sql/api/r/SQLUtils.scala | 6 +- 3 files changed, 59 insertions(+), 23 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 5b3cc0940d9c..58fc4104b0f0 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -202,8 +202,9 @@ NULL #' \itemize{ #' \item \code{from_json}: a structType object to use as the schema to use #' when parsing the JSON string. Since Spark 2.3, the DDL-formatted string is -#' also supported for the schema. -#' \item \code{from_csv}: a DDL-formatted string +#' also supported for the schema. Since Spark 3.0, \code{schema_of_json} or +#' the DDL-formatted string literal can also be accepted. +#' \item \code{from_csv}: a structType object, DDL-formatted string or \code{schema_of_csv} #' } #' @param ... additional argument(s). #' \itemize{ @@ -2254,6 +2255,8 @@ setMethod("date_format", signature(y = "Column", x = "character"), column(jc) }) +setClassUnion("characterOrstructTypeOrColumn", c("character", "structType", "Column")) + #' @details #' \code{from_json}: Parses a column containing a JSON string into a Column of \code{structType} #' with the specified \code{schema} or array of \code{structType} if \code{as.json.array} is set @@ -2261,7 +2264,7 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' #' @rdname column_collection_functions #' @param as.json.array indicating if input string is JSON array of objects or a single object. -#' @aliases from_json from_json,Column,characterOrstructType-method +#' @aliases from_json from_json,Column,characterOrstructTypeOrColumn-method #' @examples #' #' \dontrun{ @@ -2269,25 +2272,37 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' df2 <- mutate(df2, d2 = to_json(df2$d, dateFormat = 'dd/MM/yyyy')) #' schema <- structType(structField("date", "string")) #' head(select(df2, from_json(df2$d2, schema, dateFormat = 'dd/MM/yyyy'))) - #' df2 <- sql("SELECT named_struct('name', 'Bob') as people") #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' schema <- structType(structField("name", "string")) #' head(select(df2, from_json(df2$people_json, schema))) -#' head(select(df2, from_json(df2$people_json, "name STRING")))} +#' head(select(df2, from_json(df2$people_json, "name STRING"))) +#' head(select(df2, from_json(df2$people_json, schema_of_json(head(df2)$people_json))))} #' @note from_json since 2.2.0 -setMethod("from_json", signature(x = "Column", schema = "characterOrstructType"), +setMethod("from_json", signature(x = "Column", schema = "characterOrstructTypeOrColumn"), function(x, schema, as.json.array = FALSE, ...) { if (is.character(schema)) { - schema <- structType(schema) + jschema <- structType(schema)$jobj + } else if (class(schema) == "structType") { + jschema <- schema$jobj + } else { + jschema <- schema@jc } if (as.json.array) { - jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", - "createArrayType", - schema$jobj) - } else { - jschema <- schema$jobj + # This case is R-specifically different. Unlike Scala and Python side, + # R side has 'as.json.array' option to indicate if the schema should be + # treated as struct or element type of array in order to make it more + # R-friendly. + if (class(schema) == "Column") { + jschema <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createArrayType", + jschema) + } else { + jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", + "createArrayType", + jschema) + } } options <- varargsToStrEnv(...) jc <- callJStatic("org.apache.spark.sql.functions", @@ -2328,22 +2343,27 @@ setMethod("schema_of_json", signature(x = "characterOrColumn"), #' If the string is unparseable, the Column will contain the value NA. #' #' @rdname column_collection_functions -#' @aliases from_csv from_csv,Column,character-method +#' @aliases from_csv from_csv,Column,characterOrstructTypeOrColumn-method #' @examples #' #' \dontrun{ -#' df <- sql("SELECT 'Amsterdam,2018' as csv") +#' csv <- "Amsterdam,2018" +#' df <- sql(paste0("SELECT '", csv, "' as csv")) #' schema <- "city STRING, year INT" -#' head(select(df, from_csv(df$csv, schema)))} +#' head(select(df, from_csv(df$csv, schema))) +#' head(select(df, from_csv(df$csv, structType(schema)))) +#' head(select(df, from_csv(df$csv, schema_of_csv(csv))))} #' @note from_csv since 3.0.0 -setMethod("from_csv", signature(x = "Column", schema = "characterOrColumn"), +setMethod("from_csv", signature(x = "Column", schema = "characterOrstructTypeOrColumn"), function(x, schema, ...) { - if (class(schema) == "Column") { - jschema <- schema@jc - } else if (is.character(schema)) { + if (class(schema) == "structType") { + schema <- callJMethod(schema$jobj, "toDDL") + } + + if (is.character(schema)) { jschema <- callJStatic("org.apache.spark.sql.functions", "lit", schema) } else { - stop("schema argument should be a column or character") + jschema <- schema@jc } options <- varargsToStrEnv(...) jc <- callJStatic("org.apache.spark.sql.functions", diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 0d5118c127f2..a1805f57b1dc 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1626,6 +1626,12 @@ test_that("column functions", { expect_equal(c[[1]][[1]]$a, 1) c <- collect(select(df, alias(from_csv(df$col, lit("a INT")), "csv"))) expect_equal(c[[1]][[1]]$a, 1) + c <- collect(select(df, alias(from_csv(df$col, structType("a INT")), "csv"))) + expect_equal(c[[1]][[1]]$a, 1) + c <- collect(select(df, alias(from_csv(df$col, schema_of_csv("1")), "csv"))) + expect_equal(c[[1]][[1]]$`_c0`, 1) + c <- collect(select(df, alias(from_csv(df$col, schema_of_csv(lit("1"))), "csv"))) + expect_equal(c[[1]][[1]]$`_c0`, 1) df <- as.DataFrame(list(list("col" = "1"))) c <- collect(select(df, schema_of_csv("Amsterdam,2018"))) @@ -1651,7 +1657,9 @@ test_that("column functions", { expect_equal(j[order(j$json), ][1], "{\"age\":16,\"height\":176.5}") df <- as.DataFrame(j) schemas <- list(structType(structField("age", "integer"), structField("height", "double")), - "age INT, height DOUBLE") + "age INT, height DOUBLE", + schema_of_json("{\"age\":16,\"height\":176.5}"), + schema_of_json(lit("{\"age\":16,\"height\":176.5}"))) for (schema in schemas) { s <- collect(select(df, alias(from_json(df$json, schema), "structcol"))) expect_equal(ncol(s), 1) @@ -1691,7 +1699,11 @@ test_that("column functions", { # check if array type in string is correctly supported. jsonArr <- "[{\"name\":\"Bob\"}, {\"name\":\"Alice\"}]" df <- as.DataFrame(list(list("people" = jsonArr))) - for (schema in list(structType(structField("name", "string")), "name STRING")) { + schemas <- list(structType(structField("name", "string")), + "name STRING", + schema_of_json("{\"name\":\"Alice\"}"), + schema_of_json(lit("{\"name\":\"Bob\"}"))) + for (schema in schemas) { arr <- collect(select(df, alias(from_json(df$people, schema, as.json.array = TRUE), "arrcol"))) expect_equal(ncol(arr), 1) expect_equal(nrow(arr), 1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index e98cab8b56d1..f5d8d4ea0a4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -30,7 +30,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericRowWithSchema} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.command.ShowTablesCommand import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION @@ -233,4 +233,8 @@ private[sql] object SQLUtils extends Logging { } sparkSession.sessionState.catalog.listTables(db).map(_.table).toArray } + + def createArrayType(column: Column): ArrayType = { + new ArrayType(ExprUtils.evalTypeExpr(column.expr), true) + } } From d371180c01bf68ed4e5f88df836c7f2fb27a46d3 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 2 Jan 2019 08:04:36 +0800 Subject: [PATCH 0213/1072] [MINOR][R] Deduplicate RStudio setup documentation ## What changes were proposed in this pull request? This PR targets to deduplicate RStudio setup for SparkR. ## How was this patch tested? N/A Closes #23421 from HyukjinKwon/minor-doc. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- R/README.md | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/R/README.md b/R/README.md index d77a1ecffc99..e238a0efe4b5 100644 --- a/R/README.md +++ b/R/README.md @@ -39,15 +39,7 @@ To set other options like driver memory, executor memory etc. you can pass in th #### Using SparkR from RStudio -If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example -```R -# Set this to where Spark is installed -Sys.setenv(SPARK_HOME="/Users/username/spark") -# This line loads SparkR from the installed directory -.libPaths(c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"), .libPaths())) -library(SparkR) -sparkR.session() -``` +If you wish to use SparkR from RStudio, please refer [SparkR documentation](https://spark.apache.org/docs/latest/sparkr.html#starting-up-from-rstudio). #### Making changes to SparkR From 79b05481a2cff7a0aa34146c72068cc6e41e2241 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 1 Jan 2019 22:37:28 -0600 Subject: [PATCH 0214/1072] [SPARK-26508][CORE][SQL] Address warning messages in Java reported at lgtm.com ## What changes were proposed in this pull request? This PR addresses warning messages in Java files reported at [lgtm.com](https://lgtm.com). [lgtm.com](https://lgtm.com) provides automated code review of Java/Python/JavaScript files for OSS projects. [Here](https://lgtm.com/projects/g/apache/spark/alerts/?mode=list&severity=warning) are warning messages regarding Apache Spark project. This PR addresses the following warnings: - Result of multiplication cast to wider type - Implicit narrowing conversion in compound assignment - Boxed variable is never null - Useless null check NOTE: `Potential input resource leak` looks false positive for now. ## How was this patch tested? Existing UTs Closes #23420 from kiszk/SPARK-26508. Authored-by: Kazuaki Ishizaki Signed-off-by: Sean Owen --- .../java/org/apache/spark/network/util/ByteUnit.java | 12 ++++++------ .../org/apache/spark/network/util/TransportConf.java | 6 +++--- .../org/apache/spark/unsafe/map/BytesToBytesMap.java | 4 ++-- .../main/java/org/apache/spark/examples/JavaTC.java | 2 +- .../org/apache/spark/examples/ml/JavaALSExample.java | 2 +- .../examples/mllib/JavaCorrelationsExample.java | 2 +- .../mllib/JavaRandomForestClassificationExample.java | 8 ++++---- .../org/apache/spark/launcher/LauncherServer.java | 7 +++---- .../expressions/codegen/UnsafeArrayWriter.java | 2 +- .../expressions/codegen/UnsafeRowWriter.java | 2 +- .../sql/catalyst/expressions/xml/UDFXPathUtil.java | 2 +- 11 files changed, 24 insertions(+), 25 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java index 984575acaf51..6f7925c26094 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java @@ -18,11 +18,11 @@ public enum ByteUnit { BYTE(1), - KiB(1024L), - MiB((long) Math.pow(1024L, 2L)), - GiB((long) Math.pow(1024L, 3L)), - TiB((long) Math.pow(1024L, 4L)), - PiB((long) Math.pow(1024L, 5L)); + KiB(1L << 10), + MiB(1L << 20), + GiB(1L << 30), + TiB(1L << 40), + PiB(1L << 50); ByteUnit(long multiplier) { this.multiplier = multiplier; @@ -50,7 +50,7 @@ public long convertTo(long d, ByteUnit u) { } } - public double toBytes(long d) { + public long toBytes(long d) { if (d < 0) { throw new IllegalArgumentException("Negative size value. Size must be positive: " + d); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 43a6bc7dc3d0..201628b04fbe 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -309,8 +309,8 @@ public int chunkFetchHandlerThreads() { } int chunkFetchHandlerThreadsPercent = conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 100); - return (int)Math.ceil( - (this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors()) * - chunkFetchHandlerThreadsPercent/(double)100); + int threads = + this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors(); + return (int) Math.ceil(threads * (chunkFetchHandlerThreadsPercent / 100.0)); } } diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 7df8aafb2b67..2ff98a69ee1f 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -712,7 +712,7 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff final long recordOffset = offset; UnsafeAlignedOffset.putSize(base, offset, klen + vlen + uaoSize); UnsafeAlignedOffset.putSize(base, offset + uaoSize, klen); - offset += (2 * uaoSize); + offset += (2L * uaoSize); Platform.copyMemory(kbase, koff, base, offset, klen); offset += klen; Platform.copyMemory(vbase, voff, base, offset, vlen); @@ -780,7 +780,7 @@ private void allocate(int capacity) { assert (capacity >= 0); capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64); assert (capacity <= MAX_CAPACITY); - longArray = allocateArray(capacity * 2); + longArray = allocateArray(capacity * 2L); longArray.zeroOut(); this.growthThreshold = (int) (capacity * loadFactor); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index c9ca9c9b3a41..7e8df69e7e8d 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -71,7 +71,7 @@ public static void main(String[] args) { JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); - Integer slices = (args.length > 0) ? Integer.parseInt(args[0]): 2; + int slices = (args.length > 0) ? Integer.parseInt(args[0]): 2; JavaPairRDD tc = jsc.parallelizePairs(generateGraph(), slices).cache(); // Linear transitive closure: each round grows paths by one edge, diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java index 27052be87b82..b8d2c9f6a658 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -111,7 +111,7 @@ public static void main(String[] args) { .setMetricName("rmse") .setLabelCol("rating") .setPredictionCol("prediction"); - Double rmse = evaluator.evaluate(predictions); + double rmse = evaluator.evaluate(predictions); System.out.println("Root-mean-square error = " + rmse); // Generate top 10 movie recommendations for each user diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java index c0fa0b3cac1e..9bd858b59890 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java @@ -46,7 +46,7 @@ public static void main(String[] args) { // compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. // If a method is not specified, Pearson's method will be used by default. - Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); + double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); System.out.println("Correlation is: " + correlation); // note that each Vector is a row and not a column diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java index 6998ce2156c2..0707db8d3e83 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java @@ -48,14 +48,14 @@ public static void main(String[] args) { // Train a RandomForest model. // Empty categoricalFeaturesInfo indicates all features are continuous. - Integer numClasses = 2; + int numClasses = 2; Map categoricalFeaturesInfo = new HashMap<>(); Integer numTrees = 3; // Use more in practice. String featureSubsetStrategy = "auto"; // Let the algorithm choose. String impurity = "gini"; - Integer maxDepth = 5; - Integer maxBins = 32; - Integer seed = 12345; + int maxDepth = 5; + int maxBins = 32; + int seed = 12345; RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java index 607879fd02ea..3ff77878f68a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -318,9 +318,9 @@ protected void handle(Message msg) throws IOException { throw new IllegalArgumentException("Received Hello for unknown client."); } } else { + String msgClassName = msg != null ? msg.getClass().getName() : "no message"; if (handle == null) { - throw new IllegalArgumentException("Expected hello, got: " + - msg != null ? msg.getClass().getName() : null); + throw new IllegalArgumentException("Expected hello, got: " + msgClassName); } if (msg instanceof SetAppId) { SetAppId set = (SetAppId) msg; @@ -328,8 +328,7 @@ protected void handle(Message msg) throws IOException { } else if (msg instanceof SetState) { handle.setState(((SetState)msg).state); } else { - throw new IllegalArgumentException("Invalid message: " + - msg != null ? msg.getClass().getName() : null); + throw new IllegalArgumentException("Invalid message: " + msgClassName); } } } catch (Exception e) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index a78dd970d23e..997eecd839d8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -74,7 +74,7 @@ public void initialize(int numElements) { } private long getElementOffset(int ordinal) { - return startingOffset + headerInBytes + ordinal * elementSize; + return startingOffset + headerInBytes + ordinal * (long) elementSize; } private void setNullBit(int ordinal) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java index 3960d6d52047..d2298aa26364 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java @@ -132,7 +132,7 @@ public void setNull8Bytes(int ordinal) { } public long getFieldOffset(int ordinal) { - return startingOffset + nullBitsSize + 8 * ordinal; + return startingOffset + nullBitsSize + 8L * ordinal; } public void write(int ordinal, boolean value) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java index 023ec139652c..e9f18229b54c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java @@ -180,7 +180,7 @@ public long skip(long ns) throws IOException { return 0; } // Bound skip by beginning and end of the source - long n = Math.min(length - next, ns); + int n = (int) Math.min(length - next, ns); n = Math.max(-next, n); next += n; return n; From 4bdfda92a1c570d7a1142ee30eb41e37661bc240 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 2 Jan 2019 11:23:53 -0600 Subject: [PATCH 0215/1072] [SPARK-26507][CORE] Fix core tests for Java 11 ## What changes were proposed in this pull request? This should make tests in core modules pass for Java 11. ## How was this patch tested? Existing tests, with modifications. Closes #23419 from srowen/Java11. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/util/Utils.scala | 10 +++++++--- .../metrics/source/AccumulatorSourceSuite.scala | 9 ++++----- .../apache/spark/util/JsonProtocolSuite.scala | 16 +++++++++++----- .../scala/org/apache/spark/util/UtilsSuite.scala | 16 ---------------- .../launcher/SparkSubmitCommandBuilderSuite.java | 2 +- pom.xml | 2 ++ 6 files changed, 25 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index f322e92c6c8c..22f074cf9897 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2740,13 +2740,17 @@ private[spark] object Utils extends Logging { /** * Safer than Class obj's getSimpleName which may throw Malformed class name error in scala. - * This method mimicks scalatest's getSimpleNameOfAnObjectsClass. + * This method mimics scalatest's getSimpleNameOfAnObjectsClass. */ def getSimpleName(cls: Class[_]): String = { try { - return cls.getSimpleName + cls.getSimpleName } catch { - case err: InternalError => return stripDollars(stripPackages(cls.getName)) + // TODO: the value returned here isn't even quite right; it returns simple names + // like UtilsSuite$MalformedClassObject$MalformedClass instead of MalformedClass + // The exact value may not matter much as it's used in log statements + case _: InternalError => + stripDollars(stripPackages(cls.getName)) } } diff --git a/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala b/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala index 6a6c07cb068c..45e6e0b4913e 100644 --- a/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala @@ -17,9 +17,8 @@ package org.apache.spark.metrics.source -import com.codahale.metrics.MetricRegistry import org.mockito.ArgumentCaptor -import org.mockito.Mockito.{mock, never, spy, times, verify, when} +import org.mockito.Mockito.{mock, times, verify, when} import org.apache.spark.{SparkContext, SparkEnv, SparkFunSuite} import org.apache.spark.metrics.MetricsSystem @@ -37,7 +36,7 @@ class AccumulatorSourceSuite extends SparkFunSuite { val accs = Map("my-accumulator-1" -> acc1, "my-accumulator-2" -> acc2) LongAccumulatorSource.register(mockContext, accs) - val captor = new ArgumentCaptor[AccumulatorSource]() + val captor = ArgumentCaptor.forClass(classOf[AccumulatorSource]) verify(mockMetricSystem, times(1)).registerSource(captor.capture()) val source = captor.getValue() val gauges = source.metricRegistry.getGauges() @@ -59,7 +58,7 @@ class AccumulatorSourceSuite extends SparkFunSuite { val accs = Map("my-accumulator-1" -> acc1, "my-accumulator-2" -> acc2) LongAccumulatorSource.register(mockContext, accs) - val captor = new ArgumentCaptor[AccumulatorSource]() + val captor = ArgumentCaptor.forClass(classOf[AccumulatorSource]) verify(mockMetricSystem, times(1)).registerSource(captor.capture()) val source = captor.getValue() val gauges = source.metricRegistry.getGauges() @@ -81,7 +80,7 @@ class AccumulatorSourceSuite extends SparkFunSuite { "my-accumulator-1" -> acc1, "my-accumulator-2" -> acc2) DoubleAccumulatorSource.register(mockContext, accs) - val captor = new ArgumentCaptor[AccumulatorSource]() + val captor = ArgumentCaptor.forClass(classOf[AccumulatorSource]) verify(mockMetricSystem, times(1)).registerSource(captor.capture()) val source = captor.getValue() val gauges = source.metricRegistry.getGauges() 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 303ca7cb8801..b88f25726fc4 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -761,13 +761,13 @@ private[spark] object JsonProtocolSuite extends Assertions { } private def assertJsonStringEquals(expected: String, actual: String, metadata: String) { - val expectedJson = pretty(parse(expected)) - val actualJson = pretty(parse(actual)) + val expectedJson = parse(expected) + val actualJson = parse(actual) if (expectedJson != actualJson) { // scalastyle:off // This prints something useful if the JSON strings don't match - println("=== EXPECTED ===\n" + expectedJson + "\n") - println("=== ACTUAL ===\n" + actualJson + "\n") + println(s"=== EXPECTED ===\n${pretty(expectedJson)}\n") + println(s"=== ACTUAL ===\n${pretty(actualJson)}\n") // scalastyle:on throw new TestFailedException(s"$metadata JSON did not equal", 1) } @@ -807,7 +807,13 @@ private[spark] object JsonProtocolSuite extends Assertions { } private def assertStackTraceElementEquals(ste1: StackTraceElement, ste2: StackTraceElement) { - assert(ste1 === ste2) + // This mimics the equals() method from Java 8 and earlier. Java 9 adds checks for + // class loader and module, which will cause them to be not equal, when we don't + // care about those + assert(ste1.getClassName === ste2.getClassName) + assert(ste1.getMethodName === ste2.getMethodName) + assert(ste1.getLineNumber === ste2.getLineNumber) + assert(ste1.getFileName === ste2.getFileName) } /** ----------------------------------- * 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 b2ff1cce3eb0..d3f94fbe05d7 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1156,22 +1156,6 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { } } - object MalformedClassObject { - class MalformedClass - } - - test("Safe getSimpleName") { - // getSimpleName on class of MalformedClass will result in error: Malformed class name - // Utils.getSimpleName works - val err = intercept[java.lang.InternalError] { - classOf[MalformedClassObject.MalformedClass].getSimpleName - } - assert(err.getMessage === "Malformed class name") - - assert(Utils.getSimpleName(classOf[MalformedClassObject.MalformedClass]) === - "UtilsSuite$MalformedClassObject$MalformedClass") - } - test("stringHalfWidth") { // scalastyle:off nonascii assert(Utils.stringHalfWidth(null) == 0) diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index b343094b2e7b..e694e9066f12 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -158,7 +158,7 @@ public void testPySparkLauncher() throws Exception { Map env = new HashMap<>(); List cmd = buildCommand(sparkSubmitArgs, env); - assertEquals("python", cmd.get(cmd.size() - 1)); + assertTrue(Arrays.asList("python", "python2", "python3").contains(cmd.get(cmd.size() - 1))); assertEquals( String.format("\"%s\" \"foo\" \"%s\" \"bar\" \"%s\"", parser.MASTER, parser.DEPLOY_MODE, SparkSubmitCommandBuilder.PYSPARK_SHELL_RESOURCE), diff --git a/pom.xml b/pom.xml index 321de209a56a..a433659cd200 100644 --- a/pom.xml +++ b/pom.xml @@ -2060,6 +2060,8 @@ ${scala.version} + true + true incremental true From d40654861b1a051d4cfc4ec0d8e80f817e6b8e8b Mon Sep 17 00:00:00 2001 From: seancxmao Date: Wed, 2 Jan 2019 15:45:14 -0600 Subject: [PATCH 0216/1072] [SPARK-26277][SQL][TEST] WholeStageCodegen metrics should be tested with whole-stage codegen enabled ## What changes were proposed in this pull request? In `org.apache.spark.sql.execution.metric.SQLMetricsSuite`, there's a test case named "WholeStageCodegen metrics". However, it is executed with whole-stage codegen disabled. This PR fixes this by enable whole-stage codegen for this test case. ## How was this patch tested? Tested locally using exiting test cases. Closes #23224 from seancxmao/codegen-metrics. Authored-by: seancxmao Signed-off-by: Sean Owen --- .../spark/sql/execution/metric/SQLMetricsSuite.scala | 11 ++++++++--- .../sql/execution/metric/SQLMetricsTestUtils.scala | 7 +++++-- 2 files changed, 13 insertions(+), 5 deletions(-) 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 7368a6c9e1d6..6174ec4c8908 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 @@ -77,11 +77,16 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } test("WholeStageCodegen metrics") { - // Assume the execution plan is - // WholeStageCodegen(nodeId = 0, Range(nodeId = 2) -> Filter(nodeId = 1)) + // Assume the execution plan with node id is + // WholeStageCodegen(nodeId = 0) + // Filter(nodeId = 1) + // Range(nodeId = 2) // TODO: update metrics in generated operators val ds = spark.range(10).filter('id < 5) - testSparkPlanMetrics(ds.toDF(), 1, Map.empty) + testSparkPlanMetricsWithPredicates(ds.toDF(), 1, Map( + 0L -> (("WholeStageCodegen", Map( + "duration total (min, med, max)" -> {_.toString.matches(timingMetricPattern)}))) + ), true) } test("Aggregate metrics") { 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 2d245d2ba1e3..0e13f7dd55ba 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 @@ -144,6 +144,7 @@ trait SQLMetricsTestUtils extends SQLTestUtils { * @param df `DataFrame` to run * @param expectedNumOfJobs number of jobs that will run * @param expectedNodeIds the node ids of the metrics to collect from execution data. + * @param enableWholeStage enable whole-stage code generation or not. */ protected def getSparkPlanMetrics( df: DataFrame, @@ -210,13 +211,15 @@ trait SQLMetricsTestUtils extends SQLTestUtils { * @param expectedNumOfJobs number of jobs that will run * @param expectedMetricsPredicates the expected metrics predicates. The format is * `nodeId -> (operatorName, metric name -> metric predicate)`. + * @param enableWholeStage enable whole-stage code generation or not. */ protected def testSparkPlanMetricsWithPredicates( df: DataFrame, expectedNumOfJobs: Int, - expectedMetricsPredicates: Map[Long, (String, Map[String, Any => Boolean])]): Unit = { + expectedMetricsPredicates: Map[Long, (String, Map[String, Any => Boolean])], + enableWholeStage: Boolean = false): Unit = { val optActualMetrics = - getSparkPlanMetrics(df, expectedNumOfJobs, expectedMetricsPredicates.keySet) + getSparkPlanMetrics(df, expectedNumOfJobs, expectedMetricsPredicates.keySet, enableWholeStage) optActualMetrics.foreach { actualMetrics => assert(expectedMetricsPredicates.keySet === actualMetrics.keySet) for ((nodeId, (expectedNodeName, expectedMetricsPredicatesMap)) From 8be4d24a27a1e9995a53d4efb3a13a47813d1f77 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 2 Jan 2019 16:57:10 -0800 Subject: [PATCH 0217/1072] [SPARK-26023][SQL][FOLLOWUP] Dumping truncated plans and generated code to a file ## What changes were proposed in this pull request? `DataSourceScanExec` overrides "wrong" `treeString` method without `append`. In the PR, I propose to make `treeString`s **final** to prevent such mistakes in the future. And removed the `treeString` and `verboseString` since they both use `simpleString` with reduction. ## How was this patch tested? It was tested by `DataSourceScanExecRedactionSuite` Closes #23431 from MaxGekk/datasource-scan-exec-followup. Authored-by: Maxim Gekk Signed-off-by: gatorsmile --- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 4 ++-- .../apache/spark/sql/execution/DataSourceScanExec.scala | 9 ++------- 2 files changed, 4 insertions(+), 9 deletions(-) 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 570a019b2af7..d214ebb30903 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 @@ -474,9 +474,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { override def toString: String = treeString /** Returns a string representation of the nodes in this tree */ - def treeString: String = treeString(verbose = true) + final def treeString: String = treeString(verbose = true) - def treeString( + final def treeString( verbose: Boolean, addSuffix: Boolean = false, maxFields: Int = SQLConf.get.maxToStringFields): String = { 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 1d7dd73706c4..8b84eda36103 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 @@ -58,13 +58,8 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport { key + ": " + StringUtils.abbreviate(redact(value), 100) } val metadataStr = truncatedString(metadataEntries, " ", ", ", "", maxFields) - s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]", maxFields)}$metadataStr" - } - - override def verboseString(maxFields: Int): String = redact(super.verboseString(maxFields)) - - override def treeString(verbose: Boolean, addSuffix: Boolean, maxFields: Int): String = { - redact(super.treeString(verbose, addSuffix, maxFields)) + redact( + s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]", maxFields)}$metadataStr") } /** From 56967b7e288ac54e705b14a21516df5402d4c9d9 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 3 Jan 2019 11:01:54 +0800 Subject: [PATCH 0218/1072] [SPARK-26403][SQL] Support pivoting using array column for `pivot(column)` API ## What changes were proposed in this pull request? This PR fixes `pivot(Column)` can accepts `collection.mutable.WrappedArray`. Note that we return `collection.mutable.WrappedArray` from `ArrayType`, and `Literal.apply` doesn't support this. We can unwrap the array and use it for type dispatch. ```scala val df = Seq( (2, Seq.empty[String]), (2, Seq("a", "x")), (3, Seq.empty[String]), (3, Seq("a", "x"))).toDF("x", "s") df.groupBy("x").pivot("s").count().show() ``` Before: ``` Unsupported literal type class scala.collection.mutable.WrappedArray$ofRef WrappedArray() java.lang.RuntimeException: Unsupported literal type class scala.collection.mutable.WrappedArray$ofRef WrappedArray() at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:80) at org.apache.spark.sql.RelationalGroupedDataset.$anonfun$pivot$2(RelationalGroupedDataset.scala:427) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237) at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36) at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33) at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:39) at scala.collection.TraversableLike.map(TraversableLike.scala:237) at scala.collection.TraversableLike.map$(TraversableLike.scala:230) at scala.collection.AbstractTraversable.map(Traversable.scala:108) at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:425) at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:406) at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:317) at org.apache.spark.sql.DataFramePivotSuite.$anonfun$new$1(DataFramePivotSuite.scala:341) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) ``` After: ``` +---+---+------+ | x| []|[a, x]| +---+---+------+ | 3| 1| 1| | 2| 1| 1| +---+---+------+ ``` ## How was this patch tested? Manually tested and unittests were added. Closes #23349 from HyukjinKwon/SPARK-26403. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../spark/sql/catalyst/expressions/literals.scala | 1 + .../catalyst/expressions/LiteralExpressionSuite.scala | 2 ++ .../org/apache/spark/sql/DataFramePivotSuite.scala | 11 +++++++++++ 3 files changed, 14 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 34d252886ffb..48beffa18a55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -67,6 +67,7 @@ object Literal { case t: Timestamp => Literal(DateTimeUtils.fromJavaTimestamp(t), TimestampType) case d: Date => Literal(DateTimeUtils.fromJavaDate(d), DateType) case a: Array[Byte] => Literal(a, BinaryType) + case a: collection.mutable.WrappedArray[_] => apply(a.array) case a: Array[_] => val elementType = componentTypeToDataType(a.getClass.getComponentType()) val dataType = ArrayType(elementType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index 3ea6bfac9ddc..133aaa449ea4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -179,6 +179,8 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkArrayLiteral(Array("a", "b", "c")) checkArrayLiteral(Array(1.0, 4.0)) checkArrayLiteral(Array(CalendarInterval.MICROS_PER_DAY, CalendarInterval.MICROS_PER_HOUR)) + val arr = collection.mutable.WrappedArray.make(Array(1.0, 4.0)) + checkEvaluation(Literal(arr), toCatalyst(arr)) } test("seq") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index b52ca58c07d2..8c2c11be9b6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -333,4 +333,15 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext { } assert(exception.getMessage.contains("Unsupported literal type")) } + + test("SPARK-26403: pivoting by array column") { + val df = Seq( + (2, Seq.empty[String]), + (2, Seq("a", "x")), + (3, Seq.empty[String]), + (3, Seq("a", "x"))).toDF("x", "s") + val expected = Seq((3, 1, 1), (2, 1, 1)).toDF + val actual = df.groupBy("x").pivot("s").count() + checkAnswer(actual, expected) + } } From 2a30deb85ae4e42c5cbc936383dd5c3970f4a74f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 3 Jan 2019 11:27:40 +0100 Subject: [PATCH 0219/1072] [SPARK-26502][SQL] Move hiveResultString() from QueryExecution to HiveResult ## What changes were proposed in this pull request? In the PR, I propose to move `hiveResultString()` out of `QueryExecution` and put it to a separate object. Closes #23409 from MaxGekk/hive-result-string. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Herman van Hovell --- .../spark/sql/execution/HiveResult.scala | 116 ++++++++++++++++++ .../spark/sql/execution/QueryExecution.scala | 91 +------------- .../apache/spark/sql/SQLQueryTestSuite.scala | 5 +- .../hive/thriftserver/SparkSQLDriver.scala | 3 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 4 +- 6 files changed, 126 insertions(+), 95 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala 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 new file mode 100644 index 000000000000..22d3ca958a21 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * Runs a query returning the result in Hive compatible form. + */ +object HiveResult { + /** + * Returns the result as a hive compatible sequence of strings. This is used in tests and + * `SparkSQLDriver` for CLI applications. + */ + def hiveResultString(executedPlan: SparkPlan): Seq[String] = executedPlan match { + case ExecutedCommandExec(desc: DescribeTableCommand) => + // If it is a describe command for a Hive table, we want to have the output format + // be similar with Hive. + executedPlan.executeCollectPublic().map { + case Row(name: String, dataType: String, comment) => + Seq(name, dataType, + Option(comment.asInstanceOf[String]).getOrElse("")) + .map(s => String.format(s"%-20s", s)) + .mkString("\t") + } + // SHOW TABLES in Hive only output table names, while ours output database, table name, isTemp. + case command @ ExecutedCommandExec(s: ShowTablesCommand) if !s.isExtended => + command.executeCollect().map(_.getString(1)) + case other => + 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(toHiveString)).map(_.mkString("\t")) + } + + /** Formats a datum (based on the given data type) and returns the string representation. */ + private def toHiveString(a: (Any, DataType)): String = { + val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, + BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) + val timeZone = DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone) + + def formatDecimal(d: java.math.BigDecimal): String = { + if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { + java.math.BigDecimal.ZERO.toPlainString + } else { + d.stripTrailingZeros().toPlainString + } + } + + /** Hive outputs fields of structs slightly differently than top level attributes. */ + def toHiveStructString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "null" + case (s: String, StringType) => "\"" + s + "\"" + case (decimal, DecimalType()) => decimal.toString + case (interval, CalendarIntervalType) => interval.toString + case (other, tpe) if primitiveTypes contains tpe => other.toString + } + + a match { + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "NULL" + case (d: Date, DateType) => + DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d)) + case (t: Timestamp, TimestampType) => + DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t), timeZone) + case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) + case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) + case (interval, CalendarIntervalType) => interval.toString + case (other, tpe) if primitiveTypes.contains(tpe) => other.toString + } + } +} 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 7fccbf65d852..72499aa936a5 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 @@ -18,25 +18,20 @@ package org.apache.spark.sql.execution import java.io.{BufferedWriter, OutputStreamWriter} -import java.nio.charset.StandardCharsets -import java.sql.{Date, Timestamp} import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, TimestampType, _} import org.apache.spark.util.Utils /** @@ -109,90 +104,6 @@ class QueryExecution( ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf)) - /** - * Returns the result as a hive compatible sequence of strings. This is used in tests and - * `SparkSQLDriver` for CLI applications. - */ - def hiveResultString(): Seq[String] = executedPlan match { - case ExecutedCommandExec(desc: DescribeTableCommand) => - // If it is a describe command for a Hive table, we want to have the output format - // be similar with Hive. - desc.run(sparkSession).map { - case Row(name: String, dataType: String, comment) => - Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse("")) - .map(s => String.format(s"%-20s", s)) - .mkString("\t") - } - // SHOW TABLES in Hive only output table names, while ours output database, table name, isTemp. - case command @ ExecutedCommandExec(s: ShowTablesCommand) if !s.isExtended => - command.executeCollect().map(_.getString(1)) - case other => - val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq - // We need the types so we can output struct field names - val types = analyzed.output.map(_.dataType) - // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) - } - - /** Formats a datum (based on the given data type) and returns the string representation. */ - private def toHiveString(a: (Any, DataType)): String = { - val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, - BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) - - def formatDecimal(d: java.math.BigDecimal): String = { - if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { - java.math.BigDecimal.ZERO.toPlainString - } else { - d.stripTrailingZeros().toPlainString - } - } - - /** Hive outputs fields of structs slightly differently than top level attributes. */ - def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (decimal, DecimalType()) => decimal.toString - case (interval, CalendarIntervalType) => interval.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - - a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" - case (d: Date, DateType) => - DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d)) - case (t: Timestamp, TimestampType) => - DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t), - DateTimeUtils.getTimeZone(sparkSession.sessionState.conf.sessionLocalTimeZone)) - case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) - case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) - case (interval, CalendarIntervalType) => interval.toString - case (other, tpe) if primitiveTypes.contains(tpe) => other.toString - } - } - def simpleString: String = withRedaction { val concat = new StringConcat() concat.append("== Physical Plan ==\n") 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 b2515226d9a1..24b312348bd6 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 @@ -22,11 +22,11 @@ import java.util.{Locale, TimeZone} import scala.util.control.NonFatal -import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation 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} +import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeTableCommand} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -287,7 +287,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { val schema = df.schema val notIncludedMsg = "[not included in comparison]" // Get answer, but also get rid of the #1234 expression ids that show up in explain plans - val answer = df.queryExecution.hiveResultString().map(_.replaceAll("#\\d+", "#x") + val answer = hiveResultString(df.queryExecution.executedPlan) + .map(_.replaceAll("#\\d+", "#x") .replaceAll("Location.*/sql/core/", s"Location ${notIncludedMsg}sql/core/") .replaceAll("Created By.*", s"Created By $notIncludedMsg") .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") 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 677590217344..960fdd11db15 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 @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse 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 private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlContext) @@ -61,7 +62,7 @@ private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlCont context.sparkContext.setJobDescription(command) val execution = context.sessionState.executePlan(context.sql(command).logicalPlan) hiveResponse = SQLExecution.withNewExecutionId(context.sparkSession, execution) { - execution.hiveResultString() + hiveResultString(execution.executedPlan) } tableSchema = getResultSetSchema(execution) new CommandProcessorResponse(0) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 3508affda241..4c2bc62b9faf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -297,7 +297,7 @@ private[hive] class TestHiveSparkSession( protected[hive] implicit class SqlCmd(sql: String) { def cmd: () => Unit = { - () => new TestHiveQueryExecution(sql).hiveResultString(): Unit + () => new TestHiveQueryExecution(sql).executedPlan.executeCollect(): Unit } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 272e6f51f500..66426824573c 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 @@ -31,6 +31,7 @@ import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} @@ -345,7 +346,8 @@ abstract class HiveComparisonTest val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) def getResult(): Seq[String] = { - SQLExecution.withNewExecutionId(query.sparkSession, query)(query.hiveResultString()) + SQLExecution.withNewExecutionId( + query.sparkSession, query)(hiveResultString(query.executedPlan)) } try { (query, prepareAnswer(query, getResult())) } catch { case e: Throwable => From 88b074f3f06ddd236d63e8bf31edebe1d3e94fe4 Mon Sep 17 00:00:00 2001 From: Liupengcheng Date: Thu, 3 Jan 2019 10:26:14 -0600 Subject: [PATCH 0220/1072] [SPARK-26501][CORE][TEST] Fix unexpected overriden of exitFn in SparkSubmitSuite ## What changes were proposed in this pull request? The overriden of SparkSubmit's exitFn at some previous tests in SparkSubmitSuite may cause the following tests pass even they failed when they were run separately. This PR is to fix this problem. ## How was this patch tested? unittest Closes #23404 from liupc/Fix-SparkSubmitSuite-exitFn. Authored-by: Liupengcheng Signed-off-by: Sean Owen --- .../spark/deploy/SparkSubmitSuite.scala | 40 ++++++++++--------- 1 file changed, 22 insertions(+), 18 deletions(-) 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 a8973d1b60f8..2a7a55cbb903 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -72,27 +72,31 @@ trait TestPrematureExit { mainObject.printStream = printStream @volatile var exitedCleanly = false + val original = mainObject.exitFn mainObject.exitFn = (_) => exitedCleanly = true - - @volatile var exception: Exception = null - val thread = new Thread { - override def run() = try { - mainObject.main(input) - } catch { - // Capture the exception to check whether the exception contains searchString or not - case e: Exception => exception = e + try { + @volatile var exception: Exception = null + val thread = new Thread { + override def run() = try { + mainObject.main(input) + } catch { + // Capture the exception to check whether the exception contains searchString or not + case e: Exception => exception = e + } } - } - thread.start() - thread.join() - if (exitedCleanly) { - val joined = printStream.lineBuffer.mkString("\n") - assert(joined.contains(searchString)) - } else { - assert(exception != null) - if (!exception.getMessage.contains(searchString)) { - throw exception + thread.start() + thread.join() + if (exitedCleanly) { + val joined = printStream.lineBuffer.mkString("\n") + assert(joined.contains(searchString)) + } else { + assert(exception != null) + if (!exception.getMessage.contains(searchString)) { + throw exception + } } + } finally { + mainObject.exitFn = original } } } From 40711eef168716c44b873359e17822fe6b3387f4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 3 Jan 2019 10:30:47 -0600 Subject: [PATCH 0221/1072] [SPARK-26517][SQL][TEST] Avoid duplicate test in ParquetSchemaPruningSuite ## What changes were proposed in this pull request? `testExactCaseQueryPruning` and `testMixedCaseQueryPruning` don't need to set up `PARQUET_VECTORIZED_READER_ENABLED` config. Because `withMixedCaseData` will run against both Spark vectorized reader and Parquet-mr reader. ## How was this patch tested? Existing test. Closes #23427 from viirya/fix-parquet-schema-pruning-test. Authored-by: Liang-Chi Hsieh Signed-off-by: Sean Owen --- .../parquet/ParquetSchemaPruningSuite.scala | 23 ++++--------------- 1 file changed, 5 insertions(+), 18 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 434c4414edeb..9a02529a2550 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -312,15 +312,8 @@ class ParquetSchemaPruningSuite // schema's column and field names. N.B. this implies that `testThunk` should pass using either a // case-sensitive or case-insensitive query parser private def testExactCaseQueryPruning(testName: String)(testThunk: => Unit) { - test(s"Spark vectorized reader - case-sensitive parser - mixed-case schema - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", - SQLConf.CASE_SENSITIVE.key -> "true") { - withMixedCaseData(testThunk) - } - } - test(s"Parquet-mr reader - case-sensitive parser - mixed-case schema - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", - SQLConf.CASE_SENSITIVE.key -> "true") { + test(s"Case-sensitive parser - mixed-case schema - $testName") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { withMixedCaseData(testThunk) } } @@ -330,20 +323,14 @@ class ParquetSchemaPruningSuite // Tests schema pruning for a query whose column and field names may differ in case from the table // schema's column and field names private def testMixedCaseQueryPruning(testName: String)(testThunk: => Unit) { - test(s"Spark vectorized reader - case-insensitive parser - mixed-case schema - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", - SQLConf.CASE_SENSITIVE.key -> "false") { - withMixedCaseData(testThunk) - } - } - test(s"Parquet-mr reader - case-insensitive parser - mixed-case schema - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", - SQLConf.CASE_SENSITIVE.key -> "false") { + test(s"Case-insensitive parser - mixed-case schema - $testName") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { withMixedCaseData(testThunk) } } } + // Tests given test function with Spark vectorized reader and Parquet-mr reader. private def withMixedCaseData(testThunk: => Unit) { withParquetTable(mixedCaseData, "mixedcase") { testThunk From e2dbafdbc5e50fcf2554bf51939ce0cd363d8806 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 4 Jan 2019 00:37:03 +0800 Subject: [PATCH 0222/1072] [SPARK-26447][SQL] Allow OrcColumnarBatchReader to return less partition columns ## What changes were proposed in this pull request? Currently OrcColumnarBatchReader returns all the partition column values in the batch read. In data source V2, we can improve it by returning the required partition column values only. This PR is part of https://github.com/apache/spark/pull/23383 . As cloud-fan suggested, create a new PR to make review easier. Also, this PR doesn't improve `OrcFileFormat`, since in the method `buildReaderWithPartitionValues`, the `requiredSchema` filter out all the partition columns, so we can't know which partition column is required. ## How was this patch tested? Unit test Closes #23387 from gengliangwang/refactorOrcColumnarBatch. Lead-authored-by: Gengliang Wang Co-authored-by: Gengliang Wang Co-authored-by: Dongjoon Hyun Signed-off-by: Wenchen Fan --- .../orc/OrcColumnarBatchReader.java | 93 ++++++++++--------- .../datasources/orc/OrcFileFormat.scala | 10 +- .../orc/OrcColumnarBatchReaderSuite.scala | 80 ++++++++++++++++ 3 files changed, 136 insertions(+), 47 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index a0d9578a377b..7dc90df05a8f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.stream.IntStream; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.RecordReader; @@ -58,9 +59,14 @@ public class OrcColumnarBatchReader extends RecordReader { /** * The column IDs of the physical ORC file schema which are required by this reader. - * -1 means this required column doesn't exist in the ORC file. + * -1 means this required column is partition column, or it doesn't exist in the ORC file. + * Ideally partition column should never appear in the physical file, and should only appear + * in the directory name. However, Spark allows partition columns inside physical file, + * but Spark will discard the values from the file, and use the partition value got from + * directory name. The column order will be reserved though. */ - private int[] requestedColIds; + @VisibleForTesting + public int[] requestedDataColIds; // Record reader from ORC row batch. private org.apache.orc.RecordReader recordReader; @@ -68,7 +74,8 @@ public class OrcColumnarBatchReader extends RecordReader { private StructField[] requiredFields; // The result columnar batch for vectorized execution by whole-stage codegen. - private ColumnarBatch columnarBatch; + @VisibleForTesting + public ColumnarBatch columnarBatch; // Writable column vectors of the result columnar batch. private WritableColumnVector[] columnVectors; @@ -143,25 +150,33 @@ public void initialize( /** * Initialize columnar batch by setting required schema and partition information. * With this information, this creates ColumnarBatch with the full schema. + * + * @param orcSchema Schema from ORC file reader. + * @param requiredFields All the fields that are required to return, including partition fields. + * @param requestedDataColIds Requested column ids from orcSchema. -1 if not existed. + * @param requestedPartitionColIds Requested column ids from partition schema. -1 if not existed. + * @param partitionValues Values of partition columns. */ public void initBatch( TypeDescription orcSchema, - int[] requestedColIds, StructField[] requiredFields, - StructType partitionSchema, + int[] requestedDataColIds, + int[] requestedPartitionColIds, InternalRow partitionValues) { batch = orcSchema.createRowBatch(capacity); assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`. - + assert(requiredFields.length == requestedDataColIds.length); + assert(requiredFields.length == requestedPartitionColIds.length); + // If a required column is also partition column, use partition value and don't read from file. + for (int i = 0; i < requiredFields.length; i++) { + if (requestedPartitionColIds[i] != -1) { + requestedDataColIds[i] = -1; + } + } this.requiredFields = requiredFields; - this.requestedColIds = requestedColIds; - assert(requiredFields.length == requestedColIds.length); + this.requestedDataColIds = requestedDataColIds; StructType resultSchema = new StructType(requiredFields); - for (StructField f : partitionSchema.fields()) { - resultSchema = resultSchema.add(f); - } - if (copyToSpark) { if (MEMORY_MODE == MemoryMode.OFF_HEAP) { columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema); @@ -169,22 +184,18 @@ public void initBatch( columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema); } - // Initialize the missing columns once. + // Initialize the partition columns and missing columns once. for (int i = 0; i < requiredFields.length; i++) { - if (requestedColIds[i] == -1) { + if (requestedPartitionColIds[i] != -1) { + ColumnVectorUtils.populate(columnVectors[i], + partitionValues, requestedPartitionColIds[i]); + columnVectors[i].setIsConstant(); + } else if (requestedDataColIds[i] == -1) { columnVectors[i].putNulls(0, capacity); columnVectors[i].setIsConstant(); } } - if (partitionValues.numFields() > 0) { - int partitionIdx = requiredFields.length; - for (int i = 0; i < partitionValues.numFields(); i++) { - ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i); - columnVectors[i + partitionIdx].setIsConstant(); - } - } - columnarBatch = new ColumnarBatch(columnVectors); } else { // Just wrap the ORC column vector instead of copying it to Spark column vector. @@ -192,26 +203,22 @@ public void initBatch( for (int i = 0; i < requiredFields.length; i++) { DataType dt = requiredFields[i].dataType(); - int colId = requestedColIds[i]; - // Initialize the missing columns once. - if (colId == -1) { - OnHeapColumnVector missingCol = new OnHeapColumnVector(capacity, dt); - missingCol.putNulls(0, capacity); - missingCol.setIsConstant(); - orcVectorWrappers[i] = missingCol; - } else { - orcVectorWrappers[i] = new OrcColumnVector(dt, batch.cols[colId]); - } - } - - if (partitionValues.numFields() > 0) { - int partitionIdx = requiredFields.length; - for (int i = 0; i < partitionValues.numFields(); i++) { - DataType dt = partitionSchema.fields()[i].dataType(); + if (requestedPartitionColIds[i] != -1) { OnHeapColumnVector partitionCol = new OnHeapColumnVector(capacity, dt); - ColumnVectorUtils.populate(partitionCol, partitionValues, i); + ColumnVectorUtils.populate(partitionCol, partitionValues, requestedPartitionColIds[i]); partitionCol.setIsConstant(); - orcVectorWrappers[partitionIdx + i] = partitionCol; + orcVectorWrappers[i] = partitionCol; + } else { + int colId = requestedDataColIds[i]; + // Initialize the missing columns once. + if (colId == -1) { + OnHeapColumnVector missingCol = new OnHeapColumnVector(capacity, dt); + missingCol.putNulls(0, capacity); + missingCol.setIsConstant(); + orcVectorWrappers[i] = missingCol; + } else { + orcVectorWrappers[i] = new OrcColumnVector(dt, batch.cols[colId]); + } } } @@ -233,7 +240,7 @@ private boolean nextBatch() throws IOException { if (!copyToSpark) { for (int i = 0; i < requiredFields.length; i++) { - if (requestedColIds[i] != -1) { + if (requestedDataColIds[i] != -1) { ((OrcColumnVector) orcVectorWrappers[i]).setBatchSize(batchSize); } } @@ -248,8 +255,8 @@ private boolean nextBatch() throws IOException { StructField field = requiredFields[i]; WritableColumnVector toColumn = columnVectors[i]; - if (requestedColIds[i] >= 0) { - ColumnVector fromColumn = batch.cols[requestedColIds[i]]; + if (requestedDataColIds[i] >= 0) { + ColumnVector fromColumn = batch.cols[requestedDataColIds[i]]; if (fromColumn.isRepeating) { putRepeatingValues(batchSize, field, fromColumn, toColumn); 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 4574f8247af5..cd10ad21cd82 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 @@ -206,13 +206,15 @@ class OrcFileFormat // after opening a file. val iter = new RecordReaderIterator(batchReader) Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) - + val requestedDataColIds = requestedColIds ++ Array.fill(partitionSchema.length)(-1) + val requestedPartitionColIds = + Array.fill(requiredSchema.length)(-1) ++ Range(0, partitionSchema.length) batchReader.initialize(fileSplit, taskAttemptContext) batchReader.initBatch( reader.getSchema, - requestedColIds, - requiredSchema.fields, - partitionSchema, + resultSchema.fields, + requestedDataColIds, + requestedPartitionColIds, file.partitionValues) iter.asInstanceOf[Iterator[InternalRow]] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala new file mode 100644 index 000000000000..52abeb20e7f2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.orc.TypeDescription + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String.fromString + +class OrcColumnarBatchReaderSuite extends QueryTest with SQLTestUtils with SharedSQLContext { + private val dataSchema = StructType.fromDDL("col1 int, col2 int") + private val partitionSchema = StructType.fromDDL("p1 string, p2 string") + private val partitionValues = InternalRow(fromString("partValue1"), fromString("partValue2")) + private val orcFileSchemaList = Seq( + "struct", "struct", + "struct", "struct") + orcFileSchemaList.foreach { case schema => + val orcFileSchema = TypeDescription.fromString(schema) + + val isConstant = classOf[WritableColumnVector].getDeclaredField("isConstant") + isConstant.setAccessible(true) + + def getReader( + requestedDataColIds: Array[Int], + requestedPartitionColIds: Array[Int], + resultFields: Array[StructField]): OrcColumnarBatchReader = { + val reader = new OrcColumnarBatchReader(false, false, 4096) + reader.initBatch( + orcFileSchema, + resultFields, + requestedDataColIds, + requestedPartitionColIds, + partitionValues) + reader + } + + test(s"all partitions are requested: $schema") { + val requestedDataColIds = Array(0, 1, 0, 0) + val requestedPartitionColIds = Array(-1, -1, 0, 1) + val reader = getReader(requestedDataColIds, requestedPartitionColIds, + dataSchema.fields ++ partitionSchema.fields) + assert(reader.requestedDataColIds === Array(0, 1, -1, -1)) + } + + test(s"initBatch should initialize requested partition columns only: $schema") { + val requestedDataColIds = Array(0, -1) // only `col1` is requested, `col2` doesn't exist + val requestedPartitionColIds = Array(-1, 0) // only `p1` is requested + val reader = getReader(requestedDataColIds, requestedPartitionColIds, + Array(dataSchema.fields(0), partitionSchema.fields(0))) + val batch = reader.columnarBatch + assert(batch.numCols() === 2) + + assert(batch.column(0).isInstanceOf[OrcColumnVector]) + assert(batch.column(1).isInstanceOf[OnHeapColumnVector]) + + val p1 = batch.column(1).asInstanceOf[OnHeapColumnVector] + assert(isConstant.get(p1).asInstanceOf[Boolean]) // Partition column is constant. + assert(p1.getUTF8String(0) === partitionValues.getUTF8String(0)) + } + } +} From 05372d188aeaeff5e8de8866ec6e7b932bafa70f Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 3 Jan 2019 14:30:27 -0800 Subject: [PATCH 0223/1072] [SPARK-26489][CORE] Use ConfigEntry for hardcoded configs for python/r categories ## What changes were proposed in this pull request? The PR makes hardcoded configs below to use ConfigEntry. * spark.pyspark * spark.python * spark.r This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties, python source code) ## How was this patch tested? Existing tests. Closes #23428 from HeartSaVioR/SPARK-26489. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../spark/api/python/PythonRunner.scala | 6 +-- .../api/python/PythonWorkerFactory.scala | 15 +++--- .../org/apache/spark/api/r/RBackend.scala | 10 ++-- .../apache/spark/api/r/RBackendHandler.scala | 7 ++- .../org/apache/spark/api/r/RRunner.scala | 8 ++-- .../org/apache/spark/deploy/RRunner.scala | 9 ++-- .../apache/spark/internal/config/Python.scala | 47 +++++++++++++++++++ .../config/R.scala} | 26 ++++++---- .../spark/internal/config/package.scala | 4 -- .../features/BasicExecutorFeatureStep.scala | 1 + .../BasicExecutorFeatureStepSuite.scala | 1 + .../org/apache/spark/deploy/yarn/Client.scala | 1 + .../spark/deploy/yarn/YarnAllocator.scala | 1 + 13 files changed, 96 insertions(+), 40 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/internal/config/Python.scala rename core/src/main/scala/org/apache/spark/{api/r/SparkRDefaults.scala => internal/config/R.scala} (56%) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index f73e95eac8f7..6b748c825d29 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -27,7 +27,7 @@ import scala.collection.JavaConverters._ import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.internal.config.Python._ import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -71,7 +71,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( private val conf = SparkEnv.get.conf private val bufferSize = conf.getInt("spark.buffer.size", 65536) - private val reuseWorker = conf.getBoolean("spark.python.worker.reuse", true) + private val reuseWorker = conf.get(PYTHON_WORKER_REUSE) // each python worker gets an equal part of the allocation. the worker pool will grow to the // number of concurrent tasks, which is determined by the number of cores in this executor. private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY) @@ -496,7 +496,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( extends Thread(s"Worker Monitor for $pythonExec") { /** How long to wait before killing the python worker if a task cannot be interrupted. */ - private val taskKillTimeout = env.conf.getTimeAsMs("spark.python.task.killTimeout", "2s") + private val taskKillTimeout = env.conf.get(PYTHON_TASK_KILL_TIMEOUT) setDaemon(true) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 1f2f503a28d4..09e219fef5a1 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -28,6 +28,7 @@ import scala.collection.mutable import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Python._ import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util.{RedirectThread, Utils} @@ -41,7 +42,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // currently only works on UNIX-based systems now because it uses signals for child management, // so we can also fall back to launching workers, pyspark/worker.py (by default) directly. private val useDaemon = { - val useDaemonEnabled = SparkEnv.get.conf.getBoolean("spark.python.use.daemon", true) + val useDaemonEnabled = SparkEnv.get.conf.get(PYTHON_USE_DAEMON) // This flag is ignored on Windows as it's unable to fork. !System.getProperty("os.name").startsWith("Windows") && useDaemonEnabled @@ -53,21 +54,21 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // This configuration indicates the module to run the daemon to execute its Python workers. private val daemonModule = - SparkEnv.get.conf.getOption("spark.python.daemon.module").map { value => + SparkEnv.get.conf.get(PYTHON_DAEMON_MODULE).map { value => logInfo( - s"Python daemon module in PySpark is set to [$value] in 'spark.python.daemon.module', " + + s"Python daemon module in PySpark is set to [$value] in '${PYTHON_DAEMON_MODULE.key}', " + "using this to start the daemon up. Note that this configuration only has an effect when " + - "'spark.python.use.daemon' is enabled and the platform is not Windows.") + s"'${PYTHON_USE_DAEMON.key}' is enabled and the platform is not Windows.") value }.getOrElse("pyspark.daemon") // This configuration indicates the module to run each Python worker. private val workerModule = - SparkEnv.get.conf.getOption("spark.python.worker.module").map { value => + SparkEnv.get.conf.get(PYTHON_WORKER_MODULE).map { value => logInfo( - s"Python worker module in PySpark is set to [$value] in 'spark.python.worker.module', " + + s"Python worker module in PySpark is set to [$value] in '${PYTHON_WORKER_MODULE.key}', " + "using this to start the worker up. Note that this configuration only has an effect when " + - "'spark.python.use.daemon' is disabled or the platform is Windows.") + s"'${PYTHON_USE_DAEMON.key}' is disabled or the platform is Windows.") value }.getOrElse("pyspark.worker") diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala index 50c8fdf5316d..36b4132088b5 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala @@ -32,6 +32,7 @@ import io.netty.handler.timeout.ReadTimeoutHandler import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.R._ /** * Netty-based backend server that is used to communicate between R and Java. @@ -47,10 +48,8 @@ private[spark] class RBackend { def init(): (Int, RAuthHelper) = { val conf = new SparkConf() - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) - bossGroup = new NioEventLoopGroup( - conf.getInt("spark.r.numRBackendThreads", SparkRDefaults.DEFAULT_NUM_RBACKEND_THREADS)) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) + bossGroup = new NioEventLoopGroup(conf.get(R_NUM_BACKEND_THREADS)) val workerGroup = bossGroup val handler = new RBackendHandler(this) val authHelper = new RAuthHelper(conf) @@ -126,8 +125,7 @@ private[spark] object RBackend extends Logging { // Connection timeout is set by socket client. To make it configurable we will pass the // timeout value to client inside the temp file val conf = new SparkConf() - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) // tell the R process via temporary file val path = args(0) diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 18fc595301f4..7b74efa41044 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -29,6 +29,7 @@ import io.netty.handler.timeout.ReadTimeoutException import org.apache.spark.SparkConf import org.apache.spark.api.r.SerDe._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.R._ import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -98,10 +99,8 @@ private[r] class RBackendHandler(server: RBackend) } } val conf = new SparkConf() - val heartBeatInterval = conf.getInt( - "spark.r.heartBeatInterval", SparkRDefaults.DEFAULT_HEARTBEAT_INTERVAL) - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val heartBeatInterval = conf.get(R_HEARTBEAT_INTERVAL) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) val interval = Math.min(heartBeatInterval, backendConnectionTimeout - 1) execService.scheduleAtFixedRate(pingRunner, interval, interval, TimeUnit.SECONDS) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index e7fdc3963945..3fdea04cdf7a 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -27,6 +27,7 @@ import scala.util.Try import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.R._ import org.apache.spark.util.Utils /** @@ -340,11 +341,10 @@ private[r] object RRunner { // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", // but kept here for backward compatibility. val sparkConf = SparkEnv.get.conf - var rCommand = sparkConf.get("spark.sparkr.r.command", "Rscript") - rCommand = sparkConf.get("spark.r.command", rCommand) + var rCommand = sparkConf.get(SPARKR_COMMAND) + rCommand = sparkConf.get(R_COMMAND).orElse(Some(rCommand)).get - val rConnectionTimeout = sparkConf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val rConnectionTimeout = sparkConf.get(R_BACKEND_CONNECTION_TIMEOUT) val rOptions = "--vanilla" val rLibDir = RUtils.sparkRPackagePath(isDriver = false) val rExecScript = rLibDir(0) + "/SparkR/worker/" + script diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala index e86b362639e5..6284e6a6448f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -25,7 +25,8 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkUserAppException} -import org.apache.spark.api.r.{RBackend, RUtils, SparkRDefaults} +import org.apache.spark.api.r.{RBackend, RUtils} +import org.apache.spark.internal.config.R._ import org.apache.spark.util.RedirectThread /** @@ -43,8 +44,8 @@ object RRunner { val rCommand = { // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", // but kept here for backward compatibility. - var cmd = sys.props.getOrElse("spark.sparkr.r.command", "Rscript") - cmd = sys.props.getOrElse("spark.r.command", cmd) + var cmd = sys.props.getOrElse(SPARKR_COMMAND.key, SPARKR_COMMAND.defaultValue.get) + cmd = sys.props.getOrElse(R_COMMAND.key, cmd) if (sys.props.getOrElse("spark.submit.deployMode", "client") == "client") { cmd = sys.props.getOrElse("spark.r.driver.command", cmd) } @@ -53,7 +54,7 @@ object RRunner { // Connection timeout set by R process on its connection to RBackend in seconds. val backendConnectionTimeout = sys.props.getOrElse( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT.toString) + R_BACKEND_CONNECTION_TIMEOUT.key, R_BACKEND_CONNECTION_TIMEOUT.defaultValue.get.toString) // Check if the file path exists. // If not, change directory to current working directory for YARN cluster mode diff --git a/core/src/main/scala/org/apache/spark/internal/config/Python.scala b/core/src/main/scala/org/apache/spark/internal/config/Python.scala new file mode 100644 index 000000000000..26a0598f4941 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Python.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.internal.config + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit + +private[spark] object Python { + val PYTHON_WORKER_REUSE = ConfigBuilder("spark.python.worker.reuse") + .booleanConf + .createWithDefault(true) + + val PYTHON_TASK_KILL_TIMEOUT = ConfigBuilder("spark.python.task.killTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("2s") + + val PYTHON_USE_DAEMON = ConfigBuilder("spark.python.use.daemon") + .booleanConf + .createWithDefault(true) + + val PYTHON_DAEMON_MODULE = ConfigBuilder("spark.python.daemon.module") + .stringConf + .createOptional + + val PYTHON_WORKER_MODULE = ConfigBuilder("spark.python.worker.module") + .stringConf + .createOptional + + val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory") + .bytesConf(ByteUnit.MiB) + .createOptional +} diff --git a/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala b/core/src/main/scala/org/apache/spark/internal/config/R.scala similarity index 56% rename from core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala rename to core/src/main/scala/org/apache/spark/internal/config/R.scala index af67cbbce4e5..26e06a5231c4 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/R.scala @@ -14,17 +14,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.spark.internal.config -package org.apache.spark.api.r +private[spark] object R { -private[spark] object SparkRDefaults { + val R_BACKEND_CONNECTION_TIMEOUT = ConfigBuilder("spark.r.backendConnectionTimeout") + .intConf + .createWithDefault(6000) - // Default value for spark.r.backendConnectionTimeout config - val DEFAULT_CONNECTION_TIMEOUT: Int = 6000 + val R_NUM_BACKEND_THREADS = ConfigBuilder("spark.r.numRBackendThreads") + .intConf + .createWithDefault(2) - // Default value for spark.r.heartBeatInterval config - val DEFAULT_HEARTBEAT_INTERVAL: Int = 100 + val R_HEARTBEAT_INTERVAL = ConfigBuilder("spark.r.heartBeatInterval") + .intConf + .createWithDefault(100) - // Default value for spark.r.numRBackendThreads config - val DEFAULT_NUM_RBACKEND_THREADS = 2 + val SPARKR_COMMAND = ConfigBuilder("spark.sparkr.r.command") + .stringConf + .createWithDefault("Rscript") + + val R_COMMAND = ConfigBuilder("spark.r.command") + .stringConf + .createOptional } 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 d8e9c099028f..da8060459477 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 @@ -166,10 +166,6 @@ package object config { .checkValue(_ >= 0, "The off-heap memory size must not be negative") .createWithDefault(0) - private[spark] val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory") - .bytesConf(ByteUnit.MiB) - .createOptional - private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() .booleanConf.createWithDefault(false) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index c8bf7cdb4224..dd73a5e52281 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -25,6 +25,7 @@ import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index c2efab01e424..e28c650a571e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils 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 184fb6a8ad13..44a60b835f12 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 @@ -53,6 +53,7 @@ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} import org.apache.spark.util.{CallerContext, Utils} 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 a3feca5dfd22..8c6eff991513 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 @@ -36,6 +36,7 @@ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor From f65dc9593ee4b84343fea04fdcace14096788be8 Mon Sep 17 00:00:00 2001 From: "Liu,Linhong" Date: Fri, 4 Jan 2019 10:51:33 +0800 Subject: [PATCH 0224/1072] [SPARK-26526][SQL][TEST] Fix invalid test case about non-deterministic expression ## What changes were proposed in this pull request? Test case in SPARK-10316 is used to make sure non-deterministic `Filter` won't be pushed through `Project` But in current code base this test case can't cover this purpose. Change LogicalRDD to HadoopFsRelation can fix this issue. ## How was this patch tested? Modified test pass. Closes #23440 from LinhongLiu/fix-test. Authored-by: Liu,Linhong Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/DataFrameSuite.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) 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 b51c51e66350..3082e0bb97df 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 @@ -1398,11 +1398,14 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("SPARK-10316: respect non-deterministic expressions in PhysicalOperation") { - val input = spark.read.json((1 to 10).map(i => s"""{"id": $i}""").toDS()) + withTempDir { dir => + (1 to 10).toDF("id").write.mode(SaveMode.Overwrite).json(dir.getCanonicalPath) + val input = spark.read.json(dir.getCanonicalPath) - val df = input.select($"id", rand(0).as('r)) - df.as("a").join(df.filter($"r" < 0.5).as("b"), $"a.id" === $"b.id").collect().foreach { row => - assert(row.getDouble(1) - row.getDouble(3) === 0.0 +- 0.001) + val df = input.select($"id", rand(0).as('r)) + df.as("a").join(df.filter($"r" < 0.5).as("b"), $"a.id" === $"b.id").collect().foreach { row => + assert(row.getDouble(1) - row.getDouble(3) === 0.0 +- 0.001) + } } } From 27e42c1de502da80fa3e22bb69de47fb00158174 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 3 Jan 2019 20:01:19 -0800 Subject: [PATCH 0225/1072] [MINOR][NETWORK][TEST] Fix TransportFrameDecoderSuite to use ByteBuf instead of ByteBuffer ## What changes were proposed in this pull request? `fireChannelRead` expects `io.netty.buffer.ByteBuf`.I checked that this is the only place which misuse `java.nio.ByteBuffer` in `network` module. ## How was this patch tested? Pass the Jenkins with the existing tests. Closes #23442 from dongjoon-hyun/SPARK-NETWORK-COMMON. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/network/util/TransportFrameDecoderSuite.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java index b53e41303751..7d40387c5f1a 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.network.util; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -69,7 +68,7 @@ public void testInterception() throws Exception { decoder.channelRead(ctx, len); decoder.channelRead(ctx, dataBuf); verify(interceptor, times(interceptedReads)).handle(any(ByteBuf.class)); - verify(ctx).fireChannelRead(any(ByteBuffer.class)); + verify(ctx).fireChannelRead(any(ByteBuf.class)); assertEquals(0, len.refCnt()); assertEquals(0, dataBuf.refCnt()); } finally { From 4419e1daca6c5de373d5f3f13c417b791d768c96 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 4 Jan 2019 22:12:35 +0800 Subject: [PATCH 0226/1072] [SPARK-26445][CORE] Use ConfigEntry for hardcoded configs for driver/executor categories. ## What changes were proposed in this pull request? The PR makes hardcoded spark.driver, spark.executor, and spark.cores.max configs to use `ConfigEntry`. Note that some config keys are from `SparkLauncher` instead of defining in the config package object because the string is already defined in it and it does not depend on core module. ## How was this patch tested? Existing tests. Closes #23415 from ueshin/issues/SPARK-26445/hardcoded_driver_executor_configs. Authored-by: Takuya UESHIN Signed-off-by: Hyukjin Kwon --- .../spark/ExecutorAllocationManager.scala | 4 +- .../scala/org/apache/spark/SparkConf.scala | 25 ++++----- .../scala/org/apache/spark/SparkContext.scala | 8 +-- .../scala/org/apache/spark/SparkEnv.scala | 8 +-- .../spark/api/python/PythonRunner.scala | 4 +- .../org/apache/spark/deploy/Client.scala | 8 +-- .../spark/deploy/FaultToleranceTest.scala | 6 +- .../org/apache/spark/deploy/SparkSubmit.scala | 24 ++++---- .../spark/deploy/SparkSubmitArguments.scala | 20 +++---- .../deploy/rest/StandaloneRestServer.scala | 13 +++-- .../rest/SubmitRestProtocolRequest.scala | 11 ++-- .../spark/deploy/worker/DriverWrapper.scala | 6 +- .../spark/internal/config/package.scala | 55 ++++++++++++++++++- .../spark/memory/StaticMemoryManager.scala | 9 +-- .../spark/memory/UnifiedMemoryManager.scala | 9 +-- .../apache/spark/metrics/MetricsSystem.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../cluster/StandaloneSchedulerBackend.scala | 16 +++--- .../local/LocalSchedulerBackend.scala | 4 +- .../org/apache/spark/util/RpcUtils.scala | 5 +- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../spark/util/logging/FileAppender.scala | 10 ++-- .../util/logging/RollingFileAppender.scala | 19 ++----- .../ExecutorAllocationManagerSuite.scala | 2 +- .../org/apache/spark/SparkConfSuite.scala | 2 +- .../StandaloneDynamicAllocationSuite.scala | 6 +- .../memory/UnifiedMemoryManagerSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../BlockManagerReplicationSuite.scala | 4 +- .../spark/storage/BlockManagerSuite.scala | 2 +- .../apache/spark/util/FileAppenderSuite.scala | 13 ++--- .../k8s/features/BasicDriverFeatureStep.scala | 4 +- .../features/BasicExecutorFeatureStep.scala | 6 +- .../features/DriverServiceFeatureStep.scala | 15 +++-- .../BasicDriverFeatureStepSuite.scala | 2 +- .../BasicExecutorFeatureStepSuite.scala | 16 +++--- .../DriverServiceFeatureStepSuite.scala | 6 +- .../apache/spark/deploy/mesos/config.scala | 3 + .../deploy/rest/mesos/MesosRestServer.scala | 13 +++-- .../cluster/mesos/MesosClusterScheduler.scala | 10 ++-- .../MesosCoarseGrainedSchedulerBackend.scala | 20 +++---- .../MesosFineGrainedSchedulerBackend.scala | 12 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 6 +- .../org/apache/spark/deploy/yarn/config.scala | 10 +--- .../cluster/YarnClientSchedulerBackend.scala | 8 +-- .../yarn/ResourceRequestHelperSuite.scala | 2 +- 46 files changed, 236 insertions(+), 200 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 3f0b71bbe17f..d966582295b3 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -127,7 +127,7 @@ private[spark] class ExecutorAllocationManager( // allocation is only supported for YARN and the default number of cores per executor in YARN is // 1, but it might need to be attained differently for different cluster managers private val tasksPerExecutorForFullParallelism = - conf.getInt("spark.executor.cores", 1) / conf.getInt("spark.task.cpus", 1) + conf.get(EXECUTOR_CORES) / conf.getInt("spark.task.cpus", 1) private val executorAllocationRatio = conf.get(DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO) @@ -223,7 +223,7 @@ private[spark] class ExecutorAllocationManager( "shuffle service. You may enable this through spark.shuffle.service.enabled.") } if (tasksPerExecutorForFullParallelism == 0) { - throw new SparkException("spark.executor.cores must not be < spark.task.cpus.") + throw new SparkException(s"${EXECUTOR_CORES.key} must not be < spark.task.cpus.") } if (executorAllocationRatio > 1.0 || executorAllocationRatio <= 0.0) { diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0b47da12b5b4..681e4378a4dd 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -503,12 +503,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria logWarning(msg) } - val executorOptsKey = "spark.executor.extraJavaOptions" - val executorClasspathKey = "spark.executor.extraClassPath" - val driverOptsKey = "spark.driver.extraJavaOptions" - val driverClassPathKey = "spark.driver.extraClassPath" - val driverLibraryPathKey = "spark.driver.extraLibraryPath" - val sparkExecutorInstances = "spark.executor.instances" + val executorOptsKey = EXECUTOR_JAVA_OPTIONS.key // Used by Yarn in 1.1 and before sys.props.get("spark.driver.libraryPath").foreach { value => @@ -517,7 +512,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria |spark.driver.libraryPath was detected (set to '$value'). |This is deprecated in Spark 1.2+. | - |Please instead use: $driverLibraryPathKey + |Please instead use: ${DRIVER_LIBRARY_PATH.key} """.stripMargin logWarning(warning) } @@ -594,9 +589,9 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } - if (contains("spark.cores.max") && contains("spark.executor.cores")) { - val totalCores = getInt("spark.cores.max", 1) - val executorCores = getInt("spark.executor.cores", 1) + if (contains(CORES_MAX) && contains(EXECUTOR_CORES)) { + val totalCores = getInt(CORES_MAX.key, 1) + val executorCores = get(EXECUTOR_CORES) val leftCores = totalCores % executorCores if (leftCores != 0) { logWarning(s"Total executor cores: ${totalCores} is not " + @@ -605,12 +600,12 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } - if (contains("spark.executor.cores") && contains("spark.task.cpus")) { - val executorCores = getInt("spark.executor.cores", 1) + if (contains(EXECUTOR_CORES) && contains("spark.task.cpus")) { + val executorCores = get(EXECUTOR_CORES) val taskCpus = getInt("spark.task.cpus", 1) if (executorCores < taskCpus) { - throw new SparkException("spark.executor.cores must not be less than spark.task.cpus.") + throw new SparkException(s"${EXECUTOR_CORES.key} must not be less than spark.task.cpus.") } } @@ -680,7 +675,7 @@ private[spark] object SparkConf extends Logging { * TODO: consolidate it with `ConfigBuilder.withAlternative`. */ private val configsWithAlternatives = Map[String, Seq[AlternateConfig]]( - "spark.executor.userClassPathFirst" -> Seq( + EXECUTOR_USER_CLASS_PATH_FIRST.key -> Seq( AlternateConfig("spark.files.userClassPathFirst", "1.3")), UPDATE_INTERVAL_S.key -> Seq( AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), @@ -703,7 +698,7 @@ private[spark] object SparkConf extends Logging { AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4")), "spark.shuffle.file.buffer" -> Seq( AlternateConfig("spark.shuffle.file.buffer.kb", "1.4")), - "spark.executor.logs.rolling.maxSize" -> Seq( + EXECUTOR_LOGS_ROLLING_MAX_SIZE.key -> Seq( AlternateConfig("spark.executor.logs.rolling.size.maxBytes", "1.4")), "spark.io.compression.snappy.blockSize" -> Seq( AlternateConfig("spark.io.compression.snappy.block.size", "1.4")), diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3475859c3ed6..89be9de08307 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -386,9 +386,9 @@ class SparkContext(config: SparkConf) extends Logging { // Set Spark driver host and port system properties. This explicitly sets the configuration // instead of relying on the default value of the config constant. _conf.set(DRIVER_HOST_ADDRESS, _conf.get(DRIVER_HOST_ADDRESS)) - _conf.setIfMissing("spark.driver.port", "0") + _conf.setIfMissing(DRIVER_PORT, 0) - _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) + _conf.set(EXECUTOR_ID, SparkContext.DRIVER_IDENTIFIER) _jars = Utils.getUserJars(_conf) _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.nonEmpty)) @@ -461,7 +461,7 @@ class SparkContext(config: SparkConf) extends Logging { files.foreach(addFile) } - _executorMemory = _conf.getOption("spark.executor.memory") + _executorMemory = _conf.getOption(EXECUTOR_MEMORY.key) .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) .orElse(Option(System.getenv("SPARK_MEM")) .map(warnSparkMem)) @@ -2639,7 +2639,7 @@ object SparkContext extends Logging { case SparkMasterRegex.LOCAL_N_FAILURES_REGEX(threads, _) => convertToInt(threads) case "yarn" => if (conf != null && conf.getOption("spark.submit.deployMode").contains("cluster")) { - conf.getInt("spark.driver.cores", 0) + conf.getInt(DRIVER_CORES.key, 0) } else { 0 } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index de0c8579d9ac..9222781fa083 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -163,10 +163,10 @@ object SparkEnv extends Logging { mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = { assert(conf.contains(DRIVER_HOST_ADDRESS), s"${DRIVER_HOST_ADDRESS.key} is not set on the driver!") - assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") + assert(conf.contains(DRIVER_PORT), s"${DRIVER_PORT.key} is not set on the driver!") val bindAddress = conf.get(DRIVER_BIND_ADDRESS) val advertiseAddress = conf.get(DRIVER_HOST_ADDRESS) - val port = conf.get("spark.driver.port").toInt + val port = conf.get(DRIVER_PORT) val ioEncryptionKey = if (conf.get(IO_ENCRYPTION_ENABLED)) { Some(CryptoStreamUtils.createKey(conf)) } else { @@ -251,7 +251,7 @@ object SparkEnv extends Logging { // Figure out which port RpcEnv actually bound to in case the original port is 0 or occupied. if (isDriver) { - conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set(DRIVER_PORT, rpcEnv.address.port) } // Create an instance of the class with the given name, possibly initializing it with our conf @@ -359,7 +359,7 @@ object SparkEnv extends Logging { // We need to set the executor ID before the MetricsSystem is created because sources and // sinks specified in the metrics configuration file will want to incorporate this executor's // ID into the metrics they report. - conf.set("spark.executor.id", executorId) + conf.set(EXECUTOR_ID, executorId) val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) ms.start() ms diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 6b748c825d29..5168e9330965 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -27,6 +27,7 @@ import scala.collection.JavaConverters._ import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.EXECUTOR_CORES import org.apache.spark.internal.config.Python._ import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -74,8 +75,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( private val reuseWorker = conf.get(PYTHON_WORKER_REUSE) // each python worker gets an equal part of the allocation. the worker pool will grow to the // number of concurrent tasks, which is determined by the number of cores in this executor. - private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY) - .map(_ / conf.getInt("spark.executor.cores", 1)) + private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY).map(_ / conf.get(EXECUTOR_CORES)) // All the Python functions should have the same exec, version and envvars. protected val envVars = funcs.head.funcs.head.envVars diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index d5145094ec07..d94b174d8d86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -27,7 +27,7 @@ import org.apache.log4j.Logger import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.util.{SparkExitCode, ThreadUtils, Utils} @@ -68,17 +68,17 @@ private class ClientEndpoint( // people call `addJar` assuming the jar is in the same directory. val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" - val classPathConf = "spark.driver.extraClassPath" + val classPathConf = config.DRIVER_CLASS_PATH.key val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val libraryPathConf = "spark.driver.extraLibraryPath" + val libraryPathConf = config.DRIVER_LIBRARY_PATH.key val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val extraJavaOptsConf = "spark.driver.extraJavaOptions" + val extraJavaOptsConf = config.DRIVER_JAVA_OPTIONS.key val extraJavaOpts = sys.props.get(extraJavaOptsConf) .map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) 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 c6307da61c7e..0679bdf7c707 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -34,7 +34,7 @@ import org.json4s.jackson.JsonMethods import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.master.RecoveryState -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -77,7 +77,7 @@ private object FaultToleranceTest extends App with Logging { private val containerSparkHome = "/opt/spark" private val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) - System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip + System.setProperty(config.DRIVER_HOST_ADDRESS.key, "172.17.42.1") // default docker host ip private def afterEach() { if (sc != null) { @@ -216,7 +216,7 @@ private object FaultToleranceTest extends App with Logging { if (sc != null) { sc.stop() } // Counter-hack: Because of a hack in SparkEnv#create() that changes this // property, we need to reset it. - System.setProperty("spark.driver.port", "0") + System.setProperty(config.DRIVER_PORT.key, "0") sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome) } 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 763bd0a70a03..a4c65aeaae3f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -514,13 +514,13 @@ private[spark] class SparkSubmit extends Logging { OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = "spark.app.name"), OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, confKey = "spark.jars.ivy"), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, - confKey = "spark.driver.memory"), + confKey = DRIVER_MEMORY.key), OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraClassPath"), + confKey = DRIVER_CLASS_PATH.key), OptionAssigner(args.driverExtraJavaOptions, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraJavaOptions"), + confKey = DRIVER_JAVA_OPTIONS.key), OptionAssigner(args.driverExtraLibraryPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraLibraryPath"), + confKey = DRIVER_LIBRARY_PATH.key), OptionAssigner(args.principal, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = PRINCIPAL.key), OptionAssigner(args.keytab, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, @@ -537,7 +537,7 @@ private[spark] class SparkSubmit extends Logging { // Yarn only OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.queue"), OptionAssigner(args.numExecutors, YARN, ALL_DEPLOY_MODES, - confKey = "spark.executor.instances"), + confKey = EXECUTOR_INSTANCES.key), OptionAssigner(args.pyFiles, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.pyFiles"), OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.jars"), OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.files"), @@ -545,22 +545,22 @@ private[spark] class SparkSubmit extends Logging { // Other options OptionAssigner(args.executorCores, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.executor.cores"), + confKey = EXECUTOR_CORES.key), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.executor.memory"), + confKey = EXECUTOR_MEMORY.key), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.cores.max"), + confKey = CORES_MAX.key), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, confKey = "spark.files"), OptionAssigner(args.jars, LOCAL, CLIENT, confKey = "spark.jars"), OptionAssigner(args.jars, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, confKey = "spark.jars"), OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, - confKey = "spark.driver.memory"), + confKey = DRIVER_MEMORY.key), OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, - confKey = "spark.driver.cores"), + confKey = DRIVER_CORES.key), OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, - confKey = "spark.driver.supervise"), + confKey = DRIVER_SUPERVISE.key), OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, confKey = "spark.jars.ivy"), // An internal option used only for spark-shell to add user jars to repl's classloader, @@ -727,7 +727,7 @@ private[spark] class SparkSubmit extends Logging { // Ignore invalid spark.driver.host in cluster modes. if (deployMode == CLUSTER) { - sparkConf.remove("spark.driver.host") + sparkConf.remove(DRIVER_HOST_ADDRESS) } // Resolve paths in certain spark properties diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 4cf08a7980f5..34facd5a58c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -31,7 +31,7 @@ import scala.util.Try import org.apache.spark.{SparkException, SparkUserAppException} import org.apache.spark.deploy.SparkSubmitAction._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.SparkSubmitArgumentsParser import org.apache.spark.network.util.JavaUtils import org.apache.spark.util.Utils @@ -155,31 +155,31 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orElse(env.get("MASTER")) .orNull driverExtraClassPath = Option(driverExtraClassPath) - .orElse(sparkProperties.get("spark.driver.extraClassPath")) + .orElse(sparkProperties.get(config.DRIVER_CLASS_PATH.key)) .orNull driverExtraJavaOptions = Option(driverExtraJavaOptions) - .orElse(sparkProperties.get("spark.driver.extraJavaOptions")) + .orElse(sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key)) .orNull driverExtraLibraryPath = Option(driverExtraLibraryPath) - .orElse(sparkProperties.get("spark.driver.extraLibraryPath")) + .orElse(sparkProperties.get(config.DRIVER_LIBRARY_PATH.key)) .orNull driverMemory = Option(driverMemory) - .orElse(sparkProperties.get("spark.driver.memory")) + .orElse(sparkProperties.get(config.DRIVER_MEMORY.key)) .orElse(env.get("SPARK_DRIVER_MEMORY")) .orNull driverCores = Option(driverCores) - .orElse(sparkProperties.get("spark.driver.cores")) + .orElse(sparkProperties.get(config.DRIVER_CORES.key)) .orNull executorMemory = Option(executorMemory) - .orElse(sparkProperties.get("spark.executor.memory")) + .orElse(sparkProperties.get(config.EXECUTOR_MEMORY.key)) .orElse(env.get("SPARK_EXECUTOR_MEMORY")) .orNull executorCores = Option(executorCores) - .orElse(sparkProperties.get("spark.executor.cores")) + .orElse(sparkProperties.get(config.EXECUTOR_CORES.key)) .orElse(env.get("SPARK_EXECUTOR_CORES")) .orNull totalExecutorCores = Option(totalExecutorCores) - .orElse(sparkProperties.get("spark.cores.max")) + .orElse(sparkProperties.get(config.CORES_MAX.key)) .orNull name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull @@ -197,7 +197,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orElse(env.get("DEPLOY_MODE")) .orNull numExecutors = Option(numExecutors) - .getOrElse(sparkProperties.get("spark.executor.instances").orNull) + .getOrElse(sparkProperties.get(config.EXECUTOR_INSTANCES.key).orNull) queue = Option(queue).orElse(sparkProperties.get("spark.yarn.queue")).orNull keytab = Option(keytab) .orElse(sparkProperties.get("spark.kerberos.keytab")) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index afa1a5fbba79..c75e684df226 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -23,6 +23,7 @@ import javax.servlet.http.HttpServletResponse import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ +import org.apache.spark.internal.config import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -132,12 +133,12 @@ private[rest] class StandaloneSubmitRequestServlet( // Optional fields val sparkProperties = request.sparkProperties - val driverMemory = sparkProperties.get("spark.driver.memory") - val driverCores = sparkProperties.get("spark.driver.cores") - val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") - val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") - val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") - val superviseDriver = sparkProperties.get("spark.driver.supervise") + val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) + val driverCores = sparkProperties.get(config.DRIVER_CORES.key) + val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) + val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) + val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) + val superviseDriver = sparkProperties.get(config.DRIVER_SUPERVISE.key) // The semantics of "spark.master" and the masterUrl are different. While the // property "spark.master" could contain all registered masters, masterUrl // contains only the active master. To make sure a Spark driver can recover diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala index 86ddf954ca12..7f462148c71a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.rest import scala.util.Try +import org.apache.spark.internal.config import org.apache.spark.util.Utils /** @@ -49,11 +50,11 @@ private[rest] class CreateSubmissionRequest extends SubmitRestProtocolRequest { assertFieldIsSet(appArgs, "appArgs") assertFieldIsSet(environmentVariables, "environmentVariables") assertPropertyIsSet("spark.app.name") - assertPropertyIsBoolean("spark.driver.supervise") - assertPropertyIsNumeric("spark.driver.cores") - assertPropertyIsNumeric("spark.cores.max") - assertPropertyIsMemory("spark.driver.memory") - assertPropertyIsMemory("spark.executor.memory") + assertPropertyIsBoolean(config.DRIVER_SUPERVISE.key) + assertPropertyIsNumeric(config.DRIVER_CORES.key) + assertPropertyIsNumeric(config.CORES_MAX.key) + assertPropertyIsMemory(config.DRIVER_MEMORY.key) + assertPropertyIsMemory(config.EXECUTOR_MEMORY.key) } private def assertPropertyIsSet(key: String): Unit = diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 8d6a2b80ef5f..1e8ad0b6af6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -23,7 +23,7 @@ import org.apache.commons.lang3.StringUtils import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{DependencyUtils, SparkHadoopUtil, SparkSubmit} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util._ @@ -43,7 +43,7 @@ object DriverWrapper extends Logging { case workerUrl :: userJar :: mainClass :: extraArgs => val conf = new SparkConf() val host: String = Utils.localHostName() - val port: Int = sys.props.getOrElse("spark.driver.port", "0").toInt + val port: Int = sys.props.getOrElse(config.DRIVER_PORT.key, "0").toInt val rpcEnv = RpcEnv.create("Driver", host, port, conf, new SecurityManager(conf)) logInfo(s"Driver address: ${rpcEnv.address}") rpcEnv.setupEndpoint("workerWatcher", new WorkerWatcher(rpcEnv, workerUrl)) @@ -51,7 +51,7 @@ object DriverWrapper extends Logging { val currentLoader = Thread.currentThread.getContextClassLoader val userJarUrl = new File(userJar).toURI().toURL() val loader = - if (sys.props.getOrElse("spark.driver.userClassPathFirst", "false").toBoolean) { + if (sys.props.getOrElse(config.DRIVER_USER_CLASS_PATH_FIRST.key, "false").toBoolean) { new ChildFirstURLClassLoader(Array(userJarUrl), currentLoader) } else { new MutableURLClassLoader(Array(userJarUrl), currentLoader) 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 da8060459477..8caaa73b0227 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 @@ -39,7 +39,12 @@ package object config { private[spark] val DRIVER_USER_CLASS_PATH_FIRST = ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.createWithDefault(false) - private[spark] val DRIVER_MEMORY = ConfigBuilder("spark.driver.memory") + private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores") + .doc("Number of cores to use for the driver process, only in cluster mode.") + .intConf + .createWithDefault(1) + + private[spark] val DRIVER_MEMORY = ConfigBuilder(SparkLauncher.DRIVER_MEMORY) .doc("Amount of memory to use for the driver process, in MiB unless otherwise specified.") .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") @@ -113,6 +118,9 @@ package object config { private[spark] val EVENT_LOG_CALLSITE_LONG_FORM = ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false) + private[spark] val EXECUTOR_ID = + ConfigBuilder("spark.executor.id").stringConf.createOptional + private[spark] val EXECUTOR_CLASS_PATH = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional @@ -139,7 +147,11 @@ package object config { private[spark] val EXECUTOR_USER_CLASS_PATH_FIRST = ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false) - private[spark] val EXECUTOR_MEMORY = ConfigBuilder("spark.executor.memory") + private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) + .intConf + .createWithDefault(1) + + private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) .doc("Amount of memory to use per executor process, in MiB unless otherwise specified.") .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") @@ -150,6 +162,15 @@ package object config { .bytesConf(ByteUnit.MiB) .createOptional + private[spark] val CORES_MAX = ConfigBuilder("spark.cores.max") + .doc("When running on a standalone deploy cluster or a Mesos cluster in coarse-grained " + + "sharing mode, the maximum amount of CPU cores to request for the application from across " + + "the cluster (not from each machine). If not set, the default will be " + + "`spark.deploy.defaultCores` on Spark's standalone cluster manager, or infinite " + + "(all available cores) on Mesos.") + .intConf + .createOptional + private[spark] val MEMORY_OFFHEAP_ENABLED = ConfigBuilder("spark.memory.offHeap.enabled") .doc("If true, Spark will attempt to use off-heap memory for certain operations. " + "If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive.") @@ -347,6 +368,17 @@ package object config { .stringConf .createWithDefault(Utils.localCanonicalHostName()) + private[spark] val DRIVER_PORT = ConfigBuilder("spark.driver.port") + .doc("Port of driver endpoints.") + .intConf + .createWithDefault(0) + + private[spark] val DRIVER_SUPERVISE = ConfigBuilder("spark.driver.supervise") + .doc("If true, restarts the driver automatically if it fails with a non-zero exit status. " + + "Only has effect in Spark standalone mode or Mesos cluster deploy mode.") + .booleanConf + .createWithDefault(false) + private[spark] val DRIVER_BIND_ADDRESS = ConfigBuilder("spark.driver.bindAddress") .doc("Address where to bind network listen sockets on the driver.") .fallbackConf(DRIVER_HOST_ADDRESS) @@ -729,4 +761,23 @@ package object config { .stringConf .toSequence .createWithDefault(Nil) + + private[spark] val EXECUTOR_LOGS_ROLLING_STRATEGY = + ConfigBuilder("spark.executor.logs.rolling.strategy").stringConf.createWithDefault("") + + private[spark] val EXECUTOR_LOGS_ROLLING_TIME_INTERVAL = + ConfigBuilder("spark.executor.logs.rolling.time.interval").stringConf.createWithDefault("daily") + + private[spark] val EXECUTOR_LOGS_ROLLING_MAX_SIZE = + ConfigBuilder("spark.executor.logs.rolling.maxSize") + .stringConf + .createWithDefault((1024 * 1024).toString) + + private[spark] val EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES = + ConfigBuilder("spark.executor.logs.rolling.maxRetainedFiles").intConf.createWithDefault(-1) + + private[spark] val EXECUTOR_LOGS_ROLLING_ENABLE_COMPRESSION = + ConfigBuilder("spark.executor.logs.rolling.enableCompression") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index a6f7db0600e6..828608704274 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.memory import org.apache.spark.SparkConf +import org.apache.spark.internal.config import org.apache.spark.storage.BlockId /** @@ -127,14 +128,14 @@ private[spark] object StaticMemoryManager { if (systemMaxMemory < MIN_MEMORY_BYTES) { throw new IllegalArgumentException(s"System memory $systemMaxMemory must " + s"be at least $MIN_MEMORY_BYTES. Please increase heap size using the --driver-memory " + - s"option or spark.driver.memory in Spark configuration.") + s"option or ${config.DRIVER_MEMORY.key} in Spark configuration.") } - if (conf.contains("spark.executor.memory")) { - val executorMemory = conf.getSizeAsBytes("spark.executor.memory") + if (conf.contains(config.EXECUTOR_MEMORY)) { + val executorMemory = conf.getSizeAsBytes(config.EXECUTOR_MEMORY.key) if (executorMemory < MIN_MEMORY_BYTES) { throw new IllegalArgumentException(s"Executor memory $executorMemory must be at least " + s"$MIN_MEMORY_BYTES. Please increase executor memory using the " + - s"--executor-memory option or spark.executor.memory in Spark configuration.") + s"--executor-memory option or ${config.EXECUTOR_MEMORY.key} in Spark configuration.") } } val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2) diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 78edd2c4d7fa..9260fd3a6fb3 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.memory import org.apache.spark.SparkConf +import org.apache.spark.internal.config import org.apache.spark.storage.BlockId /** @@ -216,15 +217,15 @@ object UnifiedMemoryManager { if (systemMemory < minSystemMemory) { throw new IllegalArgumentException(s"System memory $systemMemory must " + s"be at least $minSystemMemory. Please increase heap size using the --driver-memory " + - s"option or spark.driver.memory in Spark configuration.") + s"option or ${config.DRIVER_MEMORY.key} in Spark configuration.") } // SPARK-12759 Check executor memory to fail fast if memory is insufficient - if (conf.contains("spark.executor.memory")) { - val executorMemory = conf.getSizeAsBytes("spark.executor.memory") + if (conf.contains(config.EXECUTOR_MEMORY)) { + val executorMemory = conf.getSizeAsBytes(config.EXECUTOR_MEMORY.key) if (executorMemory < minSystemMemory) { throw new IllegalArgumentException(s"Executor memory $executorMemory must be at least " + s"$minSystemMemory. Please increase executor memory using the " + - s"--executor-memory option or spark.executor.memory in Spark configuration.") + s"--executor-memory option or ${config.EXECUTOR_MEMORY.key} in Spark configuration.") } } val usableMemory = systemMemory - reservedMemory 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 301317a79dfc..b1e311ada459 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -130,7 +130,7 @@ private[spark] class MetricsSystem private ( private[spark] def buildRegistryName(source: Source): String = { val metricsNamespace = conf.get(METRICS_NAMESPACE).orElse(conf.getOption("spark.app.id")) - val executorId = conf.getOption("spark.executor.id") + val executorId = conf.get(EXECUTOR_ID) val defaultName = MetricRegistry.name(source.sourceName) if (instance == "driver" || instance == "executor") { 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 6bf60dd8e9df..41f032ccf82b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -717,7 +717,7 @@ private[spark] class TaskSetManager( calculatedTasks += 1 if (maxResultSize > 0 && totalResultSize > maxResultSize) { val msg = s"Total size of serialized results of ${calculatedTasks} tasks " + - s"(${Utils.bytesToString(totalResultSize)}) is bigger than spark.driver.maxResultSize " + + s"(${Utils.bytesToString(totalResultSize)}) is bigger than ${config.MAX_RESULT_SIZE.key} " + s"(${Utils.bytesToString(maxResultSize)})" logError(msg) abort(msg) 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 f73a58ff5d48..adef20d3077d 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 @@ -25,7 +25,7 @@ import scala.concurrent.Future import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ @@ -54,7 +54,7 @@ private[spark] class StandaloneSchedulerBackend( private val registrationBarrier = new Semaphore(0) - private val maxCores = conf.getOption("spark.cores.max").map(_.toInt) + private val maxCores = conf.get(config.CORES_MAX) private val totalExpectedCores = maxCores.getOrElse(0) override def start() { @@ -69,8 +69,8 @@ private[spark] class StandaloneSchedulerBackend( // The endpoint for executors to talk to us val driverUrl = RpcEndpointAddress( - sc.conf.get("spark.driver.host"), - sc.conf.get("spark.driver.port").toInt, + sc.conf.get(config.DRIVER_HOST_ADDRESS), + sc.conf.get(config.DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString val args = Seq( "--driver-url", driverUrl, @@ -79,11 +79,11 @@ private[spark] class StandaloneSchedulerBackend( "--cores", "{{CORES}}", "--app-id", "{{APP_ID}}", "--worker-url", "{{WORKER_URL}}") - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + val extraJavaOpts = sc.conf.get(config.EXECUTOR_JAVA_OPTIONS) .map(Utils.splitCommandString).getOrElse(Seq.empty) - val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath") + val classPathEntries = sc.conf.get(config.EXECUTOR_CLASS_PATH) .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) - val libraryPathEntries = sc.conf.getOption("spark.executor.extraLibraryPath") + val libraryPathEntries = sc.conf.get(config.EXECUTOR_LIBRARY_PATH) .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) // When testing, expose the parent class path to the child. This is processed by @@ -102,7 +102,7 @@ private[spark] class StandaloneSchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) val webUrl = sc.ui.map(_.webUrl).getOrElse("") - val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) + val coresPerExecutor = conf.getOption(config.EXECUTOR_CORES.key).map(_.toInt) // If we're using dynamic allocation, set our initial executor limit to 0 for now. // ExecutorAllocationManager will send the real initial limit to the Master later. val initialExecutorLimit = diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index 0de57fbd5600..6ff8bf29b006 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -24,7 +24,7 @@ import java.nio.ByteBuffer import org.apache.spark.{SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ @@ -116,7 +116,7 @@ private[spark] class LocalSchedulerBackend( * @param conf Spark configuration. */ def getUserClasspath(conf: SparkConf): Seq[URL] = { - val userClassPathStr = conf.getOption("spark.executor.extraClassPath") + val userClassPathStr = conf.get(config.EXECUTOR_CLASS_PATH) userClassPathStr.map(_.split(File.pathSeparator)).toSeq.flatten.map(new File(_).toURI.toURL) } 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 e5cccf39f945..902e48fed391 100644 --- a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import org.apache.spark.SparkConf +import org.apache.spark.internal.config import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, RpcTimeout} private[spark] object RpcUtils { @@ -26,8 +27,8 @@ private[spark] object RpcUtils { * Retrieve a `RpcEndpointRef` which is located in the driver via its name. */ def makeDriverRef(name: String, conf: SparkConf, rpcEnv: RpcEnv): RpcEndpointRef = { - val driverHost: String = conf.get("spark.driver.host", "localhost") - val driverPort: Int = conf.getInt("spark.driver.port", 7077) + val driverHost: String = conf.get(config.DRIVER_HOST_ADDRESS.key, "localhost") + val driverPort: Int = conf.getInt(config.DRIVER_PORT.key, 7077) Utils.checkHost(driverHost) rpcEnv.setupEndpointRef(RpcAddress(driverHost, driverPort), name) } 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 22f074cf9897..3527fee68939 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2231,7 +2231,7 @@ private[spark] object Utils extends Logging { s"${e.getMessage}: Service$serviceString failed after " + s"$maxRetries retries (on a random free port)! " + s"Consider explicitly setting the appropriate binding address for " + - s"the service$serviceString (for example spark.driver.bindAddress " + + s"the service$serviceString (for example ${DRIVER_BIND_ADDRESS.key} " + s"for SparkDriver) to the correct binding address." } else { s"${e.getMessage}: Service$serviceString failed after " + diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 2f9ad4c8cc3e..3188e0bd2b70 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -20,7 +20,7 @@ package org.apache.spark.util.logging import java.io.{File, FileOutputStream, InputStream, IOException} import org.apache.spark.SparkConf -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{IntParam, Utils} /** @@ -115,11 +115,9 @@ private[spark] object FileAppender extends Logging { /** Create the right appender based on Spark configuration */ def apply(inputStream: InputStream, file: File, conf: SparkConf): FileAppender = { - import RollingFileAppender._ - - val rollingStrategy = conf.get(STRATEGY_PROPERTY, STRATEGY_DEFAULT) - val rollingSizeBytes = conf.get(SIZE_PROPERTY, STRATEGY_DEFAULT) - val rollingInterval = conf.get(INTERVAL_PROPERTY, INTERVAL_DEFAULT) + val rollingStrategy = conf.get(config.EXECUTOR_LOGS_ROLLING_STRATEGY) + val rollingSizeBytes = conf.get(config.EXECUTOR_LOGS_ROLLING_MAX_SIZE) + val rollingInterval = conf.get(config.EXECUTOR_LOGS_ROLLING_TIME_INTERVAL) def createTimeBasedAppender(): FileAppender = { val validatedParams: Option[(Long, String)] = rollingInterval match { diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index 5d8cec8447b5..59439b68792e 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -24,6 +24,7 @@ import com.google.common.io.Files import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf +import org.apache.spark.internal.config /** * Continuously appends data from input stream into the given file, and rolls @@ -44,10 +45,8 @@ private[spark] class RollingFileAppender( bufferSize: Int = RollingFileAppender.DEFAULT_BUFFER_SIZE ) extends FileAppender(inputStream, activeFile, bufferSize) { - import RollingFileAppender._ - - private val maxRetainedFiles = conf.getInt(RETAINED_FILES_PROPERTY, -1) - private val enableCompression = conf.getBoolean(ENABLE_COMPRESSION, false) + private val maxRetainedFiles = conf.get(config.EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES) + private val enableCompression = conf.get(config.EXECUTOR_LOGS_ROLLING_ENABLE_COMPRESSION) /** Stop the appender */ override def stop() { @@ -82,7 +81,7 @@ private[spark] class RollingFileAppender( // Roll the log file and compress if enableCompression is true. private def rotateFile(activeFile: File, rolloverFile: File): Unit = { if (enableCompression) { - val gzFile = new File(rolloverFile.getAbsolutePath + GZIP_LOG_SUFFIX) + val gzFile = new File(rolloverFile.getAbsolutePath + RollingFileAppender.GZIP_LOG_SUFFIX) var gzOutputStream: GZIPOutputStream = null var inputStream: InputStream = null try { @@ -103,7 +102,7 @@ private[spark] class RollingFileAppender( // Check if the rollover file already exists. private def rolloverFileExist(file: File): Boolean = { - file.exists || new File(file.getAbsolutePath + GZIP_LOG_SUFFIX).exists + file.exists || new File(file.getAbsolutePath + RollingFileAppender.GZIP_LOG_SUFFIX).exists } /** Move the active log file to a new rollover file */ @@ -164,15 +163,7 @@ private[spark] class RollingFileAppender( * names of configurations that configure rolling file appenders. */ private[spark] object RollingFileAppender { - val STRATEGY_PROPERTY = "spark.executor.logs.rolling.strategy" - val STRATEGY_DEFAULT = "" - val INTERVAL_PROPERTY = "spark.executor.logs.rolling.time.interval" - val INTERVAL_DEFAULT = "daily" - val SIZE_PROPERTY = "spark.executor.logs.rolling.maxSize" - val SIZE_DEFAULT = (1024 * 1024).toString - val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" val DEFAULT_BUFFER_SIZE = 8192 - val ENABLE_COMPRESSION = "spark.executor.logs.rolling.enableCompression" val GZIP_LOG_SUFFIX = ".gz" diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 5c718cb654ce..d0389235cb72 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -155,7 +155,7 @@ class ExecutorAllocationManagerSuite .set("spark.dynamicAllocation.maxExecutors", "15") .set("spark.dynamicAllocation.minExecutors", "3") .set("spark.dynamicAllocation.executorAllocationRatio", divisor.toString) - .set("spark.executor.cores", cores.toString) + .set(config.EXECUTOR_CORES, cores) val sc = new SparkContext(conf) contexts += sc var manager = sc.executorAllocationManager.get diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index e14a5dcb5ef8..9a6abbdb0a46 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -140,7 +140,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst test("creating SparkContext with cpus per tasks bigger than cores per executors") { val conf = new SparkConf(false) - .set("spark.executor.cores", "1") + .set(EXECUTOR_CORES, 1) .set("spark.task.cpus", "2") intercept[SparkException] { sc = new SparkContext(conf) } } 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 a1d2a1283db1..8567dd1f0823 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -243,7 +243,7 @@ class StandaloneDynamicAllocationSuite } test("dynamic allocation with cores per executor") { - sc = new SparkContext(appConf.set("spark.executor.cores", "2")) + sc = new SparkContext(appConf.set(config.EXECUTOR_CORES, 2)) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() @@ -296,7 +296,7 @@ class StandaloneDynamicAllocationSuite test("dynamic allocation with cores per executor AND max cores") { sc = new SparkContext(appConf - .set("spark.executor.cores", "2") + .set(config.EXECUTOR_CORES, 2) .set("spark.cores.max", "8")) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { @@ -526,7 +526,7 @@ class StandaloneDynamicAllocationSuite new SparkConf() .setMaster(masterRpcEnv.address.toSparkURL) .setAppName("test") - .set("spark.executor.memory", "256m") + .set(config.EXECUTOR_MEMORY.key, "256m") } /** Make a master to which our application will send executor requests. */ diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index d56cfc183d92..5ce3453b682f 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -248,7 +248,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val mm = UnifiedMemoryManager(conf, numCores = 1) // Try using an executor memory that's too small - val conf2 = conf.clone().set("spark.executor.memory", (reservedMemory / 2).toString) + val conf2 = conf.clone().set(EXECUTOR_MEMORY.key, (reservedMemory / 2).toString) val exception = intercept[IllegalArgumentException] { UnifiedMemoryManager(conf2, numCores = 1) } 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 d264adaef90a..f73ff67837c6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -655,7 +655,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } test("abort the job if total size of results is too large") { - val conf = new SparkConf().set("spark.driver.maxResultSize", "2m") + val conf = new SparkConf().set(config.MAX_RESULT_SIZE.key, "2m") sc = new SparkContext("local", "test", conf) def genBytes(size: Int): (Int) => Array[Byte] = { (x: Int) => 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 3962bdc27d22..19116cf22d2f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE +import org.apache.spark.internal.config.{DRIVER_PORT, MEMORY_OFFHEAP_SIZE} import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService @@ -86,7 +86,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.authenticate", "false") - conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set(DRIVER_PORT, rpcEnv.address.port) conf.set("spark.testing", "true") conf.set("spark.memory.fraction", "1") conf.set("spark.memory.storageFraction", "1") 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 cf00c1c3aad3..e866342e4472 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -124,7 +124,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE .set("spark.storage.unrollMemoryThreshold", "512") rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) - conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set(DRIVER_PORT, rpcEnv.address.port) // 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. diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 52cd5378bc71..242163931f7a 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -34,7 +34,7 @@ import org.mockito.Mockito.{atLeast, mock, verify} import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.logging.{FileAppender, RollingFileAppender, SizeBasedRollingPolicy, TimeBasedRollingPolicy} class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { @@ -136,7 +136,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { // setup input stream and appender val testOutputStream = new PipedOutputStream() val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) - val conf = new SparkConf().set(RollingFileAppender.RETAINED_FILES_PROPERTY, "10") + val conf = new SparkConf().set(config.EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES, 10) val appender = new RollingFileAppender(testInputStream, testFile, new SizeBasedRollingPolicy(1000, false), conf, 10) @@ -200,13 +200,12 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { appender.awaitTermination() } - import RollingFileAppender._ - def rollingStrategy(strategy: String): Seq[(String, String)] = - Seq(STRATEGY_PROPERTY -> strategy) - def rollingSize(size: String): Seq[(String, String)] = Seq(SIZE_PROPERTY -> size) + Seq(config.EXECUTOR_LOGS_ROLLING_STRATEGY.key -> strategy) + def rollingSize(size: String): Seq[(String, String)] = + Seq(config.EXECUTOR_LOGS_ROLLING_MAX_SIZE.key -> size) def rollingInterval(interval: String): Seq[(String, String)] = - Seq(INTERVAL_PROPERTY -> interval) + Seq(config.EXECUTOR_LOGS_ROLLING_TIME_INTERVAL.key -> interval) val msInDay = 24 * 60 * 60 * 1000L val msInHour = 60 * 60 * 1000L diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 8362c14fb289..d52988df58d6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -42,7 +42,7 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) .getOrElse(throw new SparkException("Must specify the driver container image")) // CPU settings - private val driverCpuCores = conf.get("spark.driver.cores", "1") + private val driverCpuCores = conf.get(DRIVER_CORES.key, "1") private val driverLimitCores = conf.get(KUBERNETES_DRIVER_LIMIT_CORES) // Memory settings @@ -85,7 +85,7 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) } - val driverPort = conf.sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) + val driverPort = conf.sparkConf.getInt(DRIVER_PORT.key, DEFAULT_DRIVER_PORT) val driverBlockManagerPort = conf.sparkConf.getInt( DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index dd73a5e52281..6c3a6b39fa5c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -46,8 +46,8 @@ private[spark] class BasicExecutorFeatureStep( private val executorPodNamePrefix = kubernetesConf.resourceNamePrefix private val driverUrl = RpcEndpointAddress( - kubernetesConf.get("spark.driver.host"), - kubernetesConf.sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), + kubernetesConf.get(DRIVER_HOST_ADDRESS), + kubernetesConf.sparkConf.getInt(DRIVER_PORT.key, DEFAULT_DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString private val executorMemoryMiB = kubernetesConf.get(EXECUTOR_MEMORY) private val executorMemoryString = kubernetesConf.get( @@ -67,7 +67,7 @@ private[spark] class BasicExecutorFeatureStep( executorMemoryWithOverhead } - private val executorCores = kubernetesConf.sparkConf.getInt("spark.executor.cores", 1) + private val executorCores = kubernetesConf.sparkConf.get(EXECUTOR_CORES) private val executorCoresRequest = if (kubernetesConf.sparkConf.contains(KUBERNETES_EXECUTOR_REQUEST_CORES)) { kubernetesConf.get(KUBERNETES_EXECUTOR_REQUEST_CORES).get diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala index 42305457f4ff..15671179b18b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.{HasMetadata, ServiceBuilder} import org.apache.spark.deploy.k8s.{KubernetesDriverConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{Clock, SystemClock} private[spark] class DriverServiceFeatureStep( @@ -51,18 +51,17 @@ private[spark] class DriverServiceFeatureStep( } private val driverPort = kubernetesConf.sparkConf.getInt( - "spark.driver.port", DEFAULT_DRIVER_PORT) + config.DRIVER_PORT.key, DEFAULT_DRIVER_PORT) private val driverBlockManagerPort = kubernetesConf.sparkConf.getInt( - org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) override def configurePod(pod: SparkPod): SparkPod = pod override def getAdditionalPodSystemProperties(): Map[String, String] = { val driverHostname = s"$resolvedServiceName.${kubernetesConf.namespace}.svc" Map(DRIVER_HOST_KEY -> driverHostname, - "spark.driver.port" -> driverPort.toString, - org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key -> - driverBlockManagerPort.toString) + config.DRIVER_PORT.key -> driverPort.toString, + config.DRIVER_BLOCK_MANAGER_PORT.key -> driverBlockManagerPort.toString) } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { @@ -90,8 +89,8 @@ private[spark] class DriverServiceFeatureStep( } private[spark] object DriverServiceFeatureStep { - val DRIVER_BIND_ADDRESS_KEY = org.apache.spark.internal.config.DRIVER_BIND_ADDRESS.key - val DRIVER_HOST_KEY = org.apache.spark.internal.config.DRIVER_HOST_ADDRESS.key + val DRIVER_BIND_ADDRESS_KEY = config.DRIVER_BIND_ADDRESS.key + val DRIVER_HOST_KEY = config.DRIVER_HOST_ADDRESS.key val DRIVER_SVC_POSTFIX = "-driver-svc" val MAX_SERVICE_NAME_LENGTH = 63 } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 5ceb9d6d6fcd..27d59dd7f3e5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -46,7 +46,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { test("Check the pod respects all configurations from the user.") { val sparkConf = new SparkConf() .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") - .set("spark.driver.cores", "2") + .set(DRIVER_CORES, 2) .set(KUBERNETES_DRIVER_LIMIT_CORES, "4") .set(DRIVER_MEMORY.key, "256M") .set(DRIVER_MEMORY_OVERHEAD, 200L) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index e28c650a571e..36bfb7d41ec3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config import org.apache.spark.internal.config.Python._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -74,8 +74,8 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, RESOURCE_NAME_PREFIX) .set(CONTAINER_IMAGE, EXECUTOR_IMAGE) .set(KUBERNETES_DRIVER_SUBMIT_CHECK, true) - .set(DRIVER_HOST_ADDRESS, DRIVER_HOSTNAME) - .set("spark.driver.port", DRIVER_PORT.toString) + .set(config.DRIVER_HOST_ADDRESS, DRIVER_HOSTNAME) + .set(config.DRIVER_PORT, DRIVER_PORT) .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS) .set("spark.kubernetes.resource.type", "java") } @@ -125,8 +125,8 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("classpath and extra java options get translated into environment variables") { - baseConf.set(EXECUTOR_JAVA_OPTIONS, "foo=bar") - baseConf.set(EXECUTOR_CLASS_PATH, "bar=baz") + baseConf.set(config.EXECUTOR_JAVA_OPTIONS, "foo=bar") + baseConf.set(config.EXECUTOR_CLASS_PATH, "bar=baz") val kconf = newExecutorConf(environment = Map("qux" -> "quux")) val step = new BasicExecutorFeatureStep(kconf, new SecurityManager(baseConf)) val executor = step.configurePod(SparkPod.initialPod()) @@ -150,7 +150,7 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { test("auth secret propagation") { val conf = baseConf.clone() - .set(NETWORK_AUTH_ENABLED, true) + .set(config.NETWORK_AUTH_ENABLED, true) .set("spark.master", "k8s://127.0.0.1") val secMgr = new SecurityManager(conf) @@ -168,8 +168,8 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { val secretFile = new File(secretDir, "secret-file.txt") Files.write(secretFile.toPath, "some-secret".getBytes(StandardCharsets.UTF_8)) val conf = baseConf.clone() - .set(NETWORK_AUTH_ENABLED, true) - .set(AUTH_SECRET_FILE, secretFile.getAbsolutePath) + .set(config.NETWORK_AUTH_ENABLED, true) + .set(config.AUTH_SECRET_FILE, secretFile.getAbsolutePath) .set("spark.master", "k8s://127.0.0.1") val secMgr = new SecurityManager(conf) secMgr.initializeAuth() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala index 045278939dff..822f1e32968c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala @@ -39,7 +39,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite { test("Headless service has a port for the driver RPC and the block manager.") { val sparkConf = new SparkConf(false) - .set("spark.driver.port", "9000") + .set(DRIVER_PORT, 9000) .set(DRIVER_BLOCK_MANAGER_PORT, 8080) val kconf = KubernetesTestConf.createDriverConf( sparkConf = sparkConf, @@ -61,7 +61,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite { test("Hostname and ports are set according to the service name.") { val sparkConf = new SparkConf(false) - .set("spark.driver.port", "9000") + .set(DRIVER_PORT, 9000) .set(DRIVER_BLOCK_MANAGER_PORT, 8080) .set(KUBERNETES_NAMESPACE, "my-namespace") val kconf = KubernetesTestConf.createDriverConf( @@ -87,7 +87,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite { s"${kconf.resourceNamePrefix}${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", resolvedService) val additionalProps = configurationStep.getAdditionalPodSystemProperties() - assert(additionalProps("spark.driver.port") === DEFAULT_DRIVER_PORT.toString) + assert(additionalProps(DRIVER_PORT.key) === DEFAULT_DRIVER_PORT.toString) assert(additionalProps(DRIVER_BLOCK_MANAGER_PORT.key) === DEFAULT_BLOCKMANAGER_PORT.toString) } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala index d134847dc74d..dd0b2bad1ecb 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -129,4 +129,7 @@ package object config { "when launching drivers. Default is to accept all offers with sufficient resources.") .stringConf .createWithDefault("") + + private[spark] val EXECUTOR_URI = + ConfigBuilder("spark.executor.uri").stringConf.createOptional } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index 68f6921153d8..a4aba3e9c0d0 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -27,6 +27,7 @@ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.deploy.Command import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest._ +import org.apache.spark.internal.config import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler import org.apache.spark.util.Utils @@ -92,12 +93,12 @@ private[mesos] class MesosSubmitRequestServlet( // Optional fields val sparkProperties = request.sparkProperties - val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") - val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") - val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") - val superviseDriver = sparkProperties.get("spark.driver.supervise") - val driverMemory = sparkProperties.get("spark.driver.memory") - val driverCores = sparkProperties.get("spark.driver.cores") + val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) + val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) + val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) + val superviseDriver = sparkProperties.get(config.DRIVER_SUPERVISE.key) + val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) + val driverCores = sparkProperties.get(config.DRIVER_CORES.key) val name = request.sparkProperties.getOrElse("spark.app.name", mainClass) // Construct driver description 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 cb1bcba651be..021b1ac84805 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 @@ -32,6 +32,7 @@ import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState} import org.apache.spark.deploy.mesos.{config, MesosDriverDescription} import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} +import org.apache.spark.internal.config.{CORES_MAX, EXECUTOR_LIBRARY_PATH, EXECUTOR_MEMORY} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.Utils @@ -365,8 +366,7 @@ private[spark] class MesosClusterScheduler( } private def getDriverExecutorURI(desc: MesosDriverDescription): Option[String] = { - desc.conf.getOption("spark.executor.uri") - .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) + desc.conf.get(config.EXECUTOR_URI).orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) } private def getDriverFrameworkID(desc: MesosDriverDescription): String = { @@ -474,7 +474,7 @@ private[spark] class MesosClusterScheduler( } else if (executorUri.isDefined) { val folderBasename = executorUri.get.split('/').last.split('.').head - val entries = conf.getOption("spark.executor.extraLibraryPath") + val entries = conf.get(EXECUTOR_LIBRARY_PATH) .map(path => Seq(path) ++ desc.command.libraryPathEntries) .getOrElse(desc.command.libraryPathEntries) @@ -528,10 +528,10 @@ private[spark] class MesosClusterScheduler( options ++= Seq("--class", desc.command.mainClass) } - desc.conf.getOption("spark.executor.memory").foreach { v => + desc.conf.getOption(EXECUTOR_MEMORY.key).foreach { v => options ++= Seq("--executor-memory", v) } - desc.conf.getOption("spark.cores.max").foreach { v => + desc.conf.getOption(CORES_MAX.key).foreach { v => options ++= Seq("--total-executor-cores", v) } desc.conf.getOption("spark.submit.pyFiles").foreach { pyFiles => 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 f5866651dc90..d0174516c236 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 @@ -33,7 +33,6 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkExceptio import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config -import org.apache.spark.internal.config.EXECUTOR_HEARTBEAT_INTERVAL import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient @@ -63,9 +62,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // Blacklist a slave after this many failures private val MAX_SLAVE_FAILURES = 2 - private val maxCoresOption = conf.getOption("spark.cores.max").map(_.toInt) + private val maxCoresOption = conf.get(config.CORES_MAX) - private val executorCoresOption = conf.getOption("spark.executor.cores").map(_.toInt) + private val executorCoresOption = conf.getOption(config.EXECUTOR_CORES.key).map(_.toInt) private val minCoresPerExecutor = executorCoresOption.getOrElse(1) @@ -220,18 +219,18 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = { val environment = Environment.newBuilder() - val extraClassPath = conf.getOption("spark.executor.extraClassPath") + val extraClassPath = conf.get(config.EXECUTOR_CLASS_PATH) extraClassPath.foreach { cp => environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_EXECUTOR_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions").map { + val extraJavaOpts = conf.get(config.EXECUTOR_JAVA_OPTIONS).map { Utils.substituteAppNExecIds(_, appId, taskId) }.getOrElse("") // Set the environment variable through a command prefix // to append to the existing value of the variable - val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => + val prefixEnv = conf.get(config.EXECUTOR_LIBRARY_PATH).map { p => Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") @@ -261,8 +260,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = conf.getOption("spark.executor.uri") - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + val uri = conf.get(EXECUTOR_URI).orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) if (uri.isEmpty) { val executorSparkHome = conf.getOption("spark.mesos.executor.home") @@ -304,8 +302,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( "driverURL" } else { RpcEndpointAddress( - conf.get("spark.driver.host"), - conf.get("spark.driver.port").toInt, + conf.get(config.DRIVER_HOST_ADDRESS), + conf.get(config.DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString } } @@ -633,7 +631,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( externalShufflePort, sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", s"${sc.conf.getTimeAsSeconds("spark.network.timeout", "120s")}s"), - sc.conf.get(EXECUTOR_HEARTBEAT_INTERVAL)) + sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)) slave.shuffleRegistered = true } 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 0bb6fe0fa4bd..192f9407a1ba 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 @@ -28,8 +28,9 @@ import org.apache.mesos.SchedulerDriver import org.apache.mesos.protobuf.ByteString import org.apache.spark.{SparkContext, SparkException, TaskState} -import org.apache.spark.deploy.mesos.config +import org.apache.spark.deploy.mesos.config.EXECUTOR_URI import org.apache.spark.executor.MesosExecutorBackend +import org.apache.spark.internal.config import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils @@ -107,15 +108,15 @@ private[spark] class MesosFineGrainedSchedulerBackend( throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") } val environment = Environment.newBuilder() - sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => + sc.conf.get(config.EXECUTOR_CLASS_PATH).foreach { cp => environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_EXECUTOR_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").map { + val extraJavaOpts = sc.conf.get(config.EXECUTOR_JAVA_OPTIONS).map { Utils.substituteAppNExecIds(_, appId, execId) }.getOrElse("") - val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => + val prefixEnv = sc.conf.get(config.EXECUTOR_LIBRARY_PATH).map { p => Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") @@ -132,8 +133,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = sc.conf.getOption("spark.executor.uri") - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + val uri = sc.conf.get(EXECUTOR_URI).orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) val executorBackendName = classOf[MesosExecutorBackend].getName if (uri.isEmpty) { 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 e46c4f970c4a..8dbdac168f70 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 @@ -470,8 +470,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends rpcEnv = sc.env.rpcEnv val userConf = sc.getConf - val host = userConf.get("spark.driver.host") - val port = userConf.get("spark.driver.port").toInt + val host = userConf.get(DRIVER_HOST_ADDRESS) + val port = userConf.get(DRIVER_PORT) registerAM(host, port, userConf, sc.ui.map(_.webUrl)) val driverRef = rpcEnv.setupEndpointRef( @@ -505,7 +505,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends amCores, true) // The client-mode AM doesn't listen for incoming connections, so report an invalid port. - registerAM(hostname, -1, sparkConf, sparkConf.getOption("spark.driver.appUIAddress")) + registerAM(hostname, -1, sparkConf, sparkConf.get(DRIVER_APP_UI_ADDRESS)) // The driver should be up and listening, so unlike cluster mode, just try to connect to it // with no waiting or retrying. 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 b257d8fdd3b1..7e9cd409daf3 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 @@ -224,16 +224,12 @@ package object config { /* Driver configuration. */ - private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores") - .intConf - .createWithDefault(1) + private[spark] val DRIVER_APP_UI_ADDRESS = ConfigBuilder("spark.driver.appUIAddress") + .stringConf + .createOptional /* Executor configuration. */ - private[spark] val EXECUTOR_CORES = ConfigBuilder("spark.executor.cores") - .intConf - .createWithDefault(1) - private[spark] val EXECUTOR_NODE_LABEL_EXPRESSION = ConfigBuilder("spark.yarn.executor.nodeLabelExpression") .doc("Node label expression for executors.") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 9397a1e3de9a..167eef19ed85 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnAppReport} import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.SparkAppHandle import org.apache.spark.scheduler.TaskSchedulerImpl @@ -42,10 +42,10 @@ private[spark] class YarnClientSchedulerBackend( * This waits until the application is running. */ override def start() { - val driverHost = conf.get("spark.driver.host") - val driverPort = conf.get("spark.driver.port") + val driverHost = conf.get(config.DRIVER_HOST_ADDRESS) + val driverPort = conf.get(config.DRIVER_PORT) val hostport = driverHost + ":" + driverPort - sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.webUrl) } + sc.ui.foreach { ui => conf.set(DRIVER_APP_UI_ADDRESS, ui.webUrl) } val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ("--arg", hostport) 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 8032213602c9..9e3cc6ec01df 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 @@ -24,7 +24,7 @@ import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy.yarn.ResourceRequestTestHelper.ResourceInformation import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_MEMORY} +import org.apache.spark.internal.config.{DRIVER_CORES, DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_MEMORY} class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { From 36440e64476610ec4037fb14f50cf7f06495e384 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 4 Jan 2019 15:35:23 -0600 Subject: [PATCH 0227/1072] [SPARK-26306][TEST][BUILD] More memory to de-flake SorterSuite ## What changes were proposed in this pull request? Increase test memory to avoid OOM in TimSort-related tests. ## How was this patch tested? Existing tests. Closes #23425 from srowen/SPARK-26306. Authored-by: Sean Owen Signed-off-by: Sean Owen --- pom.xml | 4 ++-- project/SparkBuild.scala | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- sql/hive/pom.xml | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index a433659cd200..d0c525adf355 100644 --- a/pom.xml +++ b/pom.xml @@ -2115,7 +2115,7 @@ **/*Suite.java ${project.build.directory}/surefire-reports - -ea -Xmx3g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} - -da -Xmx3g -XX:ReservedCodeCacheSize=${CodeCacheSize} + -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} From 89cebf4932ff966cc876ba8a9ecd9d9c034fb071 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 4 Jan 2019 15:37:09 -0600 Subject: [PATCH 0228/1072] [SPARK-24421][CORE][FOLLOWUP] Use normal direct ByteBuffer allocation if Cleaner can't be set ## What changes were proposed in this pull request? In Java 9+ we can't use sun.misc.Cleaner by default anymore, and this was largely handled in https://github.com/apache/spark/pull/22993 However I think the change there left a significant problem. If a DirectByteBuffer is allocated using the reflective hack in Platform, now, we by default can't set a Cleaner. But I believe this means the memory isn't freed promptly or possibly at all. If a Cleaner can't be set, I think we need to use normal APIs to allocate the direct ByteBuffer. According to comments in the code, the downside is simply that the normal APIs will check and impose limits on how much off-heap memory can be allocated. Per the original review on https://github.com/apache/spark/pull/22993 this much seems fine, as either way in this case the user would have to add a JVM setting (increase max, or allow the reflective access). ## How was this patch tested? Existing tests. This resolved an OutOfMemoryError in Java 11 from TimSort tests without increasing test heap size. (See https://github.com/apache/spark/pull/23419#issuecomment-450772125 ) This suggests there is a problem and that this resolves it. Closes #23424 from srowen/SPARK-24421.2. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../org/apache/spark/unsafe/Platform.java | 31 +++++++++++++------ 1 file changed, 21 insertions(+), 10 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index 076b693f81c8..1adf7abfc8a6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -209,22 +209,33 @@ public static long reallocateMemory(long address, long oldSize, long newSize) { } /** - * Uses internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's - * MaxDirectMemorySize limit (the default limit is too low and we do not want to require users - * to increase it). + * Allocate a DirectByteBuffer, potentially bypassing the JVM's MaxDirectMemorySize limit. */ public static ByteBuffer allocateDirectBuffer(int size) { try { - long memory = allocateMemory(size); - ByteBuffer buffer = (ByteBuffer) DBB_CONSTRUCTOR.newInstance(memory, size); - if (CLEANER_CREATE_METHOD != null) { + if (CLEANER_CREATE_METHOD == null) { + // Can't set a Cleaner (see comments on field), so need to allocate via normal Java APIs try { - DBB_CLEANER_FIELD.set(buffer, - CLEANER_CREATE_METHOD.invoke(null, buffer, (Runnable) () -> freeMemory(memory))); - } catch (IllegalAccessException | InvocationTargetException e) { - throw new IllegalStateException(e); + return ByteBuffer.allocateDirect(size); + } catch (OutOfMemoryError oome) { + // checkstyle.off: RegexpSinglelineJava + throw new OutOfMemoryError("Failed to allocate direct buffer (" + oome.getMessage() + + "); try increasing -XX:MaxDirectMemorySize=... to, for example, your heap size"); + // checkstyle.on: RegexpSinglelineJava } } + // Otherwise, use internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's + // MaxDirectMemorySize limit (the default limit is too low and we do not want to + // require users to increase it). + long memory = allocateMemory(size); + ByteBuffer buffer = (ByteBuffer) DBB_CONSTRUCTOR.newInstance(memory, size); + try { + DBB_CLEANER_FIELD.set(buffer, + CLEANER_CREATE_METHOD.invoke(null, buffer, (Runnable) () -> freeMemory(memory))); + } catch (IllegalAccessException | InvocationTargetException e) { + freeMemory(memory); + throw new IllegalStateException(e); + } return buffer; } catch (Exception e) { throwException(e); From bccb8602d7bc78894689e9b2e5fe685763d32d23 Mon Sep 17 00:00:00 2001 From: shane knapp Date: Fri, 4 Jan 2019 18:27:26 -0800 Subject: [PATCH 0229/1072] [SPARK-26537][BUILD] change git-wip-us to gitbox ## What changes were proposed in this pull request? due to apache recently moving from git-wip-us.apache.org to gitbox.apache.org, we need to update the packaging scripts to point to the new repo location. this will also need to be backported to 2.4, 2.3, 2.1, 2.0 and 1.6. ## How was this patch tested? the build system will test this. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23454 from shaneknapp/update-apache-repo. Authored-by: shane knapp Signed-off-by: Dongjoon Hyun --- dev/create-release/release-tag.sh | 2 +- dev/create-release/release-util.sh | 4 ++-- pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) mode change 100644 => 100755 dev/create-release/release-util.sh diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index 628bc0504c9c..010082d960a2 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -61,7 +61,7 @@ done init_java init_maven_sbt -ASF_SPARK_REPO="git-wip-us.apache.org/repos/asf/spark.git" +ASF_SPARK_REPO="gitbox.apache.org/repos/asf/spark.git" rm -rf spark git clone "https://$ASF_USERNAME:$ASF_PASSWORD@$ASF_SPARK_REPO" -b $GIT_BRANCH diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh old mode 100644 new mode 100755 index 7426b0d6ca08..c925de9be52d --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -19,8 +19,8 @@ DRY_RUN=${DRY_RUN:-0} GPG="gpg --no-tty --batch" -ASF_REPO="https://git-wip-us.apache.org/repos/asf/spark.git" -ASF_REPO_WEBUI="https://git-wip-us.apache.org/repos/asf?p=spark.git" +ASF_REPO="https://gitbox.apache.org/repos/asf/spark.git" +ASF_REPO_WEBUI="https://gitbox.apache.org/repos/asf?p=spark.git" function error { echo "$*" diff --git a/pom.xml b/pom.xml index d0c525adf355..40b0e328c035 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ scm:git:git@github.com:apache/spark.git - scm:git:https://git-wip-us.apache.org/repos/asf/spark.git + scm:git:https://gitbox.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git HEAD From e15a319ccd1125584c09c38ca90b252324df6998 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 4 Jan 2019 19:23:38 -0800 Subject: [PATCH 0230/1072] [SPARK-26536][BUILD][TEST] Upgrade Mockito to 2.23.4 ## What changes were proposed in this pull request? This PR upgrades Mockito from 1.10.19 to 2.23.4. The following changes are required. - Replace `org.mockito.Matchers` with `org.mockito.ArgumentMatchers` - Replace `anyObject` with `any` - Replace `getArgumentAt` with `getArgument` and add type annotation. - Use `isNull` matcher in case of `null` is invoked. ```scala saslHandler.channelInactive(null); - verify(handler).channelInactive(any(TransportClient.class)); + verify(handler).channelInactive(isNull()); ``` - Make and use `doReturn` wrapper to avoid [SI-4775](https://issues.scala-lang.org/browse/SI-4775) ```scala private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) ``` ## How was this patch tested? Pass the Jenkins with the existing tests. Closes #23452 from dongjoon-hyun/SPARK-26536. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../spark/network/sasl/SparkSaslSuite.java | 4 ++-- .../ExternalShuffleBlockHandlerSuite.java | 4 +++- .../shuffle/OneForOneBlockFetcherSuite.java | 8 ++++---- .../unsafe/map/AbstractBytesToBytesMapSuite.java | 4 ++-- .../spark/ExecutorAllocationManagerSuite.scala | 2 +- .../apache/spark/HeartbeatReceiverSuite.scala | 13 ++++++------- .../org/apache/spark/MapOutputTrackerSuite.scala | 2 +- .../StandaloneDynamicAllocationSuite.scala | 2 +- .../deploy/history/ApplicationCacheSuite.scala | 2 +- .../deploy/history/FsHistoryProviderSuite.scala | 4 ++-- .../history/HistoryServerDiskManagerSuite.scala | 6 ++++-- .../spark/deploy/worker/DriverRunnerTest.scala | 2 +- .../apache/spark/deploy/worker/WorkerSuite.scala | 2 +- .../apache/spark/executor/ExecutorSuite.scala | 2 +- .../apache/spark/memory/MemoryManagerSuite.scala | 2 +- .../scala/org/apache/spark/rpc/RpcEnvSuite.scala | 2 +- .../spark/rpc/netty/NettyRpcHandlerSuite.scala | 2 +- .../spark/scheduler/BlacklistTrackerSuite.scala | 2 +- .../apache/spark/scheduler/MapStatusSuite.scala | 3 ++- .../scheduler/OutputCommitCoordinatorSuite.scala | 16 +++++++++------- .../spark/scheduler/TaskContextSuite.scala | 2 +- .../spark/scheduler/TaskResultGetterSuite.scala | 2 +- .../spark/scheduler/TaskSchedulerImplSuite.scala | 10 +++++----- .../spark/scheduler/TaskSetBlacklistSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 4 ++-- .../spark/security/CryptoStreamUtilsSuite.scala | 2 +- .../sort/BypassMergeSortShuffleWriterSuite.scala | 2 +- .../sort/IndexShuffleBlockResolverSuite.scala | 2 +- .../shuffle/sort/SortShuffleManagerSuite.scala | 4 +++- .../apache/spark/storage/BlockManagerSuite.scala | 2 +- .../storage/PartiallyUnrolledIteratorSuite.scala | 4 ++-- .../ShuffleBlockFetcherIteratorSuite.scala | 7 +++++-- .../sql/kafka010/KafkaDelegationTokenTest.scala | 4 +++- .../kinesis/KinesisCheckpointerSuite.scala | 2 +- .../streaming/kinesis/KinesisReceiverSuite.scala | 5 ++--- .../launcher/SparkSubmitOptionParserSuite.java | 6 +++++- .../org/apache/spark/ml/PipelineSuite.scala | 2 +- pom.xml | 2 +- .../spark/repl/ExecutorClassLoaderSuite.scala | 2 +- .../spark/deploy/k8s/PodBuilderSuite.scala | 2 +- .../features/KubernetesFeaturesTestUtils.scala | 8 ++++---- .../spark/deploy/k8s/submit/ClientSuite.scala | 4 +++- .../cluster/k8s/ExecutorPodsAllocatorSuite.scala | 4 ++-- .../k8s/ExecutorPodsLifecycleManagerSuite.scala | 4 ++-- .../KubernetesClusterSchedulerBackendSuite.scala | 2 +- .../mesos/MesosClusterSchedulerSuite.scala | 7 ++++--- ...MesosCoarseGrainedSchedulerBackendSuite.scala | 9 ++++----- .../MesosFineGrainedSchedulerBackendSuite.scala | 12 ++++++------ .../spark/scheduler/cluster/mesos/Utils.scala | 11 ++++++----- .../apache/spark/deploy/yarn/ClientSuite.scala | 5 +++-- .../yarn/YarnShuffleServiceMetricsSuite.scala | 8 ++++---- .../continuous/EpochCoordinatorSuite.scala | 4 ++-- .../test/DataStreamReaderWriterSuite.scala | 2 +- .../streaming/ReceivedBlockTrackerSuite.scala | 2 +- .../ExecutorAllocationManagerSuite.scala | 4 ++-- .../streaming/util/WriteAheadLogSuite.scala | 4 ++-- 56 files changed, 131 insertions(+), 111 deletions(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index 6f15718bd870..59adf9704cbf 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -347,10 +347,10 @@ public void testRpcHandlerDelegate() throws Exception { verify(handler).getStreamManager(); saslHandler.channelInactive(null); - verify(handler).channelInactive(any(TransportClient.class)); + verify(handler).channelInactive(isNull()); saslHandler.exceptionCaught(null, null); - verify(handler).exceptionCaught(any(Throwable.class), any(TransportClient.class)); + verify(handler).exceptionCaught(isNull(), isNull()); } @Test diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 7846b71d5a8b..4cc9a16e1449 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -27,7 +27,7 @@ import org.mockito.ArgumentCaptor; import static org.junit.Assert.*; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.*; import org.apache.spark.network.buffer.ManagedBuffer; @@ -79,6 +79,8 @@ public void testRegisterExecutor() { @SuppressWarnings("unchecked") @Test public void testOpenShuffleBlocks() { + when(client.getClientId()).thenReturn("app0"); + RpcResponseCallback callback = mock(RpcResponseCallback.class); ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index dc947a619bf0..95460637db89 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -28,10 +28,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index e5fbafc23d95..ecfebf8f8287 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -50,8 +50,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.mockito.Answers.RETURNS_SMART_NULLS; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.when; diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index d0389235cb72..38f5e8c9f0ac 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import scala.collection.mutable -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{mock, never, verify, when} import org.scalatest.{BeforeAndAfter, PrivateMethodTester} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index de479db5fbc0..a69e589743ef 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -23,8 +23,7 @@ import scala.collection.mutable import scala.concurrent.Future import scala.concurrent.duration._ -import org.mockito.Matchers -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} @@ -151,7 +150,7 @@ class HeartbeatReceiverSuite heartbeatReceiverClock.advance(executorTimeout) heartbeatReceiverRef.askSync[Boolean](ExpireDeadHosts) // Only the second executor should be expired as a dead host - verify(scheduler).executorLost(Matchers.eq(executorId2), any()) + verify(scheduler).executorLost(meq(executorId2), any()) val trackedExecutors = getTrackedExecutors assert(trackedExecutors.size === 1) assert(trackedExecutors.contains(executorId1)) @@ -223,10 +222,10 @@ class HeartbeatReceiverSuite assert(!response.reregisterBlockManager) // Additionally verify that the scheduler callback is called with the correct parameters verify(scheduler).executorHeartbeatReceived( - Matchers.eq(executorId), - Matchers.eq(Array(1L -> metrics.accumulators())), - Matchers.eq(blockManagerId), - Matchers.eq(executorUpdates)) + meq(executorId), + meq(Array(1L -> metrics.accumulators())), + meq(blockManagerId), + meq(executorUpdates)) } } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 3e1a3d4f7306..c088da8fbf3b 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.apache.spark.LocalSparkContext._ 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 8567dd1f0823..8c3c38dbc7ea 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy import scala.collection.mutable import scala.concurrent.duration._ -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, verify, when} import org.scalatest.{BeforeAndAfterAll, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ 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 44f9c566a380..0402d949e904 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 @@ -24,7 +24,7 @@ import scala.collection.mutable import com.codahale.metrics.Counter import org.eclipse.jetty.servlet.ServletContextHandler -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer 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 6d2e329094ae..7d6efd95fbab 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,7 +34,7 @@ import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} import org.apache.hadoop.security.AccessControlException import org.json4s.jackson.JsonMethods._ import org.mockito.ArgumentMatcher -import org.mockito.Matchers.{any, argThat} +import org.mockito.ArgumentMatchers.{any, argThat} import org.mockito.Mockito.{doThrow, mock, spy, verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.Matchers @@ -933,7 +933,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val mockedFs = spy(provider.fs) doThrow(new AccessControlException("Cannot read accessDenied file")).when(mockedFs).open( argThat(new ArgumentMatcher[Path]() { - override def matches(path: Any): Boolean = { + override def matches(path: Path): Boolean = { path.asInstanceOf[Path].getName.toLowerCase(Locale.ROOT) == "accessdenied" } })) 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 341a1e2443df..f78469e13249 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 @@ -20,8 +20,8 @@ package org.apache.spark.deploy.history import java.io.File import org.mockito.AdditionalAnswers -import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{anyBoolean, anyLong, eq => meq} +import org.mockito.Mockito.{doAnswer, spy} import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} @@ -32,6 +32,8 @@ import org.apache.spark.util.kvstore.KVStore class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + private val MAX_USAGE = 3L private var testDir: File = _ diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index 52956045d598..1deac43897f9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -21,7 +21,7 @@ import java.io.File import scala.concurrent.duration._ -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer 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 e3fe2b696aa1..e5e5b5e428c4 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 @@ -22,7 +22,7 @@ import java.util.function.Supplier import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer 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 32a94e60484e..a5fe2026c0f7 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -30,7 +30,7 @@ import scala.concurrent.duration._ import scala.language.postfixOps import org.mockito.ArgumentCaptor -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{inOrder, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 85eeb5055ae0..8b35f1dfddb0 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration -import org.mockito.Matchers.{any, anyLong} +import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 5cb2b561d6bc..558b7fa49832 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -29,7 +29,7 @@ import scala.concurrent.duration._ import scala.language.postfixOps import com.google.common.io.Files -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, never, verify, when} import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala index a71d8726e706..4bc001fe8f7c 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala @@ -21,7 +21,7 @@ import java.net.InetSocketAddress import java.nio.ByteBuffer import io.netty.channel.Channel -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 96c8404327e2..aea4c5f96bbe 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{never, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 2155a0f2b6c2..f41ffb7f2c0b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -21,7 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, import scala.util.Random -import org.mockito.Mockito._ +import org.mockito.Mockito.mock import org.roaringbitmap.RoaringBitmap import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} @@ -31,6 +31,7 @@ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.storage.BlockManagerId class MapStatusSuite extends SparkFunSuite { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) test("compressSize") { assert(MapStatus.compressSize(0L) === 0) diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 158c9eb75f2b..a560013dba96 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -26,8 +26,8 @@ import scala.language.postfixOps import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.TaskType -import org.mockito.Matchers -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.{doAnswer, spy, times, verify} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter @@ -71,6 +71,8 @@ import org.apache.spark.util.{ThreadUtils, Utils} */ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + var outputCommitCoordinator: OutputCommitCoordinator = null var tempDir: File = null var sc: SparkContext = null @@ -103,7 +105,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { invoke.callRealMethod() mockTaskScheduler.backend.reviveOffers() } - }).when(mockTaskScheduler).submitTasks(Matchers.any()) + }).when(mockTaskScheduler).submitTasks(any()) doAnswer(new Answer[TaskSetManager]() { override def answer(invoke: InvocationOnMock): TaskSetManager = { @@ -123,7 +125,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { } } } - }).when(mockTaskScheduler).createTaskSetManager(Matchers.any(), Matchers.any()) + }).when(mockTaskScheduler).createTaskSetManager(any(), any()) sc.taskScheduler = mockTaskScheduler val dagSchedulerWithMockTaskScheduler = new DAGScheduler(sc, mockTaskScheduler) @@ -154,7 +156,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { test("Job should not complete if all commits are denied") { // Create a mock OutputCommitCoordinator that denies all attempts to commit doReturn(false).when(outputCommitCoordinator).handleAskPermissionToCommit( - Matchers.any(), Matchers.any(), Matchers.any(), Matchers.any()) + any(), any(), any(), any()) val rdd: RDD[Int] = sc.parallelize(Seq(1), 1) def resultHandler(x: Int, y: Unit): Unit = {} val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd, @@ -268,8 +270,8 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { assert(retriedStage.size === 1) assert(sc.dagScheduler.outputCommitCoordinator.isEmpty) verify(sc.env.outputCommitCoordinator, times(2)) - .stageStart(Matchers.eq(retriedStage.head), Matchers.any()) - verify(sc.env.outputCommitCoordinator).stageEnd(Matchers.eq(retriedStage.head)) + .stageStart(meq(retriedStage.head), any()) + verify(sc.env.outputCommitCoordinator).stageEnd(meq(retriedStage.head)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index aa9c36c0aaac..3bfc97b80184 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.util.Properties -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter 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 efb8b15cf6b4..ea1439cfebca 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import com.google.common.util.concurrent.MoreExecutors import org.mockito.ArgumentCaptor -import org.mockito.Matchers.{any, anyLong} +import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{spy, times, verify} import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ 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 29172b4664e3..9c555a923d62 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.HashMap import scala.concurrent.duration._ -import org.mockito.Matchers.{anyInt, anyObject, anyString, eq => meq} +import org.mockito.ArgumentMatchers.{any, anyInt, anyString, eq => meq} import org.mockito.Mockito.{atLeast, atMost, never, spy, times, verify, when} import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually @@ -430,7 +430,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B verify(blacklist, never).updateBlacklistForSuccessfulTaskSet( stageId = meq(2), stageAttemptId = anyInt(), - failuresByExec = anyObject()) + failuresByExec = any()) } test("scheduled tasks obey node and executor blacklists") { @@ -504,7 +504,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B WorkerOffer("executor3", "host1", 2) )).flatten.size === 0) assert(tsm.isZombie) - verify(tsm).abort(anyString(), anyObject()) + verify(tsm).abort(anyString(), any()) } test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " + @@ -1184,7 +1184,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(finalTsm.isZombie) // no taskset has completed all of its tasks, so no updates to the blacklist tracker yet - verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(anyInt(), anyInt(), anyObject()) + verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(anyInt(), anyInt(), any()) // finally, lets complete all the tasks. We simulate failures in attempt 1, but everything // else succeeds, to make sure we get the right updates to the blacklist in all cases. @@ -1202,7 +1202,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // we update the blacklist for the stage attempts with all successful tasks. Even though // some tasksets had failures, we still consider them all successful from a blacklisting // perspective, as the failures weren't from a problem w/ the tasks themselves. - verify(blacklist).updateBlacklistForSuccessfulTaskSet(meq(0), meq(stageAttempt), anyObject()) + verify(blacklist).updateBlacklistForSuccessfulTaskSet(meq(0), meq(stageAttempt), any()) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala index 6e2709dbe1e8..b3bc76687ce1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.scheduler -import org.mockito.Matchers.isA +import org.mockito.ArgumentMatchers.isA import org.mockito.Mockito.{never, verify} import org.scalatest.BeforeAndAfterEach import org.scalatest.mockito.MockitoSugar 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 f73ff67837c6..f9dfd2c456c5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,7 +22,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.Matchers.{any, anyInt, anyString} +import org.mockito.ArgumentMatchers.{any, anyInt, anyString} import org.mockito.Mockito.{mock, never, spy, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -1319,7 +1319,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg when(taskSetManagerSpy.addPendingTask(anyInt())).thenAnswer( new Answer[Unit] { override def answer(invocationOnMock: InvocationOnMock): Unit = { - val task = invocationOnMock.getArgumentAt(0, classOf[Int]) + val task: Int = invocationOnMock.getArgument(0) assert(taskSetManager.taskSetBlacklistHelperOpt.get. isExecutorBlacklistedForTask(exec, task)) } diff --git a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala index 0d3611c80b8d..e5d1bf4fde9e 100644 --- a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala @@ -24,7 +24,7 @@ import java.nio.file.Files import java.util.{Arrays, Random, UUID} import com.google.common.io.ByteStreams -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.apache.spark._ diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 4467c3241a94..7f956c26d0ff 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer 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 4ce379b76b55..0154d0b6ef6f 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 @@ -21,7 +21,7 @@ import java.io.{DataInputStream, File, FileInputStream, FileOutputStream} import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer 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 f29dac965c80..e5f3aab6a6a1 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 @@ -17,7 +17,7 @@ package org.apache.spark.shuffle.sort -import org.mockito.Mockito._ +import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.Matchers @@ -31,6 +31,8 @@ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, Serializer} */ class SortShuffleManagerSuite extends SparkFunSuite with Matchers { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + import SortShuffleManager.canUseSerializedShuffle private class RuntimeExceptionAnswer extends Answer[Object] { 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 e866342e4472..a7bb2a03360a 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -27,7 +27,7 @@ import scala.language.{implicitConversions, postfixOps} import scala.reflect.ClassTag import org.apache.commons.lang3.RandomUtils -import org.mockito.{Matchers => mc} +import org.mockito.{ArgumentMatchers => mc} import org.mockito.Mockito.{mock, times, verify, when} import org.scalatest._ import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala index cbc903f17ad7..56860b2e5570 100644 --- a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import org.mockito.Matchers +import org.mockito.ArgumentMatchers.{eq => meq} import org.mockito.Mockito._ import org.scalatest.mockito.MockitoSugar @@ -45,7 +45,7 @@ class PartiallyUnrolledIteratorSuite extends SparkFunSuite with MockitoSugar { joinIterator.hasNext joinIterator.hasNext verify(memoryStore, times(1)) - .releaseUnrollMemoryForThisTask(Matchers.eq(ON_HEAP), Matchers.eq(unrollSize.toLong)) + .releaseUnrollMemoryForThisTask(meq(ON_HEAP), meq(unrollSize.toLong)) // Secondly, iterate over rest iterator (unrollSize until unrollSize + restSize).foreach { value => 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 01ee9ef0825f..6b83243fe496 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -24,8 +24,8 @@ import java.util.concurrent.Semaphore import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future -import org.mockito.Matchers.{any, eq => meq} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.{mock, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.PrivateMethodTester @@ -40,6 +40,9 @@ import org.apache.spark.util.Utils class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodTester { + + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + // Some of the tests are quite tricky because we are testing the cleanup behavior // in the presence of faults. diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala index 1899c65c721b..31247ab21908 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala @@ -22,7 +22,7 @@ import javax.security.auth.login.{AppConfigurationEntry, Configuration} import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.Token -import org.mockito.Mockito.{doReturn, mock} +import org.mockito.Mockito.mock import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} @@ -35,6 +35,8 @@ import org.apache.spark.deploy.security.KafkaTokenUtil.KafkaDelegationTokenIdent trait KafkaDelegationTokenTest extends BeforeAndAfterEach { self: SparkFunSuite => + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + protected val tokenId = "tokenId" + ju.UUID.randomUUID().toString protected val tokenPassword = "tokenPassword" + ju.UUID.randomUUID().toString diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala index e26f4477d1d7..bd31b7dc49a6 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala @@ -24,7 +24,7 @@ import scala.concurrent.duration._ import scala.language.postfixOps import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers._ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer 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 2fadda271ea2..7531a9cc400d 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 @@ -24,9 +24,8 @@ import com.amazonaws.services.kinesis.clientlibrary.exceptions._ import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason import com.amazonaws.services.kinesis.model.Record -import org.mockito.Matchers._ -import org.mockito.Matchers.{eq => meq} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{anyListOf, anyString, eq => meq} +import org.mockito.Mockito.{never, times, verify, when} import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.mockito.MockitoSugar diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java index 9ff7aceb581f..4e26cf6c109c 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java @@ -23,6 +23,7 @@ import org.junit.Before; import org.junit.Test; +import static org.mockito.ArgumentMatchers.isNull; import static org.mockito.Mockito.*; public class SparkSubmitOptionParserSuite extends BaseSuite { @@ -48,14 +49,17 @@ public void testAllOptions() { } } + int nullCount = 0; for (String[] switchNames : parser.switches) { int switchCount = 0; for (String name : switchNames) { parser.parse(Arrays.asList(name)); count++; + nullCount++; switchCount++; verify(parser, times(switchCount)).handle(eq(switchNames[0]), same(null)); - verify(parser, times(count)).handle(anyString(), any(String.class)); + verify(parser, times(nullCount)).handle(anyString(), isNull()); + verify(parser, times(count - nullCount)).handle(anyString(), any(String.class)); verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala index 7848eae931a0..1183cb061761 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar.mock diff --git a/pom.xml b/pom.xml index 40b0e328c035..245344a82693 100644 --- a/pom.xml +++ b/pom.xml @@ -764,7 +764,7 @@ org.mockito mockito-core - 1.10.19 + 2.23.4 test diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index ac528ecb829b..e9ed01ff2233 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -30,7 +30,7 @@ import scala.io.Source import scala.language.implicitConversions import com.google.common.io.Files -import org.mockito.Matchers.anyString +import org.mockito.ArgumentMatchers.anyString import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala index 7dde0c137716..707c823d69cf 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala @@ -21,7 +21,7 @@ import java.io.File import io.fabric8.kubernetes.api.model.{Config => _, _} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, never, verify, when} import scala.collection.JavaConverters._ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala index 076b681be239..95de7d905954 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala @@ -20,8 +20,8 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import io.fabric8.kubernetes.api.model.{Container, HasMetadata, PodBuilder, SecretBuilder} -import org.mockito.Matchers -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -37,10 +37,10 @@ object KubernetesFeaturesTestUtils { when(mockStep.getAdditionalPodSystemProperties()) .thenReturn(Map(stepType -> stepType)) - when(mockStep.configurePod(Matchers.any(classOf[SparkPod]))) + when(mockStep.configurePod(any(classOf[SparkPod]))) .thenAnswer(new Answer[SparkPod]() { override def answer(invocation: InvocationOnMock): SparkPod = { - val originalPod = invocation.getArgumentAt(0, classOf[SparkPod]) + val originalPod: SparkPod = invocation.getArgument(0) val configuredPod = new PodBuilder(originalPod.pod) .editOrNewMetadata() .addToLabels(stepType, stepType) 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 1bb926cbca23..aa421be6e841 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 @@ -20,7 +20,7 @@ import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, Watch} import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} -import org.mockito.Mockito.{doReturn, verify, when} +import org.mockito.Mockito.{verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.mockito.MockitoSugar._ @@ -31,6 +31,8 @@ import org.apache.spark.deploy.k8s.Fabric8Aliases._ class ClientSuite extends SparkFunSuite with BeforeAndAfter { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + private val DRIVER_POD_UID = "pod-id" private val DRIVER_POD_API_VERSION = "v1" private val DRIVER_POD_KIND = "pod" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 278a3821a6f3..55d9adc212f9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -20,7 +20,7 @@ import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{ArgumentMatcher, Matchers, Mock, MockitoAnnotations} -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{never, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -156,7 +156,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private def executorPodAnswer(): Answer[SparkPod] = { new Answer[SparkPod] { override def answer(invocation: InvocationOnMock): SparkPod = { - val k8sConf = invocation.getArgumentAt(0, classOf[KubernetesExecutorConf]) + val k8sConf: KubernetesExecutorConf = invocation.getArgument(0) executorPodWithId(k8sConf.executorId.toInt) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala index 7411f8f9d69e..b20ed4799e32 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala @@ -21,7 +21,7 @@ import io.fabric8.kubernetes.api.model.{DoneablePod, Pod} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, never, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -128,7 +128,7 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte private def namedPodsAnswer(): Answer[PodResource[Pod, DoneablePod]] = { new Answer[PodResource[Pod, DoneablePod]] { override def answer(invocation: InvocationOnMock): PodResource[Pod, DoneablePod] = { - val podName = invocation.getArgumentAt(0, classOf[String]) + val podName: String = invocation.getArgument(0) namedExecutorPods.getOrElseUpdate( podName, mock(classOf[PodResource[Pod, DoneablePod]])) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 6e182bed459f..8ed934d91dd7 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.client.KubernetesClient import org.jmock.lib.concurrent.DeterministicScheduler import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} -import org.mockito.Matchers.{eq => mockitoEq} +import org.mockito.ArgumentMatchers.{eq => mockitoEq} import org.mockito.Mockito.{never, verify, when} import org.scalatest.BeforeAndAfter 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 082d4bcfdf83..7adac1964e01 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 @@ -24,7 +24,8 @@ import scala.collection.JavaConverters._ import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} import org.apache.mesos.Protos.Value.{Scalar, Type} import org.apache.mesos.SchedulerDriver -import org.mockito.{ArgumentCaptor, Matchers} +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{eq => meq} import org.mockito.Mockito._ import org.scalatest.mockito.MockitoSugar @@ -133,7 +134,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi when( driver.launchTasks( - Matchers.eq(Collections.singleton(offer.getId)), + meq(Collections.singleton(offer.getId)), capture.capture()) ).thenReturn(Status.valueOf(1)) @@ -156,7 +157,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi assert(mem.exists(_.getRole() == "*")) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(offer.getId)), + meq(Collections.singleton(offer.getId)), capture.capture() ) } 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 da33d85d8fb2..0cfaa0a0c9a6 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 @@ -24,9 +24,8 @@ import scala.concurrent.duration._ import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} import org.apache.mesos.Protos._ -import org.mockito.Matchers -import org.mockito.Matchers._ -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, anyInt, anyLong, anyString, eq => meq} +import org.mockito.Mockito.{times, verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.ScalaFutures import org.scalatest.mockito.MockitoSugar @@ -697,9 +696,9 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite offerId: OfferID, filter: Boolean = false): Unit = { if (filter) { - verify(driver, times(1)).declineOffer(Matchers.eq(offerId), anyObject[Filters]) + verify(driver, times(1)).declineOffer(meq(offerId), any[Filters]()) } else { - verify(driver, times(1)).declineOffer(Matchers.eq(offerId)) + verify(driver, times(1)).declineOffer(meq(offerId)) } } 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 1ead4b1ed7c7..c9b7e6c439c4 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 @@ -30,8 +30,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} import org.apache.mesos.Protos._ import org.apache.mesos.Protos.Value.Scalar -import org.mockito.{ArgumentCaptor, Matchers} -import org.mockito.Matchers._ +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} import org.mockito.Mockito._ import org.scalatest.mockito.MockitoSugar @@ -264,7 +264,7 @@ class MesosFineGrainedSchedulerBackendSuite val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) when( driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + meq(Collections.singleton(mesosOffers.get(0).getId)), capture.capture(), any(classOf[Filters]) ) @@ -275,7 +275,7 @@ class MesosFineGrainedSchedulerBackendSuite backend.resourceOffers(driver, mesosOffers) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + meq(Collections.singleton(mesosOffers.get(0).getId)), capture.capture(), any(classOf[Filters]) ) @@ -373,7 +373,7 @@ class MesosFineGrainedSchedulerBackendSuite val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) when( driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + meq(Collections.singleton(mesosOffers.get(0).getId)), capture.capture(), any(classOf[Filters]) ) @@ -382,7 +382,7 @@ class MesosFineGrainedSchedulerBackendSuite backend.resourceOffers(driver, mesosOffers) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + meq(Collections.singleton(mesosOffers.get(0).getId)), capture.capture(), any(classOf[Filters]) ) 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 c9f47471cd75..65e595e3cf2b 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 @@ -25,8 +25,9 @@ import org.apache.mesos.Protos._ import org.apache.mesos.Protos.Value.{Range => MesosRange, Ranges, Scalar} import org.apache.mesos.SchedulerDriver import org.apache.mesos.protobuf.ByteString -import org.mockito.{ArgumentCaptor, Matchers} -import org.mockito.Mockito._ +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.{times, verify} import org.apache.spark.deploy.mesos.config.MesosSecretConfig @@ -84,15 +85,15 @@ object Utils { def verifyTaskLaunched(driver: SchedulerDriver, offerId: String): List[TaskInfo] = { val captor = ArgumentCaptor.forClass(classOf[java.util.Collection[TaskInfo]]) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(createOfferId(offerId))), + meq(Collections.singleton(createOfferId(offerId))), captor.capture()) captor.getValue.asScala.toList } def verifyTaskNotLaunched(driver: SchedulerDriver, offerId: String): Unit = { verify(driver, times(0)).launchTasks( - Matchers.eq(Collections.singleton(createOfferId(offerId))), - Matchers.any(classOf[java.util.Collection[TaskInfo]])) + meq(Collections.singleton(createOfferId(offerId))), + any(classOf[java.util.Collection[TaskInfo]])) } def createOfferId(offerId: String): OfferID = { 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 a6f57fcdb246..9acd99546c03 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 @@ -34,8 +34,8 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.YarnClientApplication import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records -import org.mockito.Matchers.{eq => meq, _} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, anyBoolean, anyShort, eq => meq} +import org.mockito.Mockito.{spy, verify} import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite, TestUtils} @@ -43,6 +43,7 @@ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.util.{SparkConfWithEnv, Utils} class ClientSuite extends SparkFunSuite with Matchers { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) import Client._ 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 952fd0b70bb7..f538cbc5b765 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 @@ -19,7 +19,7 @@ package org.apache.spark.network.yarn import scala.collection.JavaConverters._ import org.apache.hadoop.metrics2.MetricsRecordBuilder -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.{any, anyDouble, anyInt, anyLong} import org.mockito.Mockito.{mock, times, verify, when} import org.scalatest.Matchers @@ -56,8 +56,8 @@ class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers { YarnShuffleServiceMetrics.collectMetric(builder, testname, metrics.getMetrics.get(testname)) - verify(builder).addCounter(anyObject(), anyLong()) - verify(builder, times(4)).addGauge(anyObject(), anyDouble()) + verify(builder).addCounter(any(), anyLong()) + verify(builder, times(4)).addGauge(any(), anyDouble()) } } @@ -69,6 +69,6 @@ class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers { metrics.getMetrics.get("registeredExecutorsSize")) // only one - verify(builder).addGauge(anyObject(), anyInt()) + verify(builder).addGauge(any(), anyInt()) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala index 3c973d8ebc70..e644c16ddfea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.streaming.continuous +import org.mockito.ArgumentMatchers.{any, eq => eqTo} import org.mockito.InOrder -import org.mockito.Matchers.{any, eq => eqTo} -import org.mockito.Mockito._ +import org.mockito.Mockito.{inOrder, never, verify} import org.scalatest.BeforeAndAfterEach import org.scalatest.mockito.MockitoSugar diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 8212fb912ec5..4d3a54a048e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import org.apache.hadoop.fs.Path -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter 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 fd7e00b1de25..bdaef9494915 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -26,7 +26,7 @@ import scala.language.{implicitConversions, postfixOps} import scala.util.Random import org.apache.hadoop.conf.Configuration -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{doThrow, reset, spy} import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala index 8d81b582e4d3..7ec02c4782e4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.streaming.scheduler -import org.mockito.Matchers.{eq => meq} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{eq => meq} +import org.mockito.Mockito.{never, reset, times, verify, when} import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, PrivateMethodTester} import org.scalatest.concurrent.Eventually.{eventually, timeout} import org.scalatest.mockito.MockitoSugar 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 4a2549fc0a96..c20380d8490d 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 @@ -31,8 +31,8 @@ import scala.language.{implicitConversions, postfixOps} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.mockito.ArgumentCaptor -import org.mockito.Matchers.{eq => meq, _} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} +import org.mockito.Mockito.{times, verify, when} import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ From 5969b8a2edb913fe2a8e0d928010eb8f471c7b02 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 5 Jan 2019 00:55:17 -0800 Subject: [PATCH 0231/1072] [SPARK-26541][BUILD] Add `-Pdocker-integration-tests` to `dev/scalastyle` ## What changes were proposed in this pull request? This PR makes `scalastyle` to check `docker-integration-tests` module additionally and fixes one error. ## How was this patch tested? Pass the Jenkins with the updated Scalastyle. ``` ======================================================================== Running Scala style checks ======================================================================== Scalastyle checks passed. ``` Closes #23459 from dongjoon-hyun/SPARK-26541. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/scalastyle | 1 + .../org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/scalastyle b/dev/scalastyle index 2d6ee0da1d4c..ff6dba5b536a 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -28,6 +28,7 @@ ERRORS=$(echo -e "q\n" \ -Phive \ -Phive-thriftserver \ -Pspark-ganglia-lgpl \ + -Pdocker-integration-tests \ scalastyle test:scalastyle \ | awk '{if($1~/error/)print}' \ ) 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 70d294d0ca65..79fdf9c2ba43 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 @@ -154,7 +154,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo // A value with fractions from DECIMAL(3, 2) is correct: assert(row.getDecimal(1).compareTo(BigDecimal.valueOf(1.23)) == 0) // A value > Int.MaxValue from DECIMAL(10) is correct: - assert(row.getDecimal(2).compareTo(BigDecimal.valueOf(9999999999l)) == 0) + assert(row.getDecimal(2).compareTo(BigDecimal.valueOf(9999999999L)) == 0) } From 1af1190beeb1ac15205a9bd06ca67e363de03221 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Sat, 5 Jan 2019 01:14:58 -0800 Subject: [PATCH 0232/1072] [SPARK-26078][SQL][FOLLOWUP] Remove useless import ## What changes were proposed in this pull request? While backporting the patch to 2.4/2.3, I realized that the patch introduces unneeded imports (probably leftovers from intermediate changes). This PR removes the useless import. ## How was this patch tested? NA Closes #23451 from mgaido91/SPARK-26078_FOLLOWUP. Authored-by: Marco Gaido Signed-off-by: Dongjoon Hyun --- sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala | 1 - 1 file changed, 1 deletion(-) 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 c95c52f1d3a9..48c167660913 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 @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types._ class SubquerySuite extends QueryTest with SharedSQLContext { import testImplicits._ From 980e6bcd1c016139c6918d788fb4806a60740fcf Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 5 Jan 2019 21:50:27 +0800 Subject: [PATCH 0233/1072] [SPARK-26246][SQL][FOLLOWUP] Inferring TimestampType from JSON ## What changes were proposed in this pull request? Added new JSON option `inferTimestamp` (`true` by default) to control inferring of `TimestampType` from string values. ## How was this patch tested? Add new UT to `JsonInferSchemaSuite`. Closes #23455 from MaxGekk/json-infer-time-followup. Authored-by: Maxim Gekk Signed-off-by: Hyukjin Kwon --- docs/sql-migration-guide-upgrade.md | 3 +++ .../org/apache/spark/sql/catalyst/json/JSONOptions.scala | 6 ++++++ .../apache/spark/sql/catalyst/json/JsonInferSchema.scala | 3 ++- .../spark/sql/catalyst/json/JsonInferSchemaSuite.scala | 6 ++++++ 4 files changed, 17 insertions(+), 1 deletion(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index c4d2157de8b6..7e6a0c097d24 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -40,6 +40,9 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. - Since Spark 3.0, the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions use java.time API for parsing and formatting dates/timestamps from/to strings by using ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html) based on Proleptic Gregorian calendar. In Spark version 2.4 and earlier, java.text.SimpleDateFormat and java.util.GregorianCalendar (hybrid calendar that supports both the Julian and Gregorian calendar systems, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html) is used for the same purpuse. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and 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`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. + + - Since Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they matches to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. + ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. 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 eaff3fa7bec2..1ec9d5093a78 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 @@ -117,6 +117,12 @@ private[sql] class JSONOptions( */ val pretty: Boolean = parameters.get("pretty").map(_.toBoolean).getOrElse(false) + /** + * 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) + /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { factory.configure(JsonParser.Feature.ALLOW_COMMENTS, allowComments) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index 3203e626ea40..0bf3f03cdb72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -128,7 +128,8 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { } if (options.prefersDecimal && decimalTry.isDefined) { decimalTry.get - } else if ((allCatch opt timestampFormatter.parse(field)).isDefined) { + } else if (options.inferTimestamp && + (allCatch opt timestampFormatter.parse(field)).isDefined) { TimestampType } else { StringType 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 9307f9b47b80..9a6f4f5f9b0c 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 @@ -99,4 +99,10 @@ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { } } } + + test("disable timestamp inferring") { + val json = """{"a": "2019-01-04T21:11:10.123Z"}""" + checkType(Map("inferTimestamp" -> "true"), json, TimestampType) + checkType(Map("inferTimestamp" -> "false"), json, StringType) + } } From 0037bbb71725619590f5ecbc9a5a470c4889810f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 5 Jan 2019 22:53:28 +0800 Subject: [PATCH 0234/1072] [MINOR][DOC] Fix typos in the SQL migration guide ## What changes were proposed in this pull request? Fixed a few typos in the migration guide. Closes #23465 from MaxGekk/fix-typos-migration-guide. Authored-by: Maxim Gekk Signed-off-by: Hyukjin Kwon --- docs/sql-migration-guide-upgrade.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 7e6a0c097d24..0fcdd420bcfe 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -17,7 +17,7 @@ displayTitle: Spark SQL Upgrading Guide - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. - - In Spark version 2.4 and earlier, the `from_json` function produces `null`s for JSON strings and JSON datasource skips the same independetly of its mode if there is no valid root JSON token in its input (` ` for example). Since Spark 3.0, such input is treated as a bad record and handled according to specified mode. For example, in the `PERMISSIVE` mode the ` ` input is converted to `Row(null, null)` if specified schema is `key STRING, value INT`. + - In Spark version 2.4 and earlier, the `from_json` function produces `null`s for JSON strings and JSON datasource skips the same independently of its mode if there is no valid root JSON token in its input (` ` for example). Since Spark 3.0, such input is treated as a bad record and handled according to specified mode. For example, in the `PERMISSIVE` mode the ` ` input is converted to `Row(null, null)` if specified schema is `key STRING, value INT`. - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. @@ -27,21 +27,21 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal to 0.0, but users can still distinguish them via `Dataset.show`, `Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 internally, and users can't distinguish them any more. - - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be udefined. + - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be undefined. - In Spark version 2.4 and earlier, partition column value is converted as null if it can't be casted to corresponding user provided schema. Since 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. - - Since Spark 3.0, CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpuse with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. + - Since Spark 3.0, CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. - - Since Spark 3.0, the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions use java.time API for parsing and formatting dates/timestamps from/to strings by using ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html) based on Proleptic Gregorian calendar. In Spark version 2.4 and earlier, java.text.SimpleDateFormat and java.util.GregorianCalendar (hybrid calendar that supports both the Julian and Gregorian calendar systems, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html) is used for the same purpuse. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and 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`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. + - Since Spark 3.0, the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions use java.time API for parsing and formatting dates/timestamps from/to strings by using ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html) based on Proleptic Gregorian calendar. In Spark version 2.4 and earlier, java.text.SimpleDateFormat and java.util.GregorianCalendar (hybrid calendar that supports both the Julian and Gregorian calendar systems, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html) is used for the same purpose. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and 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`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. - - Since Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they matches to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. + - Since 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`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. ## Upgrading From Spark SQL 2.3 to 2.4 From 4ab5b5b9185f60f671d90d94732d0d784afa5f84 Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Sat, 5 Jan 2019 14:37:04 -0800 Subject: [PATCH 0235/1072] [SPARK-26545] Fix typo in EqualNullSafe's truth table comment ## What changes were proposed in this pull request? The truth table comment in EqualNullSafe incorrectly marked FALSE results as UNKNOWN. ## How was this patch tested? N/A Closes #23461 from rednaxelafx/fix-typo. Authored-by: Kris Mok Signed-off-by: gatorsmile --- .../apache/spark/sql/catalyst/expressions/predicates.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 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 01ecb99025ea..37fe22f4556e 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 @@ -653,9 +653,9 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp // +---------+---------+---------+---------+ // | <=> | TRUE | FALSE | UNKNOWN | // +---------+---------+---------+---------+ - // | TRUE | TRUE | FALSE | UNKNOWN | - // | FALSE | FALSE | TRUE | UNKNOWN | - // | UNKNOWN | UNKNOWN | UNKNOWN | TRUE | + // | TRUE | TRUE | FALSE | FALSE | + // | FALSE | FALSE | TRUE | FALSE | + // | UNKNOWN | FALSE | FALSE | TRUE | // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) From a17851cb95687963936c4d4a7eed132ee2c10677 Mon Sep 17 00:00:00 2001 From: Dave DeCaprio Date: Sat, 5 Jan 2019 19:20:35 -0800 Subject: [PATCH 0236/1072] [SPARK-26548][SQL] Don't hold CacheManager write lock while computing executedPlan ## What changes were proposed in this pull request? Address SPARK-26548, in Spark 2.4.0, the CacheManager holds a write lock while computing the executedPlan for a cached logicalPlan. In some cases with very large query plans this can be an expensive operation, taking minutes to run. The entire cache is blocked during this time. This PR changes that so the writeLock is only obtained after the executedPlan is generated, this reduces the time the lock is held to just the necessary time when the shared data structure is being updated. gatorsmile and cloud-fan - You can committed patches in this area before. This is a small incremental change. ## How was this patch tested? Has been tested on a live system where the blocking was causing major issues and it is working well. CacheManager has no explicit unit test but is used in many places internally as part of the SharedState. Closes #23469 from DaveDeCaprio/optimizer-unblocked. Lead-authored-by: Dave DeCaprio Co-authored-by: David DeCaprio Signed-off-by: gatorsmile --- .../org/apache/spark/sql/execution/CacheManager.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) 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 c9929935fb8a..728fde54fe69 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 @@ -88,7 +88,7 @@ class CacheManager extends Logging { def cacheQuery( query: Dataset[_], tableName: Option[String] = None, - storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock { + storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = { val planToCache = query.logicalPlan if (lookupCachedData(planToCache).nonEmpty) { logWarning("Asked to cache already cached data.") @@ -100,7 +100,13 @@ class CacheManager extends Logging { sparkSession.sessionState.executePlan(planToCache).executedPlan, tableName, planToCache) - cachedData.add(CachedData(planToCache, inMemoryRelation)) + writeLock { + if (lookupCachedData(planToCache).nonEmpty) { + logWarning("Data has already been cached.") + } else { + cachedData.add(CachedData(planToCache, inMemoryRelation)) + } + } } } From 737f08949adecbae37bb92dfad71ae5f3a82cbee Mon Sep 17 00:00:00 2001 From: SongYadong Date: Sun, 6 Jan 2019 08:46:20 -0600 Subject: [PATCH 0237/1072] [SPARK-26527][CORE] Let acquireUnrollMemory fail fast if required space exceeds memory limit ## What changes were proposed in this pull request? When acquiring unroll memory from `StaticMemoryManager`, let it fail fast if required space exceeds memory limit, just like acquiring storage memory. I think this may reduce some computation and memory evicting costs especially when required space(`numBytes`) is very big. ## How was this patch tested? Existing unit tests. Closes #23426 from SongYadong/acquireUnrollMemory_fail_fast. Authored-by: SongYadong Signed-off-by: Sean Owen --- .../spark/memory/StaticMemoryManager.scala | 27 ++++++++++++------- .../spark/storage/MemoryStoreSuite.scala | 4 +-- 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index 828608704274..0fd349dc5161 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -80,16 +80,23 @@ private[spark] class StaticMemoryManager( memoryMode: MemoryMode): Boolean = synchronized { require(memoryMode != MemoryMode.OFF_HEAP, "StaticMemoryManager does not support off-heap unroll memory") - val currentUnrollMemory = onHeapStorageMemoryPool.memoryStore.currentUnrollMemory - val freeMemory = onHeapStorageMemoryPool.memoryFree - // When unrolling, we will use all of the existing free memory, and, if necessary, - // some extra space freed from evicting cached blocks. We must place a cap on the - // amount of memory to be evicted by unrolling, however, otherwise unrolling one - // big block can blow away the entire cache. - val maxNumBytesToFree = math.max(0, maxUnrollMemory - currentUnrollMemory - freeMemory) - // Keep it within the range 0 <= X <= maxNumBytesToFree - val numBytesToFree = math.max(0, math.min(maxNumBytesToFree, numBytes - freeMemory)) - onHeapStorageMemoryPool.acquireMemory(blockId, numBytes, numBytesToFree) + if (numBytes > maxOnHeapStorageMemory) { + // Fail fast if the block simply won't fit + logInfo(s"Will not store $blockId as the required space ($numBytes bytes) exceeds our " + + s"memory limit ($maxOnHeapStorageMemory bytes)") + false + } else { + val currentUnrollMemory = onHeapStorageMemoryPool.memoryStore.currentUnrollMemory + val freeMemory = onHeapStorageMemoryPool.memoryFree + // When unrolling, we will use all of the existing free memory, and, if necessary, + // some extra space freed from evicting cached blocks. We must place a cap on the + // amount of memory to be evicted by unrolling, however, otherwise unrolling one + // big block can blow away the entire cache. + val maxNumBytesToFree = math.max(0, maxUnrollMemory - currentUnrollMemory - freeMemory) + // Keep it within the range 0 <= X <= maxNumBytesToFree + val numBytesToFree = math.max(0, math.min(maxNumBytesToFree, numBytes - freeMemory)) + onHeapStorageMemoryPool.acquireMemory(blockId, numBytes, numBytesToFree) + } } private[memory] diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala index 7274072e5049..baff672f5fb8 100644 --- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala @@ -291,11 +291,11 @@ class MemoryStoreSuite blockInfoManager.removeBlock("b3") putIteratorAsBytes("b3", smallIterator, ClassTag.Any) - // Unroll huge block with not enough space. This should fail and kick out b2 in the process. + // Unroll huge block with not enough space. This should fail. val result4 = putIteratorAsBytes("b4", bigIterator, ClassTag.Any) assert(result4.isLeft) // unroll was unsuccessful assert(!memoryStore.contains("b1")) - assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b2")) assert(memoryStore.contains("b3")) assert(!memoryStore.contains("b4")) assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator From 9d8e9b394bbc065a72076585a21393f42ce86cd1 Mon Sep 17 00:00:00 2001 From: Hirobe Keiichi Date: Sun, 6 Jan 2019 08:52:09 -0600 Subject: [PATCH 0238/1072] [SPARK-26339][SQL] Throws better exception when reading files that start with underscore ## What changes were proposed in this pull request? My pull request #23288 was resolved and merged to master, but it turned out later that my change breaks another regression test. Because we cannot reopen pull request, I create a new pull request here. Commit 92934b4 is only change after pull request #23288. `CheckFileExist` was avoided at 239cfa4 after discussing #23288 (comment). But, that change turned out to be wrong because we should not check if argument checkFileExist is false. Test https://github.com/apache/spark/blob/27e42c1de502da80fa3e22bb69de47fb00158174/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala#L2555 failed when we avoided checkFileExist, but now successed after commit 92934b4 . ## How was this patch tested? Both of below tests were passed. ``` testOnly org.apache.spark.sql.execution.datasources.csv.CSVSuite testOnly org.apache.spark.sql.SQLQuerySuite ``` Closes #23446 from KeiichiHirobe/SPARK-26339. Authored-by: Hirobe Keiichi Signed-off-by: Sean Owen --- .../execution/datasources/DataSource.scala | 19 +++++++++++++++++- .../src/test/resources/test-data/_cars.csv | 7 +++++++ .../execution/datasources/csv/CSVSuite.scala | 20 +++++++++++++++++++ 3 files changed, 45 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/resources/test-data/_cars.csv 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 fefff68c4ba8..2a438a5cbf95 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 @@ -543,7 +543,7 @@ case class DataSource( checkFilesExist: Boolean): Seq[Path] = { val allPaths = caseInsensitiveOptions.get("path") ++ paths val hadoopConf = sparkSession.sessionState.newHadoopConf() - allPaths.flatMap { path => + val allGlobPath = allPaths.flatMap { path => val hdfsPath = new Path(path) val fs = hdfsPath.getFileSystem(hadoopConf) val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) @@ -560,6 +560,23 @@ case class DataSource( } globPath }.toSeq + + if (checkFilesExist) { + val (filteredOut, filteredIn) = allGlobPath.partition { path => + InMemoryFileIndex.shouldFilterOut(path.getName) + } + if (filteredOut.nonEmpty) { + if (filteredIn.isEmpty) { + throw new AnalysisException( + s"All paths were ignored:\n${filteredOut.mkString("\n ")}") + } else { + logDebug( + s"Some paths were ignored:\n${filteredOut.mkString("\n ")}") + } + } + } + + allGlobPath } } diff --git a/sql/core/src/test/resources/test-data/_cars.csv b/sql/core/src/test/resources/test-data/_cars.csv new file mode 100644 index 000000000000..40ded573ade5 --- /dev/null +++ b/sql/core/src/test/resources/test-data/_cars.csv @@ -0,0 +1,7 @@ + +year,make,model,comment,blank +"2012","Tesla","S","No comment", + +1997,Ford,E350,"Go get one now they are going fast", +2015,Chevy,Volt + 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 d9e5d7af1967..fb1bedfaa32c 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 @@ -53,6 +53,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te private val carsEmptyValueFile = "test-data/cars-empty-value.csv" private val carsBlankColName = "test-data/cars-blank-column-name.csv" private val carsCrlf = "test-data/cars-crlf.csv" + private val carsFilteredOutFile = "test-data/_cars.csv" private val emptyFile = "test-data/empty.csv" private val commentsFile = "test-data/comments.csv" private val disableCommentsFile = "test-data/disable_comments.csv" @@ -346,6 +347,25 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te assert(result.schema.fieldNames.size === 1) } + test("SPARK-26339 Not throw an exception if some of specified paths are filtered in") { + val cars = spark + .read + .option("header", "false") + .csv(testFile(carsFile), testFile(carsFilteredOutFile)) + + verifyCars(cars, withHeader = false, checkTypes = false) + } + + test("SPARK-26339 Throw an exception only if all of the specified paths are filtered out") { + val e = intercept[AnalysisException] { + val cars = spark + .read + .option("header", "false") + .csv(testFile(carsFilteredOutFile)) + }.getMessage + assert(e.contains("All paths were ignored:")) + } + test("DDL test with empty file") { withView("carsTable") { spark.sql( From b305d71625380f6fcd7b675d423222eca1840c2a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 6 Jan 2019 17:36:06 -0800 Subject: [PATCH 0239/1072] [SPARK-26547][SQL] Remove duplicate toHiveString from HiveUtils ## What changes were proposed in this pull request? The `toHiveString()` and `toHiveStructString` methods were removed from `HiveUtils` because they have been already implemented in `HiveResult`. One related test was moved to `HiveResultSuite`. ## How was this patch tested? By tests from `hive-thriftserver`. Closes #23466 from MaxGekk/dedup-hive-result-string. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/HiveResult.scala | 115 ++++++++++-------- .../spark/sql/execution/HiveResultSuite.scala | 30 +++++ .../SparkExecuteStatementOperation.scala | 4 +- .../org/apache/spark/sql/hive/HiveUtils.scala | 46 ------- .../spark/sql/hive/HiveUtilsSuite.scala | 11 +- 5 files changed, 96 insertions(+), 110 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala 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 22d3ca958a21..c90b254a6d12 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 @@ -56,61 +56,70 @@ object HiveResult { result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) } - /** Formats a datum (based on the given data type) and returns the string representation. */ - private def toHiveString(a: (Any, DataType)): String = { - val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, - BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) - val timeZone = DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone) - - def formatDecimal(d: java.math.BigDecimal): String = { - if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { - java.math.BigDecimal.ZERO.toPlainString - } else { - d.stripTrailingZeros().toPlainString - } + private def formatDecimal(d: java.math.BigDecimal): String = { + if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { + java.math.BigDecimal.ZERO.toPlainString + } else { + d.stripTrailingZeros().toPlainString // Hive strips trailing zeros } + } - /** Hive outputs fields of structs slightly differently than top level attributes. */ - def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (decimal, DecimalType()) => decimal.toString - case (interval, CalendarIntervalType) => interval.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } + private val primitiveTypes = Seq( + StringType, + IntegerType, + LongType, + DoubleType, + FloatType, + BooleanType, + ByteType, + ShortType, + DateType, + TimestampType, + BinaryType) - a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" - case (d: Date, DateType) => - DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d)) - case (t: Timestamp, TimestampType) => - DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t), timeZone) - case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) - case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) - case (interval, CalendarIntervalType) => interval.toString - case (other, tpe) if primitiveTypes.contains(tpe) => other.toString - } + /** Hive outputs fields of structs slightly differently than top level attributes. */ + private def toHiveStructString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "null" + case (s: String, StringType) => "\"" + s + "\"" + case (decimal, DecimalType()) => decimal.toString + case (interval, CalendarIntervalType) => interval.toString + case (other, tpe) if primitiveTypes contains tpe => other.toString + } + + /** Formats a datum (based on the given data type) and returns the string representation. */ + def toHiveString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "NULL" + case (d: Date, DateType) => + DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d)) + case (t: Timestamp, TimestampType) => + val timeZone = DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone) + DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t), timeZone) + case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) + case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) + case (interval, CalendarIntervalType) => interval.toString + case (other, _ : UserDefinedType[_]) => other.toString + case (other, tpe) if primitiveTypes.contains(tpe) => 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 new file mode 100644 index 000000000000..4205b3f79a97 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT} + +class HiveResultSuite extends SparkFunSuite { + + 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)") + } +} 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 3cfc81b8a957..e68c6011c139 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 @@ -34,8 +34,8 @@ import org.apache.hive.service.cli.session.HiveSession 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.command.SetCommand -import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.{Utils => SparkUtils} @@ -103,7 +103,7 @@ private[hive] class SparkExecuteStatementOperation( case BinaryType => to += from.getAs[Array[Byte]](ordinal) case _: ArrayType | _: StructType | _: MapType | _: UserDefinedType[_] => - val hiveString = HiveUtils.toHiveString((from.get(ordinal), dataTypes(ordinal))) + val hiveString = HiveResult.toHiveString((from.get(ordinal), dataTypes(ordinal))) to += hiveString } } 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 b60d4c71f594..597eef129f63 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 @@ -444,52 +444,6 @@ private[spark] object HiveUtils extends Logging { propMap.toMap } - protected val primitiveTypes = - Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DateType, TimestampType, BinaryType) - - protected[sql] def toHiveString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" - case (d: Int, DateType) => new DateWritable(d).toString - case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString - case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) - case (decimal: java.math.BigDecimal, DecimalType()) => - // Hive strips trailing zeros so use its toString - HiveDecimal.create(decimal).toString - case (other, _ : UserDefinedType[_]) => other.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - - /** Hive outputs fields of structs slightly differently than top level attributes. */ - protected def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (decimal, DecimalType()) => decimal.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - /** * Infers the schema for Hive serde tables and returns the CatalogTable with the inferred schema. * When the tables are data source tables or the schema already exists, returns the original diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala index f2b75e4b23f0..303dd70760a1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala @@ -17,16 +17,15 @@ package org.apache.spark.sql.hive -import java.net.URL - import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.HiveResult import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SQLTestUtils} -import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader} +import org.apache.spark.util.ChildFirstURLClassLoader class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { @@ -62,10 +61,4 @@ class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton Thread.currentThread().setContextClassLoader(contextClassLoader) } } - - test("toHiveString correctly handles UDTs") { - val point = new ExamplePoint(50.0, 50.0) - val tpe = new ExamplePointUDT() - assert(HiveUtils.toHiveString((point, tpe)) === "(50.0, 50.0)") - } } From fe039faddf13c6a30f7aea69324aa4d4bb84c632 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 6 Jan 2019 19:59:31 -0800 Subject: [PATCH 0240/1072] [SPARK-26554][BUILD] Update `release-util.sh` to avoid GitBox fake 200 headers ## What changes were proposed in this pull request? Unlike the previous Apache Git repository, new GitBox repository returns a fake HTTP 200 header instead of `404 Not Found` header. This makes release scripts out of order. This PR aims to fix it to handle the html body message instead of the fake HTTP headers. This is a release blocker. ```bash $ curl -s --head --fail "https://gitbox.apache.org/repos/asf?p=spark.git;a=commit;h=v3.0.0" HTTP/1.1 200 OK Date: Sun, 06 Jan 2019 22:42:39 GMT Server: Apache/2.4.18 (Ubuntu) Vary: Accept-Encoding Access-Control-Allow-Origin: * Access-Control-Allow-Methods: POST, GET, OPTIONS Access-Control-Allow-Headers: X-PINGOTHER Access-Control-Max-Age: 1728000 Content-Type: text/html; charset=utf-8 ``` **BEFORE** ```bash $ ./do-release-docker.sh -d /tmp/test -n Branch [branch-2.4]: Current branch version is 2.4.1-SNAPSHOT. Release [2.4.1]: RC # [1]: v2.4.1-rc1 already exists. Continue anyway [y/n]? ``` **AFTER** ```bash $ ./do-release-docker.sh -d /tmp/test -n Branch [branch-2.4]: Current branch version is 2.4.1-SNAPSHOT. Release [2.4.1]: RC # [1]: This is a dry run. Please confirm the ref that will be built for testing. Ref [v2.4.1-rc1]: ``` ## How was this patch tested? Manual. Closes #23476 from dongjoon-hyun/SPARK-26554. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/release-util.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index c925de9be52d..9a340528b506 100755 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -73,7 +73,9 @@ function fcreate_secure { } function check_for_tag { - curl -s --head --fail "$ASF_REPO_WEBUI;a=commit;h=$1" >/dev/null + # Check HTML body messages instead of header status codes. Apache GitBox returns + # a header with `200 OK` status code for both existing and non-existing tag URLs + ! curl -s --fail "$ASF_REPO_WEBUI;a=commit;h=$1" | grep '404 Not Found' > /dev/null } function get_release_info { From 61133cb8a69e7814c3450e84ce9cc9226d7e8ad8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 6 Jan 2019 21:00:10 -0800 Subject: [PATCH 0241/1072] [SPARK-26536][BUILD][FOLLOWUP][TEST-MAVEN] Make StreamingReadSupport public for maven testing ## What changes were proposed in this pull request? `StreamingReadSupport` is designed to be a `package` interface. Mockito seems to complain during `Maven` testing. This doesn't fail in `sbt` and IntelliJ. For mock-testing purpose, this PR makes it `public` interface and adds explicit comments like `public interface ReadSupport` ```scala EpochCoordinatorSuite: *** RUN ABORTED *** java.lang.IllegalAccessError: tried to access class org.apache.spark.sql.sources.v2.reader.streaming.StreamingReadSupport from class org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport$MockitoMock$58628338 at org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport$MockitoMock$58628338.(Unknown Source) at sun.reflect.GeneratedSerializationConstructorAccessor632.newInstance(Unknown Source) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.objenesis.instantiator.sun.SunReflectionFactoryInstantiator.newInstance(SunReflectionFactoryInstantiator.java:48) at org.objenesis.ObjenesisBase.newInstance(ObjenesisBase.java:73) at org.mockito.internal.creation.instance.ObjenesisInstantiator.newInstance(ObjenesisInstantiator.java:19) at org.mockito.internal.creation.bytebuddy.SubclassByteBuddyMockMaker.createMock(SubclassByteBuddyMockMaker.java:47) at org.mockito.internal.creation.bytebuddy.ByteBuddyMockMaker.createMock(ByteBuddyMockMaker.java:25) at org.mockito.internal.util.MockUtil.createMock(MockUtil.java:35) at org.mockito.internal.MockitoCore.mock(MockitoCore.java:69) ``` ## How was this patch tested? Pass the Jenkins with Maven build Closes #23463 from dongjoon-hyun/SPARK-26536-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../v2/reader/streaming/StreamingReadSupport.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java index 84872d1ebc26..bd39fc858d3b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java @@ -17,14 +17,17 @@ package org.apache.spark.sql.sources.v2.reader.streaming; +import com.google.common.annotations.VisibleForTesting; + import org.apache.spark.sql.sources.v2.reader.ReadSupport; /** - * A base interface for streaming read support. This is package private and is invisible to data - * sources. Data sources should implement concrete streaming read support interfaces: - * {@link MicroBatchReadSupport} or {@link ContinuousReadSupport}. + * A base interface for streaming read support. Data sources should implement concrete streaming + * read support interfaces: {@link MicroBatchReadSupport} or {@link ContinuousReadSupport}. + * This is exposed for a testing purpose. */ -interface StreamingReadSupport extends ReadSupport { +@VisibleForTesting +public interface StreamingReadSupport extends ReadSupport { /** * Returns the initial offset for a streaming query to start reading from. Note that the From 468d25ec7419b4c55955ead877232aae5654260e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 6 Jan 2019 22:45:18 -0800 Subject: [PATCH 0242/1072] [MINOR][BUILD] Fix script name in `release-tag.sh` usage message ## What changes were proposed in this pull request? This PR fixes the old script name in `release-tag.sh`. $ ./release-tag.sh --help | head -n1 usage: tag-release.sh ## How was this patch tested? Manual. $ ./release-tag.sh --help | head -n1 usage: release-tag.sh Closes #23477 from dongjoon-hyun/SPARK-RELEASE-TAG. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/release-tag.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index 010082d960a2..8024440759eb 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -21,8 +21,9 @@ SELF=$(cd $(dirname $0) && pwd) . "$SELF/release-util.sh" function exit_with_usage { + local NAME=$(basename $0) cat << EOF -usage: tag-release.sh +usage: $NAME Tags a Spark release on a particular branch. Inputs are specified with the following environment variables: From a927c764c1eee066efc1c2c713dfee411de79245 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 7 Jan 2019 18:36:52 +0800 Subject: [PATCH 0243/1072] [SPARK-26559][ML][PYSPARK] ML image can't work with numpy versions prior to 1.9 ## What changes were proposed in this pull request? Due to [API change](https://github.com/numpy/numpy/pull/4257/files#diff-c39521d89f7e61d6c0c445d93b62f7dc) at 1.9, PySpark image doesn't work with numpy version prior to 1.9. When running image test with numpy version prior to 1.9, we can see error: ``` test_read_images (pyspark.ml.tests.test_image.ImageReaderTest) ... ERROR test_read_images_multiple_times (pyspark.ml.tests.test_image.ImageReaderTest2) ... ok ====================================================================== ERROR: test_read_images (pyspark.ml.tests.test_image.ImageReaderTest) ---------------------------------------------------------------------- Traceback (most recent call last): File "/Users/viirya/docker_tmp/repos/spark-1/python/pyspark/ml/tests/test_image.py", line 36, in test_read_images self.assertEqual(ImageSchema.toImage(array, origin=first_row[0]), first_row) File "/Users/viirya/docker_tmp/repos/spark-1/python/pyspark/ml/image.py", line 193, in toImage data = bytearray(array.astype(dtype=np.uint8).ravel().tobytes()) AttributeError: 'numpy.ndarray' object has no attribute 'tobytes' ---------------------------------------------------------------------- Ran 2 tests in 29.040s FAILED (errors=1) ``` ## How was this patch tested? Manually test with numpy version prior and after 1.9. Closes #23484 from viirya/fix-pyspark-image. Authored-by: Liang-Chi Hsieh Signed-off-by: Hyukjin Kwon --- python/pyspark/ml/image.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/image.py b/python/pyspark/ml/image.py index edb90a357854..a1aacea88e42 100644 --- a/python/pyspark/ml/image.py +++ b/python/pyspark/ml/image.py @@ -28,6 +28,7 @@ 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 @@ -190,7 +191,11 @@ def toImage(self, array, origin=""): # Running `bytearray(numpy.array([1]))` fails in specific Python versions # with a specific Numpy version, for example in Python 3.6.0 and NumPy 1.13.3. # Here, it avoids it by converting it to bytes. - data = bytearray(array.astype(dtype=np.uint8).ravel().tobytes()) + if LooseVersion(np.__version__) >= LooseVersion('1.9'): + data = bytearray(array.astype(dtype=np.uint8).ravel().tobytes()) + else: + # Numpy prior to 1.9 don't have `tobytes` method. + data = bytearray(array.astype(dtype=np.uint8).ravel()) # Creating new Row with _create_row(), because Row(name = value, ... ) # orders fields by name, which conflicts with expected schema order From 868e02533d76d45bff4200d07658105b6004cf46 Mon Sep 17 00:00:00 2001 From: ayudovin Date: Mon, 7 Jan 2019 08:58:33 -0600 Subject: [PATCH 0244/1072] [SPARK-26383][CORE] NPE when use DataFrameReader.jdbc with wrong URL ### What changes were proposed in this pull request? When passing wrong url to jdbc then It would throw IllegalArgumentException instead of NPE. ### How was this patch tested? Adding test case to Existing tests in JDBCSuite Closes #23464 from ayudovin/fixing-npe. Authored-by: ayudovin Signed-off-by: Sean Owen --- .../sql/execution/datasources/jdbc/JdbcUtils.scala | 7 ++++++- .../scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 13 +++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) 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 922bef284c98..86a27b5afc25 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 @@ -48,6 +48,7 @@ object JdbcUtils extends Logging { * Returns a factory for creating connections to the given JDBC URL. * * @param options - JDBC options that contains url, table and other information. + * @throws IllegalArgumentException if the driver could not open a JDBC connection. */ def createConnectionFactory(options: JDBCOptions): () => Connection = { val driverClass: String = options.driverClass @@ -60,7 +61,11 @@ object JdbcUtils extends Logging { throw new IllegalStateException( s"Did not find registered driver with class $driverClass") } - driver.connect(options.url, options.asConnectionProperties) + val connection: Connection = driver.connect(options.url, options.asConnectionProperties) + require(connection != null, + s"The driver could not open a JDBC connection. Check the URL: ${options.url}") + + connection } } 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 e4641631e607..aefa5da94481 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 @@ -1507,4 +1507,17 @@ class JDBCSuite extends QueryTest checkNotPushdown(sql("SELECT name, theid FROM predicateOption WHERE theid = 1")), Row("fred", 1) :: Nil) } + + test("SPARK-26383 throw IllegalArgumentException if wrong kind of driver to the given url") { + val e = intercept[IllegalArgumentException] { + val opts = Map( + "url" -> "jdbc:mysql://localhost/db", + "dbtable" -> "table", + "driver" -> "org.postgresql.Driver" + ) + spark.read.format("jdbc").options(opts).load + }.getMessage + assert(e.contains("The driver could not open a JDBC connection. " + + "Check the URL: jdbc:mysql://localhost/db")) + } } From 71183b283343a99c6fa99a41268dae412598067f Mon Sep 17 00:00:00 2001 From: Shahid Date: Mon, 7 Jan 2019 09:15:50 -0800 Subject: [PATCH 0245/1072] [SPARK-24489][ML] Check for invalid input type of weight data in ml.PowerIterationClustering ## What changes were proposed in this pull request? The test case will result the following failure. currently in ml.PIC, there is no check for the data type of weight column. ``` test("invalid input types for weight") { val invalidWeightData = spark.createDataFrame(Seq( (0L, 1L, "a"), (2L, 3L, "b") )).toDF("src", "dst", "weight") val pic = new PowerIterationClustering() .setWeightCol("weight") val result = pic.assignClusters(invalidWeightData) } ``` ``` Job aborted due to stage failure: Task 0 in stage 8077.0 failed 1 times, most recent failure: Lost task 0.0 in stage 8077.0 (TID 882, localhost, executor driver): scala.MatchError: [0,1,null] (of class org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema) at org.apache.spark.ml.clustering.PowerIterationClustering$$anonfun$3.apply(PowerIterationClustering.scala:178) at org.apache.spark.ml.clustering.PowerIterationClustering$$anonfun$3.apply(PowerIterationClustering.scala:178) at scala.collection.Iterator$$anon$11.next(Iterator.scala:409) at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434) at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at org.apache.spark.graphx.EdgeRDD$$anonfun$1.apply(EdgeRDD.scala:107) at org.apache.spark.graphx.EdgeRDD$$anonfun$1.apply(EdgeRDD.scala:105) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:847) ``` In this PR, added check types for weight column. ## How was this patch tested? UT added Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #21509 from shahidki31/testCasePic. Authored-by: Shahid Signed-off-by: Holden Karau --- .../ml/clustering/PowerIterationClustering.scala | 1 + .../PowerIterationClusteringSuite.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+) 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 d9a330f67e8d..149e99d2f195 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 @@ -166,6 +166,7 @@ class PowerIterationClustering private[clustering] ( val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) { lit(1.0) } else { + SchemaUtils.checkNumericType(dataset.schema, $(weightCol)) col($(weightCol)).cast(DoubleType) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala index 55b460f1a452..0ba3ffabb75d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala @@ -145,6 +145,21 @@ class PowerIterationClusteringSuite extends SparkFunSuite assert(msg.contains("Similarity must be nonnegative")) } + test("check for invalid input types of weight") { + val invalidWeightData = spark.createDataFrame(Seq( + (0L, 1L, "a"), + (2L, 3L, "b") + )).toDF("src", "dst", "weight") + + val msg = intercept[IllegalArgumentException] { + new PowerIterationClustering() + .setWeightCol("weight") + .assignClusters(invalidWeightData) + }.getMessage + assert(msg.contains("requirement failed: Column weight must be of type numeric" + + " but was actually of type string.")) + } + test("test default weight") { val dataWithoutWeight = data.sample(0.5, 1L).select('src, 'dst) From 669e8a155987995a1a5d49a96b88c05f39e41723 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 7 Jan 2019 14:40:08 -0600 Subject: [PATCH 0246/1072] [SPARK-25689][YARN] Make driver, not AM, manage delegation tokens. This change modifies the behavior of the delegation token code when running on YARN, so that the driver controls the renewal, in both client and cluster mode. For that, a few different things were changed: * The AM code only runs code that needs DTs when DTs are available. In a way, this restores the AM behavior to what it was pre-SPARK-23361, but keeping the fix added in that bug. Basically, all the AM code is run in a "UGI.doAs()" block; but code that needs to talk to HDFS (basically the distributed cache handling code) was delayed to the point where the driver is up and running, and thus when valid delegation tokens are available. * SparkSubmit / ApplicationMaster now handle user login, not the token manager. The previous AM code was relying on the token manager to keep the user logged in when keytabs are used. This required some odd APIs in the token manager and the AM so that the right UGI was exposed and used in the right places. After this change, the logged in user is handled separately from the token manager, so the API was cleaned up, and, as explained above, the whole AM runs under the logged in user, which also helps with simplifying some more code. * Distributed cache configs are sent separately to the AM. Because of the delayed initialization of the cached resources in the AM, it became easier to write the cache config to a separate properties file instead of bundling it with the rest of the Spark config. This also avoids having to modify the SparkConf to hide things from the UI. * Finally, the AM doesn't manage the token manager anymore. The above changes allow the token manager to be completely handled by the driver's scheduler backend code also in YARN mode (whether client or cluster), making it similar to other RMs. To maintain the fix added in SPARK-23361 also in client mode, the AM now sends an extra message to the driver on initialization to fetch delegation tokens; and although it might not really be needed, the driver also keeps the running AM updated when new tokens are created. Tested in a kerberized cluster with the same tests used to validate SPARK-23361, in both client and cluster mode. Also tested with a non-kerberized cluster. Closes #23338 from vanzin/SPARK-25689. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid --- .../HadoopDelegationTokenManager.scala | 110 ++++++-------- .../HiveDelegationTokenProvider.scala | 16 ++- .../cluster/CoarseGrainedClusterMessage.scala | 3 + .../CoarseGrainedSchedulerBackend.scala | 40 ++++-- .../HadoopDelegationTokenManagerSuite.scala | 8 +- .../KerberosConfDriverFeatureStep.scala | 2 +- .../KubernetesClusterSchedulerBackend.scala | 7 +- .../MesosCoarseGrainedSchedulerBackend.scala | 7 +- .../spark/deploy/yarn/ApplicationMaster.scala | 135 +++++++++--------- .../yarn/ApplicationMasterArguments.scala | 5 + .../org/apache/spark/deploy/yarn/Client.scala | 100 +++++++------ .../spark/deploy/yarn/YarnRMClient.scala | 8 +- .../org/apache/spark/deploy/yarn/config.scala | 10 -- .../YARNHadoopDelegationTokenManager.scala | 7 +- .../cluster/YarnClientSchedulerBackend.scala | 6 + .../cluster/YarnSchedulerBackend.scala | 17 ++- ...ARNHadoopDelegationTokenManagerSuite.scala | 2 +- 17 files changed, 246 insertions(+), 237 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index f7e3ddecee09..d97857a39fc2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -21,7 +21,6 @@ import java.io.File import java.net.URI import java.security.PrivilegedExceptionAction import java.util.concurrent.{ScheduledExecutorService, TimeUnit} -import java.util.concurrent.atomic.AtomicReference import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem @@ -39,32 +38,24 @@ import org.apache.spark.util.ThreadUtils /** * Manager for delegation tokens in a Spark application. * - * This manager has two modes of operation: - * - * 1. When configured with a principal and a keytab, it will make sure long-running apps can run - * without interruption while accessing secured services. It periodically logs in to the KDC with - * user-provided credentials, and contacts all the configured secure services to obtain delegation - * tokens to be distributed to the rest of the application. - * - * Because the Hadoop UGI API does not expose the TTL of the TGT, a configuration controls how often - * to check that a relogin is necessary. This is done reasonably often since the check is a no-op - * when the relogin is not yet needed. The check period can be overridden in the configuration. + * When configured with a principal and a keytab, this manager will make sure long-running apps can + * run without interruption while accessing secured services. It periodically logs in to the KDC + * with user-provided credentials, and contacts all the configured secure services to obtain + * delegation tokens to be distributed to the rest of the application. * * New delegation tokens are created once 75% of the renewal interval of the original tokens has - * elapsed. The new tokens are sent to the Spark driver endpoint once it's registered with the AM. - * The driver is tasked with distributing the tokens to other processes that might need them. + * elapsed. The new tokens are sent to the Spark driver endpoint. The driver is tasked with + * distributing the tokens to other processes that might need them. * - * 2. When operating without an explicit principal and keytab, token renewal will not be available. - * Starting the manager will distribute an initial set of delegation tokens to the provided Spark - * driver, but the app will not get new tokens when those expire. - * - * It can also be used just to create delegation tokens, by calling the `obtainDelegationTokens` - * method. This option does not require calling the `start` method, but leaves it up to the - * caller to distribute the tokens that were generated. + * This class can also be used just to create delegation tokens, by calling the + * `obtainDelegationTokens` method. This option does not require calling the `start` method nor + * providing a driver reference, but leaves it up to the caller to distribute the tokens that were + * generated. */ private[spark] class HadoopDelegationTokenManager( protected val sparkConf: SparkConf, - protected val hadoopConf: Configuration) extends Logging { + protected val hadoopConf: Configuration, + protected val schedulerRef: RpcEndpointRef) extends Logging { private val deprecatedProviderEnabledConfigs = List( "spark.yarn.security.tokens.%s.enabled", @@ -85,60 +76,44 @@ private[spark] class HadoopDelegationTokenManager( s"${delegationTokenProviders.keys.mkString(", ")}.") private var renewalExecutor: ScheduledExecutorService = _ - private val driverRef = new AtomicReference[RpcEndpointRef]() - - /** Set the endpoint used to send tokens to the driver. */ - def setDriverRef(ref: RpcEndpointRef): Unit = { - driverRef.set(ref) - } /** @return Whether delegation token renewal is enabled. */ def renewalEnabled: Boolean = principal != null /** - * Start the token renewer. Requires a principal and keytab. Upon start, the renewer will: + * Start the token renewer. Requires a principal and keytab. Upon start, the renewer will + * obtain delegation tokens for all configured services and send them to the driver, and + * set up tasks to periodically get fresh tokens as needed. * - * - log in the configured principal, and set up a task to keep that user's ticket renewed - * - obtain delegation tokens from all available providers - * - send the tokens to the driver, if it's already registered - * - schedule a periodic task to update the tokens when needed. + * This method requires that a keytab has been provided to Spark, and will try to keep the + * logged in user's TGT valid while this manager is active. * - * @return The newly logged in user. + * @return New set of delegation tokens created for the configured principal. */ - def start(): UserGroupInformation = { + def start(): Array[Byte] = { require(renewalEnabled, "Token renewal must be enabled to start the renewer.") + require(schedulerRef != null, "Token renewal requires a scheduler endpoint.") renewalExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Renewal Thread") - val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() - val ugi = doLogin() - - val tgtRenewalTask = new Runnable() { - override def run(): Unit = { - ugi.checkTGTAndReloginFromKeytab() + val ugi = UserGroupInformation.getCurrentUser() + if (ugi.isFromKeytab()) { + // In Hadoop 2.x, renewal of the keytab-based login seems to be automatic, but in Hadoop 3.x, + // it is configurable (see hadoop.kerberos.keytab.login.autorenewal.enabled, added in + // HADOOP-9567). This task will make sure that the user stays logged in regardless of that + // configuration's value. Note that checkTGTAndReloginFromKeytab() is a no-op if the TGT does + // not need to be renewed yet. + val tgtRenewalTask = new Runnable() { + override def run(): Unit = { + ugi.checkTGTAndReloginFromKeytab() + } } + val tgtRenewalPeriod = sparkConf.get(KERBEROS_RELOGIN_PERIOD) + renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod, + TimeUnit.SECONDS) } - val tgtRenewalPeriod = sparkConf.get(KERBEROS_RELOGIN_PERIOD) - renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod, - TimeUnit.SECONDS) - val creds = obtainTokensAndScheduleRenewal(ugi) - ugi.addCredentials(creds) - - val driver = driverRef.get() - if (driver != null) { - val tokens = SparkHadoopUtil.get.serialize(creds) - driver.send(UpdateDelegationTokens(tokens)) - } - - // Transfer the original user's tokens to the new user, since it may contain needed tokens - // (such as those user to connect to YARN). Explicitly avoid overwriting tokens that already - // exist in the current user's credentials, since those were freshly obtained above - // (see SPARK-23361). - val existing = ugi.getCredentials() - existing.mergeAll(originalCreds) - ugi.addCredentials(existing) - ugi + updateTokensTask() } def stop(): Unit = { @@ -218,27 +193,22 @@ private[spark] class HadoopDelegationTokenManager( * Periodic task to login to the KDC and create new delegation tokens. Re-schedules itself * to fetch the next set of tokens when needed. */ - private def updateTokensTask(): Unit = { + private def updateTokensTask(): Array[Byte] = { try { val freshUGI = doLogin() val creds = obtainTokensAndScheduleRenewal(freshUGI) val tokens = SparkHadoopUtil.get.serialize(creds) - val driver = driverRef.get() - if (driver != null) { - logInfo("Updating delegation tokens.") - driver.send(UpdateDelegationTokens(tokens)) - } else { - // This shouldn't really happen, since the driver should register way before tokens expire. - logWarning("Delegation tokens close to expiration but no driver has registered yet.") - SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) - } + logInfo("Updating delegation tokens.") + schedulerRef.send(UpdateDelegationTokens(tokens)) + tokens } catch { case e: Exception => val delay = TimeUnit.SECONDS.toMillis(sparkConf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)) logWarning(s"Failed to update tokens, will try again in ${UIUtils.formatDuration(delay)}!" + " If this happens too often tasks will fail.", e) scheduleRenewal(delay) + null } } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala index 90f705138157..4ca0136424fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala @@ -67,11 +67,17 @@ private[spark] class HiveDelegationTokenProvider // Other modes (such as client with or without keytab, or cluster mode with keytab) do not need // a delegation token, since there's a valid kerberos TGT for the right user available to the // driver, which is the only process that connects to the HMS. - val deployMode = sparkConf.get("spark.submit.deployMode", "client") - UserGroupInformation.isSecurityEnabled && + // + // Note that this means Hive tokens are not re-created periodically by the token manager. + // This is because HMS connections are only performed by the Spark driver, and the driver + // either has a TGT, in which case it does not need tokens, or it has a token created + // elsewhere, in which case it cannot create new ones. The check for an existing token avoids + // printing an exception to the logs in the latter case. + val currentToken = UserGroupInformation.getCurrentUser().getCredentials().getToken(tokenAlias) + currentToken == null && UserGroupInformation.isSecurityEnabled && hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty && (SparkHadoopUtil.get.isProxyUser(UserGroupInformation.getCurrentUser()) || - (deployMode == "cluster" && !sparkConf.contains(KEYTAB))) + (!Utils.isClientMode(sparkConf) && !sparkConf.contains(KEYTAB))) } override def obtainDelegationTokens( @@ -98,7 +104,7 @@ private[spark] class HiveDelegationTokenProvider val hive2Token = new Token[DelegationTokenIdentifier]() hive2Token.decodeFromUrlString(tokenStr) logDebug(s"Get Token from hive metastore: ${hive2Token.toString}") - creds.addToken(new Text("hive.server2.delegation.token"), hive2Token) + creds.addToken(tokenAlias, hive2Token) } None @@ -134,4 +140,6 @@ private[spark] class HiveDelegationTokenProvider case e: UndeclaredThrowableException => throw Option(e.getCause()).getOrElse(e) } } + + private def tokenAlias: Text = new Text("hive.server2.delegation.token") } 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 e8b7fc0ef100..9e768c22c17e 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 @@ -104,6 +104,9 @@ private[spark] object CoarseGrainedClusterMessages { case class RegisterClusterManager(am: RpcEndpointRef) extends CoarseGrainedClusterMessage + // Used by YARN's client mode AM to retrieve the current set of delegation tokens. + object RetrieveDelegationTokens extends CoarseGrainedClusterMessage + // Request executors by specifying the new total number of executors desired // This includes executors already pending or running case class RequestExecutors( 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 329158a44d36..98ed2fffc0ac 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 @@ -162,11 +162,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } case UpdateDelegationTokens(newDelegationTokens) => - SparkHadoopUtil.get.addDelegationTokens(newDelegationTokens, conf) - delegationTokens.set(newDelegationTokens) - executorDataMap.values.foreach { ed => - ed.executorEndpoint.send(UpdateDelegationTokens(newDelegationTokens)) - } + updateDelegationTokens(newDelegationTokens) case RemoveExecutor(executorId, reason) => // We will remove the executor's state and cannot restore it. However, the connection @@ -404,17 +400,18 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp driverEndpoint = createDriverEndpointRef(properties) if (UserGroupInformation.isSecurityEnabled()) { - delegationTokenManager = createTokenManager() + delegationTokenManager = createTokenManager(driverEndpoint) delegationTokenManager.foreach { dtm => - dtm.setDriverRef(driverEndpoint) - val creds = if (dtm.renewalEnabled) { - dtm.start().getCredentials() + val tokens = if (dtm.renewalEnabled) { + dtm.start() } else { val creds = UserGroupInformation.getCurrentUser().getCredentials() dtm.obtainDelegationTokens(creds) - creds + SparkHadoopUtil.get.serialize(creds) + } + if (tokens != null) { + delegationTokens.set(tokens) } - delegationTokens.set(SparkHadoopUtil.get.serialize(creds)) } } } @@ -716,8 +713,27 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * Create the delegation token manager to be used for the application. This method is called * once during the start of the scheduler backend (so after the object has already been * fully constructed), only if security is enabled in the Hadoop configuration. + * + * @param schedulerRef RPC endpoint for the scheduler, where updated delegation tokens should be + * sent. */ - protected def createTokenManager(): Option[HadoopDelegationTokenManager] = None + protected def createTokenManager( + schedulerRef: RpcEndpointRef): Option[HadoopDelegationTokenManager] = None + + /** + * Called when a new set of delegation tokens is sent to the driver. Child classes can override + * this method but should always call this implementation, which handles token distribution to + * executors. + */ + protected def updateDelegationTokens(tokens: Array[Byte]): Unit = { + SparkHadoopUtil.get.addDelegationTokens(tokens, conf) + delegationTokens.set(tokens) + executorDataMap.values.foreach { ed => + ed.executorEndpoint.send(UpdateDelegationTokens(tokens)) + } + } + + protected def currentDelegationTokens: Array[Byte] = delegationTokens.get() } diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index def9e626a2df..af7d44b160fe 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -27,7 +27,7 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { private val hadoopConf = new Configuration() test("default configuration") { - val manager = new HadoopDelegationTokenManager(new SparkConf(false), hadoopConf) + val manager = new HadoopDelegationTokenManager(new SparkConf(false), hadoopConf, null) assert(manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) assert(manager.isProviderLoaded("hive")) @@ -36,7 +36,7 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { test("disable hive credential provider") { val sparkConf = new SparkConf(false).set("spark.security.credentials.hive.enabled", "false") - val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf) + val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null) assert(manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) assert(!manager.isProviderLoaded("hive")) @@ -47,7 +47,7 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set("spark.yarn.security.tokens.hadoopfs.enabled", "false") .set("spark.yarn.security.credentials.hive.enabled", "false") - val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf) + val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null) assert(!manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) assert(!manager.isProviderLoaded("hive")) @@ -99,7 +99,7 @@ private object NoHiveTest { def runTest(): Unit = { try { - val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration()) + val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration(), null) require(!manager.isProviderLoaded("hive")) } catch { case e: Throwable => diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index 721d7e97b21f..a77e8d4dbcff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -91,7 +91,7 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri private lazy val delegationTokens: Array[Byte] = { if (keytab.isEmpty && existingSecretName.isEmpty) { val tokenManager = new HadoopDelegationTokenManager(kubernetesConf.sparkConf, - SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf)) + SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf), null) val creds = UserGroupInformation.getCurrentUser().getCredentials() tokenManager.obtainDelegationTokens(creds) // If no tokens and no secrets are stored in the credentials, make sure nothing is returned, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index cd298971e02a..e285e202a148 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -26,7 +26,7 @@ import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.rpc.{RpcAddress, RpcEnv} +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} import org.apache.spark.util.{ThreadUtils, Utils} @@ -147,8 +147,9 @@ private[spark] class KubernetesClusterSchedulerBackend( new KubernetesDriverEndpoint(sc.env.rpcEnv, properties) } - override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { - Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration)) + override protected def createTokenManager( + schedulerRef: RpcEndpointRef): Option[HadoopDelegationTokenManager] = { + Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration, schedulerRef)) } private class KubernetesDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) 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 d0174516c236..03cd2583b9b2 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 @@ -36,7 +36,7 @@ import org.apache.spark.internal.config import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient -import org.apache.spark.rpc.RpcEndpointAddress +import org.apache.spark.rpc.{RpcEndpointAddress, RpcEndpointRef} import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -772,8 +772,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } } - override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { - Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration)) + override protected def createTokenManager( + schedulerRef: RpcEndpointRef): Option[HadoopDelegationTokenManager] = { + Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration, schedulerRef)) } private def numExecutors(): Int = { 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 8dbdac168f70..1ece7bdc979c 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 @@ -30,6 +30,7 @@ import scala.util.control.NonFatal import org.apache.commons.lang3.{StringUtils => ComStrUtils} import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ @@ -41,7 +42,6 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.metrics.MetricsSystem @@ -58,6 +58,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. + private val appAttemptId = YarnSparkHadoopUtil.getContainerId.getApplicationAttemptId() private val isClusterMode = args.userClass != null private val sparkConf = new SparkConf() @@ -99,25 +100,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } } - private val tokenManager: Option[YARNHadoopDelegationTokenManager] = { - sparkConf.get(KEYTAB).map { _ => - new YARNHadoopDelegationTokenManager(sparkConf, yarnConf) - } - } - - private val ugi = tokenManager match { - case Some(tm) => - // Set the context class loader so that the token renewer has access to jars distributed - // by the user. - Utils.withContextClassLoader(userClassLoader) { - tm.start() - } - - case _ => - SparkHadoopUtil.get.createSparkUser() - } - - private val client = doAsUser { new YarnRMClient() } + private val client = new YarnRMClient() // Default to twice the number of executors (twice the maximum number of executors if dynamic // allocation is enabled), with a minimum of 3. @@ -174,11 +157,19 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends // In cluster mode, used to tell the AM when the user's SparkContext has been initialized. private val sparkContextPromise = Promise[SparkContext]() - // Load the list of localized files set by the client. This is used when launching executors, - // and is loaded here so that these configs don't pollute the Web UI's environment page in - // cluster mode. - private val localResources = doAsUser { + /** + * Load the list of localized files set by the client, used when launching executors. This should + * be called in a context where the needed credentials to access HDFS are available. + */ + private def prepareLocalResources(): Map[String, LocalResource] = { logInfo("Preparing Local resources") + val distCacheConf = new SparkConf(false) + if (args.distCacheConf != null) { + Utils.getPropertiesFromFile(args.distCacheConf).foreach { case (k, v) => + distCacheConf.set(k, v) + } + } + val resources = HashMap[String, LocalResource]() def setupDistributedCache( @@ -199,11 +190,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends resources(fileName) = amJarRsrc } - val distFiles = sparkConf.get(CACHED_FILES) - val fileSizes = sparkConf.get(CACHED_FILES_SIZES) - val timeStamps = sparkConf.get(CACHED_FILES_TIMESTAMPS) - val visibilities = sparkConf.get(CACHED_FILES_VISIBILITIES) - val resTypes = sparkConf.get(CACHED_FILES_TYPES) + val distFiles = distCacheConf.get(CACHED_FILES) + val fileSizes = distCacheConf.get(CACHED_FILES_SIZES) + val timeStamps = distCacheConf.get(CACHED_FILES_TIMESTAMPS) + val visibilities = distCacheConf.get(CACHED_FILES_VISIBILITIES) + val resTypes = distCacheConf.get(CACHED_FILES_TYPES) for (i <- 0 to distFiles.size - 1) { val resType = LocalResourceType.valueOf(resTypes(i)) @@ -212,7 +203,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } // Distribute the conf archive to executors. - sparkConf.get(CACHED_CONF_ARCHIVE).foreach { path => + distCacheConf.get(CACHED_CONF_ARCHIVE).foreach { path => val uri = new URI(path) val fs = FileSystem.get(uri, yarnConf) val status = fs.getFileStatus(new Path(uri)) @@ -225,33 +216,12 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends LocalResourceVisibility.PRIVATE.name()) } - // Clean up the configuration so it doesn't show up in the Web UI (since it's really noisy). - CACHE_CONFIGS.foreach { e => - sparkConf.remove(e) - sys.props.remove(e.key) - } - resources.toMap } - def getAttemptId(): ApplicationAttemptId = { - client.getAttemptId() - } - final def run(): Int = { - doAsUser { - runImpl() - } - exitCode - } - - private def runImpl(): Unit = { try { - val appAttemptId = client.getAttemptId() - - var attemptID: Option[String] = None - - if (isClusterMode) { + 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("spark.ui.port", "0") @@ -264,7 +234,9 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends // configuration will be checked in SparkContext to avoid misuse of yarn cluster mode. System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) - attemptID = Option(appAttemptId.getAttemptId.toString) + Option(appAttemptId.getAttemptId.toString) + } else { + None } new CallerContext( @@ -277,7 +249,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends val priority = ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY - 1 ShutdownHookManager.addShutdownHook(priority) { () => val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf) - val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts + val isLastAttempt = appAttemptId.getAttemptId() >= maxAppAttempts if (!finished) { // The default state of ApplicationMaster is failed if it is invoked by shut down hook. @@ -322,6 +294,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends logWarning("Exception during stopping of the metric system: ", e) } } + + exitCode } /** @@ -377,9 +351,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends logDebug("shutting down user thread") userClassThread.interrupt() } - if (!inShutdown) { - tokenManager.foreach(_.stop()) - } } } } @@ -405,8 +376,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends port: Int, _sparkConf: SparkConf, uiAddress: Option[String]): Unit = { - val appId = client.getAttemptId().getApplicationId().toString() - val attemptId = client.getAttemptId().getAttemptId().toString() + val appId = appAttemptId.getApplicationId().toString() + val attemptId = appAttemptId.getAttemptId().toString() val historyAddress = ApplicationMaster .getHistoryServerAddress(_sparkConf, yarnConf, appId, attemptId) @@ -415,9 +386,20 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } private def createAllocator(driverRef: RpcEndpointRef, _sparkConf: SparkConf): Unit = { - val appId = client.getAttemptId().getApplicationId().toString() + // In client mode, the AM may be restarting after delegation tokens have reached their TTL. So + // always contact the driver to get the current set of valid tokens, so that local resources can + // be initialized below. + if (!isClusterMode) { + val tokens = driverRef.askSync[Array[Byte]](RetrieveDelegationTokens) + if (tokens != null) { + SparkHadoopUtil.get.addDelegationTokens(tokens, _sparkConf) + } + } + + val appId = appAttemptId.getApplicationId().toString() val driverUrl = RpcEndpointAddress(driverRef.address.host, driverRef.address.port, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + val localResources = prepareLocalResources() // Before we initialize the allocator, let's log the information about how executors will // be run up front, to avoid printing this out for every single executor being launched. @@ -433,13 +415,12 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends allocator = client.createAllocator( yarnConf, _sparkConf, + appAttemptId, driverUrl, driverRef, securityMgr, localResources) - tokenManager.foreach(_.setDriverRef(driverRef)) - // Initialize the AM endpoint *after* the allocator has been initialized. This ensures // that when the driver sends an initial executor request (e.g. after an AM restart), // the allocator is ready to service requests. @@ -755,6 +736,9 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends case None => logWarning("Container allocator is not ready to find executor loss reasons yet.") } + + case UpdateDelegationTokens(tokens) => + SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) } override def onDisconnected(remoteAddress: RpcAddress): Unit = { @@ -767,12 +751,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } } - private def doAsUser[T](fn: => T): T = { - ugi.doAs(new PrivilegedExceptionAction[T]() { - override def run: T = fn - }) - } - } object ApplicationMaster extends Logging { @@ -793,7 +771,24 @@ object ApplicationMaster extends Logging { SignalUtils.registerLogger(log) val amArgs = new ApplicationMasterArguments(args) master = new ApplicationMaster(amArgs) - System.exit(master.run()) + + val ugi = master.sparkConf.get(PRINCIPAL) match { + case Some(principal) => + val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() + SparkHadoopUtil.get.loginUserFromKeytab(principal, master.sparkConf.get(KEYTAB).orNull) + val newUGI = UserGroupInformation.getCurrentUser() + // Transfer the original user's tokens to the new user, since it may contain needed tokens + // (such as those user to connect to YARN). + newUGI.addCredentials(originalCreds) + newUGI + + case _ => + SparkHadoopUtil.get.createSparkUser() + } + + ugi.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = System.exit(master.run()) + }) } private[spark] def sparkContextInitialized(sc: SparkContext): Unit = { @@ -801,7 +796,7 @@ object ApplicationMaster extends Logging { } private[spark] def getAttemptId(): ApplicationAttemptId = { - master.getAttemptId + master.appAttemptId } private[spark] def getHistoryServerAddress( diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index cc76a7c8f13f..c10206c84727 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -26,6 +26,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var primaryRFile: String = null var userArgs: Seq[String] = Nil var propertiesFile: String = null + var distCacheConf: String = null parseArgs(args.toList) @@ -62,6 +63,10 @@ class ApplicationMasterArguments(val args: Array[String]) { propertiesFile = value args = tail + case ("--dist-cache-conf") :: value :: tail => + distCacheConf = value + args = tail + case _ => printUsageAndExit(1, args) } 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 44a60b835f12..9f09dc031754 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 @@ -100,21 +100,19 @@ private[spark] class Client( } private val distCacheMgr = new ClientDistributedCacheManager() + private val cachedResourcesConf = new SparkConf(false) - private val principal = sparkConf.get(PRINCIPAL).orNull private val keytab = sparkConf.get(KEYTAB).orNull - private val loginFromKeytab = principal != null - private val amKeytabFileName: String = { + private val amKeytabFileName: Option[String] = if (keytab != null && isClusterMode) { + val principal = sparkConf.get(PRINCIPAL).orNull require((principal == null) == (keytab == null), "Both principal and keytab must be defined, or neither.") - if (loginFromKeytab) { - logInfo(s"Kerberos credentials: principal = $principal, keytab = $keytab") - // Generate a file name that can be used for the keytab file, that does not conflict - // with any user file. - new File(keytab).getName() + "-" + UUID.randomUUID().toString - } else { - null - } + logInfo(s"Kerberos credentials: principal = $principal, keytab = $keytab") + // Generate a file name that can be used for the keytab file, that does not conflict + // with any user file. + Some(new File(keytab).getName() + "-" + UUID.randomUUID().toString) + } else { + None } require(keytab == null || !Utils.isLocalUri(keytab), "Keytab should reference a local file.") @@ -220,16 +218,7 @@ private[spark] class Client( } } - if (isClusterMode && principal != null && keytab != null) { - val newUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) - newUgi.doAs(new PrivilegedExceptionAction[Unit] { - override def run(): Unit = { - cleanupStagingDirInternal() - } - }) - } else { - cleanupStagingDirInternal() - } + cleanupStagingDirInternal() } /** @@ -312,7 +301,7 @@ private[spark] class Client( */ private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { val credentials = UserGroupInformation.getCurrentUser().getCredentials() - val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) + val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf, null) credentialManager.obtainDelegationTokens(credentials) // When using a proxy user, copy the delegation tokens to the user's credentials. Avoid @@ -496,11 +485,11 @@ private[spark] class Client( // If we passed in a keytab, make sure we copy the keytab to the staging directory on // HDFS, and setup the relevant environment vars, so the AM can login again. - if (loginFromKeytab) { + amKeytabFileName.foreach { kt => logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" + " via the YARN Secure Distributed Cache.") val (_, localizedPath) = distribute(keytab, - destName = Some(amKeytabFileName), + destName = Some(kt), appMasterOnly = true) require(localizedPath != null, "Keytab file already distributed.") } @@ -636,7 +625,7 @@ private[spark] class Client( // Update the configuration with all the distributed files, minus the conf archive. The // conf archive will be handled by the AM differently so that we avoid having to send // this configuration by other means. See SPARK-14602 for one reason of why this is needed. - distCacheMgr.updateConfiguration(sparkConf) + distCacheMgr.updateConfiguration(cachedResourcesConf) // Upload the conf archive to HDFS manually, and record its location in the configuration. // This will allow the AM to know where the conf archive is in HDFS, so that it can be @@ -648,7 +637,7 @@ private[spark] class Client( // system. val remoteConfArchivePath = new Path(destDir, LOCALIZED_CONF_ARCHIVE) val remoteFs = FileSystem.get(remoteConfArchivePath.toUri(), hadoopConf) - sparkConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString()) + cachedResourcesConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString()) val localConfArchive = new Path(createConfArchive().toURI()) copyFileToRemote(destDir, localConfArchive, replication, symlinkCache, force = true, @@ -660,11 +649,6 @@ private[spark] class Client( remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE, LOCALIZED_CONF_DIR, statCache, appMasterOnly = false) - // Clear the cache-related entries from the configuration to avoid them polluting the - // UI's environment page. This works for client mode; for cluster mode, this is handled - // by the AM. - CACHE_CONFIGS.foreach(sparkConf.remove) - localResources } @@ -768,19 +752,25 @@ private[spark] class Client( hadoopConf.writeXml(confStream) confStream.closeEntry() - // Save Spark configuration to a file in the archive, but filter out the app's secret. - val props = new Properties() - sparkConf.getAll.foreach { case (k, v) => - props.setProperty(k, v) + // Save Spark configuration to a file in the archive. + val props = confToProperties(sparkConf) + + // If propagating the keytab to the AM, override the keytab name with the name of the + // distributed file. Otherwise remove princpal/keytab from the conf, so they're not seen + // by the AM at all. + amKeytabFileName match { + case Some(kt) => + props.setProperty(KEYTAB.key, kt) + case None => + props.remove(PRINCIPAL.key) + props.remove(KEYTAB.key) } - // Override spark.yarn.key to point to the location in distributed cache which will be used - // by AM. - Option(amKeytabFileName).foreach { k => props.setProperty(KEYTAB.key, k) } - confStream.putNextEntry(new ZipEntry(SPARK_CONF_FILE)) - val writer = new OutputStreamWriter(confStream, StandardCharsets.UTF_8) - props.store(writer, "Spark configuration.") - writer.flush() - confStream.closeEntry() + + writePropertiesToArchive(props, SPARK_CONF_FILE, confStream) + + // Write the distributed cache config to the archive. + writePropertiesToArchive(confToProperties(cachedResourcesConf), DIST_CACHE_CONF_FILE, + confStream) } finally { confStream.close() } @@ -984,7 +974,10 @@ private[spark] class Client( } val amArgs = Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ primaryRFile ++ userArgs ++ - Seq("--properties-file", buildPath(Environment.PWD.$$(), LOCALIZED_CONF_DIR, SPARK_CONF_FILE)) + Seq("--properties-file", + buildPath(Environment.PWD.$$(), LOCALIZED_CONF_DIR, SPARK_CONF_FILE)) ++ + Seq("--dist-cache-conf", + buildPath(Environment.PWD.$$(), LOCALIZED_CONF_DIR, DIST_CACHE_CONF_FILE)) // Command for the ApplicationMaster val commands = prefixEnv ++ @@ -1213,6 +1206,9 @@ private object Client extends Logging { // Name of the file in the conf archive containing Spark configuration. val SPARK_CONF_FILE = "__spark_conf__.properties" + // Name of the file in the conf archive containing the distributed cache info. + val DIST_CACHE_CONF_FILE = "__spark_dist_cache__.properties" + // Subdirectory where the user's python files (not archives) will be placed. val LOCALIZED_PYTHON_DIR = "__pyfiles__" @@ -1512,6 +1508,22 @@ private object Client extends Logging { } getClusterPath(conf, cmdPrefix) } + + def confToProperties(conf: SparkConf): Properties = { + val props = new Properties() + conf.getAll.foreach { case (k, v) => + props.setProperty(k, v) + } + props + } + + def writePropertiesToArchive(props: Properties, name: String, out: ZipOutputStream): Unit = { + out.putNextEntry(new ZipEntry(name)) + val writer = new OutputStreamWriter(out, StandardCharsets.UTF_8) + props.store(writer, "Spark configuration.") + writer.flush() + out.closeEntry() + } } private[spark] class YarnClusterApplication extends SparkApplication { 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 05a7b1e1310c..cf16edf16c03 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 @@ -76,12 +76,13 @@ private[spark] class YarnRMClient extends Logging { def createAllocator( conf: YarnConfiguration, sparkConf: SparkConf, + appAttemptId: ApplicationAttemptId, driverUrl: String, driverRef: RpcEndpointRef, securityMgr: SecurityManager, localResources: Map[String, LocalResource]): YarnAllocator = { require(registered, "Must register AM before creating allocator.") - new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr, + new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, appAttemptId, securityMgr, localResources, new SparkRackResolver()) } @@ -100,11 +101,6 @@ private[spark] class YarnRMClient extends Logging { } } - /** Returns the attempt ID. */ - def getAttemptId(): ApplicationAttemptId = { - YarnSparkHadoopUtil.getContainerId.getApplicationAttemptId() - } - /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = { // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, 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 7e9cd409daf3..6091cd496c03 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 @@ -321,16 +321,6 @@ package object config { .stringConf .createOptional - // The list of cache-related config entries. This is used by Client and the AM to clean - // up the environment so that these settings do not appear on the web UI. - private[yarn] val CACHE_CONFIGS = Seq( - CACHED_FILES, - CACHED_FILES_SIZES, - CACHED_FILES_TIMESTAMPS, - CACHED_FILES_VISIBILITIES, - CACHED_FILES_TYPES, - CACHED_CONF_ARCHIVE) - /* YARN allocator-level blacklisting related config entries. */ private[spark] val YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED = ConfigBuilder("spark.yarn.blacklist.executor.launch.blacklisting.enabled") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala index 2d9a3f0c83fd..bb40ea801519 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala @@ -36,10 +36,11 @@ import org.apache.spark.util.Utils * [[ServiceCredentialProvider]] interface, as well as the builtin providers defined * in [[HadoopDelegationTokenManager]]. */ -private[yarn] class YARNHadoopDelegationTokenManager( +private[spark] class YARNHadoopDelegationTokenManager( _sparkConf: SparkConf, - _hadoopConf: Configuration) - extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf) { + _hadoopConf: Configuration, + _schedulerRef: RpcEndpointRef) + extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf, _schedulerRef) { private val credentialProviders = { ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 167eef19ed85..934fba3e6ff3 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -27,6 +27,7 @@ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.SparkAppHandle import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ private[spark] class YarnClientSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -166,4 +167,9 @@ private[spark] class YarnClientSchedulerBackend( logInfo("Stopped") } + override protected def updateDelegationTokens(tokens: Array[Byte]): Unit = { + super.updateDelegationTokens(tokens) + amEndpoint.foreach(_.send(UpdateDelegationTokens(tokens))) + } + } 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 1289d4be79ea..6357d4adbcd9 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 @@ -27,6 +27,8 @@ import scala.util.control.NonFatal import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} import org.apache.spark.SparkContext +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.rpc._ import org.apache.spark.scheduler._ @@ -55,6 +57,7 @@ private[spark] abstract class YarnSchedulerBackend( protected var totalExpectedExecutors = 0 private val yarnSchedulerEndpoint = new YarnSchedulerEndpoint(rpcEnv) + protected var amEndpoint: Option[RpcEndpointRef] = None private val yarnSchedulerEndpointRef = rpcEnv.setupEndpoint( YarnSchedulerBackend.ENDPOINT_NAME, yarnSchedulerEndpoint) @@ -191,6 +194,11 @@ private[spark] abstract class YarnSchedulerBackend( sc.executorAllocationManager.foreach(_.reset()) } + override protected def createTokenManager( + schedulerRef: RpcEndpointRef): Option[HadoopDelegationTokenManager] = { + Some(new YARNHadoopDelegationTokenManager(sc.conf, sc.hadoopConfiguration, schedulerRef)) + } + /** * Override the DriverEndpoint to add extra logic for the case when an executor is disconnected. * This endpoint communicates with the executors and queries the AM for an executor's exit @@ -226,7 +234,6 @@ private[spark] abstract class YarnSchedulerBackend( */ private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint with Logging { - private var amEndpoint: Option[RpcEndpointRef] = None private[YarnSchedulerBackend] def handleExecutorDisconnectedFromDriver( executorId: String, @@ -266,11 +273,6 @@ private[spark] abstract class YarnSchedulerBackend( logWarning(s"Requesting driver to remove executor $executorId for reason $reason") driverEndpoint.send(r) } - - case u @ UpdateDelegationTokens(tokens) => - // Add the tokens to the current user and send a message to the scheduler so that it - // notifies all registered executors of the new tokens. - driverEndpoint.send(u) } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -304,6 +306,9 @@ private[spark] abstract class YarnSchedulerBackend( case RetrieveLastAllocatedExecutorId => context.reply(currentExecutorIdCounter) + + case RetrieveDelegationTokens => + context.reply(currentDelegationTokens) } override def onDisconnected(remoteAddress: RpcAddress): Unit = { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala index 98315e423574..f00453cb9c59 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala @@ -34,7 +34,7 @@ class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite { } test("Correctly loads credential providers") { - credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) + credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf, null) assert(credentialManager.isProviderLoaded("yarn-test")) } } From 98be8953c75c026c1cb432cc8f66dd312feed0c6 Mon Sep 17 00:00:00 2001 From: maryannxue Date: Mon, 7 Jan 2019 13:59:40 -0800 Subject: [PATCH 0247/1072] [SPARK-26065][SQL] Change query hint from a `LogicalPlan` to a field ## What changes were proposed in this pull request? The existing query hint implementation relies on a logical plan node `ResolvedHint` to store query hints in logical plans, and on `Statistics` in physical plans. Since `ResolvedHint` is not really a logical operator and can break the pattern matching for existing and future optimization rules, it is a issue to the Optimizer as the old `AnalysisBarrier` was to the Analyzer. Given the fact that all our query hints are either 1) a join hint, i.e., broadcast hint; or 2) a re-partition hint, which is indeed an operator, we only need to add a hint field on the Join plan and that will be a good enough solution for the current hint usage. This PR is to let `Join` node have a hint for its left sub-tree and another hint for its right sub-tree and each hint is a merged result of all the effective hints specified in the corresponding sub-tree. The "effectiveness" of a hint, i.e., whether that hint should be propagated to the `Join` node, is currently consistent with the hint propagation rules originally implemented in the `Statistics` approach. Note that the `ResolvedHint` node still has to live through the analysis stage because of the `Dataset` interface, but it will be got rid of and moved to the `Join` node in the "pre-optimization" stage. This PR also introduces a change in how hints work with join reordering. Before this PR, hints would stop join reordering. For example, in "a.join(b).join(c).hint("broadcast").join(d)", the broadcast hint would stop d from participating in the cost-based join reordering while still allowing reordering from under the hint node. After this PR, though, the broadcast hint will not interfere with join reordering at all, and after reordering if a relation associated with a hint stays unchanged or equivalent to the original relation, the hint will be retained, otherwise will be discarded. For example, the original plan is like "a.join(b).hint("broadcast").join(c).hint("broadcast").join(d)", thus the join order is "a JOIN b JOIN c JOIN d". So if after reordering the join order becomes "a JOIN b JOIN (c JOIN d)", the plan will be like "a.join(b).hint("broadcast").join(c.join(d))"; but if after reordering the join order becomes "a JOIN c JOIN b JOIN d", the plan will be like "a.join(c).join(b).hint("broadcast").join(d)". ## How was this patch tested? Added new tests. Closes #23036 from maryannxue/query-hint. Authored-by: maryannxue Signed-off-by: gatorsmile --- .../sql/catalyst/analysis/Analyzer.scala | 16 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 4 +- .../analysis/StreamingJoinHelper.scala | 2 +- .../UnsupportedOperationChecker.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../optimizer/CostBasedJoinReorder.scala | 84 ++++++-- .../optimizer/EliminateResolvedHint.scala | 59 ++++++ .../sql/catalyst/optimizer/Optimizer.scala | 36 ++-- .../optimizer/PropagateEmptyRelation.scala | 2 +- .../ReplaceNullWithFalseInPredicate.scala | 2 +- .../sql/catalyst/optimizer/expressions.scala | 3 +- .../spark/sql/catalyst/optimizer/joins.scala | 27 ++- .../sql/catalyst/optimizer/subquery.scala | 14 +- .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../sql/catalyst/planning/patterns.scala | 41 ++-- .../plans/logical/LogicalPlanVisitor.scala | 3 - .../catalyst/plans/logical/Statistics.scala | 7 +- .../plans/logical/basicLogicalOperators.scala | 14 +- .../sql/catalyst/plans/logical/hints.scala | 27 ++- .../statsEstimation/AggregateEstimation.scala | 3 +- .../BasicStatsPlanVisitor.scala | 2 - .../statsEstimation/JoinEstimation.scala | 2 +- .../SizeInBytesOnlyStatsPlanVisitor.scala | 22 +- .../analysis/AnalysisErrorSuite.scala | 8 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 4 +- .../catalyst/analysis/ResolveHintsSuite.scala | 2 +- .../optimizer/ColumnPruningSuite.scala | 6 +- .../optimizer/FilterPushdownSuite.scala | 14 -- .../optimizer/JoinOptimizationSuite.scala | 28 +-- .../catalyst/optimizer/JoinReorderSuite.scala | 83 +++++++- .../optimizer/ReplaceOperatorSuite.scala | 8 +- .../spark/sql/catalyst/plans/PlanTest.scala | 10 +- .../sql/catalyst/plans/SameResultSuite.scala | 16 +- .../BasicStatsEstimationSuite.scala | 18 -- .../FilterEstimationSuite.scala | 2 +- .../statsEstimation/JoinEstimationSuite.scala | 31 +-- .../scala/org/apache/spark/sql/Dataset.scala | 16 +- .../spark/sql/execution/SparkStrategies.scala | 52 ++--- .../execution/columnar/InMemoryRelation.scala | 9 +- .../apache/spark/sql/CachedTableSuite.scala | 21 ++ .../apache/spark/sql/DataFrameJoinSuite.scala | 10 +- .../org/apache/spark/sql/JoinHintSuite.scala | 193 ++++++++++++++++++ .../spark/sql/StatisticsCollectionSuite.scala | 3 +- .../execution/joins/BroadcastJoinSuite.scala | 14 +- .../execution/joins/ExistenceJoinSuite.scala | 11 +- .../sql/execution/joins/InnerJoinSuite.scala | 15 +- .../sql/execution/joins/OuterJoinSuite.scala | 11 +- 47 files changed, 680 insertions(+), 283 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.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 198645d875c4..2aa0f2117364 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 @@ -943,7 +943,7 @@ class Analyzer( failAnalysis("Invalid usage of '*' in explode/json_tuple/UDTF") // To resolve duplicate expression IDs for Join and Intersect - case j @ Join(left, right, _, _) if !j.duplicateResolved => + case j @ Join(left, right, _, _, _) if !j.duplicateResolved => j.copy(right = dedupRight(left, right)) case i @ Intersect(left, right, _) if !i.duplicateResolved => i.copy(right = dedupRight(left, right)) @@ -2249,13 +2249,14 @@ class Analyzer( */ object ResolveNaturalAndUsingJoin extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case j @ Join(left, right, UsingJoin(joinType, usingCols), _) + case j @ Join(left, right, UsingJoin(joinType, usingCols), _, hint) if left.resolved && right.resolved && j.duplicateResolved => - commonNaturalJoinProcessing(left, right, joinType, usingCols, None) - case j @ Join(left, right, NaturalJoin(joinType), condition) if j.resolvedExceptNatural => + commonNaturalJoinProcessing(left, right, joinType, usingCols, None, hint) + case j @ Join(left, right, NaturalJoin(joinType), condition, hint) + if j.resolvedExceptNatural => // find common column names from both sides val joinNames = left.output.map(_.name).intersect(right.output.map(_.name)) - commonNaturalJoinProcessing(left, right, joinType, joinNames, condition) + commonNaturalJoinProcessing(left, right, joinType, joinNames, condition, hint) } } @@ -2360,7 +2361,8 @@ class Analyzer( right: LogicalPlan, joinType: JoinType, joinNames: Seq[String], - condition: Option[Expression]) = { + condition: Option[Expression], + hint: JoinHint) = { val leftKeys = joinNames.map { keyName => left.output.find(attr => resolver(attr.name, keyName)).getOrElse { throw new AnalysisException(s"USING column `$keyName` cannot be resolved on the left " + @@ -2401,7 +2403,7 @@ class Analyzer( sys.error("Unsupported natural join type " + joinType) } // use Project to trim unnecessary fields - Project(projectList, Join(left, right, joinType, newCondition)) + Project(projectList, Join(left, right, joinType, newCondition, hint)) } /** 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 c28a97839fe4..18c40b370cb5 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 @@ -172,7 +172,7 @@ trait CheckAnalysis extends PredicateHelper { failAnalysis("Null-aware predicate sub-queries cannot be used in nested " + s"conditions: $condition") - case j @ Join(_, _, _, Some(condition)) if condition.dataType != BooleanType => + case j @ Join(_, _, _, Some(condition), _) if condition.dataType != BooleanType => failAnalysis( s"join condition '${condition.sql}' " + s"of type ${condition.dataType.catalogString} is not a boolean.") @@ -609,7 +609,7 @@ trait CheckAnalysis extends PredicateHelper { failOnNonEqualCorrelatedPredicate(foundNonEqualCorrelatedPred, a) // Join can host correlated expressions. - case j @ Join(left, right, joinType, _) => + case j @ Join(left, right, joinType, _, _) => joinType match { // Inner join, like Filter, can be anywhere. case _: InnerLike => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala index 7a0aa08289ef..76733dd6dac3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala @@ -41,7 +41,7 @@ object StreamingJoinHelper extends PredicateHelper with Logging { */ def isWatermarkInJoinKeys(plan: LogicalPlan): Boolean = { plan match { - case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, _) => + case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, _, _) => (leftKeys ++ rightKeys).exists { case a: AttributeReference => a.metadata.contains(EventTimeWatermark.delayKey) case _ => false 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 cff4cee09427..41ba6d34b549 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 @@ -229,7 +229,7 @@ object UnsupportedOperationChecker { throwError("dropDuplicates is not supported after aggregation on a " + "streaming DataFrame/Dataset") - case Join(left, right, joinType, condition) => + case Join(left, right, joinType, condition, _) => joinType match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 151481c80ee9..846ee3b38652 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -325,7 +325,7 @@ package object dsl { otherPlan: LogicalPlan, joinType: JoinType = Inner, condition: Option[Expression] = None): LogicalPlan = - Join(logicalPlan, otherPlan, joinType, condition) + Join(logicalPlan, otherPlan, joinType, condition, JoinHint.NONE) def cogroup[Key: Encoder, Left: Encoder, Right: Encoder, Result: Encoder]( otherPlan: LogicalPlan, 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 01634a9d852c..743d3ce944fe 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 @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper} import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike, JoinType} -import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -31,6 +31,40 @@ import org.apache.spark.sql.internal.SQLConf * Cost-based join reorder. * We may have several join reorder algorithms in the future. This class is the entry of these * algorithms, and chooses which one to use. + * + * Note that join strategy hints, e.g. the broadcast hint, do not interfere with the reordering. + * Such hints will be applied on the equivalent counterparts (i.e., join between the same relations + * regardless of the join order) of the original nodes after reordering. + * For example, the plan before reordering is like: + * + * Join + * / \ + * Hint1 t4 + * / + * Join + * / \ + * Join t3 + * / \ + * Hint2 t2 + * / + * t1 + * + * The original join order as illustrated above is "((t1 JOIN t2) JOIN t3) JOIN t4", and after + * reordering, the new join order is "((t1 JOIN t3) JOIN t2) JOIN t4", so the new plan will be like: + * + * Join + * / \ + * Hint1 t4 + * / + * Join + * / \ + * Join t2 + * / \ + * t1 t3 + * + * "Hint1" is applied on "(t1 JOIN t3) JOIN t2" as it is equivalent to the original hinted node, + * "(t1 JOIN t2) JOIN t3"; while "Hint2" has disappeared from the new plan since there is no + * equivalent node to "t1 JOIN t2". */ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { @@ -40,24 +74,30 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { if (!conf.cboEnabled || !conf.joinReorderEnabled) { plan } else { + // Use a map to track the hints on the join items. + val hintMap = new mutable.HashMap[AttributeSet, HintInfo] val result = plan transformDown { // Start reordering with a joinable item, which is an InnerLike join with conditions. - case j @ Join(_, _, _: InnerLike, Some(cond)) => - reorder(j, j.output) - case p @ Project(projectList, Join(_, _, _: InnerLike, Some(cond))) + case j @ Join(_, _, _: InnerLike, Some(cond), _) => + reorder(j, j.output, hintMap) + case p @ Project(projectList, Join(_, _, _: InnerLike, Some(cond), _)) if projectList.forall(_.isInstanceOf[Attribute]) => - reorder(p, p.output) + reorder(p, p.output, hintMap) } - - // After reordering is finished, convert OrderedJoin back to Join - result transformDown { - case OrderedJoin(left, right, jt, cond) => Join(left, right, jt, cond) + // After reordering is finished, convert OrderedJoin back to Join. + result transform { + case OrderedJoin(left, right, jt, cond) => + val joinHint = JoinHint(hintMap.get(left.outputSet), hintMap.get(right.outputSet)) + Join(left, right, jt, cond, joinHint) } } } - private def reorder(plan: LogicalPlan, output: Seq[Attribute]): LogicalPlan = { - val (items, conditions) = extractInnerJoins(plan) + private def reorder( + plan: LogicalPlan, + output: Seq[Attribute], + hintMap: mutable.HashMap[AttributeSet, HintInfo]): LogicalPlan = { + val (items, conditions) = extractInnerJoins(plan, hintMap) val result = // Do reordering if the number of items is appropriate and join conditions exist. // We also need to check if costs of all items can be evaluated. @@ -75,27 +115,31 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { * Extracts items of consecutive inner joins and join conditions. * This method works for bushy trees and left/right deep trees. */ - private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = { + private def extractInnerJoins( + plan: LogicalPlan, + hintMap: mutable.HashMap[AttributeSet, HintInfo]): (Seq[LogicalPlan], Set[Expression]) = { plan match { - case Join(left, right, _: InnerLike, Some(cond)) => - val (leftPlans, leftConditions) = extractInnerJoins(left) - val (rightPlans, rightConditions) = extractInnerJoins(right) + case Join(left, right, _: InnerLike, Some(cond), hint) => + hint.leftHint.foreach(hintMap.put(left.outputSet, _)) + hint.rightHint.foreach(hintMap.put(right.outputSet, _)) + val (leftPlans, leftConditions) = extractInnerJoins(left, hintMap) + val (rightPlans, rightConditions) = extractInnerJoins(right, hintMap) (leftPlans ++ rightPlans, splitConjunctivePredicates(cond).toSet ++ leftConditions ++ rightConditions) - case Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond))) + case Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond), _)) if projectList.forall(_.isInstanceOf[Attribute]) => - extractInnerJoins(j) + extractInnerJoins(j, hintMap) case _ => (Seq(plan), Set()) } } private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match { - case j @ Join(left, right, jt: InnerLike, Some(cond)) => + case j @ Join(left, right, jt: InnerLike, Some(cond), _) => val replacedLeft = replaceWithOrderedJoin(left) val replacedRight = replaceWithOrderedJoin(right) OrderedJoin(replacedLeft, replacedRight, jt, Some(cond)) - case p @ Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond))) => + case p @ Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond), _)) => p.copy(child = replaceWithOrderedJoin(j)) case _ => plan @@ -295,7 +339,7 @@ object JoinReorderDP extends PredicateHelper with Logging { } else { (otherPlan, onePlan) } - val newJoin = Join(left, right, Inner, joinConds.reduceOption(And)) + val newJoin = Join(left, right, Inner, joinConds.reduceOption(And), JoinHint.NONE) val collectedJoinConds = joinConds ++ oneJoinPlan.joinConds ++ otherJoinPlan.joinConds val remainingConds = conditions -- collectedJoinConds val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala new file mode 100644 index 000000000000..bbe4eee4b432 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * Replaces [[ResolvedHint]] operators from the plan. Move the [[HintInfo]] to associated [[Join]] + * operators, otherwise remove it if no [[Join]] operator is matched. + */ +object EliminateResolvedHint extends Rule[LogicalPlan] { + // This is also called in the beginning of the optimization phase, and as a result + // is using transformUp rather than resolveOperators. + def apply(plan: LogicalPlan): LogicalPlan = { + val pulledUp = plan transformUp { + case j: Join => + val leftHint = mergeHints(collectHints(j.left)) + val rightHint = mergeHints(collectHints(j.right)) + j.copy(hint = JoinHint(leftHint, rightHint)) + } + pulledUp.transform { + case h: ResolvedHint => h.child + } + } + + private def mergeHints(hints: Seq[HintInfo]): Option[HintInfo] = { + hints.reduceOption((h1, h2) => HintInfo( + broadcast = h1.broadcast || h2.broadcast)) + } + + private def collectHints(plan: LogicalPlan): Seq[HintInfo] = { + plan match { + case h: ResolvedHint => collectHints(h.child) :+ h.hints + case u: UnaryNode => collectHints(u.child) + // TODO revisit this logic: + // except and intersect are semi/anti-joins which won't return more data then + // their left argument, so the broadcast hint should be propagated here + case i: Intersect => collectHints(i.left) + case e: Except => collectHints(e.left) + case _ => Seq.empty + } + } +} 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 44d554311490..06f908281dd3 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 @@ -115,6 +115,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) // However, because we also use the analyzer to canonicalized queries (for view definition), // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, + EliminateResolvedHint, EliminateSubqueryAliases, EliminateView, ReplaceExpressions, @@ -192,6 +193,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) */ def nonExcludableRules: Seq[String] = EliminateDistinct.ruleName :: + EliminateResolvedHint.ruleName :: EliminateSubqueryAliases.ruleName :: EliminateView.ruleName :: ReplaceExpressions.ruleName :: @@ -356,7 +358,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { // 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 // attribute is not on the black list. - case Join(left, right, joinType, condition) => + case Join(left, right, joinType, condition, hint) => val newLeft = removeRedundantAliases(left, blacklist ++ right.outputSet) val newRight = removeRedundantAliases(right, blacklist ++ newLeft.outputSet) val mapping = AttributeMap( @@ -365,7 +367,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { val newCondition = condition.map(_.transform { case a: Attribute => mapping.getOrElse(a, a) }) - Join(newLeft, newRight, joinType, newCondition) + Join(newLeft, newRight, joinType, newCondition, hint) case _ => // Remove redundant aliases in the subtree(s). @@ -460,7 +462,7 @@ object LimitPushDown extends Rule[LogicalPlan] { // on both sides if it is applied multiple times. Therefore: // - If one side is already limited, stack another limit on top if the new limit is smaller. // The redundant limit will be collapsed by the CombineLimits rule. - case LocalLimit(exp, join @ Join(left, right, joinType, _)) => + case LocalLimit(exp, join @ Join(left, right, joinType, _, _)) => val newJoin = joinType match { case RightOuter => join.copy(right = maybePushLocalLimit(exp, right)) case LeftOuter => join.copy(left = maybePushLocalLimit(exp, left)) @@ -578,7 +580,7 @@ object ColumnPruning extends Rule[LogicalPlan] { p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices)) // Eliminate unneeded attributes from right side of a Left Existence Join. - case j @ Join(_, right, LeftExistence(_), _) => + case j @ Join(_, right, LeftExistence(_), _, _) => j.copy(right = prunedChild(right, j.references)) // all the columns will be used to compare, so we can't prune them @@ -792,7 +794,7 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] filter } - case join @ Join(left, right, joinType, conditionOpt) => + case join @ Join(left, right, joinType, conditionOpt, _) => joinType match { // For inner join, we can infer additional filters for both sides. LeftSemi is kind of an // inner join, it just drops the right side in the final output. @@ -919,7 +921,6 @@ object RemoveRedundantSorts extends Rule[LogicalPlan] { def canEliminateSort(plan: LogicalPlan): Boolean = plan match { case p: Project => p.projectList.forall(_.deterministic) case f: Filter => f.condition.deterministic - case _: ResolvedHint => true case _ => false } } @@ -1094,7 +1095,6 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { // Note that some operators (e.g. project, aggregate, union) are being handled separately // (earlier in this rule). case _: AppendColumns => true - case _: ResolvedHint => true case _: Distinct => true case _: Generate => true case _: Pivot => true @@ -1179,7 +1179,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // push the where condition down into join filter - case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition)) => + case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition, hint)) => val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = split(splitConjunctivePredicates(filterCondition), left, right) joinType match { @@ -1193,7 +1193,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { commonFilterCondition.partition(canEvaluateWithinJoin) val newJoinCond = (newJoinConditions ++ joinCondition).reduceLeftOption(And) - val join = Join(newLeft, newRight, joinType, newJoinCond) + val join = Join(newLeft, newRight, joinType, newJoinCond, hint) if (others.nonEmpty) { Filter(others.reduceLeft(And), join) } else { @@ -1205,7 +1205,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newRight = rightFilterConditions. reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val newJoinCond = joinCondition - val newJoin = Join(newLeft, newRight, RightOuter, newJoinCond) + val newJoin = Join(newLeft, newRight, RightOuter, newJoinCond, hint) (leftFilterConditions ++ commonFilterCondition). reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) @@ -1215,7 +1215,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) val newRight = right val newJoinCond = joinCondition - val newJoin = Join(newLeft, newRight, joinType, newJoinCond) + val newJoin = Join(newLeft, newRight, joinType, newJoinCond, hint) (rightFilterConditions ++ commonFilterCondition). reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) @@ -1225,7 +1225,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { } // push down the join filter into sub query scanning if applicable - case j @ Join(left, right, joinType, joinCondition) => + case j @ Join(left, right, joinType, joinCondition, hint) => val (leftJoinConditions, rightJoinConditions, commonJoinCondition) = split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) @@ -1238,7 +1238,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val newJoinCond = commonJoinCondition.reduceLeftOption(And) - Join(newLeft, newRight, joinType, newJoinCond) + Join(newLeft, newRight, joinType, newJoinCond, hint) case RightOuter => // push down the left side only join filter for left side sub query val newLeft = leftJoinConditions. @@ -1246,7 +1246,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newRight = right val newJoinCond = (rightJoinConditions ++ commonJoinCondition).reduceLeftOption(And) - Join(newLeft, newRight, RightOuter, newJoinCond) + Join(newLeft, newRight, RightOuter, newJoinCond, hint) case LeftOuter | LeftAnti | ExistenceJoin(_) => // push down the right side only join filter for right sub query val newLeft = left @@ -1254,7 +1254,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val newJoinCond = (leftJoinConditions ++ commonJoinCondition).reduceLeftOption(And) - Join(newLeft, newRight, joinType, newJoinCond) + Join(newLeft, newRight, joinType, newJoinCond, hint) case FullOuter => j case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") case UsingJoin(_, _) => sys.error("Untransformed Using join node") @@ -1310,7 +1310,7 @@ object CheckCartesianProducts extends Rule[LogicalPlan] with PredicateHelper { if (SQLConf.get.crossJoinEnabled) { plan } else plan transform { - case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, _) + case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, _, _) if isCartesianProduct(j) => throw new AnalysisException( s"""Detected implicit cartesian product for ${j.joinType.sql} join between logical plans @@ -1449,7 +1449,7 @@ object ReplaceIntersectWithSemiJoin extends Rule[LogicalPlan] { case Intersect(left, right, false) => assert(left.output.size == right.output.size) val joinCond = left.output.zip(right.output).map { case (l, r) => EqualNullSafe(l, r) } - Distinct(Join(left, right, LeftSemi, joinCond.reduceLeftOption(And))) + Distinct(Join(left, right, LeftSemi, joinCond.reduceLeftOption(And), JoinHint.NONE)) } } @@ -1470,7 +1470,7 @@ object ReplaceExceptWithAntiJoin extends Rule[LogicalPlan] { case Except(left, right, false) => assert(left.output.size == right.output.size) val joinCond = left.output.zip(right.output).map { case (l, r) => EqualNullSafe(l, r) } - Distinct(Join(left, right, LeftAnti, joinCond.reduceLeftOption(And))) + Distinct(Join(left, right, LeftAnti, joinCond.reduceLeftOption(And), JoinHint.NONE)) } } 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 c3fdb924243d..b19e13870aa6 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 @@ -56,7 +56,7 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper wit // Joins on empty LocalRelations generated from streaming sources are not eliminated // as stateful streaming joins need to perform other state management operations other than // just processing the input data. - case p @ Join(_, _, joinType, _) + case p @ Join(_, _, joinType, _, _) if !p.children.exists(_.isStreaming) => val isLeftEmpty = isEmptyLocalRelation(p.left) val isRightEmpty = isEmptyLocalRelation(p.right) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala index 72a60f692ac7..689915a98534 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala @@ -52,7 +52,7 @@ object ReplaceNullWithFalseInPredicate extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case f @ Filter(cond, _) => f.copy(condition = replaceNullWithFalse(cond)) - case j @ Join(_, _, _, Some(cond)) => j.copy(condition = Some(replaceNullWithFalse(cond))) + case j @ Join(_, _, _, Some(cond), _) => j.copy(condition = Some(replaceNullWithFalse(cond))) case p: LogicalPlan => p transformExpressions { case i @ If(pred, _, _) => i.copy(predicate = replaceNullWithFalse(pred)) case cw @ CaseWhen(branches, _) => 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 468a950fb108..39709529c00d 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 @@ -600,7 +600,7 @@ object FoldablePropagation extends Rule[LogicalPlan] { // propagating the foldable expressions. // TODO(cloud-fan): It seems more reasonable to use new attributes as the output attributes // of outer join. - case j @ Join(left, right, joinType, _) if foldableMap.nonEmpty => + case j @ Join(left, right, joinType, _, _) if foldableMap.nonEmpty => val newJoin = j.transformExpressions(replaceFoldable) val missDerivedAttrsSet: AttributeSet = AttributeSet(joinType match { case _: InnerLike | LeftExistence(_) => Nil @@ -648,7 +648,6 @@ object FoldablePropagation extends Rule[LogicalPlan] { case _: Distinct => true case _: AppendColumns => true case _: AppendColumnsWithObject => true - case _: ResolvedHint => true case _: RepartitionByExpression => true case _: Repartition => true case _: Sort => true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 0b6471289a47..82aefca8a1af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -43,10 +43,13 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { * * @param input a list of LogicalPlans to inner join and the type of inner join. * @param conditions a list of condition for join. + * @param hintMap a map of relation output attribute sets to their corresponding hints. */ @tailrec - final def createOrderedJoin(input: Seq[(LogicalPlan, InnerLike)], conditions: Seq[Expression]) - : LogicalPlan = { + final def createOrderedJoin( + input: Seq[(LogicalPlan, InnerLike)], + conditions: Seq[Expression], + hintMap: Map[AttributeSet, HintInfo]): LogicalPlan = { assert(input.size >= 2) if (input.size == 2) { val (joinConditions, others) = conditions.partition(canEvaluateWithinJoin) @@ -55,7 +58,8 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { case (Inner, Inner) => Inner case (_, _) => Cross } - val join = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And)) + val join = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And), + JoinHint(hintMap.get(left.outputSet), hintMap.get(right.outputSet))) if (others.nonEmpty) { Filter(others.reduceLeft(And), join) } else { @@ -78,26 +82,27 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val joinedRefs = left.outputSet ++ right.outputSet val (joinConditions, others) = conditions.partition( e => e.references.subsetOf(joinedRefs) && canEvaluateWithinJoin(e)) - val joined = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And)) + val joined = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And), + JoinHint(hintMap.get(left.outputSet), hintMap.get(right.outputSet))) // should not have reference to same logical plan - createOrderedJoin(Seq((joined, Inner)) ++ rest.filterNot(_._1 eq right), others) + createOrderedJoin(Seq((joined, Inner)) ++ rest.filterNot(_._1 eq right), others, hintMap) } } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p @ ExtractFiltersAndInnerJoins(input, conditions) + case p @ ExtractFiltersAndInnerJoins(input, conditions, hintMap) if input.size > 2 && conditions.nonEmpty => val reordered = if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { val starJoinPlan = StarSchemaDetection.reorderStarJoins(input, conditions) if (starJoinPlan.nonEmpty) { val rest = input.filterNot(starJoinPlan.contains(_)) - createOrderedJoin(starJoinPlan ++ rest, conditions) + createOrderedJoin(starJoinPlan ++ rest, conditions, hintMap) } else { - createOrderedJoin(input, conditions) + createOrderedJoin(input, conditions, hintMap) } } else { - createOrderedJoin(input, conditions) + createOrderedJoin(input, conditions, hintMap) } if (p.sameOutput(reordered)) { @@ -156,7 +161,7 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case f @ Filter(condition, j @ Join(_, _, RightOuter | LeftOuter | FullOuter, _)) => + case f @ Filter(condition, j @ Join(_, _, RightOuter | LeftOuter | FullOuter, _, _)) => val newJoinType = buildNewJoinType(f, j) if (j.joinType == newJoinType) f else Filter(condition, j.copy(joinType = newJoinType)) } @@ -176,7 +181,7 @@ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateH } override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case j @ Join(_, _, joinType, Some(cond)) if hasUnevaluablePythonUDF(cond, j) => + case j @ Join(_, _, joinType, Some(cond), _) if hasUnevaluablePythonUDF(cond, j) => if (!joinType.isInstanceOf[InnerLike] && joinType != LeftSemi) { // The current strategy only support InnerLike and LeftSemi join because for other type, // it breaks SQL semantic if we run the join condition as a filter after join. If we pass 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 34840c6c977a..e78ed1c3c5d9 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 @@ -51,7 +51,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { condition: Option[Expression]): Join = { // Deduplicate conflicting attributes if any. val dedupSubplan = dedupSubqueryOnSelfJoin(outerPlan, subplan, None, condition) - Join(outerPlan, dedupSubplan, joinType, condition) + Join(outerPlan, dedupSubplan, joinType, condition, JoinHint.NONE) } private def dedupSubqueryOnSelfJoin( @@ -116,7 +116,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) val inConditions = values.zip(newSub.output).map(EqualTo.tupled) val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p) - Join(outerPlan, newSub, LeftSemi, joinCond) + Join(outerPlan, newSub, LeftSemi, joinCond, JoinHint.NONE) case (p, Not(InSubquery(values, ListQuery(sub, conditions, _, _)))) => // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr // Construct the condition. A NULL in one of the conditions is regarded as a positive @@ -142,7 +142,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // will have the final conditions in the LEFT ANTI as // (A.A1 = B.B1 OR ISNULL(A.A1 = B.B1)) AND (B.B2 = A.A2) AND B.B3 > 1 val finalJoinCond = (nullAwareJoinConds ++ conditions).reduceLeft(And) - Join(outerPlan, newSub, LeftAnti, Option(finalJoinCond)) + Join(outerPlan, newSub, LeftAnti, Option(finalJoinCond), JoinHint.NONE) case (p, predicate) => val (newCond, inputPlan) = rewriteExistentialExpr(Seq(predicate), p) Project(p.output, Filter(newCond.get, inputPlan)) @@ -172,7 +172,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { val newSub = dedupSubqueryOnSelfJoin(newPlan, sub, Some(values)) val inConditions = values.zip(newSub.output).map(EqualTo.tupled) val newConditions = (inConditions ++ conditions).reduceLeftOption(And) - newPlan = Join(newPlan, newSub, ExistenceJoin(exists), newConditions) + newPlan = Join(newPlan, newSub, ExistenceJoin(exists), newConditions, JoinHint.NONE) exists } } @@ -450,7 +450,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { // CASE 1: Subquery guaranteed not to have the COUNT bug Project( currentChild.output :+ origOutput, - Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) + Join(currentChild, query, LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) } else { // Subquery might have the COUNT bug. Add appropriate corrections. val (topPart, havingNode, aggNode) = splitSubquery(query) @@ -477,7 +477,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { aggValRef), origOutput.name)(exprId = origOutput.exprId), Join(currentChild, Project(query.output :+ alwaysTrueExpr, query), - LeftOuter, conditions.reduceOption(And))) + LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) } else { // CASE 3: Subquery with HAVING clause. Pull the HAVING clause above the join. @@ -507,7 +507,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { currentChild.output :+ caseExpr, Join(currentChild, Project(subqueryRoot.output :+ alwaysTrueExpr, subqueryRoot), - LeftOuter, conditions.reduceOption(And))) + LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) } } 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 8959f78b656d..a27c6d3c3671 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 @@ -515,7 +515,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) { val from = ctx.relation.asScala.foldLeft(null: LogicalPlan) { (left, relation) => val right = plan(relation.relationPrimary) - val join = right.optionalMap(left)(Join(_, _, Inner, None)) + val join = right.optionalMap(left)(Join(_, _, Inner, None, JoinHint.NONE)) withJoinRelations(join, relation) } if (ctx.pivotClause() != null) { @@ -727,7 +727,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case None => (baseJoinType, None) } - Join(left, plan(join.right), joinType, condition) + Join(left, plan(join.right), joinType, condition, JoinHint.NONE) } } } 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 84be677e438a..dfc3b2d22129 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.planning +import scala.collection.mutable + import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ @@ -98,12 +100,13 @@ object PhysicalOperation extends PredicateHelper { * value). */ object ExtractEquiJoinKeys extends Logging with PredicateHelper { - /** (joinType, leftKeys, rightKeys, condition, leftChild, rightChild) */ + /** (joinType, leftKeys, rightKeys, condition, leftChild, rightChild, joinHint) */ type ReturnType = - (JoinType, Seq[Expression], Seq[Expression], Option[Expression], LogicalPlan, LogicalPlan) + (JoinType, Seq[Expression], Seq[Expression], + Option[Expression], LogicalPlan, LogicalPlan, JoinHint) def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { - case join @ Join(left, right, joinType, condition) => + case join @ Join(left, right, joinType, condition, hint) => logDebug(s"Considering join on: $condition") // Find equi-join predicates that can be evaluated before the join, and thus can be used // as join keys. @@ -133,7 +136,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { if (joinKeys.nonEmpty) { val (leftKeys, rightKeys) = joinKeys.unzip logDebug(s"leftKeys:$leftKeys | rightKeys:$rightKeys") - Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) + Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right, hint)) } else { None } @@ -164,25 +167,35 @@ object ExtractFiltersAndInnerJoins extends PredicateHelper { * was involved in an explicit cross join. Also returns the entire list of join conditions for * the left-deep tree. */ - def flattenJoin(plan: LogicalPlan, parentJoinType: InnerLike = Inner) + def flattenJoin( + plan: LogicalPlan, + hintMap: mutable.HashMap[AttributeSet, HintInfo], + parentJoinType: InnerLike = Inner) : (Seq[(LogicalPlan, InnerLike)], Seq[Expression]) = plan match { - case Join(left, right, joinType: InnerLike, cond) => - val (plans, conditions) = flattenJoin(left, joinType) + case Join(left, right, joinType: InnerLike, cond, hint) => + val (plans, conditions) = flattenJoin(left, hintMap, joinType) + hint.leftHint.map(hintMap.put(left.outputSet, _)) + hint.rightHint.map(hintMap.put(right.outputSet, _)) (plans ++ Seq((right, joinType)), conditions ++ cond.toSeq.flatMap(splitConjunctivePredicates)) - case Filter(filterCondition, j @ Join(left, right, _: InnerLike, joinCondition)) => - val (plans, conditions) = flattenJoin(j) + case Filter(filterCondition, j @ Join(_, _, _: InnerLike, _, _)) => + val (plans, conditions) = flattenJoin(j, hintMap) (plans, conditions ++ splitConjunctivePredicates(filterCondition)) case _ => (Seq((plan, parentJoinType)), Seq.empty) } - def unapply(plan: LogicalPlan): Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])] + def unapply(plan: LogicalPlan) + : Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression], Map[AttributeSet, HintInfo])] = plan match { - case f @ Filter(filterCondition, j @ Join(_, _, joinType: InnerLike, _)) => - Some(flattenJoin(f)) - case j @ Join(_, _, joinType, _) => - Some(flattenJoin(j)) + case f @ Filter(filterCondition, j @ Join(_, _, joinType: InnerLike, _, _)) => + val hintMap = new mutable.HashMap[AttributeSet, HintInfo] + val flattened = flattenJoin(f, hintMap) + Some((flattened._1, flattened._2, hintMap.toMap)) + case j @ Join(_, _, joinType, _, _) => + val hintMap = new mutable.HashMap[AttributeSet, HintInfo] + val flattened = flattenJoin(j, hintMap) + Some((flattened._1, flattened._2, hintMap.toMap)) case _ => None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala index 2c248d74869c..18baced8f3d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala @@ -37,7 +37,6 @@ trait LogicalPlanVisitor[T] { case p: Project => visitProject(p) case p: Repartition => visitRepartition(p) case p: RepartitionByExpression => visitRepartitionByExpr(p) - case p: ResolvedHint => visitHint(p) case p: Sample => visitSample(p) case p: ScriptTransformation => visitScriptTransform(p) case p: Union => visitUnion(p) @@ -61,8 +60,6 @@ trait LogicalPlanVisitor[T] { def visitGlobalLimit(p: GlobalLimit): T - def visitHint(p: ResolvedHint): T - def visitIntersect(p: Intersect): T def visitJoin(p: Join): T diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index b3a48860aa63..5a388117a6c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -52,13 +52,11 @@ import org.apache.spark.util.Utils * defaults to the product of children's `sizeInBytes`. * @param rowCount Estimated number of rows. * @param attributeStats Statistics for Attributes. - * @param hints Query hints. */ case class Statistics( sizeInBytes: BigInt, rowCount: Option[BigInt] = None, - attributeStats: AttributeMap[ColumnStat] = AttributeMap(Nil), - hints: HintInfo = HintInfo()) { + attributeStats: AttributeMap[ColumnStat] = AttributeMap(Nil)) { override def toString: String = "Statistics(" + simpleString + ")" @@ -70,8 +68,7 @@ case class Statistics( s"rowCount=${BigDecimal(rowCount.get, new MathContext(3, RoundingMode.HALF_UP)).toString()}" } else { "" - }, - s"hints=$hints" + } ).filter(_.nonEmpty).mkString(", ") } } 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 d8b3a4af4f7b..639d68f4ecd7 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 @@ -288,7 +288,8 @@ case class Join( left: LogicalPlan, right: LogicalPlan, joinType: JoinType, - condition: Option[Expression]) + condition: Option[Expression], + hint: JoinHint) extends BinaryNode with PredicateHelper { override def output: Seq[Attribute] = { @@ -350,6 +351,17 @@ case class Join( case UsingJoin(_, _) => false case _ => resolvedExceptNatural } + + // Ignore hint for canonicalization + protected override def doCanonicalize(): LogicalPlan = + super.doCanonicalize().asInstanceOf[Join].copy(hint = JoinHint.NONE) + + // Do not include an empty join hint in string description + protected override def stringArgs: Iterator[Any] = super.stringArgs.filter { e => + (!e.isInstanceOf[JoinHint] + || e.asInstanceOf[JoinHint].leftHint.isDefined + || e.asInstanceOf[JoinHint].rightHint.isDefined) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala index cbb626590d1d..b2ba725e9d44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala @@ -35,6 +35,7 @@ case class UnresolvedHint(name: String, parameters: Seq[Any], child: LogicalPlan /** * A resolved hint node. The analyzer should convert all [[UnresolvedHint]] into [[ResolvedHint]]. + * This node will be eliminated before optimization starts. */ case class ResolvedHint(child: LogicalPlan, hints: HintInfo = HintInfo()) extends UnaryNode { @@ -44,11 +45,31 @@ case class ResolvedHint(child: LogicalPlan, hints: HintInfo = HintInfo()) override def doCanonicalize(): LogicalPlan = child.canonicalized } +/** + * Hint that is associated with a [[Join]] node, with [[HintInfo]] on its left child and on its + * right child respectively. + */ +case class JoinHint(leftHint: Option[HintInfo], rightHint: Option[HintInfo]) { -case class HintInfo(broadcast: Boolean = false) { + override def toString: String = { + Seq( + leftHint.map("leftHint=" + _), + rightHint.map("rightHint=" + _)) + .filter(_.isDefined).map(_.get).mkString(", ") + } +} - /** Must be called when computing stats for a join operator to reset hints. */ - def resetForJoin(): HintInfo = copy(broadcast = false) +object JoinHint { + val NONE = JoinHint(None, None) +} + +/** + * The hint attributes to be applied on a specific node. + * + * @param broadcast If set to true, it indicates that the broadcast hash join is the preferred join + * strategy and the node with this hint is preferred to be the build side. + */ +case class HintInfo(broadcast: Boolean = false) { override def toString: String = { val hints = scala.collection.mutable.ArrayBuffer.empty[String] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala index 111c594a53e5..eb56ab43ea9d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala @@ -56,8 +56,7 @@ object AggregateEstimation { Some(Statistics( sizeInBytes = getOutputSize(agg.output, outputRows, outputAttrStats), rowCount = Some(outputRows), - attributeStats = outputAttrStats, - hints = childStats.hints)) + attributeStats = outputAttrStats)) } else { None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala index b6c16079d198..b8c652dc8f12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala @@ -47,8 +47,6 @@ object BasicStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { override def visitGlobalLimit(p: GlobalLimit): Statistics = fallback(p) - override def visitHint(p: ResolvedHint): Statistics = fallback(p) - override def visitIntersect(p: Intersect): Statistics = fallback(p) override def visitJoin(p: Join): Statistics = { 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 2543e38a92c0..19a0d1279cc3 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 @@ -56,7 +56,7 @@ case class JoinEstimation(join: Join) extends Logging { case _ if !rowCountsExist(join.left, join.right) => None - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, _) => + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, _, _) => // 1. Compute join selectivity val joinKeyPairs = extractJoinKeysWithColStats(leftKeys, rightKeys) val (numInnerJoinedRows, keyStatsAfterJoin) = computeCardinalityAndStats(joinKeyPairs) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala index ee43f9126386..da36db7ae1f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala @@ -44,7 +44,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { } // Don't propagate rowCount and attributeStats, since they are not estimated here. - Statistics(sizeInBytes = sizeInBytes, hints = p.child.stats.hints) + Statistics(sizeInBytes = sizeInBytes) } /** @@ -60,8 +60,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { if (p.groupingExpressions.isEmpty) { Statistics( sizeInBytes = EstimationUtils.getOutputSize(p.output, outputRowCount = 1), - rowCount = Some(1), - hints = p.child.stats.hints) + rowCount = Some(1)) } else { visitUnaryNode(p) } @@ -87,19 +86,15 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { // Don't propagate column stats, because we don't know the distribution after limit Statistics( sizeInBytes = EstimationUtils.getOutputSize(p.output, rowCount, childStats.attributeStats), - rowCount = Some(rowCount), - hints = childStats.hints) + rowCount = Some(rowCount)) } - override def visitHint(p: ResolvedHint): Statistics = p.child.stats.copy(hints = p.hints) - override def visitIntersect(p: Intersect): Statistics = { val leftSize = p.left.stats.sizeInBytes val rightSize = p.right.stats.sizeInBytes val sizeInBytes = if (leftSize < rightSize) leftSize else rightSize Statistics( - sizeInBytes = sizeInBytes, - hints = p.left.stats.hints.resetForJoin()) + sizeInBytes = sizeInBytes) } override def visitJoin(p: Join): Statistics = { @@ -108,10 +103,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { // LeftSemi and LeftAnti won't ever be bigger than left p.left.stats case _ => - // Make sure we don't propagate isBroadcastable in other joins, because - // they could explode the size. - val stats = default(p) - stats.copy(hints = stats.hints.resetForJoin()) + default(p) } } @@ -121,7 +113,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { if (limit == 0) { // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also be zero // (product of children). - Statistics(sizeInBytes = 1, rowCount = Some(0), hints = childStats.hints) + Statistics(sizeInBytes = 1, rowCount = Some(0)) } else { // The output row count of LocalLimit should be the sum of row counts from each partition. // However, since the number of partitions is not available here, we just use statistics of @@ -147,7 +139,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { } val sampleRows = p.child.stats.rowCount.map(c => EstimationUtils.ceil(BigDecimal(c) * ratio)) // Don't propagate column stats, because we don't know the distribution after a sample operation - Statistics(sizeInBytes, sampleRows, hints = p.child.stats.hints) + Statistics(sizeInBytes, sampleRows) } override def visitScriptTransform(p: ScriptTransformation): Statistics = default(p) 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 117e96175e92..129ce3b1105e 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 @@ -443,7 +443,7 @@ class AnalysisErrorSuite extends AnalysisTest { } test("error test for self-join") { - val join = Join(testRelation, testRelation, Cross, None) + val join = Join(testRelation, testRelation, Cross, None, JoinHint.NONE) val error = intercept[AnalysisException] { SimpleAnalyzer.checkAnalysis(join) } @@ -565,7 +565,8 @@ class AnalysisErrorSuite extends AnalysisTest { LocalRelation(b), Filter(EqualTo(UnresolvedAttribute("a"), c), LocalRelation(c)), LeftOuter, - Option(EqualTo(b, c)))), + Option(EqualTo(b, c)), + JoinHint.NONE)), LocalRelation(a)) assertAnalysisError(plan1, "Accessing outer query column is not allowed in" :: Nil) @@ -575,7 +576,8 @@ class AnalysisErrorSuite extends AnalysisTest { Filter(EqualTo(UnresolvedAttribute("a"), c), LocalRelation(c)), LocalRelation(b), RightOuter, - Option(EqualTo(b, c)))), + Option(EqualTo(b, c)), + JoinHint.NONE)), LocalRelation(a)) assertAnalysisError(plan2, "Accessing outer query column is not allowed in" :: Nil) 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 da3ae72c3682..982948483fa1 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 @@ -397,7 +397,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { Join( Project(Seq($"x.key"), SubqueryAlias("x", input)), Project(Seq($"y.key"), SubqueryAlias("y", input)), - Cross, None)) + Cross, None, JoinHint.NONE)) assertAnalysisSuccess(query) } @@ -578,7 +578,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { Seq(UnresolvedAttribute("a")), pythonUdf, output, project) val left = SubqueryAlias("temp0", flatMapGroupsInPandas) val right = SubqueryAlias("temp1", flatMapGroupsInPandas) - val join = Join(left, right, Inner, None) + val join = Join(left, right, Inner, None, JoinHint.NONE) assertAnalysisSuccess( Project(Seq(UnresolvedAttribute("temp0.a"), UnresolvedAttribute("temp1.a")), join)) } 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 bd66ee5355f4..563e8adf87ed 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 @@ -60,7 +60,7 @@ class ResolveHintsSuite extends AnalysisTest { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table", "table2"), table("table").join(table("table2"))), Join(ResolvedHint(testRelation, HintInfo(broadcast = true)), - ResolvedHint(testRelation2, HintInfo(broadcast = true)), Inner, None), + ResolvedHint(testRelation2, HintInfo(broadcast = true)), Inner, None, JoinHint.NONE), caseSensitive = false) } 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 57195d5fda7c..0cd6e092e203 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 @@ -353,15 +353,15 @@ class ColumnPruningSuite extends PlanTest { Project(Seq($"x.key", $"y.key"), Join( SubqueryAlias("x", input), - ResolvedHint(SubqueryAlias("y", input)), Inner, None)).analyze + SubqueryAlias("y", input), Inner, None, JoinHint.NONE)).analyze val optimized = Optimize.execute(query) val expected = Join( Project(Seq($"x.key"), SubqueryAlias("x", input)), - ResolvedHint(Project(Seq($"y.key"), SubqueryAlias("y", input))), - Inner, None).analyze + Project(Seq($"y.key"), SubqueryAlias("y", input)), + Inner, None, JoinHint.NONE).analyze comparePlans(optimized, expected) } 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 82a10254d846..cf4e9fcea2c6 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 @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -822,19 +821,6 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("broadcast hint") { - val originalQuery = ResolvedHint(testRelation) - .where('a === 2L && 'b + Rand(10).as("rnd") === 3) - - val optimized = Optimize.execute(originalQuery.analyze) - - val correctAnswer = ResolvedHint(testRelation.where('a === 2L)) - .where('b + Rand(10).as("rnd") === 3) - .analyze - - comparePlans(optimized, correctAnswer) - } - test("union") { val testRelation2 = LocalRelation('d.int, 'e.int, 'f.int) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala index 6fe5e619d03a..9093d7fecb0f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala @@ -65,7 +65,8 @@ class JoinOptimizationSuite extends PlanTest { def testExtractCheckCross (plan: LogicalPlan, expected: Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])]) { - assert(ExtractFiltersAndInnerJoins.unapply(plan) === expected) + assert( + ExtractFiltersAndInnerJoins.unapply(plan) === expected.map(e => (e._1, e._2, Map.empty))) } testExtract(x, None) @@ -124,29 +125,4 @@ class JoinOptimizationSuite extends PlanTest { comparePlans(optimized, queryAnswerPair._2.analyze) } } - - test("broadcasthint sets relation statistics to smallest value") { - val input = LocalRelation('key.int, 'value.string) - - val query = - Project(Seq($"x.key", $"y.key"), - Join( - SubqueryAlias("x", input), - ResolvedHint(SubqueryAlias("y", input)), Cross, None)).analyze - - val optimized = Optimize.execute(query) - - val expected = - Join( - Project(Seq($"x.key"), SubqueryAlias("x", input)), - ResolvedHint(Project(Seq($"y.key"), SubqueryAlias("y", input))), - Cross, None).analyze - - comparePlans(optimized, expected) - - val broadcastChildren = optimized.collect { - case Join(_, r, _, _) if r.stats.sizeInBytes == 1 => r - } - assert(broadcastChildren.size == 1) - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala index c94a8b9e318f..0dee84620586 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala @@ -31,6 +31,8 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { object Optimize extends RuleExecutor[LogicalPlan] { val batches = + Batch("Resolve Hints", Once, + EliminateResolvedHint) :: Batch("Operator Optimizations", FixedPoint(100), CombineFilters, PushDownPredicate, @@ -42,6 +44,12 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { CostBasedJoinReorder) :: Nil } + object ResolveHints extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Resolve Hints", Once, + EliminateResolvedHint) :: Nil + } + var originalConfCBOEnabled = false var originalConfJoinReorderEnabled = false @@ -284,12 +292,85 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { assertEqualPlans(originalPlan, bestPlan) } + test("hints preservation") { + // Apply hints if we find an equivalent node in the new plan, otherwise discard them. + val originalPlan = + t1.join(t2.hint("broadcast")).hint("broadcast").join(t4.join(t3).hint("broadcast")) + .where((nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")) && + (nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && + (nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + + val bestPlan = + t1.join(t2.hint("broadcast"), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .hint("broadcast") + .join( + t4.join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + .hint("broadcast"), + Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + + assertEqualPlans(originalPlan, bestPlan) + + val originalPlan2 = + t1.join(t2).hint("broadcast").join(t3).hint("broadcast").join(t4.hint("broadcast")) + .where((nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")) && + (nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && + (nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + + val bestPlan2 = + t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .hint("broadcast") + .join( + t4.hint("broadcast") + .join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), + Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + .select(outputsOf(t1, t2, t3, t4): _*) + + assertEqualPlans(originalPlan2, bestPlan2) + + val originalPlan3 = + t1.join(t4).hint("broadcast") + .join(t2.hint("broadcast")).hint("broadcast") + .join(t3.hint("broadcast")) + .where((nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")) && + (nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && + (nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + + val bestPlan3 = + t1.join(t2.hint("broadcast"), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .join( + t4.join(t3.hint("broadcast"), + Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), + Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + .select(outputsOf(t1, t4, t2, t3): _*) + + assertEqualPlans(originalPlan3, bestPlan3) + + val originalPlan4 = + t2.hint("broadcast") + .join(t4).hint("broadcast") + .join(t3.hint("broadcast")).hint("broadcast") + .join(t1) + .where((nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")) && + (nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && + (nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + + val bestPlan4 = + t1.join(t2.hint("broadcast"), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .join( + t4.join(t3.hint("broadcast"), + Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), + Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + .select(outputsOf(t2, t4, t3, t1): _*) + + assertEqualPlans(originalPlan4, bestPlan4) + } + private def assertEqualPlans( originalPlan: LogicalPlan, groundTruthBestPlan: LogicalPlan): Unit = { val analyzed = originalPlan.analyze val optimized = Optimize.execute(analyzed) - val expected = groundTruthBestPlan.analyze + val expected = ResolveHints.execute(groundTruthBestPlan.analyze) assert(analyzed.sameOutput(expected)) // if this fails, the expected plan itself is incorrect assert(analyzed.sameOutput(optimized)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala index c8e15c7da763..6d1af12e68b2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala @@ -48,7 +48,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Join(table1, table2, LeftSemi, Option('a <=> 'c && 'b <=> 'd))).analyze + Join(table1, table2, LeftSemi, Option('a <=> 'c && 'b <=> 'd), JoinHint.NONE)).analyze comparePlans(optimized, correctAnswer) } @@ -160,7 +160,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Join(table1, table2, LeftAnti, Option('a <=> 'c && 'b <=> 'd))).analyze + Join(table1, table2, LeftAnti, Option('a <=> 'c && 'b <=> 'd), JoinHint.NONE)).analyze comparePlans(optimized, correctAnswer) } @@ -175,7 +175,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(left.output, right.output, - Join(left, right, LeftAnti, Option($"left.a" <=> $"right.a"))).analyze + Join(left, right, LeftAnti, Option($"left.a" <=> $"right.a"), JoinHint.NONE)).analyze comparePlans(optimized, correctAnswer) } @@ -248,7 +248,7 @@ class ReplaceOperatorSuite extends PlanTest { val condition = basePlan.output.zip(otherPlan.output).map { case (a1, a2) => a1 <=> a2 }.reduce( _ && _) val correctAnswer = Aggregate(basePlan.output, otherPlan.output, - Join(basePlan, otherPlan, LeftAnti, Option(condition))).analyze + Join(basePlan, otherPlan, LeftAnti, Option(condition), JoinHint.NONE)).analyze comparePlans(result, correctAnswer) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 3081ff935f04..5394732f41f2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -99,11 +99,11 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => .reduce(And), child) case sample: Sample => sample.copy(seed = 0L) - case Join(left, right, joinType, condition) if condition.isDefined => + case Join(left, right, joinType, condition, hint) if condition.isDefined => val newCondition = splitConjunctivePredicates(condition.get).map(rewriteEqual).sortBy(_.hashCode()) .reduce(And) - Join(left, right, joinType, Some(newCondition)) + Join(left, right, joinType, Some(newCondition), hint) } } @@ -165,8 +165,10 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => private def sameJoinPlan(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { (plan1, plan2) match { case (j1: Join, j2: Join) => - (sameJoinPlan(j1.left, j2.left) && sameJoinPlan(j1.right, j2.right)) || - (sameJoinPlan(j1.left, j2.right) && sameJoinPlan(j1.right, j2.left)) + (sameJoinPlan(j1.left, j2.left) && sameJoinPlan(j1.right, j2.right) + && j1.hint.leftHint == j2.hint.leftHint && j1.hint.rightHint == j2.hint.rightHint) || + (sameJoinPlan(j1.left, j2.right) && sameJoinPlan(j1.right, j2.left) + && j1.hint.leftHint == j2.hint.rightHint && j1.hint.rightHint == j2.hint.leftHint) case (p1: Project, p2: Project) => p1.projectList == p2.projectList && sameJoinPlan(p1.child, p2.child) case _ => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala index 7c8ed78a4911..fbaaf807af5d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, ResolvedHint, Union} +import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util._ /** @@ -30,6 +32,10 @@ class SameResultSuite extends SparkFunSuite { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("EliminateResolvedHint", Once, EliminateResolvedHint) :: Nil + } + def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true): Unit = { val aAnalyzed = a.analyze val bAnalyzed = b.analyze @@ -72,4 +78,12 @@ class SameResultSuite extends SparkFunSuite { val df2 = testRelation.join(testRelation) assertSameResult(df1, df2) } + + test("join hint") { + val df1 = testRelation.join(testRelation.hint("broadcast")) + val df2 = testRelation.join(testRelation) + val df1Optimized = Optimize.execute(df1.analyze) + val df2Optimized = Optimize.execute(df2.analyze) + assertSameResult(df1Optimized, df2Optimized) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 953094cb0dd5..16a5c2d3001a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -38,24 +38,6 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { // row count * (overhead + column size) size = Some(10 * (8 + 4))) - test("BroadcastHint estimation") { - val filter = Filter(Literal(true), plan) - val filterStatsCboOn = Statistics(sizeInBytes = 10 * (8 +4), - rowCount = Some(10), attributeStats = AttributeMap(Seq(attribute -> colStat))) - val filterStatsCboOff = Statistics(sizeInBytes = 10 * (8 +4)) - checkStats( - filter, - expectedStatsCboOn = filterStatsCboOn, - expectedStatsCboOff = filterStatsCboOff) - - val broadcastHint = ResolvedHint(filter, HintInfo(broadcast = true)) - checkStats( - broadcastHint, - expectedStatsCboOn = filterStatsCboOn.copy(hints = HintInfo(broadcast = true)), - expectedStatsCboOff = filterStatsCboOff.copy(hints = HintInfo(broadcast = true)) - ) - } - test("range") { val range = Range(1, 5, 1, None) val rangeStats = Statistics(sizeInBytes = 4 * 8) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala index b0a47e783512..1cf888519077 100755 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala @@ -528,7 +528,7 @@ class FilterEstimationSuite extends StatsEstimationTestBase { rowCount = 30, attributeStats = AttributeMap(Seq(attrIntLargerRange -> colStatIntLargerRange))) val nonLeafChild = Join(largerTable, smallerTable, LeftOuter, - Some(EqualTo(attrIntLargerRange, attrInt))) + Some(EqualTo(attrIntLargerRange, attrInt)), JoinHint.NONE) Seq(IsNull(attrIntLargerRange), IsNotNull(attrIntLargerRange)).foreach { predicate => validateEstimatedStats( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala index 12c0a7be2129..6c5a2b247fc2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala @@ -79,8 +79,8 @@ class JoinEstimationSuite extends StatsEstimationTestBase { val c1 = generateJoinChild(col1, leftHistogram, expectedMin, expectedMax) val c2 = generateJoinChild(col2, rightHistogram, expectedMin, expectedMax) - val c1JoinC2 = Join(c1, c2, Inner, Some(EqualTo(col1, col2))) - val c2JoinC1 = Join(c2, c1, Inner, Some(EqualTo(col2, col1))) + val c1JoinC2 = Join(c1, c2, Inner, Some(EqualTo(col1, col2)), JoinHint.NONE) + val c2JoinC1 = Join(c2, c1, Inner, Some(EqualTo(col2, col1)), JoinHint.NONE) val expectedStatsAfterJoin = Statistics( sizeInBytes = expectedRows * (8 + 2 * 4), rowCount = Some(expectedRows), @@ -284,7 +284,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { test("cross join") { // table1 (key-1-5 int, key-5-9 int): (1, 9), (2, 8), (3, 7), (4, 6), (5, 5) // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) - val join = Join(table1, table2, Cross, None) + val join = Join(table1, table2, Cross, None, JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 5 * 3 * (8 + 4 * 4), rowCount = Some(5 * 3), @@ -299,7 +299,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // key-5-9 and key-2-4 are disjoint val join = Join(table1, table2, Inner, - Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 1, rowCount = Some(0), @@ -312,7 +312,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // key-5-9 and key-2-4 are disjoint val join = Join(table1, table2, LeftOuter, - Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 5 * (8 + 4 * 4), rowCount = Some(5), @@ -328,7 +328,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // key-5-9 and key-2-4 are disjoint val join = Join(table1, table2, RightOuter, - Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 3 * (8 + 4 * 4), rowCount = Some(3), @@ -344,7 +344,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // key-5-9 and key-2-4 are disjoint val join = Join(table1, table2, FullOuter, - Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = (5 + 3) * (8 + 4 * 4), rowCount = Some(5 + 3), @@ -361,7 +361,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table1 (key-1-5 int, key-5-9 int): (1, 9), (2, 8), (3, 7), (4, 6), (5, 5) // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) val join = Join(table1, table2, Inner, - Some(EqualTo(nameToAttr("key-1-5"), nameToAttr("key-1-2")))) + Some(EqualTo(nameToAttr("key-1-5"), nameToAttr("key-1-2"))), JoinHint.NONE) // Update column stats for equi-join keys (key-1-5 and key-1-2). val joinedColStat = ColumnStat(distinctCount = Some(2), min = Some(1), max = Some(2), nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) @@ -383,7 +383,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) val join = Join(table2, table3, Inner, Some( And(EqualTo(nameToAttr("key-1-2"), nameToAttr("key-1-2")), - EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))))) + EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3")))), JoinHint.NONE) // Update column stats for join keys. val joinedColStat1 = ColumnStat(distinctCount = Some(2), min = Some(1), max = Some(2), @@ -404,7 +404,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) val join = Join(table3, table2, LeftOuter, - Some(EqualTo(nameToAttr("key-2-3"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-2-3"), nameToAttr("key-2-4"))), JoinHint.NONE) val joinedColStat = ColumnStat(distinctCount = Some(2), min = Some(2), max = Some(3), nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) @@ -422,7 +422,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) val join = Join(table2, table3, RightOuter, - Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3")))) + Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))), JoinHint.NONE) val joinedColStat = ColumnStat(distinctCount = Some(2), min = Some(2), max = Some(3), nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) @@ -440,7 +440,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) val join = Join(table2, table3, FullOuter, - Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3")))) + Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 3 * (8 + 4 * 4), @@ -456,7 +456,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) Seq(LeftSemi, LeftAnti).foreach { jt => val join = Join(table2, table3, jt, - Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3")))) + Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))), JoinHint.NONE) // For now we just propagate the statistics from left side for left semi/anti join. val expectedStats = Statistics( sizeInBytes = 3 * (8 + 4 * 2), @@ -525,7 +525,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { withClue(s"For data type ${key1.dataType}") { // All values in two tables are the same, so column stats after join are also the same. val join = Join(Project(Seq(key1), table1), Project(Seq(key2), table2), Inner, - Some(EqualTo(key1, key2))) + Some(EqualTo(key1, key2)), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 1 * (8 + 2 * getColSize(key1, columnInfo1(key1))), rowCount = Some(1), @@ -543,7 +543,8 @@ class JoinEstimationSuite extends StatsEstimationTestBase { outputList = Seq(nullColumn), rowCount = 1, attributeStats = AttributeMap(Seq(nullColumn -> nullColStat))) - val join = Join(table1, nullTable, Inner, Some(EqualTo(nameToAttr("key-1-5"), nullColumn))) + val join = Join(table1, nullTable, Inner, + Some(EqualTo(nameToAttr("key-1-5"), nullColumn)), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 1, rowCount = Some(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 a664c7338bad..44cada086489 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 @@ -862,7 +862,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ def join(right: Dataset[_]): DataFrame = withPlan { - Join(logicalPlan, right.logicalPlan, joinType = Inner, None) + Join(logicalPlan, right.logicalPlan, joinType = Inner, None, JoinHint.NONE) } /** @@ -940,7 +940,7 @@ class Dataset[T] private[sql]( // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right // by creating a new instance for one of the branch. val joined = sparkSession.sessionState.executePlan( - Join(logicalPlan, right.logicalPlan, joinType = JoinType(joinType), None)) + Join(logicalPlan, right.logicalPlan, joinType = JoinType(joinType), None, JoinHint.NONE)) .analyzed.asInstanceOf[Join] withPlan { @@ -948,7 +948,8 @@ class Dataset[T] private[sql]( joined.left, joined.right, UsingJoin(JoinType(joinType), usingColumns), - None) + None, + JoinHint.NONE) } } @@ -1001,7 +1002,7 @@ class Dataset[T] private[sql]( // Trigger analysis so in the case of self-join, the analyzer will clone the plan. // After the cloning, left and right side will have distinct expression ids. val plan = withPlan( - Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr))) + Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr), JoinHint.NONE)) .queryExecution.analyzed.asInstanceOf[Join] // If auto self join alias is disabled, return the plan. @@ -1048,7 +1049,7 @@ class Dataset[T] private[sql]( * @since 2.1.0 */ def crossJoin(right: Dataset[_]): DataFrame = withPlan { - Join(logicalPlan, right.logicalPlan, joinType = Cross, None) + Join(logicalPlan, right.logicalPlan, joinType = Cross, None, JoinHint.NONE) } /** @@ -1083,7 +1084,8 @@ class Dataset[T] private[sql]( this.logicalPlan, other.logicalPlan, JoinType(joinType), - Some(condition.expr))).analyzed.asInstanceOf[Join] + Some(condition.expr), + JoinHint.NONE)).analyzed.asInstanceOf[Join] if (joined.joinType == LeftSemi || joined.joinType == LeftAnti) { throw new AnalysisException("Invalid join type in joinWith: " + joined.joinType.sql) @@ -1135,7 +1137,7 @@ class Dataset[T] private[sql]( implicit val tuple2Encoder: Encoder[(T, U)] = ExpressionEncoder.tuple(this.exprEnc, other.exprEnc) - withTypedPlan(Join(left, right, joined.joinType, Some(conditionExpr))) + withTypedPlan(Join(left, right, joined.joinType, Some(conditionExpr), JoinHint.NONE)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index dbc6db62bd82..b7cc373b2df1 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 @@ -208,17 +208,17 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } - private def canBroadcastByHints(joinType: JoinType, left: LogicalPlan, right: LogicalPlan) - : Boolean = { - val buildLeft = canBuildLeft(joinType) && left.stats.hints.broadcast - val buildRight = canBuildRight(joinType) && right.stats.hints.broadcast + private def canBroadcastByHints( + joinType: JoinType, left: LogicalPlan, right: LogicalPlan, hint: JoinHint): Boolean = { + val buildLeft = canBuildLeft(joinType) && hint.leftHint.exists(_.broadcast) + val buildRight = canBuildRight(joinType) && hint.rightHint.exists(_.broadcast) buildLeft || buildRight } - private def broadcastSideByHints(joinType: JoinType, left: LogicalPlan, right: LogicalPlan) - : BuildSide = { - val buildLeft = canBuildLeft(joinType) && left.stats.hints.broadcast - val buildRight = canBuildRight(joinType) && right.stats.hints.broadcast + private def broadcastSideByHints( + joinType: JoinType, left: LogicalPlan, right: LogicalPlan, hint: JoinHint): BuildSide = { + val buildLeft = canBuildLeft(joinType) && hint.leftHint.exists(_.broadcast) + val buildRight = canBuildRight(joinType) && hint.rightHint.exists(_.broadcast) broadcastSide(buildLeft, buildRight, left, right) } @@ -241,14 +241,14 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // --- BroadcastHashJoin -------------------------------------------------------------------- // broadcast hints were specified - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) - if canBroadcastByHints(joinType, left, right) => - val buildSide = broadcastSideByHints(joinType, left, right) + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, hint) + if canBroadcastByHints(joinType, left, right, hint) => + val buildSide = broadcastSideByHints(joinType, left, right, hint) Seq(joins.BroadcastHashJoinExec( leftKeys, rightKeys, joinType, buildSide, condition, planLater(left), planLater(right))) // broadcast hints were not specified, so need to infer it from size and configuration. - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, _) if canBroadcastBySizes(joinType, left, right) => val buildSide = broadcastSideBySizes(joinType, left, right) Seq(joins.BroadcastHashJoinExec( @@ -256,14 +256,14 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // --- ShuffledHashJoin --------------------------------------------------------------------- - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, _) if !conf.preferSortMergeJoin && canBuildRight(joinType) && canBuildLocalHashMap(right) && muchSmaller(right, left) || !RowOrdering.isOrderable(leftKeys) => Seq(joins.ShuffledHashJoinExec( leftKeys, rightKeys, joinType, BuildRight, condition, planLater(left), planLater(right))) - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, _) if !conf.preferSortMergeJoin && canBuildLeft(joinType) && canBuildLocalHashMap(left) && muchSmaller(left, right) || !RowOrdering.isOrderable(leftKeys) => @@ -272,7 +272,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // --- SortMergeJoin ------------------------------------------------------------ - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, _) if RowOrdering.isOrderable(leftKeys) => joins.SortMergeJoinExec( leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil @@ -280,25 +280,25 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // --- Without joining keys ------------------------------------------------------------ // Pick BroadcastNestedLoopJoin if one side could be broadcast - case j @ logical.Join(left, right, joinType, condition) - if canBroadcastByHints(joinType, left, right) => - val buildSide = broadcastSideByHints(joinType, left, right) + case j @ logical.Join(left, right, joinType, condition, hint) + if canBroadcastByHints(joinType, left, right, hint) => + val buildSide = broadcastSideByHints(joinType, left, right, hint) joins.BroadcastNestedLoopJoinExec( planLater(left), planLater(right), buildSide, joinType, condition) :: Nil - case j @ logical.Join(left, right, joinType, condition) + case j @ logical.Join(left, right, joinType, condition, _) if canBroadcastBySizes(joinType, left, right) => val buildSide = broadcastSideBySizes(joinType, left, right) joins.BroadcastNestedLoopJoinExec( planLater(left), planLater(right), buildSide, joinType, condition) :: Nil // Pick CartesianProduct for InnerJoin - case logical.Join(left, right, _: InnerLike, condition) => + case logical.Join(left, right, _: InnerLike, condition, _) => joins.CartesianProductExec(planLater(left), planLater(right), condition) :: Nil - case logical.Join(left, right, joinType, condition) => + case logical.Join(left, right, joinType, condition, hint) => val buildSide = broadcastSide( - left.stats.hints.broadcast, right.stats.hints.broadcast, left, right) + hint.leftHint.exists(_.broadcast), hint.rightHint.exists(_.broadcast), left, right) // This join could be very slow or OOM joins.BroadcastNestedLoopJoinExec( planLater(left), planLater(right), buildSide, joinType, condition) :: Nil @@ -380,13 +380,13 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object StreamingJoinStrategy extends Strategy { override def apply(plan: LogicalPlan): Seq[SparkPlan] = { plan match { - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, _) if left.isStreaming && right.isStreaming => new StreamingSymmetricHashJoinExec( leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil - case Join(left, right, _, _) if left.isStreaming && right.isStreaming => + case Join(left, right, _, _, _) if left.isStreaming && right.isStreaming => throw new AnalysisException( "Stream-stream join without equality predicate is not supported", plan = Some(plan)) @@ -561,6 +561,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { throw new IllegalStateException( "logical except (all) operator should have been replaced by union, aggregate" + " and generate operators in the optimizer") + case logical.ResolvedHint(child, hints) => + throw new IllegalStateException( + "ResolvedHint operator should have been replaced by join hint in the optimizer") case logical.DeserializeToObject(deserializer, objAttr, child) => execution.DeserializeToObjectExec(deserializer, objAttr, planLater(child)) :: Nil @@ -632,7 +635,6 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil - case h: ResolvedHint => planLater(h.child) :: Nil case _ => Nil } } 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 4109d9994dd8..41f406d6c299 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 @@ -26,7 +26,7 @@ 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.{HintInfo, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.storage.StorageLevel @@ -184,12 +184,7 @@ case class InMemoryRelation( override def computeStats(): Statistics = { if (cacheBuilder.sizeInBytesStats.value == 0L) { // Underlying columnar RDD hasn't been materialized, use the stats from the plan to cache. - // Note that we should drop the hint info here. We may cache a plan whose root node is a hint - // node. When we lookup the cache with a semantically same plan without hint info, the plan - // returned by cache lookup should not have hint info. If we lookup the cache with a - // semantically same plan with a different hint info, `CacheManager.useCachedData` will take - // care of it and retain the hint info in the lookup input plan. - statsOfPlanToCache.copy(hints = HintInfo()) + statsOfPlanToCache } else { Statistics(sizeInBytes = cacheBuilder.sizeInBytesStats.value.longValue) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 6e805c4f3c39..2141be4d680f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -27,9 +27,11 @@ import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.execution.{RDDScanExec, SparkPlan} import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.storage.{RDDBlockId, StorageLevel} @@ -925,4 +927,23 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } } } + + test("Cache should respect the broadcast hint") { + val df = broadcast(spark.range(1000)).cache() + val df2 = spark.range(1000).cache() + df.count() + df2.count() + + // Test the broadcast hint. + val joinPlan = df.join(df2, "id").queryExecution.optimizedPlan + val hint = joinPlan.collect { + case Join(_, _, _, _, hint) => hint + } + assert(hint.size == 1) + assert(hint(0).leftHint.get.broadcast) + assert(hint(0).rightHint.isEmpty) + + // Clean-up + df.unpersist() + } } 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 c9f41ab1c017..a4a3e2a62d1a 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 @@ -198,7 +198,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // outer -> left val outerJoin2Left = df.join(df2, $"a.int" === $"b.int", "outer").where($"a.int" >= 3) assert(outerJoin2Left.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, LeftOuter, _) => j }.size === 1) + case j @ Join(_, _, LeftOuter, _, _) => j }.size === 1) checkAnswer( outerJoin2Left, Row(3, 4, "3", null, null, null) :: Nil) @@ -206,7 +206,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // outer -> right val outerJoin2Right = df.join(df2, $"a.int" === $"b.int", "outer").where($"b.int" >= 3) assert(outerJoin2Right.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, RightOuter, _) => j }.size === 1) + case j @ Join(_, _, RightOuter, _, _) => j }.size === 1) checkAnswer( outerJoin2Right, Row(null, null, null, 5, 6, "5") :: Nil) @@ -215,7 +215,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { val outerJoin2Inner = df.join(df2, $"a.int" === $"b.int", "outer"). where($"a.int" === 1 && $"b.int2" === 3) assert(outerJoin2Inner.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, Inner, _) => j }.size === 1) + case j @ Join(_, _, Inner, _, _) => j }.size === 1) checkAnswer( outerJoin2Inner, Row(1, 2, "1", 1, 3, "1") :: Nil) @@ -223,7 +223,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // right -> inner val rightJoin2Inner = df.join(df2, $"a.int" === $"b.int", "right").where($"a.int" > 0) assert(rightJoin2Inner.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, Inner, _) => j }.size === 1) + case j @ Join(_, _, Inner, _, _) => j }.size === 1) checkAnswer( rightJoin2Inner, Row(1, 2, "1", 1, 3, "1") :: Nil) @@ -231,7 +231,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // left -> inner val leftJoin2Inner = df.join(df2, $"a.int" === $"b.int", "left").where($"b.int2" > 0) assert(leftJoin2Inner.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, Inner, _) => j }.size === 1) + case j @ Join(_, _, Inner, _, _) => j }.size === 1) checkAnswer( leftJoin2Inner, Row(1, 2, "1", 1, 3, "1") :: Nil) 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 new file mode 100644 index 000000000000..3652895ff43d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.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 + +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.test.SharedSQLContext + +class JoinHintSuite extends PlanTest with SharedSQLContext { + import testImplicits._ + + lazy val df = spark.range(10) + lazy val df1 = df.selectExpr("id as a1", "id as a2") + lazy val df2 = df.selectExpr("id as b1", "id as b2") + lazy val df3 = df.selectExpr("id as c1", "id as c2") + + def verifyJoinHint(df: DataFrame, expectedHints: Seq[JoinHint]): Unit = { + val optimized = df.queryExecution.optimizedPlan + val joinHints = optimized collect { + case Join(_, _, _, _, hint) => hint + case _: ResolvedHint => fail("ResolvedHint should not appear after optimize.") + } + assert(joinHints == expectedHints) + } + + test("single join") { + verifyJoinHint( + df.hint("broadcast").join(df, "id"), + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: Nil + ) + verifyJoinHint( + df.join(df.hint("broadcast"), "id"), + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: Nil + ) + } + + test("multiple joins") { + verifyJoinHint( + df1.join(df2.hint("broadcast").join(df3, 'b1 === 'c1).hint("broadcast"), 'a1 === 'c1), + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: Nil + ) + verifyJoinHint( + df1.hint("broadcast").join(df2, 'a1 === 'b1).hint("broadcast").join(df3, 'a1 === 'c1), + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: Nil + ) + } + + test("hint scope") { + withTempView("a", "b", "c") { + df1.createOrReplaceTempView("a") + df2.createOrReplaceTempView("b") + verifyJoinHint( + sql( + """ + |select /*+ broadcast(a, b)*/ * from ( + | select /*+ broadcast(b)*/ * from a join b on a.a1 = b.b1 + |) a join ( + | select /*+ broadcast(a)*/ * from a join b on a.a1 = b.b1 + |) b on a.a1 = b.b1 + """.stripMargin), + JoinHint( + Some(HintInfo(broadcast = true)), + Some(HintInfo(broadcast = true))) :: + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: Nil + ) + } + } + + test("hint preserved after join reorder") { + withTempView("a", "b", "c") { + df1.createOrReplaceTempView("a") + df2.createOrReplaceTempView("b") + df3.createOrReplaceTempView("c") + verifyJoinHint( + sql("select /*+ broadcast(a, c)*/ * from a, b, c " + + "where a.a1 = b.b1 and b.b1 = c.c1"), + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: + JoinHint( + Some(HintInfo(broadcast = true)), + None):: Nil + ) + verifyJoinHint( + sql("select /*+ broadcast(a, c)*/ * from a, c, b " + + "where a.a1 = b.b1 and b.b1 = c.c1"), + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: + JoinHint( + Some(HintInfo(broadcast = true)), + None):: Nil + ) + verifyJoinHint( + sql("select /*+ broadcast(b, c)*/ * from a, c, b " + + "where a.a1 = b.b1 and b.b1 = c.c1"), + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: + JoinHint( + None, + Some(HintInfo(broadcast = true))):: Nil + ) + + verifyJoinHint( + df1.join(df2, 'a1 === 'b1 && 'a1 > 5).hint("broadcast") + .join(df3, 'b1 === 'c1 && 'a1 < 10), + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: + JoinHint.NONE:: Nil + ) + + verifyJoinHint( + df1.join(df2, 'a1 === 'b1 && 'a1 > 5).hint("broadcast") + .join(df3, 'b1 === 'c1 && 'a1 < 10) + .join(df, 'b1 === 'id), + JoinHint.NONE :: + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: + JoinHint.NONE:: Nil + ) + } + } + + test("intersect/except") { + val dfSub = spark.range(2) + verifyJoinHint( + df.hint("broadcast").except(dfSub).join(df, "id"), + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: + JoinHint.NONE :: Nil + ) + verifyJoinHint( + df.join(df.hint("broadcast").intersect(dfSub), "id"), + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: + JoinHint.NONE :: Nil + ) + } + + test("hint merge") { + verifyJoinHint( + df.hint("broadcast").filter('id > 2).hint("broadcast").join(df, "id"), + JoinHint( + Some(HintInfo(broadcast = true)), + None) :: Nil + ) + verifyJoinHint( + df.join(df.hint("broadcast").limit(2).hint("broadcast"), "id"), + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: Nil + ) + } +} 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 02dc32d5f90b..99842680cedf 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 @@ -237,8 +237,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared ) numbers.foreach { case (input, (expectedSize, expectedRows)) => val stats = Statistics(sizeInBytes = input, rowCount = Some(input)) - val expectedString = s"sizeInBytes=$expectedSize, rowCount=$expectedRows," + - s" hints=none" + val expectedString = s"sizeInBytes=$expectedSize, rowCount=$expectedRows" assert(stats.simpleString == expectedString) } } 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 42dd0024b258..f238148e61c3 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 @@ -203,7 +203,7 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { } test("broadcast hint in SQL") { - import org.apache.spark.sql.catalyst.plans.logical.{ResolvedHint, Join} + import org.apache.spark.sql.catalyst.plans.logical.Join spark.range(10).createOrReplaceTempView("t") spark.range(10).createOrReplaceTempView("u") @@ -216,12 +216,12 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { val plan3 = sql(s"SELECT /*+ $name(v) */ * FROM t JOIN u ON t.id = u.id").queryExecution .optimizedPlan - assert(plan1.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) - assert(!plan1.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) - assert(!plan2.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) - assert(plan2.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) - assert(!plan3.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) - assert(!plan3.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) + assert(plan1.asInstanceOf[Join].hint.leftHint.get.broadcast) + assert(plan1.asInstanceOf[Join].hint.rightHint.isEmpty) + assert(plan2.asInstanceOf[Join].hint.leftHint.isEmpty) + assert(plan2.asInstanceOf[Join].hint.rightHint.get.broadcast) + assert(plan3.asInstanceOf[Join].hint.leftHint.isEmpty) + assert(plan3.asInstanceOf[Join].hint.rightHint.isEmpty) } } 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 22279a3a43ef..771a9730247a 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 @@ -21,7 +21,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf @@ -85,7 +85,8 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { expectedAnswer: Seq[Row]): Unit = { def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { - val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition)) + val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, + Inner, Some(condition), JoinHint.NONE) ExtractEquiJoinKeys.unapply(join) } @@ -102,7 +103,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using ShuffledHashJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( @@ -121,7 +122,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { } testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastHashJoin") { _ => - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( @@ -140,7 +141,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using SortMergeJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index f5edd6bbd5e6..f99a278bb242 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf @@ -80,7 +80,8 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { expectedAnswer: Seq[Product]): Unit = { def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { - val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition())) + val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, + Inner, Some(condition()), JoinHint.NONE) ExtractEquiJoinKeys.unapply(join) } @@ -128,7 +129,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastHashJoin (build=left)") { _ => - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeBroadcastHashJoin( @@ -140,7 +141,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastHashJoin (build=right)") { _ => - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeBroadcastHashJoin( @@ -152,7 +153,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using ShuffledHashJoin (build=left)") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeShuffledHashJoin( @@ -164,7 +165,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using ShuffledHashJoin (build=right)") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeShuffledHashJoin( @@ -176,7 +177,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } testWithWholeStageCodegenOnAndOff(s"$testName using SortMergeJoin") { _ => - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeSortMergeJoin(leftKeys, rightKeys, boundCondition, leftPlan, rightPlan), 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 513248dae48b..1f04fcf6ca45 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 @@ -21,7 +21,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf @@ -72,13 +72,14 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { expectedAnswer: Seq[Product]): Unit = { def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { - val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition)) + val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, + Inner, Some(condition), JoinHint.NONE) ExtractEquiJoinKeys.unapply(join) } if (joinType != FullOuter) { test(s"$testName using ShuffledHashJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { val buildSide = if (joinType == LeftOuter) BuildRight else BuildLeft checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => @@ -99,7 +100,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { case RightOuter => BuildLeft case _ => fail(s"Unsupported join type $joinType") } - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => BroadcastHashJoinExec( @@ -112,7 +113,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using SortMergeJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(spark.sessionState.conf).apply( From 1a641525e60039cc6b10816e946cb6f44b3e2696 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Mon, 7 Jan 2019 15:35:33 -0800 Subject: [PATCH 0248/1072] [SPARK-26491][CORE][TEST] Use ConfigEntry for hardcoded configs for test categories ## What changes were proposed in this pull request? The PR makes hardcoded `spark.test` and `spark.testing` configs to use `ConfigEntry` and put them in the config package. ## How was this patch tested? existing UTs Closes #23413 from mgaido91/SPARK-26491. Authored-by: Marco Gaido Signed-off-by: Marcelo Vanzin --- .../spark/ExecutorAllocationManager.scala | 4 +- .../scala/org/apache/spark/SparkContext.scala | 3 +- .../deploy/history/FsHistoryProvider.scala | 3 +- .../apache/spark/deploy/worker/Worker.scala | 4 +- .../spark/executor/ProcfsMetricsGetter.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 3 +- .../apache/spark/internal/config/Tests.scala | 56 +++++++++++++++++++ .../spark/memory/StaticMemoryManager.scala | 5 +- .../spark/memory/UnifiedMemoryManager.scala | 7 ++- .../apache/spark/scheduler/DAGScheduler.scala | 3 +- .../cluster/StandaloneSchedulerBackend.scala | 3 +- .../org/apache/spark/util/SizeEstimator.scala | 5 +- .../scala/org/apache/spark/util/Utils.scala | 5 +- .../org/apache/spark/DistributedSuite.scala | 5 +- .../ExecutorAllocationManagerSuite.scala | 3 +- .../scala/org/apache/spark/ShuffleSuite.scala | 5 +- .../org/apache/spark/SparkFunSuite.scala | 3 +- .../history/HistoryServerArgumentsSuite.scala | 6 +- .../deploy/history/HistoryServerSuite.scala | 7 ++- .../memory/StaticMemoryManagerSuite.scala | 5 +- .../memory/UnifiedMemoryManagerSuite.scala | 33 +++++------ .../scheduler/BarrierTaskContextSuite.scala | 7 ++- .../scheduler/BlacklistIntegrationSuite.scala | 19 ++++--- .../sort/ShuffleExternalSorterSuite.scala | 5 +- .../BlockManagerReplicationSuite.scala | 9 +-- .../spark/storage/BlockManagerSuite.scala | 10 ++-- .../spark/storage/MemoryStoreSuite.scala | 1 - .../spark/util/SizeEstimatorSuite.scala | 5 +- .../ExternalAppendOnlyMapSuite.scala | 3 +- .../util/collection/ExternalSorterSuite.scala | 3 +- .../KubernetesTestComponents.scala | 3 +- .../MesosCoarseGrainedSchedulerBackend.scala | 3 +- .../spark/sql/execution/SQLExecution.scala | 4 +- .../apache/spark/sql/BenchmarkQueryTest.scala | 3 +- .../execution/UnsafeRowSerializerSuite.scala | 3 +- 35 files changed, 165 insertions(+), 83 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/internal/config/Tests.scala diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index d966582295b3..0807e653b41a 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -27,6 +27,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.source.Source import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMaster @@ -157,7 +158,7 @@ private[spark] class ExecutorAllocationManager( // Polling loop interval (ms) private val intervalMillis: Long = if (Utils.isTesting) { - conf.getLong(TESTING_SCHEDULE_INTERVAL_KEY, 100) + conf.get(TEST_SCHEDULE_INTERVAL) } else { 100 } @@ -899,5 +900,4 @@ private[spark] class ExecutorAllocationManager( private object ExecutorAllocationManager { val NOT_SET = Long.MaxValue - val TESTING_SCHEDULE_INTERVAL_KEY = "spark.testing.dynamicAllocation.scheduleInterval" } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 89be9de08307..3a1e1b931002 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -45,6 +45,7 @@ import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.io.CompressionCodec import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ @@ -470,7 +471,7 @@ class SparkContext(config: SparkConf) extends Logging { // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) + for { (envKey, propKey) <- Seq(("SPARK_TESTING", IS_TESTING.key)) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { executorEnvs(envKey) = value } 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 709a380dfb63..3c5648434fa6 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 @@ -45,6 +45,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DRIVER_LOG_DFS_DIR, History} import org.apache.spark.internal.config.History._ import org.apache.spark.internal.config.Status._ +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.scheduler.ReplayListenerBus._ @@ -267,7 +268,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } // Disable the background thread during tests. - if (!conf.contains("spark.testing")) { + if (!conf.contains(IS_TESTING)) { // A task that periodically checks for event log updates on disk. logDebug(s"Scheduling update thread every $UPDATE_INTERVAL_S seconds") pool.scheduleWithFixedDelay( 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 d5ea2523c628..467df26c4735 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 @@ -37,6 +37,7 @@ import org.apache.spark.deploy.ExternalShuffleService import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} @@ -103,7 +104,6 @@ private[deploy] class Worker( private val CLEANUP_NON_SHUFFLE_FILES_ENABLED = conf.getBoolean("spark.storage.cleanupFilesAfterExecutorExit", true) - private val testing: Boolean = sys.props.contains("spark.testing") private var master: Option[RpcEndpointRef] = None /** @@ -127,7 +127,7 @@ private[deploy] class Worker( private var connected = false private val workerId = generateWorkerId() private val sparkHome = - if (testing) { + if (sys.props.contains(IS_TESTING.key)) { assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!") new File(sys.props("spark.test.home")) } else { diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala index af67f41e94af..f354d603c2e3 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -43,7 +43,7 @@ private[spark] case class ProcfsMetrics( // project. private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends Logging { private val procfsStatFile = "stat" - private val testing = sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + private val testing = Utils.isTesting private val pageSize = computePageSize() private var isAvailable: Boolean = isProcfsAvailable private val pid = computePid() 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 85b2745a2aec..ea79c7310349 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util._ @@ -202,7 +203,7 @@ class TaskMetrics private[spark] () extends Serializable { } // Only used for test - private[spark] val testAccum = sys.props.get("spark.testing").map(_ => new LongAccumulator) + private[spark] val testAccum = sys.props.get(IS_TESTING.key).map(_ => new LongAccumulator) import InternalAccumulator._ diff --git a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala new file mode 100644 index 000000000000..21660ab3a951 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +private[spark] object Tests { + + val TEST_USE_COMPRESSED_OOPS_KEY = "spark.test.useCompressedOops" + + val TEST_MEMORY = ConfigBuilder("spark.testing.memory") + .longConf + .createWithDefault(Runtime.getRuntime.maxMemory) + + val TEST_SCHEDULE_INTERVAL = + ConfigBuilder("spark.testing.dynamicAllocation.scheduleInterval") + .longConf + .createWithDefault(100) + + val IS_TESTING = ConfigBuilder("spark.testing") + .booleanConf + .createOptional + + val TEST_NO_STAGE_RETRY = ConfigBuilder("spark.test.noStageRetry") + .booleanConf + .createWithDefault(false) + + val TEST_RESERVED_MEMORY = ConfigBuilder("spark.testing.reservedMemory") + .longConf + .createOptional + + val TEST_N_HOSTS = ConfigBuilder("spark.testing.nHosts") + .intConf + .createWithDefault(5) + + val TEST_N_EXECUTORS_HOST = ConfigBuilder("spark.testing.nExecutorsPerHost") + .intConf + .createWithDefault(4) + + val TEST_N_CORES_EXECUTOR = ConfigBuilder("spark.testing.nCoresPerExecutor") + .intConf + .createWithDefault(2) +} diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index 0fd349dc5161..7e052c02c937 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -19,6 +19,7 @@ package org.apache.spark.memory import org.apache.spark.SparkConf import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.storage.BlockId /** @@ -120,7 +121,7 @@ private[spark] object StaticMemoryManager { * Return the total amount of memory available for the storage region, in bytes. */ private def getMaxStorageMemory(conf: SparkConf): Long = { - val systemMaxMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) + val systemMaxMemory = conf.get(TEST_MEMORY) val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) val safetyFraction = conf.getDouble("spark.storage.safetyFraction", 0.9) (systemMaxMemory * memoryFraction * safetyFraction).toLong @@ -130,7 +131,7 @@ private[spark] object StaticMemoryManager { * Return the total amount of memory available for the execution region, in bytes. */ private def getMaxExecutionMemory(conf: SparkConf): Long = { - val systemMaxMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) + val systemMaxMemory = conf.get(TEST_MEMORY) if (systemMaxMemory < MIN_MEMORY_BYTES) { throw new IllegalArgumentException(s"System memory $systemMaxMemory must " + diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 9260fd3a6fb3..7801bb87050f 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -19,6 +19,7 @@ package org.apache.spark.memory import org.apache.spark.SparkConf import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests._ import org.apache.spark.storage.BlockId /** @@ -210,9 +211,9 @@ object UnifiedMemoryManager { * Return the total amount of memory shared between execution and storage, in bytes. */ private def getMaxMemory(conf: SparkConf): Long = { - val systemMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) - val reservedMemory = conf.getLong("spark.testing.reservedMemory", - if (conf.contains("spark.testing")) 0 else RESERVED_SYSTEM_MEMORY_BYTES) + val systemMemory = conf.get(TEST_MEMORY) + val reservedMemory = conf.getLong(TEST_RESERVED_MEMORY.key, + if (conf.contains(IS_TESTING)) 0 else RESERVED_SYSTEM_MEMORY_BYTES) val minSystemMemory = (reservedMemory * 1.5).ceil.toLong if (systemMemory < minSystemMemory) { throw new IllegalArgumentException(s"System memory $systemMemory must " + 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 6f4c326442e1..f6ade180ee25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -38,6 +38,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData} @@ -186,7 +187,7 @@ private[spark] class DAGScheduler( private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() /** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */ - private val disallowStageRetryForTest = sc.getConf.getBoolean("spark.test.noStageRetry", false) + private val disallowStageRetryForTest = sc.getConf.get(TEST_NO_STAGE_RETRY) /** * Whether to unregister all the outputs on the host in condition that we receive a FetchFailure, 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 adef20d3077d..66080b6e6b4f 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 @@ -26,6 +26,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ @@ -90,7 +91,7 @@ private[spark] class StandaloneSchedulerBackend( // compute-classpath.{cmd,sh} and makes all needed jars available to child processes // when the assembly is built with the "*-provided" profiles enabled. val testingClassPath = - if (sys.props.contains("spark.testing")) { + if (sys.props.contains(IS_TESTING.key)) { sys.props("java.class.path").split(java.io.File.pathSeparator).toSeq } else { Nil diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 3bfdf95db84c..e12b6b71578c 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -28,6 +28,7 @@ import com.google.common.collect.MapMaker import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Tests.TEST_USE_COMPRESSED_OOPS_KEY import org.apache.spark.util.collection.OpenHashSet /** @@ -126,8 +127,8 @@ object SizeEstimator extends Logging { private def getIsCompressedOops: Boolean = { // This is only used by tests to override the detection of compressed oops. The test // actually uses a system property instead of a SparkConf, so we'll stick with that. - if (System.getProperty("spark.test.useCompressedOops") != null) { - return System.getProperty("spark.test.useCompressedOops").toBoolean + if (System.getProperty(TEST_USE_COMPRESSED_OOPS_KEY) != null) { + return System.getProperty(TEST_USE_COMPRESSED_OOPS_KEY).toBoolean } // java.vm.info provides compressed ref info for IBM JDKs 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 3527fee68939..16ef38142ad9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -60,6 +60,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} @@ -1847,7 +1848,7 @@ private[spark] object Utils extends Logging { * Indicates whether Spark is currently running unit tests. */ def isTesting: Boolean = { - sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + sys.env.contains("SPARK_TESTING") || sys.props.contains(IS_TESTING.key) } /** @@ -2175,7 +2176,7 @@ private[spark] object Utils extends Logging { */ def portMaxRetries(conf: SparkConf): Int = { val maxRetries = conf.getOption("spark.port.maxRetries").map(_.toInt) - if (conf.contains("spark.testing")) { + if (conf.contains(IS_TESTING)) { // Set a higher number of retries for tests... maxRetries.getOrElse(100) } else { diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 4083b20c2359..21050e44414f 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.time.{Millis, Span} import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests._ import org.apache.spark.security.EncryptionFunSuite import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.io.ChunkedByteBuffer @@ -217,7 +218,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val size = 10000 val conf = new SparkConf() .set("spark.storage.unrollMemoryThreshold", "1024") - .set("spark.testing.memory", (size / 2).toString) + .set(TEST_MEMORY, size.toLong / 2) sc = new SparkContext(clusterUrl, "test", conf) val data = sc.parallelize(1 to size, 2).persist(StorageLevel.MEMORY_ONLY) assert(data.count() === size) @@ -233,7 +234,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val numPartitions = 20 val conf = new SparkConf() .set("spark.storage.unrollMemoryThreshold", "1024") - .set("spark.testing.memory", size.toString) + .set(TEST_MEMORY, size.toLong) sc = new SparkContext(clusterUrl, "test", conf) val data = sc.parallelize(1 to size, numPartitions).persist(StorageLevel.MEMORY_ONLY) assert(data.count() === size) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 38f5e8c9f0ac..6b310b9cb67a 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.scheduler._ import org.apache.spark.scheduler.ExternalClusterManager import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -1166,7 +1167,7 @@ class ExecutorAllocationManagerSuite .set("spark.dynamicAllocation.testing", "true") // SPARK-22864: effectively disable the allocation schedule by setting the period to a // really long value. - .set(TESTING_SCHEDULE_INTERVAL_KEY, "10000") + .set(TEST_SCHEDULE_INTERVAL, 10000L) val sc = new SparkContext(conf) contexts += sc sc diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 35f728cd57fe..ffa70425ea36 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -23,6 +23,7 @@ import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListenerTaskEnd} @@ -37,7 +38,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // Ensure that the DAGScheduler doesn't retry stages whose fetches fail, so that we accurately // test that the shuffle works (rather than retrying until all blocks are local to one Executor). - conf.set("spark.test.noStageRetry", "true") + conf.set(TEST_NO_STAGE_RETRY, true) test("groupByKey without compression") { val myConf = conf.clone().set("spark.shuffle.compress", "false") @@ -269,7 +270,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC } test("[SPARK-4085] rerun map stage if reduce stage cannot find its local shuffle file") { - val myConf = conf.clone().set("spark.test.noStageRetry", "false") + val myConf = conf.clone().set(TEST_NO_STAGE_RETRY, false) sc = new SparkContext("local", "test", myConf) val rdd = sc.parallelize(1 to 10, 2).map((_, 1)).reduceByKey(_ + _) rdd.count() diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index dad24d7c01b8..7d114b1b0c14 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -23,6 +23,7 @@ import java.io.File import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.util.{AccumulatorContext, Utils} /** @@ -59,7 +60,7 @@ abstract class SparkFunSuite protected val enableAutoThreadAudit = true protected override def beforeAll(): Unit = { - System.setProperty("spark.testing", "true") + System.setProperty(IS_TESTING.key, "true") if (enableAutoThreadAudit) { doThreadPreAudit() } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala index 6b479873f69f..5903ae71ec66 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala @@ -23,7 +23,7 @@ import com.google.common.io.Files import org.apache.spark._ import org.apache.spark.internal.config.History._ -import org.apache.spark.util.Utils +import org.apache.spark.internal.config.Tests._ class HistoryServerArgumentsSuite extends SparkFunSuite { @@ -31,14 +31,14 @@ class HistoryServerArgumentsSuite extends SparkFunSuite { private val conf = new SparkConf() .set(HISTORY_LOG_DIR, logDir.getAbsolutePath) .set(UPDATE_INTERVAL_S, 1L) - .set("spark.testing", "true") + .set(IS_TESTING, true) test("No Arguments Parsing") { val argStrings = Array.empty[String] val hsa = new HistoryServerArguments(conf, argStrings) assert(conf.get(HISTORY_LOG_DIR) === logDir.getAbsolutePath) assert(conf.get(UPDATE_INTERVAL_S) === 1L) - assert(conf.get("spark.testing") === "true") + assert(conf.get(IS_TESTING).getOrElse(false)) } test("Properties File Arguments Parsing --properties-file") { 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 96458c55b5f5..bb7d3c52bc9c 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 @@ -47,6 +47,7 @@ import org.scalatest.selenium.WebBrowser import org.apache.spark._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.status.api.v1.ApplicationInfo import org.apache.spark.status.api.v1.JobData import org.apache.spark.ui.SparkUI @@ -81,7 +82,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val conf = new SparkConf() .set(HISTORY_LOG_DIR, logDir) .set(UPDATE_INTERVAL_S.key, "0") - .set("spark.testing", "true") + .set(IS_TESTING, true) .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) .set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) .set(EVENT_LOG_PROCESS_TREE_METRICS, true) @@ -400,7 +401,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers */ test("security manager starts with spark.authenticate set") { val conf = new SparkConf() - .set("spark.testing", "true") + .set(IS_TESTING, true) .set(SecurityManager.SPARK_AUTH_CONF, "true") HistoryServer.createSecurityManager(conf) } @@ -422,7 +423,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers .set(UPDATE_INTERVAL_S.key, "1s") .set(EVENT_LOG_ENABLED, true) .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) - .remove("spark.testing") + .remove(IS_TESTING) val provider = new FsHistoryProvider(myConf) val securityManager = HistoryServer.createSecurityManager(myConf) diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala index 0f32fe4059fb..c3275add50f4 100644 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala @@ -21,6 +21,7 @@ import org.mockito.Mockito.when import org.apache.spark.SparkConf import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE +import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.storage.TestBlockId import org.apache.spark.storage.memory.MemoryStore @@ -48,8 +49,8 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { new StaticMemoryManager( conf.clone .set("spark.memory.fraction", "1") - .set("spark.testing.memory", maxOnHeapExecutionMemory.toString) - .set(MEMORY_OFFHEAP_SIZE.key, maxOffHeapExecutionMemory.toString), + .set(TEST_MEMORY, maxOnHeapExecutionMemory) + .set(MEMORY_OFFHEAP_SIZE, maxOffHeapExecutionMemory), maxOnHeapExecutionMemory = maxOnHeapExecutionMemory, maxOnHeapStorageMemory = 0, numCores = 1) diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 5ce3453b682f..8556e920daeb 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.SparkConf import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.storage.TestBlockId import org.apache.spark.storage.memory.MemoryStore @@ -43,8 +44,8 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes maxOffHeapExecutionMemory: Long): UnifiedMemoryManager = { val conf = new SparkConf() .set("spark.memory.fraction", "1") - .set("spark.testing.memory", maxOnHeapExecutionMemory.toString) - .set(MEMORY_OFFHEAP_SIZE.key, maxOffHeapExecutionMemory.toString) + .set(TEST_MEMORY, maxOnHeapExecutionMemory) + .set(MEMORY_OFFHEAP_SIZE, maxOffHeapExecutionMemory) .set("spark.memory.storageFraction", storageFraction.toString) UnifiedMemoryManager(conf, numCores = 1) } @@ -218,19 +219,19 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes } test("small heap") { - val systemMemory = 1024 * 1024 - val reservedMemory = 300 * 1024 + val systemMemory = 1024L * 1024 + val reservedMemory = 300L * 1024 val memoryFraction = 0.8 val conf = new SparkConf() .set("spark.memory.fraction", memoryFraction.toString) - .set("spark.testing.memory", systemMemory.toString) - .set("spark.testing.reservedMemory", reservedMemory.toString) + .set(TEST_MEMORY, systemMemory) + .set(TEST_RESERVED_MEMORY, reservedMemory) val mm = UnifiedMemoryManager(conf, numCores = 1) val expectedMaxMemory = ((systemMemory - reservedMemory) * memoryFraction).toLong assert(mm.maxHeapMemory === expectedMaxMemory) // Try using a system memory that's too small - val conf2 = conf.clone().set("spark.testing.memory", (reservedMemory / 2).toString) + val conf2 = conf.clone().set(TEST_MEMORY, reservedMemory / 2) val exception = intercept[IllegalArgumentException] { UnifiedMemoryManager(conf2, numCores = 1) } @@ -238,13 +239,13 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes } test("insufficient executor memory") { - val systemMemory = 1024 * 1024 - val reservedMemory = 300 * 1024 + val systemMemory = 1024L * 1024 + val reservedMemory = 300L * 1024 val memoryFraction = 0.8 val conf = new SparkConf() .set("spark.memory.fraction", memoryFraction.toString) - .set("spark.testing.memory", systemMemory.toString) - .set("spark.testing.reservedMemory", reservedMemory.toString) + .set(TEST_MEMORY, systemMemory) + .set(TEST_RESERVED_MEMORY, reservedMemory) val mm = UnifiedMemoryManager(conf, numCores = 1) // Try using an executor memory that's too small @@ -259,7 +260,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val conf = new SparkConf() .set("spark.memory.fraction", "1") .set("spark.memory.storageFraction", "0") - .set("spark.testing.memory", "1000") + .set(TEST_MEMORY, 1000L) val mm = UnifiedMemoryManager(conf, numCores = 2) val ms = makeMemoryStore(mm) val memoryMode = MemoryMode.ON_HEAP @@ -285,7 +286,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val conf = new SparkConf() .set("spark.memory.fraction", "1") .set("spark.memory.storageFraction", "0") - .set("spark.testing.memory", "1000") + .set(TEST_MEMORY, 1000L) val mm = UnifiedMemoryManager(conf, numCores = 2) makeBadMemoryStore(mm) val memoryMode = MemoryMode.ON_HEAP @@ -306,9 +307,9 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("not enough free memory in the storage pool --OFF_HEAP") { val conf = new SparkConf() - .set(MEMORY_OFFHEAP_SIZE.key, "1000") - .set("spark.testing.memory", "1000") - .set(MEMORY_OFFHEAP_ENABLED.key, "true") + .set(MEMORY_OFFHEAP_SIZE, 1000L) + .set(TEST_MEMORY, 1000L) + .set(MEMORY_OFFHEAP_ENABLED, true) val taskAttemptId = 0L val mm = UnifiedMemoryManager(conf, numCores = 1) val ms = makeMemoryStore(mm) 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 36dd620a5685..112fd31a060e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import scala.util.Random import org.apache.spark._ +import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { @@ -76,7 +77,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { test("throw exception on barrier() call timeout") { val conf = new SparkConf() .set("spark.barrier.sync.timeout", "1") - .set("spark.test.noStageRetry", "true") + .set(TEST_NO_STAGE_RETRY, true) .setMaster("local-cluster[4, 1, 1024]") .setAppName("test-cluster") sc = new SparkContext(conf) @@ -101,7 +102,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { test("throw exception if barrier() call doesn't happen on every task") { val conf = new SparkConf() .set("spark.barrier.sync.timeout", "1") - .set("spark.test.noStageRetry", "true") + .set(TEST_NO_STAGE_RETRY, true) .setMaster("local-cluster[4, 1, 1024]") .setAppName("test-cluster") sc = new SparkContext(conf) @@ -124,7 +125,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { test("throw exception if the number of barrier() calls are not the same on every task") { val conf = new SparkConf() .set("spark.barrier.sync.timeout", "1") - .set("spark.test.noStageRetry", "true") + .set(TEST_NO_STAGE_RETRY, true) .setMaster("local-cluster[4, 1, 1024]") .setAppName("test-cluster") sc = new SparkContext(conf) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 29bb8232f44f..2215f7f36621 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -20,6 +20,7 @@ import scala.concurrent.duration._ import org.apache.spark._ import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests._ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ @@ -58,9 +59,9 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", config.MAX_TASK_FAILURES.key -> "4", - "spark.testing.nHosts" -> "2", - "spark.testing.nExecutorsPerHost" -> "5", - "spark.testing.nCoresPerExecutor" -> "10" + TEST_N_HOSTS.key -> "2", + TEST_N_EXECUTORS_HOST.key -> "5", + TEST_N_CORES_EXECUTOR.key -> "10" ) ) { // To reliably reproduce the failure that would occur without blacklisting, we have to use 1 @@ -102,9 +103,9 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM "SPARK-15865 Progress with fewer executors than maxTaskFailures", extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", - "spark.testing.nHosts" -> "2", - "spark.testing.nExecutorsPerHost" -> "1", - "spark.testing.nCoresPerExecutor" -> "1", + TEST_N_HOSTS.key -> "2", + TEST_N_EXECUTORS_HOST.key -> "1", + TEST_N_CORES_EXECUTOR.key -> "1", "spark.scheduler.blacklist.unschedulableTaskSetTimeout" -> "0s" ) ) { @@ -129,9 +130,9 @@ class MultiExecutorMockBackend( conf: SparkConf, taskScheduler: TaskSchedulerImpl) extends MockBackend(conf, taskScheduler) { - val nHosts = conf.getInt("spark.testing.nHosts", 5) - val nExecutorsPerHost = conf.getInt("spark.testing.nExecutorsPerHost", 4) - val nCoresPerExecutor = conf.getInt("spark.testing.nCoresPerExecutor", 2) + val nHosts = conf.get(TEST_N_HOSTS) + val nExecutorsPerHost = conf.get(TEST_N_EXECUTORS_HOST) + val nCoresPerExecutor = conf.get(TEST_N_CORES_EXECUTOR) override val executorIdToExecutor: Map[String, ExecutorTaskStatus] = { (0 until nHosts).flatMap { hostIdx => diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala index b9f0e873375b..43621cb85762 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory._ import org.apache.spark.unsafe.Platform @@ -33,8 +34,8 @@ class ShuffleExternalSorterSuite extends SparkFunSuite with LocalSparkContext wi val conf = new SparkConf() .setMaster("local[1]") .setAppName("ShuffleExternalSorterSuite") - .set("spark.testing", "true") - .set("spark.testing.memory", "1600") + .set(IS_TESTING, true) + .set(TEST_MEMORY, 1600L) .set("spark.memory.fraction", "1") sc = new SparkContext(conf) 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 19116cf22d2f..480e07fb9399 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DRIVER_PORT, MEMORY_OFFHEAP_SIZE} +import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService @@ -69,8 +70,8 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite protected def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { - conf.set("spark.testing.memory", maxMem.toString) - conf.set(MEMORY_OFFHEAP_SIZE.key, maxMem.toString) + conf.set(TEST_MEMORY, maxMem) + conf.set(MEMORY_OFFHEAP_SIZE, maxMem) val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) @@ -87,7 +88,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite conf.set("spark.authenticate", "false") conf.set(DRIVER_PORT, rpcEnv.address.port) - conf.set("spark.testing", "true") + conf.set(IS_TESTING, true) conf.set("spark.memory.fraction", "1") conf.set("spark.memory.storageFraction", "1") conf.set("spark.storage.unrollFraction", "0.4") @@ -233,7 +234,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work when(failableTransfer.hostName).thenReturn("some-hostname") when(failableTransfer.port).thenReturn(1000) - conf.set("spark.testing.memory", "10000") + conf.set(TEST_MEMORY, 10000L) val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) val failableStore = new BlockManager("failable-store", rpcEnv, master, serializerManager, conf, 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 a7bb2a03360a..bda81365b079 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -37,6 +37,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.{BlockDataManager, BlockTransferService, TransportContext} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} @@ -89,8 +90,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE transferService: Option[BlockTransferService] = Option.empty, testConf: Option[SparkConf] = None): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) - bmConf.set("spark.testing.memory", maxMem.toString) - bmConf.set(MEMORY_OFFHEAP_SIZE.key, maxMem.toString) + bmConf.set(TEST_MEMORY, maxMem) + bmConf.set(MEMORY_OFFHEAP_SIZE, maxMem) val serializer = new KryoSerializer(bmConf) val encryptionKey = if (bmConf.get(IO_ENCRYPTION_ENABLED)) { Some(CryptoStreamUtils.createKey(bmConf)) @@ -115,11 +116,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE System.setProperty("os.arch", "amd64") conf = new SparkConf(false) .set("spark.app.id", "test") - .set("spark.testing", "true") + .set(IS_TESTING, true) .set("spark.memory.fraction", "1") .set("spark.memory.storageFraction", "1") .set("spark.kryoserializer.buffer", "1m") - .set("spark.test.useCompressedOops", "true") .set("spark.storage.unrollFraction", "0.4") .set("spark.storage.unrollMemoryThreshold", "512") @@ -901,7 +901,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("block store put failure") { // Use Java serializer so we can create an unserializable error. - conf.set("spark.testing.memory", "1200") + conf.set(TEST_MEMORY, 1200L) val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) val memoryManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala index baff672f5fb8..b02af2bfe7ac 100644 --- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala @@ -39,7 +39,6 @@ class MemoryStoreSuite with ResetSystemProperties { var conf: SparkConf = new SparkConf(false) - .set("spark.test.useCompressedOops", "true") .set("spark.storage.unrollFraction", "0.4") .set("spark.storage.unrollMemoryThreshold", "512") diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 63f9f82adf3e..8bc62db81e4f 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.Tests.TEST_USE_COMPRESSED_OOPS_KEY class DummyClass1 {} @@ -76,7 +77,7 @@ class SizeEstimatorSuite // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case super.beforeEach() System.setProperty("os.arch", "amd64") - System.setProperty("spark.test.useCompressedOops", "true") + System.setProperty(TEST_USE_COMPRESSED_OOPS_KEY, "true") } override def afterEach(): Unit = { @@ -192,7 +193,7 @@ class SizeEstimatorSuite // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("64-bit arch with no compressed oops") { System.setProperty("os.arch", "amd64") - System.setProperty("spark.test.useCompressedOops", "false") + System.setProperty(TEST_USE_COMPRESSED_OOPS_KEY, "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() 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 35fba1a3b73c..6211399005e1 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 @@ -27,6 +27,7 @@ import org.scalatest.concurrent.Eventually import org.apache.spark._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.io.CompressionCodec import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.util.CompletionIterator @@ -552,7 +553,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite val conf = createSparkConf(loadDefaults = false) .set("spark.shuffle.memoryFraction", "0.01") .set("spark.memory.useLegacyMode", "true") - .set("spark.testing.memory", "100000000") + .set(TEST_MEMORY, 100000000L) .set("spark.shuffle.sort.bypassMergeThreshold", "0") sc = new SparkContext("local", "test", conf) val N = 2e5.toInt diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 47173b89e91e..aa400dd74e9c 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark._ +import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.unsafe.array.LongArray @@ -639,7 +640,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { val conf = createSparkConf(loadDefaults = false, kryo = false) .set("spark.shuffle.memoryFraction", "0.01") .set("spark.memory.useLegacyMode", "true") - .set("spark.testing.memory", "100000000") + .set(TEST_MEMORY, 100000000L) .set("spark.shuffle.sort.bypassMergeThreshold", "0") sc = new SparkContext("local", "test", conf) val N = 2e5.toInt 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 c0b435efb8c9..cc8968394901 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 @@ -27,6 +27,7 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Tests.IS_TESTING private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { @@ -67,7 +68,7 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .set("spark.executors.instances", "1") .set("spark.app.name", "spark-test-app") .set("spark.ui.enabled", "true") - .set("spark.testing", "false") + .set(IS_TESTING, false) .set("spark.kubernetes.submission.waitAppCompletion", "false") .set("spark.kubernetes.authenticate.driver.serviceAccountName", serviceAccountName) } 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 03cd2583b9b2..fb235350700f 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 @@ -33,6 +33,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkExceptio import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient @@ -298,7 +299,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } protected def driverURL: String = { - if (conf.contains("spark.testing")) { + if (conf.contains(IS_TESTING)) { "driverURL" } else { RpcEndpointAddress( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index dda7cb55f539..5b38fe5c46bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong -import org.apache.spark.SparkContext +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} @@ -38,7 +38,7 @@ object SQLExecution { executionIdToQueryExecution.get(executionId) } - private val testing = sys.props.contains("spark.testing") + private val testing = sys.props.contains(IS_TESTING.key) private[sql] def checkSQLExecutionId(sparkSession: SparkSession): Unit = { val sc = sparkSession.sparkContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala index d95794d62403..c37d663941d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterAll +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} @@ -29,7 +30,7 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSQLContext with B // When Utils.isTesting is true, the RuleExecutor will issue an exception when hitting // the max iteration of analyzer/optimizer batches. - assert(Utils.isTesting, "spark.testing is not set to true") + assert(Utils.isTesting, s"${IS_TESTING.key} is not set to true") /** * Drop all the tables diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index ca8692290edb..963e42517b44 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -21,6 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} import java.util.Properties import org.apache.spark._ +import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.RDD import org.apache.spark.sql.{LocalSparkSession, Row, SparkSession} @@ -99,7 +100,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { val conf = new SparkConf() .set("spark.shuffle.spill.initialMemoryThreshold", "1") .set("spark.shuffle.sort.bypassMergeThreshold", "0") - .set("spark.testing.memory", "80000") + .set(TEST_MEMORY, 80000L) spark = SparkSession.builder().master("local").appName("test").config(conf).getOrCreate() val outputFile = File.createTempFile("test-unsafe-row-serializer-spill", "") outputFile.deleteOnExit() From 5102ccc4ab6e30caa5510131dee7098b4f3ad32e Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 7 Jan 2019 15:48:54 -0800 Subject: [PATCH 0249/1072] [SPARK-26339][SQL][FOLLOW-UP] Issue warning instead of throwing an exception for underscore files ## What changes were proposed in this pull request? The PR https://github.com/apache/spark/pull/23446 happened to introduce a behaviour change - empty dataframes can't be read anymore from underscore files. It looks controversial to allow or disallow this case so this PR targets to fix to issue warning instead of throwing an exception to be more conservative. **Before** ```scala scala> spark.read.schema("a int").parquet("_tmp*").show() org.apache.spark.sql.AnalysisException: All paths were ignored: file:/.../_tmp file:/.../_tmp1; at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:570) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:360) at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:231) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:219) at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:651) at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:635) ... 49 elided scala> spark.read.text("_tmp*").show() org.apache.spark.sql.AnalysisException: All paths were ignored: file:/.../_tmp file:/.../_tmp1; at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:570) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:360) at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:231) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:219) at org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:723) at org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:695) ... 49 elided ``` **After** ```scala scala> spark.read.schema("a int").parquet("_tmp*").show() 19/01/07 15:14:43 WARN DataSource: All paths were ignored: file:/.../_tmp file:/.../_tmp1 +---+ | a| +---+ +---+ scala> spark.read.text("_tmp*").show() 19/01/07 15:14:51 WARN DataSource: All paths were ignored: file:/.../_tmp file:/.../_tmp1 +-----+ |value| +-----+ +-----+ ``` ## How was this patch tested? Manually tested as above. Closes #23481 from HyukjinKwon/SPARK-26339. Authored-by: Hyukjin Kwon Signed-off-by: gatorsmile --- .../execution/datasources/DataSource.scala | 6 +++--- .../src/test/resources/test-data/_cars.csv | 7 ------- .../execution/datasources/csv/CSVSuite.scala | 20 ------------------- 3 files changed, 3 insertions(+), 30 deletions(-) delete mode 100644 sql/core/src/test/resources/test-data/_cars.csv 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 2a438a5cbf95..5dad784e45af 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 @@ -567,11 +567,11 @@ case class DataSource( } if (filteredOut.nonEmpty) { if (filteredIn.isEmpty) { - throw new AnalysisException( - s"All paths were ignored:\n${filteredOut.mkString("\n ")}") + logWarning( + s"All paths were ignored:\n ${filteredOut.mkString("\n ")}") } else { logDebug( - s"Some paths were ignored:\n${filteredOut.mkString("\n ")}") + s"Some paths were ignored:\n ${filteredOut.mkString("\n ")}") } } } diff --git a/sql/core/src/test/resources/test-data/_cars.csv b/sql/core/src/test/resources/test-data/_cars.csv deleted file mode 100644 index 40ded573ade5..000000000000 --- a/sql/core/src/test/resources/test-data/_cars.csv +++ /dev/null @@ -1,7 +0,0 @@ - -year,make,model,comment,blank -"2012","Tesla","S","No comment", - -1997,Ford,E350,"Go get one now they are going fast", -2015,Chevy,Volt - 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 fb1bedfaa32c..d9e5d7af1967 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 @@ -53,7 +53,6 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te private val carsEmptyValueFile = "test-data/cars-empty-value.csv" private val carsBlankColName = "test-data/cars-blank-column-name.csv" private val carsCrlf = "test-data/cars-crlf.csv" - private val carsFilteredOutFile = "test-data/_cars.csv" private val emptyFile = "test-data/empty.csv" private val commentsFile = "test-data/comments.csv" private val disableCommentsFile = "test-data/disable_comments.csv" @@ -347,25 +346,6 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te assert(result.schema.fieldNames.size === 1) } - test("SPARK-26339 Not throw an exception if some of specified paths are filtered in") { - val cars = spark - .read - .option("header", "false") - .csv(testFile(carsFile), testFile(carsFilteredOutFile)) - - verifyCars(cars, withHeader = false, checkTypes = false) - } - - test("SPARK-26339 Throw an exception only if all of the specified paths are filtered out") { - val e = intercept[AnalysisException] { - val cars = spark - .read - .option("header", "false") - .csv(testFile(carsFilteredOutFile)) - }.getMessage - assert(e.contains("All paths were ignored:")) - } - test("DDL test with empty file") { withView("carsTable") { spark.sql( From 5fb5a0292d9ced48860abe712a10cbb8e513b75a Mon Sep 17 00:00:00 2001 From: Adrian Tanase Date: Mon, 7 Jan 2019 19:03:38 -0600 Subject: [PATCH 0250/1072] [MINOR][K8S] add missing docs for podTemplateContainerName properties ## What changes were proposed in this pull request? Adding docs for an enhancement that came in late in this PR: #22146 Currently the docs state that we're going to use the first container in a pod template, which was the implementation for some time, until it was improved with 2 new properties. ## How was this patch tested? I tested that the properties work by combining pod templates with client-mode and a simple pod template. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23155 from aditanase/k8s-readme. Authored-by: Adrian Tanase Signed-off-by: Sean Owen --- docs/running-on-kubernetes.md | 31 +++++++++++++++++++++++++------ 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 3172b1bca8f0..3453ee912205 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -229,8 +229,11 @@ pod template that will always be overwritten by Spark. Therefore, users of this the pod template file only lets Spark start with a template pod instead of an empty pod during the pod-building process. For details, see the [full list](#pod-template-properties) of pod template values that will be overwritten by spark. -Pod template files can also define multiple containers. In such cases, Spark will always assume that the first container in -the list will be the driver or executor container. +Pod template files can also define multiple containers. In such cases, you can use the spark properties +`spark.kubernetes.driver.podTemplateContainerName` and `spark.kubernetes.executor.podTemplateContainerName` +to indicate which container should be used as a basis for the driver or executor. +If not specified, or if the container name is not valid, Spark will assume that the first container in the list +will be the driver or executor container. ## Using Kubernetes Volumes @@ -932,16 +935,32 @@ specific to Spark on Kubernetes. spark.kubernetes.driver.podTemplateFile (none) - Specify the local file that contains the driver [pod template](#pod-template). For example - spark.kubernetes.driver.podTemplateFile=/path/to/driver-pod-template.yaml` + Specify the local file that contains the driver pod template. For example + spark.kubernetes.driver.podTemplateFile=/path/to/driver-pod-template.yaml + + + + spark.kubernetes.driver.podTemplateContainerName + (none) + + Specify the container name to be used as a basis for the driver in the given pod template. + For example spark.kubernetes.driver.podTemplateContainerName=spark-driver spark.kubernetes.executor.podTemplateFile (none) - Specify the local file that contains the executor [pod template](#pod-template). For example - spark.kubernetes.executor.podTemplateFile=/path/to/executor-pod-template.yaml` + Specify the local file that contains the executor pod template. For example + spark.kubernetes.executor.podTemplateFile=/path/to/executor-pod-template.yaml + + + + spark.kubernetes.executor.podTemplateContainerName + (none) + + Specify the container name to be used as a basis for the executor in the given pod template. + For example spark.kubernetes.executor.podTemplateContainerName=spark-executor From 6f35ede31cc72a81e3852b1ac7454589d1897bfc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 7 Jan 2019 17:54:05 -0800 Subject: [PATCH 0251/1072] [SPARK-26554][BUILD][FOLLOWUP] Use GitHub instead of GitBox to check HEADER ## What changes were proposed in this pull request? This PR uses GitHub repository instead of GitBox because GitHub repo returns HTTP header status correctly. ## How was this patch tested? Manual. ``` $ ./do-release-docker.sh -d /tmp/test -n Branch [branch-2.4]: Current branch version is 2.4.1-SNAPSHOT. Release [2.4.1]: RC # [1]: This is a dry run. Please confirm the ref that will be built for testing. Ref [v2.4.1-rc1]: ``` Closes #23482 from dongjoon-hyun/SPARK-26554-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/release-util.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index 9a340528b506..5486c18e95bc 100755 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -21,6 +21,7 @@ DRY_RUN=${DRY_RUN:-0} GPG="gpg --no-tty --batch" ASF_REPO="https://gitbox.apache.org/repos/asf/spark.git" ASF_REPO_WEBUI="https://gitbox.apache.org/repos/asf?p=spark.git" +ASF_GITHUB_REPO="https://github.com/apache/spark" function error { echo "$*" @@ -73,9 +74,7 @@ function fcreate_secure { } function check_for_tag { - # Check HTML body messages instead of header status codes. Apache GitBox returns - # a header with `200 OK` status code for both existing and non-existing tag URLs - ! curl -s --fail "$ASF_REPO_WEBUI;a=commit;h=$1" | grep '404 Not Found' > /dev/null + curl -s --head --fail "$ASF_GITHUB_REPO/releases/tag/$1" > /dev/null } function get_release_info { From 29a7d2da44585d91a9e94bf88dc7b1f42a0e5674 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 7 Jan 2019 18:59:43 -0800 Subject: [PATCH 0252/1072] [SPARK-24196][SQL] Implement Spark's own GetSchemasOperation ## What changes were proposed in this pull request? This PR fix SQL Client tools can't show DBs by implementing Spark's own `GetSchemasOperation`. ## How was this patch tested? unit tests and manual tests ![image](https://user-images.githubusercontent.com/5399861/47782885-3dd5d400-dd3c-11e8-8586-59a8c15c7020.png) ![image](https://user-images.githubusercontent.com/5399861/47782899-4928ff80-dd3c-11e8-9d2d-ba9580ba4301.png) Closes #22903 from wangyum/SPARK-24196. Authored-by: Yuming Wang Signed-off-by: gatorsmile --- .../cli/operation/GetSchemasOperation.java | 2 +- .../SparkGetSchemasOperation.scala | 66 +++++++++++ .../server/SparkSQLOperationManager.scala | 17 ++- .../HiveThriftServer2Suites.scala | 16 +++ .../SparkMetadataOperationSuite.scala | 103 ++++++++++++++++++ 5 files changed, 201 insertions(+), 3 deletions(-) create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java index d6f6280f1c39..3516bc2ba242 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -41,7 +41,7 @@ public class GetSchemasOperation extends MetadataOperation { .addStringColumn("TABLE_SCHEM", "Schema name.") .addStringColumn("TABLE_CATALOG", "Catalog name."); - private RowSet rowSet; + protected RowSet rowSet; protected GetSchemasOperation(HiveSession parentSession, String catalogName, String schemaName) { 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 new file mode 100644 index 000000000000..d585049c28e3 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.GetSchemasOperation +import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.sql.SQLContext + +/** + * Spark's own GetSchemasOperation + * + * @param sqlContext SQLContext to use + * @param parentSession a HiveSession from SessionManager + * @param catalogName catalog name. null if not applicable. + * @param schemaName database name, null or a concrete database name + */ +private[hive] class SparkGetSchemasOperation( + sqlContext: SQLContext, + parentSession: HiveSession, + catalogName: String, + schemaName: String) + extends GetSchemasOperation(parentSession, catalogName, schemaName) { + + override def runInternal(): Unit = { + setState(OperationState.RUNNING) + // Always use the latest class loader provided by executionHive's state. + val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + Thread.currentThread().setContextClassLoader(executionHiveClassLoader) + + if (isAuthV2Enabled) { + val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName" + authorizeMetaGets(HiveOperationType.GET_TABLES, null, cmdStr) + } + + try { + val schemaPattern = convertSchemaPattern(schemaName) + sqlContext.sessionState.catalog.listDatabases(schemaPattern).foreach { dbName => + rowSet.addRow(Array[AnyRef](dbName, DEFAULT_HIVE_CATALOG)) + } + setState(OperationState.FINISHED) + } catch { + case e: HiveSQLException => + setState(OperationState.ERROR) + throw e + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index bf7c01f60fb5..85b6c7134755 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -21,13 +21,13 @@ import java.util.{Map => JMap} import java.util.concurrent.ConcurrentHashMap import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} +import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, GetSchemasOperation, Operation, OperationManager} 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.hive.HiveUtils -import org.apache.spark.sql.hive.thriftserver.{ReflectionUtils, SparkExecuteStatementOperation} +import org.apache.spark.sql.hive.thriftserver.{ReflectionUtils, SparkExecuteStatementOperation, SparkGetSchemasOperation} import org.apache.spark.sql.internal.SQLConf /** @@ -63,6 +63,19 @@ private[thriftserver] class SparkSQLOperationManager() operation } + override def newGetSchemasOperation( + parentSession: HiveSession, + catalogName: String, + schemaName: String): GetSchemasOperation = synchronized { + val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) + require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + " initialized or had already closed.") + val operation = new SparkGetSchemasOperation(sqlContext, parentSession, catalogName, schemaName) + handleToOperation.put(operation.getHandle, operation) + logDebug(s"Created GetSchemasOperation with session=$parentSession.") + operation + } + def setConfMap(conf: SQLConf, confMap: java.util.Map[String, String]): Unit = { val iterator = confMap.entrySet().iterator() while (iterator.hasNext) { 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 70eb28cdd0c6..f9509aed4aaa 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 @@ -818,6 +818,22 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { } } + def withDatabase(dbNames: String*)(fs: (Statement => Unit)*) { + val user = System.getProperty("user.name") + val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } + val statements = connections.map(_.createStatement()) + + try { + statements.zip(fs).foreach { case (s, f) => f(s) } + } finally { + dbNames.foreach { name => + statements(0).execute(s"DROP DATABASE IF EXISTS $name") + } + statements.foreach(_.close()) + connections.foreach(_.close()) + } + } + def withJdbcStatement(tableNames: String*)(f: Statement => Unit) { withMultipleConnectionJdbcStatement(tableNames: _*)(f) } 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 new file mode 100644 index 000000000000..9a997ae01df9 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.util.Properties + +import org.apache.hive.jdbc.{HiveConnection, HiveQueryResultSet, Utils => JdbcUtils} +import org.apache.hive.service.auth.PlainSaslHelper +import org.apache.hive.service.cli.thrift._ +import org.apache.thrift.protocol.TBinaryProtocol +import org.apache.thrift.transport.TSocket + +class SparkMetadataOperationSuite extends HiveThriftJdbcTest { + + override def mode: ServerMode.Value = ServerMode.binary + + test("Spark's own GetSchemasOperation(SparkGetSchemasOperation)") { + def testGetSchemasOperation( + catalog: String, + schemaPattern: String)(f: HiveQueryResultSet => Unit): Unit = { + val rawTransport = new TSocket("localhost", serverPort) + val connection = new HiveConnection(s"jdbc:hive2://localhost:$serverPort", new Properties) + val user = System.getProperty("user.name") + val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + val client = new TCLIService.Client(new TBinaryProtocol(transport)) + transport.open() + var rs: HiveQueryResultSet = null + try { + val openResp = client.OpenSession(new TOpenSessionReq) + val sessHandle = openResp.getSessionHandle + val schemaReq = new TGetSchemasReq(sessHandle) + + if (catalog != null) { + schemaReq.setCatalogName(catalog) + } + + if (schemaPattern == null) { + schemaReq.setSchemaName("%") + } else { + schemaReq.setSchemaName(schemaPattern) + } + + val schemaResp = client.GetSchemas(schemaReq) + JdbcUtils.verifySuccess(schemaResp.getStatus) + + rs = new HiveQueryResultSet.Builder(connection) + .setClient(client) + .setSessionHandle(sessHandle) + .setStmtHandle(schemaResp.getOperationHandle) + .build() + f(rs) + } finally { + rs.close() + connection.close() + transport.close() + rawTransport.close() + } + } + + def checkResult(dbNames: Seq[String], rs: HiveQueryResultSet): Unit = { + if (dbNames.nonEmpty) { + for (i <- dbNames.indices) { + assert(rs.next()) + assert(rs.getString("TABLE_SCHEM") === dbNames(i)) + } + } else { + assert(!rs.next()) + } + } + + withDatabase("db1", "db2") { statement => + Seq("CREATE DATABASE db1", "CREATE DATABASE db2").foreach(statement.execute) + + testGetSchemasOperation(null, "%") { rs => + checkResult(Seq("db1", "db2"), rs) + } + testGetSchemasOperation(null, "db1") { rs => + checkResult(Seq("db1"), rs) + } + testGetSchemasOperation(null, "db_not_exist") { rs => + checkResult(Seq.empty, rs) + } + testGetSchemasOperation(null, "db*") { rs => + checkResult(Seq("db1", "db2"), rs) + } + } + } +} From 72a572ffd6e156243b13f9243ed296f6d77b4241 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 8 Jan 2019 22:44:33 +0800 Subject: [PATCH 0253/1072] [SPARK-26323][SQL] Scala UDF should still check input types even if some inputs are of type Any ## What changes were proposed in this pull request? For Scala UDF, when checking input nullability, we will skip inputs with type `Any`, and only check the inputs that provide nullability info. We should do the same for checking input types. ## How was this patch tested? new tests Closes #23275 from cloud-fan/udf. Authored-by: Wenchen Fan Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/analysis/TypeCoercion.scala | 13 +- .../sql/catalyst/expressions/ScalaUDF.scala | 4 +- .../spark/sql/types/AbstractDataType.scala | 2 +- .../apache/spark/sql/UDFRegistration.scala | 216 ++++++++---------- .../sql/expressions/UserDefinedFunction.scala | 57 ++--- .../org/apache/spark/sql/functions.scala | 52 ++--- .../scala/org/apache/spark/sql/UDFSuite.scala | 15 ++ 7 files changed, 175 insertions(+), 184 deletions(-) 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 b19aa50ba215..13cc9b9c125e 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 @@ -882,7 +882,18 @@ object TypeCoercion { case udf: ScalaUDF if udf.inputTypes.nonEmpty => val children = udf.children.zip(udf.inputTypes).map { case (in, expected) => - implicitCast(in, udfInputToCastType(in.dataType, expected)).getOrElse(in) + // Currently Scala UDF will only expect `AnyDataType` at top level, so this trick works. + // In the future we should create types like `AbstractArrayType`, so that Scala UDF can + // accept inputs of array type of arbitrary element type. + if (expected == AnyDataType) { + in + } else { + implicitCast( + in, + udfInputToCastType(in.dataType, expected.asInstanceOf[DataType]) + ).getOrElse(in) + } + } udf.withNewChildren(children) } 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 a23aaa3a0b3e..fae1119c394b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.{AbstractDataType, DataType} /** * User-defined function. @@ -48,7 +48,7 @@ case class ScalaUDF( dataType: DataType, children: Seq[Expression], inputsNullSafe: Seq[Boolean], - inputTypes: Seq[DataType] = Nil, + inputTypes: Seq[AbstractDataType] = Nil, udfName: Option[String] = None, nullable: Boolean = true, udfDeterministic: Boolean = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index 5367ce2af8e9..d2ef08873187 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -96,7 +96,7 @@ private[sql] object TypeCollection { /** * An `AbstractDataType` that matches any concrete data types. */ -protected[sql] object AnyDataType extends AbstractDataType { +protected[sql] object AnyDataType extends AbstractDataType with Serializable { // Note that since AnyDataType matches any concrete types, defaultConcreteType should never // be invoked. 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 5a3f556c9c07..fe5d1afd8478 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 @@ -123,17 +123,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends |def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] | val inputSchemas: Seq[Option[ScalaReflection.Schema]] = $inputSchemas + | val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + | val finalUdf = if (nullable) udf else udf.asNonNullable() | def builder(e: Seq[Expression]) = if (e.length == $x) { - | ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - | if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - | Some(name), nullable, udfDeterministic = true) + | finalUdf.createScalaUDF(e) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $x; Found: " + e.length) | } | functionRegistry.createOrReplaceTempFunction(name, builder) - | val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - | if (nullable) udf else udf.asNonNullable() + | finalUdf |}""".stripMargin) } @@ -170,17 +169,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -191,17 +189,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -212,17 +209,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -233,17 +229,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -254,17 +249,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -275,17 +269,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -296,17 +289,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -317,17 +309,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -338,17 +329,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -359,17 +349,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -380,17 +369,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -401,17 +389,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -422,17 +409,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -443,17 +429,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -464,17 +449,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -485,17 +469,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -506,17 +489,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -527,17 +509,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -548,17 +529,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -569,17 +549,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -590,17 +569,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -611,17 +589,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Try(ScalaReflection.schemaFor[A21]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -632,17 +609,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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 inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Try(ScalaReflection.schemaFor[A21]).toOption :: Try(ScalaReflection.schemaFor[A22]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } ////////////////////////////////////////////////////////////////////////////////////////////// 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 901472d8e036..1b2d6c7ffb52 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 @@ -20,8 +20,8 @@ package org.apache.spark.sql.expressions import org.apache.spark.annotation.Stable import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.expressions.ScalaUDF -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUDF} +import org.apache.spark.sql.types.{AnyDataType, DataType} /** * A user-defined function. To create one, use the `udf` functions in `functions`. @@ -88,40 +88,47 @@ sealed abstract class UserDefinedFunction { private[sql] case class SparkUserDefinedFunction( f: AnyRef, dataType: DataType, - inputTypes: Option[Seq[DataType]], - nullableTypes: Option[Seq[Boolean]], + inputSchemas: Seq[Option[ScalaReflection.Schema]], name: Option[String] = None, nullable: Boolean = true, deterministic: Boolean = true) extends UserDefinedFunction { @scala.annotation.varargs override def apply(exprs: Column*): Column = { - // TODO: make sure this class is only instantiated through `SparkUserDefinedFunction.create()` - // and `nullableTypes` is always set. - if (inputTypes.isDefined) { - assert(inputTypes.get.length == nullableTypes.get.length) - } + Column(createScalaUDF(exprs.map(_.expr))) + } + + private[sql] def createScalaUDF(exprs: Seq[Expression]): ScalaUDF = { + // It's possible that some of the inputs don't have a specific type(e.g. `Any`), skip type + // check and null check for them. + val inputTypes = inputSchemas.map(_.map(_.dataType).getOrElse(AnyDataType)) - val inputsNullSafe = nullableTypes.getOrElse { + val inputsNullSafe = if (inputSchemas.isEmpty) { + // This is for backward compatibility of `functions.udf(AnyRef, DataType)`. We need to + // do reflection of the lambda function object and see if its arguments are nullable or not. + // This doesn't work for Scala 2.12 and we should consider removing this workaround, as Spark + // uses Scala 2.12 by default since 3.0. ScalaReflection.getParameterTypeNullability(f) + } else { + inputSchemas.map(_.map(_.nullable).getOrElse(true)) } - Column(ScalaUDF( + ScalaUDF( f, dataType, - exprs.map(_.expr), + exprs, inputsNullSafe, - inputTypes.getOrElse(Nil), + inputTypes, udfName = name, nullable = nullable, - udfDeterministic = deterministic)) + udfDeterministic = deterministic) } - override def withName(name: String): UserDefinedFunction = { + override def withName(name: String): SparkUserDefinedFunction = { copy(name = Option(name)) } - override def asNonNullable(): UserDefinedFunction = { + override def asNonNullable(): SparkUserDefinedFunction = { if (!nullable) { this } else { @@ -129,7 +136,7 @@ private[sql] case class SparkUserDefinedFunction( } } - override def asNondeterministic(): UserDefinedFunction = { + override def asNondeterministic(): SparkUserDefinedFunction = { if (!deterministic) { this } else { @@ -137,19 +144,3 @@ private[sql] case class SparkUserDefinedFunction( } } } - -private[sql] object SparkUserDefinedFunction { - - def create( - f: AnyRef, - dataType: DataType, - inputSchemas: Seq[Option[ScalaReflection.Schema]]): UserDefinedFunction = { - val inputTypes = if (inputSchemas.contains(None)) { - None - } else { - Some(inputSchemas.map(_.get.dataType)) - } - val nullableTypes = Some(inputSchemas.map(_.map(_.nullable).getOrElse(true))) - SparkUserDefinedFunction(f, dataType, inputTypes, nullableTypes) - } -} 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 645452553e6a..7572cf23cde8 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 @@ -3874,7 +3874,7 @@ object functions { |def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] | val inputSchemas = $inputSchemas - | val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + | val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) | if (nullable) udf else udf.asNonNullable() |}""".stripMargin) } @@ -3897,7 +3897,7 @@ object functions { | */ |def udf(f: UDF$i[$extTypeArgs], returnType: DataType): UserDefinedFunction = { | val func = f$anyCast.call($anyParams) - | SparkUserDefinedFunction.create($funcCall, returnType, inputSchemas = Seq.fill($i)(None)) + | SparkUserDefinedFunction($funcCall, returnType, inputSchemas = Seq.fill($i)(None)) |}""".stripMargin) } @@ -3919,7 +3919,7 @@ object functions { def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputSchemas = Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3935,7 +3935,7 @@ object functions { def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3951,7 +3951,7 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3967,7 +3967,7 @@ object functions { 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 inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3983,7 +3983,7 @@ object functions { 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 inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3999,7 +3999,7 @@ object functions { 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 inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4015,7 +4015,7 @@ object functions { 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 inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4031,7 +4031,7 @@ object functions { 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 inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4047,7 +4047,7 @@ object functions { 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 inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4063,7 +4063,7 @@ object functions { 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 inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A9])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4079,7 +4079,7 @@ object functions { 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 inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A9])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A10])).toOption :: Nil - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4098,7 +4098,7 @@ object functions { */ def udf(f: UDF0[_], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF0[Any]].call() - SparkUserDefinedFunction.create(() => func, returnType, inputSchemas = Seq.fill(0)(None)) + SparkUserDefinedFunction(() => func, returnType, inputSchemas = Seq.fill(0)(None)) } /** @@ -4112,7 +4112,7 @@ object functions { */ def udf(f: UDF1[_, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(1)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(1)(None)) } /** @@ -4126,7 +4126,7 @@ object functions { */ def udf(f: UDF2[_, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(2)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(2)(None)) } /** @@ -4140,7 +4140,7 @@ object functions { */ def udf(f: UDF3[_, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(3)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(3)(None)) } /** @@ -4154,7 +4154,7 @@ object functions { */ def udf(f: UDF4[_, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(4)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(4)(None)) } /** @@ -4168,7 +4168,7 @@ object functions { */ def udf(f: UDF5[_, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(5)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(5)(None)) } /** @@ -4182,7 +4182,7 @@ object functions { */ def udf(f: UDF6[_, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(6)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(6)(None)) } /** @@ -4196,7 +4196,7 @@ object functions { */ def udf(f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(7)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(7)(None)) } /** @@ -4210,7 +4210,7 @@ object functions { */ def udf(f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(8)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(8)(None)) } /** @@ -4224,7 +4224,7 @@ object functions { */ def udf(f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(9)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(9)(None)) } /** @@ -4238,7 +4238,7 @@ object functions { */ def udf(f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(10)(None)) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(10)(None)) } // scalastyle:on parameter.number @@ -4257,9 +4257,7 @@ object functions { * @since 2.0.0 */ def udf(f: AnyRef, dataType: DataType): UserDefinedFunction = { - // TODO: should call SparkUserDefinedFunction.create() instead but inputSchemas is currently - // unavailable. We may need to create type-safe overloaded versions of udf() methods. - SparkUserDefinedFunction(f, dataType, inputTypes = None, nullableTypes = None) + SparkUserDefinedFunction(f, dataType, inputSchemas = Nil) } /** 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 a26d306cff6b..06b9343c3758 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 @@ -450,4 +450,19 @@ class UDFSuite extends QueryTest with SharedSQLContext { }) checkAnswer(df2.select(udf2($"col1")), Seq(Row(Map("a" -> "2011000000000002456556")))) } + + test("SPARK-26323 Verify input type check - with udf()") { + val f = udf((x: Long, y: Any) => x) + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j").select(f($"i", $"j")) + checkAnswer(df, Seq(Row(1L), Row(2L))) + } + + test("SPARK-26323 Verify input type check - with udf.register") { + withTable("t") { + Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.format("json").saveAsTable("t") + spark.udf.register("f", (x: Long, y: Any) => x) + val df = spark.sql("SELECT f(i, j) FROM t") + checkAnswer(df, Seq(Row(1L), Row(2L))) + } + } } From b7113822d5f9a984d30cc7fb3b8920fcf630a96a Mon Sep 17 00:00:00 2001 From: liuxian Date: Tue, 8 Jan 2019 10:45:23 -0600 Subject: [PATCH 0254/1072] [MINOR][WEBUI] Modify the name of the column named "shuffle spill" in the StagePage ## What changes were proposed in this pull request? ![default](https://user-images.githubusercontent.com/24688163/50752687-16463f00-128a-11e9-8ee3-4d156f7631f6.png) For this DAG, it has no shuffle operation, only sorting, and sorting leads to spill. ![default](https://user-images.githubusercontent.com/24688163/50752974-0f6bfc00-128b-11e9-9362-a0f440e02359.png) So I think the name of the column named "shuffle spill" is not all right in the StagePage ## How was this patch tested? Manual testing Closes #23483 from 10110346/shufflespillwebui. Authored-by: liuxian Signed-off-by: Sean Owen --- .../resources/org/apache/spark/ui/static/stagepage.js | 8 ++++---- .../org/apache/spark/ui/static/stagespage-template.html | 8 ++++---- .../main/scala/org/apache/spark/ui/jobs/StagePage.scala | 8 ++++---- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 08de2b0fee03..5b792ffc584d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -100,10 +100,10 @@ function getColumnNameForTaskMetricSummary(columnKey) { return "Scheduler Delay"; case "diskBytesSpilled": - return "Shuffle spill (disk)"; + return "Spill (disk)"; case "memoryBytesSpilled": - return "Shuffle spill (memory)"; + return "Spill (memory)"; case "shuffleReadMetrics": return "Shuffle Read Size / Records"; @@ -842,7 +842,7 @@ $(document).ready(function () { return ""; } }, - name: "Shuffle Spill (Memory)" + name: "Spill (Memory)" }, { data : function (row, type) { @@ -852,7 +852,7 @@ $(document).ready(function () { return ""; } }, - name: "Shuffle Spill (Disk)" + name: "Spill (Disk)" }, { data : function (row, type) { diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html index 6f950c61b2d6..6b0435bb2028 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html @@ -59,8 +59,8 @@

    Aggregated Metrics by Executor

    Output Size / Records Shuffle Read Size / Records Shuffle Write Size / Records - Shuffle Spill (Memory) - Shuffle Spill (Disk) + Spill (Memory) + Spill (Disk) @@ -111,8 +111,8 @@

    Write Time Shuffle Write Size / Records Shuffle Read Size / Records - Shuffle Spill (Memory) - Shuffle Spill (Disk) + Spill (Memory) + Spill (Disk) Errors diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index a213b764abea..3bca1d574301 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -188,11 +188,11 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We }} {if (hasBytesSpilled(stageData)) {
  • - Shuffle Spill (Memory): + Spill (Memory): {Utils.bytesToString(stageData.memoryBytesSpilled)}
  • - Shuffle Spill (Disk): + Spill (Disk): {Utils.bytesToString(stageData.diskBytesSpilled)}
  • }} @@ -797,8 +797,8 @@ private[spark] object ApiHelper { val HEADER_SHUFFLE_REMOTE_READS = "Shuffle Remote Reads" val HEADER_SHUFFLE_WRITE_TIME = "Write Time" val HEADER_SHUFFLE_WRITE_SIZE = "Shuffle Write Size / Records" - val HEADER_MEM_SPILL = "Shuffle Spill (Memory)" - val HEADER_DISK_SPILL = "Shuffle Spill (Disk)" + val HEADER_MEM_SPILL = "Spill (Memory)" + val HEADER_DISK_SPILL = "Spill (Disk)" val HEADER_ERROR = "Errors" private[ui] val COLUMN_TO_INDEX = Map( From c101182b10cffd9314c44eefe4db53ba3d6553b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Wed, 9 Jan 2019 01:24:47 +0800 Subject: [PATCH 0255/1072] [SPARK-26002][SQL] Fix day of year calculation for Julian calendar days MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Fixing leap year calculations for date operators (year/month/dayOfYear) where the Julian calendars are used (before 1582-10-04). In a Julian calendar every years which are multiples of 4 are leap years (there is no extra exception for years multiples of 100). ## How was this patch tested? With a unit test ("SPARK-26002: correct day of year calculations for Julian calendar years") which focuses to these corner cases. Manually: ``` scala> sql("select year('1500-01-01')").show() +------------------------------+ |year(CAST(1500-01-01 AS DATE))| +------------------------------+ | 1500| +------------------------------+ scala> sql("select dayOfYear('1100-01-01')").show() +-----------------------------------+ |dayofyear(CAST(1100-01-01 AS DATE))| +-----------------------------------+ | 1| +-----------------------------------+ ``` Closes #23000 from attilapiros/julianOffByDays. Authored-by: “attilapiros” Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/DateTimeUtils.scala | 56 +++++++++++++++---- .../catalyst/util/DateTimeUtilsSuite.scala | 30 ++++++++++ .../resources/sql-tests/inputs/datetime.sql | 2 + .../sql-tests/results/datetime.sql.out | 10 +++- 4 files changed, 86 insertions(+), 12 deletions(-) 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 3e5e1fbc2b36..e95117f95cdb 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 @@ -53,14 +53,30 @@ object DateTimeUtils { final val NANOS_PER_MICROS = 1000L final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L - // number of days in 400 years + // number of days in 400 years by Gregorian calendar final val daysIn400Years: Int = 146097 + + // In the Julian calendar every year that is exactly divisible by 4 is a leap year without any + // exception. But in the Gregorian calendar every year that is exactly divisible by four + // is a leap year, except for years that are exactly divisible by 100, but these centurial years + // are leap years if they are exactly divisible by 400. + // So there are 3 extra days in the Julian calendar within a 400 years cycle compared to the + // Gregorian calendar. + final val extraLeapDaysIn400YearsJulian = 3 + + // number of days in 400 years by Julian calendar + final val daysIn400YearsInJulian: Int = daysIn400Years + extraLeapDaysIn400YearsJulian + // number of days between 1.1.1970 and 1.1.2001 final val to2001 = -11323 // this is year -17999, calculation: 50 * daysIn400Year final val YearZero = -17999 final val toYearZero = to2001 + 7304850 + + // days to year -17999 in Julian calendar + final val toYearZeroInJulian = toYearZero + 49 * extraLeapDaysIn400YearsJulian + final val TimeZoneGMT = TimeZone.getTimeZone("GMT") final val TimeZoneUTC = TimeZone.getTimeZone("UTC") final val MonthOf31Days = Set(1, 3, 5, 7, 8, 10, 12) @@ -575,20 +591,30 @@ object DateTimeUtils { * Return the number of days since the start of 400 year period. * The second year of a 400 year period (year 1) starts on day 365. */ - private[this] def yearBoundary(year: Int): Int = { - year * 365 + ((year / 4 ) - (year / 100) + (year / 400)) + private[this] def yearBoundary(year: Int, isGregorian: Boolean): Int = { + if (isGregorian) { + year * 365 + ((year / 4) - (year / 100) + (year / 400)) + } else { + year * 365 + (year / 4) + } } /** * Calculates the number of years for the given number of days. This depends * on a 400 year period. * @param days days since the beginning of the 400 year period + * @param isGregorian indicates whether leap years should be calculated according to Gregorian + * (or Julian) calendar * @return (number of year, days in year) */ - private[this] def numYears(days: Int): (Int, Int) = { + private[this] def numYears(days: Int, isGregorian: Boolean): (Int, Int) = { val year = days / 365 - val boundary = yearBoundary(year) - if (days > boundary) (year, days - boundary) else (year - 1, days - yearBoundary(year - 1)) + val boundary = yearBoundary(year, isGregorian) + if (days > boundary) { + (year, days - boundary) + } else { + (year - 1, days - yearBoundary(year - 1, isGregorian)) + } } /** @@ -599,18 +625,26 @@ object DateTimeUtils { * equals to the period 1.1.1601 until 31.12.2000. */ private[this] def getYearAndDayInYear(daysSince1970: SQLDate): (Int, Int) = { - // add the difference (in days) between 1.1.1970 and the artificial year 0 (-17999) - var daysSince1970Tmp = daysSince1970 // Since Julian calendar was replaced with the Gregorian calendar, // the 10 days after Oct. 4 were skipped. // (1582-10-04) -141428 days since 1970-01-01 if (daysSince1970 <= -141428) { - daysSince1970Tmp -= 10 + getYearAndDayInYear(daysSince1970 - 10, toYearZeroInJulian, daysIn400YearsInJulian, false) + } else { + getYearAndDayInYear(daysSince1970, toYearZero, daysIn400Years, true) } - val daysNormalized = daysSince1970Tmp + toYearZero + } + + private def getYearAndDayInYear( + daysSince1970: SQLDate, + toYearZero: SQLDate, + daysIn400Years: SQLDate, + isGregorian: Boolean): (Int, Int) = { + // add the difference (in days) between 1.1.1970 and the artificial year 0 (-17999) + val daysNormalized = daysSince1970 + toYearZero val numOfQuarterCenturies = daysNormalized / daysIn400Years val daysInThis400 = daysNormalized % daysIn400Years + 1 - val (years, dayInYear) = numYears(daysInThis400) + val (years, dayInYear) = numYears(daysInThis400, isGregorian) val year: Int = (2001 - 20000) + 400 * numOfQuarterCenturies + years (year, dayInYear) } 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 0182eeb17121..2cb6110e2c09 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 @@ -410,6 +410,36 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 78) } + test("SPARK-26002: correct day of year calculations for Julian calendar years") { + val c = Calendar.getInstance() + c.set(Calendar.MILLISECOND, 0) + (1000 to 1600 by 100).foreach { year => + // January 1 is the 1st day of year. + c.set(year, 0, 1, 0, 0, 0) + assert(getYear(getInUTCDays(c.getTimeInMillis)) === year) + assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 1) + assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 1) + + // March 1 is the 61st day of the year as they are leap years. It is true for + // even the multiples of 100 as before 1582-10-4 the Julian calendar leap year calculation + // is used in which every multiples of 4 are leap years + c.set(year, 2, 1, 0, 0, 0) + assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 61) + assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 3) + + // testing leap day (February 29) in leap years + c.set(year, 1, 29, 0, 0, 0) + assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 60) + + // For non-leap years: + c.set(year + 1, 2, 1, 0, 0, 0) + assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 60) + } + + c.set(1582, 2, 1, 0, 0, 0) + assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 60) + } + test("get year") { val c = Calendar.getInstance() c.set(2015, 2, 18, 0, 0, 0) 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 547c2bef02b2..8bd8bc2b94b8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -27,3 +27,5 @@ select current_date = current_date(), current_timestamp = current_timestamp(), a select a, b from ttf2 order by a, current_date; select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), weekday('1582-10-15 13:10:15'); + +select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01'); \ No newline at end of file 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 63aa00426ea3..2090633802e2 100644 --- 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: 10 +-- Number of queries: 11 -- !query 0 @@ -89,3 +89,11 @@ select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), week struct -- !query 9 output 5 3 5 NULL 4 + + +-- !query 10 +select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01') +-- !query 10 schema +struct +-- !query 10 output +1500 1 1 From 2783e4c45f55f4fc87748d1c4a454bfdf3024156 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 8 Jan 2019 11:25:33 -0600 Subject: [PATCH 0256/1072] [SPARK-24522][UI] Create filter to apply HTTP security checks consistently. Currently there is code scattered in a bunch of places to do different things related to HTTP security, such as access control, setting security-related headers, and filtering out bad content. This makes it really easy to miss these things when writing new UI code. This change creates a new filter that does all of those things, and makes sure that all servlet handlers that are attached to the UI get the new filter and any user-defined filters consistently. The extent of the actual features should be the same as before. The new filter is added at the end of the filter chain, because authentication is done by custom filters and thus needs to happen first. This means that custom filters see unfiltered HTTP requests - which is actually the current behavior anyway. As a side-effect of some of the code refactoring, handlers added after the initial set also get wrapped with a GzipHandler, which didn't happen before. Tested with added unit tests and in a history server with SPNEGO auth configured. Closes #23302 from vanzin/SPARK-24522. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid --- .../spark/deploy/history/HistoryPage.scala | 5 +- .../spark/deploy/history/HistoryServer.scala | 8 +- .../deploy/master/ui/ApplicationPage.scala | 3 +- .../spark/deploy/master/ui/MasterPage.scala | 6 +- .../spark/deploy/worker/ui/LogPage.scala | 28 ++-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 1 - .../spark/metrics/sink/MetricsServlet.scala | 2 +- .../spark/status/api/v1/SecurityFilter.scala | 36 ---- .../apache/spark/ui/HttpSecurityFilter.scala | 116 +++++++++++++ .../org/apache/spark/ui/JettyUtils.scala | 154 ++++++++--------- .../scala/org/apache/spark/ui/UIUtils.scala | 21 --- .../scala/org/apache/spark/ui/WebUI.scala | 15 +- .../ui/exec/ExecutorThreadDumpPage.scala | 4 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 16 +- .../org/apache/spark/ui/jobs/JobPage.scala | 3 +- .../org/apache/spark/ui/jobs/JobsTab.scala | 4 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 19 +-- .../org/apache/spark/ui/jobs/StageTable.scala | 15 +- .../org/apache/spark/ui/jobs/StagesTab.scala | 4 +- .../org/apache/spark/ui/storage/RDDPage.scala | 11 +- .../spark/ui/HttpSecurityFilterSuite.scala | 157 ++++++++++++++++++ .../scala/org/apache/spark/ui/UISuite.scala | 147 +++++++++++----- .../org/apache/spark/ui/UIUtilsSuite.scala | 39 ----- .../spark/deploy/mesos/ui/DriverPage.scala | 3 +- .../cluster/YarnSchedulerBackend.scala | 35 +++- .../cluster/YarnSchedulerBackendSuite.scala | 59 ++++++- .../sql/execution/ui/AllExecutionsPage.scala | 19 +-- .../sql/execution/ui/ExecutionPage.scala | 3 +- .../ui/ThriftServerSessionPage.scala | 3 +- .../apache/spark/streaming/ui/BatchPage.scala | 8 +- 31 files changed, 609 insertions(+), 338 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala create mode 100644 core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 00ca4efa4d26..7a8ab7fddd79 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -27,9 +27,8 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val requestedIncomplete = - Option(UIUtils.stripXSS(request.getParameter("showIncomplete"))).getOrElse("false").toBoolean + val requestedIncomplete = Option(request.getParameter("showIncomplete")) + .getOrElse("false").toBoolean val displayApplications = parent.getApplicationList() .exists(isApplicationCompleted(_) != requestedIncomplete) 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 b9303388638f..ff2ea3b843ee 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 @@ -150,17 +150,15 @@ class HistoryServer( ui: SparkUI, completed: Boolean) { assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") - handlers.synchronized { - ui.getHandlers.foreach(attachHandler) + ui.getHandlers.foreach { handler => + serverInfo.get.addHandler(handler, ui.securityManager) } } /** Detach a reconstructed UI from this server. Only valid after bind(). */ override def detachSparkUI(appId: String, attemptId: Option[String], ui: SparkUI): Unit = { assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") - handlers.synchronized { - ui.getHandlers.foreach(detachHandler) - } + ui.getHandlers.foreach(detachHandler) provider.onUIDetached(appId, attemptId, ui) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index fad4e46dc035..bcd7a7e4ccdb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -33,8 +33,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val appId = UIUtils.stripXSS(request.getParameter("appId")) + val appId = request.getParameter("appId") val state = master.askSync[MasterStateResponse](RequestMasterState) val app = state.activeApps.find(_.id == appId) .getOrElse(state.completedApps.find(_.id == appId).orNull) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index b8afe203fbfa..6701465c023c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -57,10 +57,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private def handleKillRequest(request: HttpServletRequest, action: String => Unit): Unit = { if (parent.killEnabled && parent.master.securityMgr.checkModifyPermissions(request.getRemoteUser)) { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val killFlag = - Option(UIUtils.stripXSS(request.getParameter("terminate"))).getOrElse("false").toBoolean - val id = Option(UIUtils.stripXSS(request.getParameter("id"))) + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val id = Option(request.getParameter("id")) if (id.isDefined && killFlag) { action(id.get) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 4fca9342c037..4e720a759a1b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -33,15 +33,13 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with private val supportedLogTypes = Set("stderr", "stdout") private val defaultBytes = 100 * 1024 - // stripXSS is called first to remove suspicious characters used in XSS attacks def renderLog(request: HttpServletRequest): String = { - val appId = Option(UIUtils.stripXSS(request.getParameter("appId"))) - val executorId = Option(UIUtils.stripXSS(request.getParameter("executorId"))) - val driverId = Option(UIUtils.stripXSS(request.getParameter("driverId"))) - val logType = UIUtils.stripXSS(request.getParameter("logType")) - val offset = Option(UIUtils.stripXSS(request.getParameter("offset"))).map(_.toLong) - val byteLength = - Option(UIUtils.stripXSS(request.getParameter("byteLength"))).map(_.toInt) + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) .getOrElse(defaultBytes) val logDir = (appId, executorId, driverId) match { @@ -58,15 +56,13 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with pre + logText } - // stripXSS is called first to remove suspicious characters used in XSS attacks def render(request: HttpServletRequest): Seq[Node] = { - val appId = Option(UIUtils.stripXSS(request.getParameter("appId"))) - val executorId = Option(UIUtils.stripXSS(request.getParameter("executorId"))) - val driverId = Option(UIUtils.stripXSS(request.getParameter("driverId"))) - val logType = UIUtils.stripXSS(request.getParameter("logType")) - val offset = Option(UIUtils.stripXSS(request.getParameter("offset"))).map(_.toLong) - val byteLength = - Option(UIUtils.stripXSS(request.getParameter("byteLength"))).map(_.toInt) + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) .getOrElse(defaultBytes) val (logDir, params, pageName) = (appId, executorId, driverId) match { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index ea67b7434a76..54886955b98f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -50,7 +50,6 @@ class WorkerWebUI( addStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE) attachHandler(createServletHandler("/log", (request: HttpServletRequest) => logPage.renderLog(request), - worker.securityMgr, worker.conf)) } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 68b58b849064..bea24ca7807e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -51,7 +51,7 @@ private[spark] class MetricsServlet( def getHandlers(conf: SparkConf): Array[ServletContextHandler] = { Array[ServletContextHandler]( createServletHandler(servletPath, - new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr, conf) + new ServletParams(request => getMetricsSnapshot(request), "text/json"), conf) ) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala deleted file mode 100644 index 1cd37185d660..000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala +++ /dev/null @@ -1,36 +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.status.api.v1 - -import javax.ws.rs.container.{ContainerRequestContext, ContainerRequestFilter} -import javax.ws.rs.core.Response -import javax.ws.rs.ext.Provider - -@Provider -private[v1] class SecurityFilter extends ContainerRequestFilter with ApiRequestContext { - override def filter(req: ContainerRequestContext): Unit = { - val user = httpRequest.getRemoteUser() - if (!uiRoot.securityManager.checkUIViewPermissions(user)) { - req.abortWith( - Response - .status(Response.Status.FORBIDDEN) - .entity(raw"""user "$user" is not authorized""") - .build() - ) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala new file mode 100644 index 000000000000..da84fdf8fe14 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui + +import java.util.{Enumeration, Map => JMap} +import javax.servlet._ +import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpServletResponse} + +import scala.collection.JavaConverters._ + +import org.apache.commons.lang3.StringEscapeUtils + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.config._ + +/** + * A servlet filter that implements HTTP security features. The following actions are taken + * for every request: + * + * - perform access control of authenticated requests. + * - check request data for disallowed content (e.g. things that could be used to create XSS + * attacks). + * - set response headers to prevent certain kinds of attacks. + * + * Request parameters are sanitized so that HTML content is escaped, and disallowed content is + * removed. + */ +private class HttpSecurityFilter( + conf: SparkConf, + securityMgr: SecurityManager) extends Filter { + + override def destroy(): Unit = { } + + override def init(config: FilterConfig): Unit = { } + + override def doFilter(req: ServletRequest, res: ServletResponse, chain: FilterChain): Unit = { + val hreq = req.asInstanceOf[HttpServletRequest] + val hres = res.asInstanceOf[HttpServletResponse] + hres.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") + + if (!securityMgr.checkUIViewPermissions(hreq.getRemoteUser())) { + hres.sendError(HttpServletResponse.SC_FORBIDDEN, + "User is not authorized to access this page.") + return + } + + // SPARK-10589 avoid frame-related click-jacking vulnerability, using X-Frame-Options + // (see http://tools.ietf.org/html/rfc7034). By default allow framing only from the + // same origin, but allow framing for a specific named URI. + // Example: spark.ui.allowFramingFrom = https://example.com/ + val xFrameOptionsValue = conf.getOption("spark.ui.allowFramingFrom") + .map { uri => s"ALLOW-FROM $uri" } + .getOrElse("SAMEORIGIN") + + hres.setHeader("X-Frame-Options", xFrameOptionsValue) + hres.setHeader("X-XSS-Protection", conf.get(UI_X_XSS_PROTECTION)) + if (conf.get(UI_X_CONTENT_TYPE_OPTIONS)) { + hres.setHeader("X-Content-Type-Options", "nosniff") + } + if (hreq.getScheme() == "https") { + conf.get(UI_STRICT_TRANSPORT_SECURITY).foreach( + hres.setHeader("Strict-Transport-Security", _)) + } + + chain.doFilter(new XssSafeRequest(hreq), res) + } + +} + +private class XssSafeRequest(req: HttpServletRequest) extends HttpServletRequestWrapper(req) { + + private val NEWLINE_AND_SINGLE_QUOTE_REGEX = raw"(?i)(\r\n|\n|\r|%0D%0A|%0A|%0D|'|%27)".r + + private val parameterMap: Map[String, Array[String]] = { + super.getParameterMap().asScala.map { case (name, values) => + stripXSS(name) -> values.map(stripXSS) + }.toMap + } + + override def getParameterMap(): JMap[String, Array[String]] = parameterMap.asJava + + override def getParameterNames(): Enumeration[String] = { + parameterMap.keys.iterator.asJavaEnumeration + } + + override def getParameterValues(name: String): Array[String] = parameterMap.get(name).orNull + + override def getParameter(name: String): String = { + parameterMap.get(name).flatMap(_.headOption).orNull + } + + private def stripXSS(str: String): String = { + if (str != null) { + // Remove new lines and single quotes, followed by escaping HTML version 4.0 + StringEscapeUtils.escapeHtml4(NEWLINE_AND_SINGLE_QUOTE_REGEX.replaceAllIn(str, "")) + } else { + null + } + } + +} 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 316af9b79d28..08f5fb937da7 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.ui import java.net.{URI, URL} +import java.util.EnumSet import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} @@ -68,43 +69,16 @@ private[spark] object JettyUtils extends Logging { implicit def textResponderToServlet(responder: Responder[String]): ServletParams[String] = new ServletParams(responder, "text/plain") - def createServlet[T <: AnyRef]( + private def createServlet[T <: AnyRef]( servletParams: ServletParams[T], - securityMgr: SecurityManager, conf: SparkConf): HttpServlet = { - - // SPARK-10589 avoid frame-related click-jacking vulnerability, using X-Frame-Options - // (see http://tools.ietf.org/html/rfc7034). By default allow framing only from the - // same origin, but allow framing for a specific named URI. - // Example: spark.ui.allowFramingFrom = https://example.com/ - val allowFramingFrom = conf.getOption("spark.ui.allowFramingFrom") - val xFrameOptionsValue = - allowFramingFrom.map(uri => s"ALLOW-FROM $uri").getOrElse("SAMEORIGIN") - new HttpServlet { override def doGet(request: HttpServletRequest, response: HttpServletResponse) { try { - if (securityMgr.checkUIViewPermissions(request.getRemoteUser)) { - response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) - response.setStatus(HttpServletResponse.SC_OK) - val result = servletParams.responder(request) - response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.setHeader("X-Frame-Options", xFrameOptionsValue) - response.setHeader("X-XSS-Protection", conf.get(UI_X_XSS_PROTECTION)) - if (conf.get(UI_X_CONTENT_TYPE_OPTIONS)) { - response.setHeader("X-Content-Type-Options", "nosniff") - } - if (request.getScheme == "https") { - conf.get(UI_STRICT_TRANSPORT_SECURITY).foreach( - response.setHeader("Strict-Transport-Security", _)) - } - response.getWriter.print(servletParams.extractFn(result)) - } else { - response.setStatus(HttpServletResponse.SC_FORBIDDEN) - response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.sendError(HttpServletResponse.SC_FORBIDDEN, - "User is not authorized to access this page.") - } + response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) + response.setStatus(HttpServletResponse.SC_OK) + val result = servletParams.responder(request) + response.getWriter.print(servletParams.extractFn(result)) } catch { case e: IllegalArgumentException => response.sendError(HttpServletResponse.SC_BAD_REQUEST, e.getMessage) @@ -124,10 +98,9 @@ private[spark] object JettyUtils extends Logging { def createServletHandler[T <: AnyRef]( path: String, servletParams: ServletParams[T], - securityMgr: SecurityManager, conf: SparkConf, basePath: String = ""): ServletContextHandler = { - createServletHandler(path, createServlet(servletParams, securityMgr, conf), basePath) + createServletHandler(path, createServlet(servletParams, conf), basePath) } /** Create a context handler that responds to a request with the given path prefix */ @@ -257,36 +230,6 @@ private[spark] object JettyUtils extends Logging { contextHandler } - /** Add filters, if any, to the given list of ServletContextHandlers */ - def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { - val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) - filters.foreach { - case filter : String => - if (!filter.isEmpty) { - logInfo(s"Adding filter $filter to ${handlers.map(_.getContextPath).mkString(", ")}.") - val holder : FilterHolder = new FilterHolder() - holder.setClassName(filter) - // Get any parameters for each filter - conf.get("spark." + filter + ".params", "").split(',').map(_.trim()).toSet.foreach { - param: String => - if (!param.isEmpty) { - val parts = param.split("=") - if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) - } - } - - val prefix = s"spark.$filter.param." - conf.getAll - .filter { case (k, v) => k.length() > prefix.length() && k.startsWith(prefix) } - .foreach { case (k, v) => holder.setInitParameter(k.substring(prefix.length()), v) } - - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, - DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) - handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } - } - } - } - /** * Attempt to start a Jetty server bound to the supplied hostName:port using the given * context handlers. @@ -298,12 +241,9 @@ private[spark] object JettyUtils extends Logging { hostName: String, port: Int, sslOptions: SSLOptions, - handlers: Seq[ServletContextHandler], conf: SparkConf, serverName: String = ""): ServerInfo = { - addFilters(handlers, conf) - // Start the server first, with no connectors. val pool = new QueuedThreadPool if (serverName.nonEmpty) { @@ -398,16 +338,6 @@ private[spark] object JettyUtils extends Logging { } server.addConnector(httpConnector) - - // Add all the known handlers now that connectors are configured. - handlers.foreach { h => - h.setVirtualHosts(toVirtualHosts(SPARK_CONNECTOR_NAME)) - val gzipHandler = new GzipHandler() - gzipHandler.setHandler(h) - collection.addHandler(gzipHandler) - gzipHandler.start() - } - pool.setMaxThreads(math.max(pool.getMaxThreads, minThreads)) ServerInfo(server, httpPort, securePort, conf, collection) } catch { @@ -489,6 +419,16 @@ private[spark] object JettyUtils extends Logging { } } + def addFilter( + handler: ServletContextHandler, + filter: String, + params: Map[String, String]): Unit = { + val holder = new FilterHolder() + holder.setClassName(filter) + params.foreach { case (k, v) => holder.setInitParameter(k, v) } + handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) + } + // 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) = { @@ -509,20 +449,37 @@ private[spark] case class ServerInfo( server: Server, boundPort: Int, securePort: Option[Int], - conf: SparkConf, - private val rootHandler: ContextHandlerCollection) { + private val conf: SparkConf, + private val rootHandler: ContextHandlerCollection) extends Logging { - def addHandler(handler: ServletContextHandler): Unit = { + def addHandler( + handler: ServletContextHandler, + securityMgr: SecurityManager): Unit = synchronized { handler.setVirtualHosts(JettyUtils.toVirtualHosts(JettyUtils.SPARK_CONNECTOR_NAME)) - JettyUtils.addFilters(Seq(handler), conf) - rootHandler.addHandler(handler) + addFilters(handler, securityMgr) + + val gzipHandler = new GzipHandler() + gzipHandler.setHandler(handler) + rootHandler.addHandler(gzipHandler) + if (!handler.isStarted()) { handler.start() } + gzipHandler.start() } - def removeHandler(handler: ContextHandler): Unit = { - rootHandler.removeHandler(handler) + def removeHandler(handler: ServletContextHandler): Unit = synchronized { + // Since addHandler() always adds a wrapping gzip handler, find the container handler + // and remove it. + rootHandler.getHandlers() + .find { h => + h.isInstanceOf[GzipHandler] && h.asInstanceOf[GzipHandler].getHandler() == handler + } + .foreach { h => + rootHandler.removeHandler(h) + h.stop() + } + if (handler.isStarted) { handler.stop() } @@ -537,4 +494,33 @@ private[spark] case class ServerInfo( threadPool.asInstanceOf[LifeCycle].stop } } + + /** + * Add filters, if any, to the given ServletContextHandlers. Always adds a filter at the end + * of the chain to perform security-related functions. + */ + private def addFilters(handler: ServletContextHandler, securityMgr: SecurityManager): Unit = { + conf.getOption("spark.ui.filters").toSeq.flatMap(Utils.stringToSeq).foreach { filter => + logInfo(s"Adding filter to ${handler.getContextPath()}: $filter") + val oldParams = conf.getOption(s"spark.$filter.params").toSeq + .flatMap(Utils.stringToSeq) + .flatMap { param => + val parts = param.split("=") + if (parts.length == 2) Some(parts(0) -> parts(1)) else None + } + .toMap + + val newParams = conf.getAllWithPrefix(s"spark.$filter.param.").toMap + + JettyUtils.addFilter(handler, filter, oldParams ++ newParams) + } + + // This filter must come after user-installed filters, since that's where authentication + // filters are installed. This means that custom filters will see the request before it's + // been validated by the security filter. + val securityFilter = new HttpSecurityFilter(conf, securityMgr) + val holder = new FilterHolder(securityFilter) + handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) + } + } 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 60a929375baa..967435030bc4 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -27,8 +27,6 @@ import scala.util.control.NonFatal import scala.xml._ import scala.xml.transform.{RewriteRule, RuleTransformer} -import org.apache.commons.lang3.StringEscapeUtils - import org.apache.spark.internal.Logging import org.apache.spark.ui.scope.RDDOperationGraph @@ -38,8 +36,6 @@ private[spark] object UIUtils extends Logging { val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" val TABLE_CLASS_STRIPED_SORTABLE = TABLE_CLASS_STRIPED + " sortable" - private val NEWLINE_AND_SINGLE_QUOTE_REGEX = raw"(?i)(\r\n|\n|\r|%0D%0A|%0A|%0D|'|%27)".r - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { override def initialValue(): SimpleDateFormat = @@ -552,23 +548,6 @@ private[spark] object UIUtils extends Logging { } } - /** - * Remove suspicious characters of user input to prevent Cross-Site scripting (XSS) attacks - * - * For more information about XSS testing: - * https://www.owasp.org/index.php/XSS_Filter_Evasion_Cheat_Sheet and - * https://www.owasp.org/index.php/Testing_for_Reflected_Cross_site_scripting_(OTG-INPVAL-001) - */ - def stripXSS(requestParameter: String): String = { - if (requestParameter == null) { - null - } else { - // Remove new lines and single quotes, followed by escaping HTML version 4.0 - StringEscapeUtils.escapeHtml4( - NEWLINE_AND_SINGLE_QUOTE_REGEX.replaceAllIn(requestParameter, "")) - } - } - def buildErrorResponse(status: Response.Status, msg: String): Response = { Response.status(status).entity(msg).`type`(MediaType.TEXT_PLAIN).build() } 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 2e43f17e6a8e..ebf8655ce8c2 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -58,7 +58,6 @@ private[spark] abstract class WebUI( def getBasePath: String = basePath def getTabs: Seq[WebUITab] = tabs def getHandlers: Seq[ServletContextHandler] = handlers - def getSecurityManager: SecurityManager = securityManager /** Attaches a tab to this UI, along with all of its attached pages. */ def attachTab(tab: WebUITab): Unit = { @@ -81,9 +80,9 @@ private[spark] abstract class WebUI( def attachPage(page: WebUIPage): Unit = { val pagePath = "/" + page.prefix val renderHandler = createServletHandler(pagePath, - (request: HttpServletRequest) => page.render(request), securityManager, conf, basePath) + (request: HttpServletRequest) => page.render(request), conf, basePath) val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json", - (request: HttpServletRequest) => page.renderJson(request), securityManager, conf, basePath) + (request: HttpServletRequest) => page.renderJson(request), conf, basePath) attachHandler(renderHandler) attachHandler(renderJsonHandler) val handlers = pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) @@ -91,13 +90,13 @@ private[spark] abstract class WebUI( } /** Attaches a handler to this UI. */ - def attachHandler(handler: ServletContextHandler): Unit = { + def attachHandler(handler: ServletContextHandler): Unit = synchronized { handlers += handler - serverInfo.foreach(_.addHandler(handler)) + serverInfo.foreach(_.addHandler(handler, securityManager)) } /** Detaches a handler from this UI. */ - def detachHandler(handler: ServletContextHandler): Unit = { + def detachHandler(handler: ServletContextHandler): Unit = synchronized { handlers -= handler serverInfo.foreach(_.removeHandler(handler)) } @@ -129,7 +128,9 @@ private[spark] abstract class WebUI( assert(serverInfo.isEmpty, s"Attempted to bind $className more than once!") try { val host = Option(conf.getenv("SPARK_LOCAL_IP")).getOrElse("0.0.0.0") - serverInfo = Some(startJettyServer(host, port, sslOptions, handlers, conf, name)) + val server = startJettyServer(host, port, sslOptions, conf, name) + handlers.foreach(server.addHandler(_, securityManager)) + serverInfo = Some(server) logInfo(s"Bound $className to $host, and started at $webUrl") } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index f9713fb5b4a3..a13037b5e24d 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -28,10 +28,8 @@ private[ui] class ExecutorThreadDumpPage( parent: SparkUITab, sc: Option[SparkContext]) extends WebUIPage("threadDump") { - // stripXSS is called first to remove suspicious characters used in XSS attacks def render(request: HttpServletRequest): Seq[Node] = { - val executorId = - Option(UIUtils.stripXSS(request.getParameter("executorId"))).map { executorId => + val executorId = Option(request.getParameter("executorId")).map { executorId => UIUtils.decodeURLParameter(executorId) }.getOrElse { throw new IllegalArgumentException(s"Missing executorId parameter") 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 2c22e0555fcb..b35ea5b52549 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 @@ -205,21 +205,17 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We jobTag: String, jobs: Seq[v1.JobData], killEnabled: Boolean): Seq[Node] = { - // stripXSS is called to remove suspicious characters used in XSS attacks - val allParameters = request.getParameterMap.asScala.toMap.map { case (k, v) => - UIUtils.stripXSS(k) -> v.map(UIUtils.stripXSS).toSeq - } - val parameterOtherTable = allParameters.filterNot(_._1.startsWith(jobTag)) + val parameterOtherTable = request.getParameterMap().asScala + .filterNot(_._1.startsWith(jobTag)) .map(para => para._1 + "=" + para._2(0)) val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) val jobIdTitle = if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id" - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterJobPage = UIUtils.stripXSS(request.getParameter(jobTag + ".page")) - val parameterJobSortColumn = UIUtils.stripXSS(request.getParameter(jobTag + ".sort")) - val parameterJobSortDesc = UIUtils.stripXSS(request.getParameter(jobTag + ".desc")) - val parameterJobPageSize = UIUtils.stripXSS(request.getParameter(jobTag + ".pageSize")) + val parameterJobPage = request.getParameter(jobTag + ".page") + val parameterJobSortColumn = request.getParameter(jobTag + ".sort") + val parameterJobSortDesc = request.getParameter(jobTag + ".desc") + val parameterJobPageSize = request.getParameter(jobTag + ".pageSize") val jobPage = Option(parameterJobPage).map(_.toInt).getOrElse(1) val jobSortColumn = Option(parameterJobSortColumn).map { sortColumn => 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 cd82439223b0..46295e73e086 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 @@ -184,8 +184,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP } def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) + val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") val jobId = parameterId.toInt diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index ff1b75e5c506..37bb292bd595 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -47,9 +47,7 @@ private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore) def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val jobId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) - jobId.foreach { id => + Option(request.getParameter("id")).map(_.toInt).foreach { id => store.asOption(store.job(id)).foreach { job => if (job.status == JobExecutionStatus.RUNNING) { sc.foreach(_.cancelJob(id)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 22a40101e33d..6d2710385d9d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -29,8 +29,7 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val poolName = Option(UIUtils.stripXSS(request.getParameter("poolname"))).map { poolname => + val poolName = Option(request.getParameter("poolname")).map { poolname => UIUtils.decodeURLParameter(poolname) }.getOrElse { throw new IllegalArgumentException(s"Missing poolname parameter") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 3bca1d574301..8ec625da042f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -80,22 +80,19 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We } def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) + val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val parameterAttempt = UIUtils.stripXSS(request.getParameter("attempt")) + val parameterAttempt = request.getParameter("attempt") require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") - val parameterTaskPage = UIUtils.stripXSS(request.getParameter("task.page")) - val parameterTaskSortColumn = UIUtils.stripXSS(request.getParameter("task.sort")) - val parameterTaskSortDesc = UIUtils.stripXSS(request.getParameter("task.desc")) - val parameterTaskPageSize = UIUtils.stripXSS(request.getParameter("task.pageSize")) + val parameterTaskPage = request.getParameter("task.page") + val parameterTaskSortColumn = request.getParameter("task.sort") + val parameterTaskSortDesc = request.getParameter("task.desc") + val parameterTaskPageSize = request.getParameter("task.pageSize") - val eventTimelineParameterTaskPage = UIUtils.stripXSS( - request.getParameter("task.eventTimelinePageNumber")) - val eventTimelineParameterTaskPageSize = UIUtils.stripXSS( - request.getParameter("task.eventTimelinePageSize")) + val eventTimelineParameterTaskPage = request.getParameter("task.eventTimelinePageNumber") + val eventTimelineParameterTaskPageSize = request.getParameter("task.eventTimelinePageSize") var eventTimelineTaskPage = Option(eventTimelineParameterTaskPage).map(_.toInt).getOrElse(1) var eventTimelineTaskPageSize = Option( eventTimelineParameterTaskPageSize).map(_.toInt).getOrElse(100) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 766efc15e26b..330b6422a13a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -42,17 +42,14 @@ private[ui] class StageTableBase( isFairScheduler: Boolean, killEnabled: Boolean, isFailedStage: Boolean) { - // stripXSS is called to remove suspicious characters used in XSS attacks - val allParameters = request.getParameterMap.asScala.toMap.map { case (k, v) => - UIUtils.stripXSS(k) -> v.map(UIUtils.stripXSS).toSeq - } - val parameterOtherTable = allParameters.filterNot(_._1.startsWith(stageTag)) + val parameterOtherTable = request.getParameterMap().asScala + .filterNot(_._1.startsWith(stageTag)) .map(para => para._1 + "=" + para._2(0)) - val parameterStagePage = UIUtils.stripXSS(request.getParameter(stageTag + ".page")) - val parameterStageSortColumn = UIUtils.stripXSS(request.getParameter(stageTag + ".sort")) - val parameterStageSortDesc = UIUtils.stripXSS(request.getParameter(stageTag + ".desc")) - val parameterStagePageSize = UIUtils.stripXSS(request.getParameter(stageTag + ".pageSize")) + val parameterStagePage = request.getParameter(stageTag + ".page") + val parameterStageSortColumn = request.getParameter(stageTag + ".sort") + val parameterStageSortDesc = request.getParameter(stageTag + ".desc") + val parameterStagePageSize = request.getParameter(stageTag + ".pageSize") val stagePage = Option(parameterStagePage).map(_.toInt).getOrElse(1) val stageSortColumn = Option(parameterStageSortColumn).map { sortColumn => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 10b032084ce4..e16c337ba164 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -45,9 +45,7 @@ private[ui] class StagesTab(val parent: SparkUI, val store: AppStatusStore) def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val stageId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) - stageId.foreach { id => + Option(request.getParameter("id")).map(_.toInt).foreach { id => store.asOption(store.lastStageAttempt(id)).foreach { stage => val status = stage.status if (status == StageStatus.ACTIVE || status == StageStatus.PENDING) { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 87da290c8305..dde441abe590 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -31,14 +31,13 @@ import org.apache.spark.util.Utils private[ui] class RDDPage(parent: SparkUITab, store: AppStatusStore) extends WebUIPage("rdd") { def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) + val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val parameterBlockPage = UIUtils.stripXSS(request.getParameter("block.page")) - val parameterBlockSortColumn = UIUtils.stripXSS(request.getParameter("block.sort")) - val parameterBlockSortDesc = UIUtils.stripXSS(request.getParameter("block.desc")) - val parameterBlockPageSize = UIUtils.stripXSS(request.getParameter("block.pageSize")) + val parameterBlockPage = request.getParameter("block.page") + val parameterBlockSortColumn = request.getParameter("block.sort") + val parameterBlockSortDesc = request.getParameter("block.desc") + val parameterBlockPageSize = request.getParameter("block.pageSize") val blockPage = Option(parameterBlockPage).map(_.toInt).getOrElse(1) val blockSortColumn = Option(parameterBlockSortColumn).getOrElse("Block Name") diff --git a/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala b/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala new file mode 100644 index 000000000000..f46cc293ed27 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui + +import java.util.UUID +import javax.servlet.FilterChain +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} + +import scala.collection.JavaConverters._ + +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.{mock, times, verify, when} + +import org.apache.spark._ +import org.apache.spark.internal.config._ + +class HttpSecurityFilterSuite extends SparkFunSuite { + + test("filter bad user input") { + val badValues = Map( + "encoded" -> "Encoding:base64%0d%0a%0d%0aPGh0bWw%2bjcmlwdD48L2h0bWw%2b", + "alert1" -> """>"'> - + @@ -195,14 +195,14 @@ private[spark] object UIUtils extends Logging { def dataTablesHeaderNodes(request: HttpServletRequest): Seq[Node] = { + "/static/jquery.dataTables.1.10.18.min.css")} type="text/css"/> - + diff --git a/dev/.rat-excludes b/dev/.rat-excludes index ccf266c3218d..7c2a720e8694 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -22,7 +22,7 @@ docs slaves spark-env.cmd bootstrap-tooltip.js -jquery-1.11.1.min.js +jquery-1.12.4.min.js d3.min.js dagre-d3.min.js graphlib-dot.min.js @@ -34,8 +34,8 @@ dataTables.bootstrap.min.js dataTables.rowsGroup.js jquery.blockUI.min.js jquery.cookies.2.2.0.min.js -jquery.dataTables.1.10.4.min.css -jquery.dataTables.1.10.4.min.js +jquery.dataTables.1.10.18.min.css +jquery.dataTables.1.10.18.min.js jquery.mustache.js jsonFormatter.min.css jsonFormatter.min.js diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index cbe4306799d4..63e9a4277562 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -159,7 +159,7 @@

    {{ page.title }}

    - + diff --git a/docs/js/vendor/jquery-1.12.4.min.js b/docs/js/vendor/jquery-1.12.4.min.js new file mode 100755 index 000000000000..e836475870da --- /dev/null +++ b/docs/js/vendor/jquery-1.12.4.min.js @@ -0,0 +1,5 @@ +/*! jQuery v1.12.4 | (c) jQuery Foundation | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=a.document,e=c.slice,f=c.concat,g=c.push,h=c.indexOf,i={},j=i.toString,k=i.hasOwnProperty,l={},m="1.12.4",n=function(a,b){return new n.fn.init(a,b)},o=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,p=/^-ms-/,q=/-([\da-z])/gi,r=function(a,b){return b.toUpperCase()};n.fn=n.prototype={jquery:m,constructor:n,selector:"",length:0,toArray:function(){return e.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:e.call(this)},pushStack:function(a){var b=n.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a){return n.each(this,a)},map:function(a){return this.pushStack(n.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(e.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor()},push:g,sort:c.sort,splice:c.splice},n.extend=n.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||n.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(n.isPlainObject(c)||(b=n.isArray(c)))?(b?(b=!1,f=a&&n.isArray(a)?a:[]):f=a&&n.isPlainObject(a)?a:{},g[d]=n.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},n.extend({expando:"jQuery"+(m+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===n.type(a)},isArray:Array.isArray||function(a){return"array"===n.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){var b=a&&a.toString();return!n.isArray(a)&&b-parseFloat(b)+1>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==n.type(a)||a.nodeType||n.isWindow(a))return!1;try{if(a.constructor&&!k.call(a,"constructor")&&!k.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(!l.ownFirst)for(b in a)return k.call(a,b);for(b in a);return void 0===b||k.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?i[j.call(a)]||"object":typeof a},globalEval:function(b){b&&n.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(p,"ms-").replace(q,r)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b){var c,d=0;if(s(a)){for(c=a.length;c>d;d++)if(b.call(a[d],d,a[d])===!1)break}else for(d in a)if(b.call(a[d],d,a[d])===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(o,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(s(Object(a))?n.merge(c,"string"==typeof a?[a]:a):g.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(h)return h.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,e,g=0,h=[];if(s(a))for(d=a.length;d>g;g++)e=b(a[g],g,c),null!=e&&h.push(e);else for(g in a)e=b(a[g],g,c),null!=e&&h.push(e);return f.apply([],h)},guid:1,proxy:function(a,b){var c,d,f;return"string"==typeof b&&(f=a[b],b=a,a=f),n.isFunction(a)?(c=e.call(arguments,2),d=function(){return a.apply(b||this,c.concat(e.call(arguments)))},d.guid=a.guid=a.guid||n.guid++,d):void 0},now:function(){return+new Date},support:l}),"function"==typeof Symbol&&(n.fn[Symbol.iterator]=c[Symbol.iterator]),n.each("Boolean Number String Function Array Date RegExp Object Error Symbol".split(" "),function(a,b){i["[object "+b+"]"]=b.toLowerCase()});function s(a){var b=!!a&&"length"in a&&a.length,c=n.type(a);return"function"===c||n.isWindow(a)?!1:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var t=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+1*new Date,v=a.document,w=0,x=0,y=ga(),z=ga(),A=ga(),B=function(a,b){return a===b&&(l=!0),0},C=1<<31,D={}.hasOwnProperty,E=[],F=E.pop,G=E.push,H=E.push,I=E.slice,J=function(a,b){for(var c=0,d=a.length;d>c;c++)if(a[c]===b)return c;return-1},K="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",L="[\\x20\\t\\r\\n\\f]",M="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",N="\\["+L+"*("+M+")(?:"+L+"*([*^$|!~]?=)"+L+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+M+"))|)"+L+"*\\]",O=":("+M+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+N+")*)|.*)\\)|)",P=new RegExp(L+"+","g"),Q=new RegExp("^"+L+"+|((?:^|[^\\\\])(?:\\\\.)*)"+L+"+$","g"),R=new RegExp("^"+L+"*,"+L+"*"),S=new RegExp("^"+L+"*([>+~]|"+L+")"+L+"*"),T=new RegExp("="+L+"*([^\\]'\"]*?)"+L+"*\\]","g"),U=new RegExp(O),V=new RegExp("^"+M+"$"),W={ID:new RegExp("^#("+M+")"),CLASS:new RegExp("^\\.("+M+")"),TAG:new RegExp("^("+M+"|[*])"),ATTR:new RegExp("^"+N),PSEUDO:new RegExp("^"+O),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+L+"*(even|odd|(([+-]|)(\\d*)n|)"+L+"*(?:([+-]|)"+L+"*(\\d+)|))"+L+"*\\)|)","i"),bool:new RegExp("^(?:"+K+")$","i"),needsContext:new RegExp("^"+L+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+L+"*((?:-\\d)?\\d*)"+L+"*\\)|)(?=[^-]|$)","i")},X=/^(?:input|select|textarea|button)$/i,Y=/^h\d$/i,Z=/^[^{]+\{\s*\[native \w/,$=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,_=/[+~]/,aa=/'|\\/g,ba=new RegExp("\\\\([\\da-f]{1,6}"+L+"?|("+L+")|.)","ig"),ca=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)},da=function(){m()};try{H.apply(E=I.call(v.childNodes),v.childNodes),E[v.childNodes.length].nodeType}catch(ea){H={apply:E.length?function(a,b){G.apply(a,I.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fa(a,b,d,e){var f,h,j,k,l,o,r,s,w=b&&b.ownerDocument,x=b?b.nodeType:9;if(d=d||[],"string"!=typeof a||!a||1!==x&&9!==x&&11!==x)return d;if(!e&&((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,p)){if(11!==x&&(o=$.exec(a)))if(f=o[1]){if(9===x){if(!(j=b.getElementById(f)))return d;if(j.id===f)return d.push(j),d}else if(w&&(j=w.getElementById(f))&&t(b,j)&&j.id===f)return d.push(j),d}else{if(o[2])return H.apply(d,b.getElementsByTagName(a)),d;if((f=o[3])&&c.getElementsByClassName&&b.getElementsByClassName)return H.apply(d,b.getElementsByClassName(f)),d}if(c.qsa&&!A[a+" "]&&(!q||!q.test(a))){if(1!==x)w=b,s=a;else if("object"!==b.nodeName.toLowerCase()){(k=b.getAttribute("id"))?k=k.replace(aa,"\\$&"):b.setAttribute("id",k=u),r=g(a),h=r.length,l=V.test(k)?"#"+k:"[id='"+k+"']";while(h--)r[h]=l+" "+qa(r[h]);s=r.join(","),w=_.test(a)&&oa(b.parentNode)||b}if(s)try{return H.apply(d,w.querySelectorAll(s)),d}catch(y){}finally{k===u&&b.removeAttribute("id")}}}return i(a.replace(Q,"$1"),b,d,e)}function ga(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function ha(a){return a[u]=!0,a}function ia(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function ja(a,b){var c=a.split("|"),e=c.length;while(e--)d.attrHandle[c[e]]=b}function ka(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||C)-(~a.sourceIndex||C);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function la(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function ma(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function na(a){return ha(function(b){return b=+b,ha(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function oa(a){return a&&"undefined"!=typeof a.getElementsByTagName&&a}c=fa.support={},f=fa.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fa.setDocument=function(a){var b,e,g=a?a.ownerDocument||a:v;return g!==n&&9===g.nodeType&&g.documentElement?(n=g,o=n.documentElement,p=!f(n),(e=n.defaultView)&&e.top!==e&&(e.addEventListener?e.addEventListener("unload",da,!1):e.attachEvent&&e.attachEvent("onunload",da)),c.attributes=ia(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ia(function(a){return a.appendChild(n.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=Z.test(n.getElementsByClassName),c.getById=ia(function(a){return o.appendChild(a).id=u,!n.getElementsByName||!n.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c=b.getElementById(a);return c?[c]:[]}},d.filter.ID=function(a){var b=a.replace(ba,ca);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(ba,ca);return function(a){var c="undefined"!=typeof a.getAttributeNode&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return"undefined"!=typeof b.getElementsByTagName?b.getElementsByTagName(a):c.qsa?b.querySelectorAll(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return"undefined"!=typeof b.getElementsByClassName&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=Z.test(n.querySelectorAll))&&(ia(function(a){o.appendChild(a).innerHTML="",a.querySelectorAll("[msallowcapture^='']").length&&q.push("[*^$]="+L+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+L+"*(?:value|"+K+")"),a.querySelectorAll("[id~="+u+"-]").length||q.push("~="),a.querySelectorAll(":checked").length||q.push(":checked"),a.querySelectorAll("a#"+u+"+*").length||q.push(".#.+[+~]")}),ia(function(a){var b=n.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+L+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=Z.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ia(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",O)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=Z.test(o.compareDocumentPosition),t=b||Z.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===n||a.ownerDocument===v&&t(v,a)?-1:b===n||b.ownerDocument===v&&t(v,b)?1:k?J(k,a)-J(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,e=a.parentNode,f=b.parentNode,g=[a],h=[b];if(!e||!f)return a===n?-1:b===n?1:e?-1:f?1:k?J(k,a)-J(k,b):0;if(e===f)return ka(a,b);c=a;while(c=c.parentNode)g.unshift(c);c=b;while(c=c.parentNode)h.unshift(c);while(g[d]===h[d])d++;return d?ka(g[d],h[d]):g[d]===v?-1:h[d]===v?1:0},n):n},fa.matches=function(a,b){return fa(a,null,null,b)},fa.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(T,"='$1']"),c.matchesSelector&&p&&!A[b+" "]&&(!r||!r.test(b))&&(!q||!q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fa(b,n,null,[a]).length>0},fa.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fa.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&D.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fa.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fa.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=fa.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=fa.selectors={cacheLength:50,createPseudo:ha,match:W,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(ba,ca),a[3]=(a[3]||a[4]||a[5]||"").replace(ba,ca),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||fa.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&fa.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return W.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&U.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(ba,ca).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+L+")"+a+"("+L+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||"undefined"!=typeof a.getAttribute&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fa.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e.replace(P," ")+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h,t=!1;if(q){if(f){while(p){m=b;while(m=m[p])if(h?m.nodeName.toLowerCase()===r:1===m.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){m=q,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n&&j[2],m=n&&q.childNodes[n];while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if(1===m.nodeType&&++t&&m===b){k[a]=[w,n,t];break}}else if(s&&(m=b,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n),t===!1)while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if((h?m.nodeName.toLowerCase()===r:1===m.nodeType)&&++t&&(s&&(l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),k[a]=[w,t]),m===b))break;return t-=e,t===d||t%d===0&&t/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||fa.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?ha(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=J(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:ha(function(a){var b=[],c=[],d=h(a.replace(Q,"$1"));return d[u]?ha(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),b[0]=null,!c.pop()}}),has:ha(function(a){return function(b){return fa(a,b).length>0}}),contains:ha(function(a){return a=a.replace(ba,ca),function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:ha(function(a){return V.test(a||"")||fa.error("unsupported lang: "+a),a=a.replace(ba,ca).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Y.test(a.nodeName)},input:function(a){return X.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:na(function(){return[0]}),last:na(function(a,b){return[b-1]}),eq:na(function(a,b,c){return[0>c?c+b:c]}),even:na(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:na(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:na(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:na(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function ra(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j,k=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(j=b[u]||(b[u]={}),i=j[b.uniqueID]||(j[b.uniqueID]={}),(h=i[d])&&h[0]===w&&h[1]===f)return k[2]=h[2];if(i[d]=k,k[2]=a(b,c,g))return!0}}}function sa(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function ta(a,b,c){for(var d=0,e=b.length;e>d;d++)fa(a,b[d],c);return c}function ua(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(c&&!c(f,d,e)||(g.push(f),j&&b.push(h)));return g}function va(a,b,c,d,e,f){return d&&!d[u]&&(d=va(d)),e&&!e[u]&&(e=va(e,f)),ha(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||ta(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ua(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ua(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?J(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ua(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):H.apply(g,r)})}function wa(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=ra(function(a){return a===b},h,!0),l=ra(function(a){return J(b,a)>-1},h,!0),m=[function(a,c,d){var e=!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d));return b=null,e}];f>i;i++)if(c=d.relative[a[i].type])m=[ra(sa(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return va(i>1&&sa(m),i>1&&qa(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(Q,"$1"),c,e>i&&wa(a.slice(i,e)),f>e&&wa(a=a.slice(e)),f>e&&qa(a))}m.push(c)}return sa(m)}function xa(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,o,q,r=0,s="0",t=f&&[],u=[],v=j,x=f||e&&d.find.TAG("*",k),y=w+=null==v?1:Math.random()||.1,z=x.length;for(k&&(j=g===n||g||k);s!==z&&null!=(l=x[s]);s++){if(e&&l){o=0,g||l.ownerDocument===n||(m(l),h=!p);while(q=a[o++])if(q(l,g||n,h)){i.push(l);break}k&&(w=y)}c&&((l=!q&&l)&&r--,f&&t.push(l))}if(r+=s,c&&s!==r){o=0;while(q=b[o++])q(t,u,g,h);if(f){if(r>0)while(s--)t[s]||u[s]||(u[s]=F.call(i));u=ua(u)}H.apply(i,u),k&&!f&&u.length>0&&r+b.length>1&&fa.uniqueSort(i)}return k&&(w=y,j=v),t};return c?ha(f):f}return h=fa.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wa(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xa(e,d)),f.selector=a}return f},i=fa.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(ba,ca),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=W.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(ba,ca),_.test(j[0].type)&&oa(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qa(j),!a)return H.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,!b||_.test(a)&&oa(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ia(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ia(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||ja("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ia(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||ja("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ia(function(a){return null==a.getAttribute("disabled")})||ja(K,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fa}(a);n.find=t,n.expr=t.selectors,n.expr[":"]=n.expr.pseudos,n.uniqueSort=n.unique=t.uniqueSort,n.text=t.getText,n.isXMLDoc=t.isXML,n.contains=t.contains;var u=function(a,b,c){var d=[],e=void 0!==c;while((a=a[b])&&9!==a.nodeType)if(1===a.nodeType){if(e&&n(a).is(c))break;d.push(a)}return d},v=function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c},w=n.expr.match.needsContext,x=/^<([\w-]+)\s*\/?>(?:<\/\1>|)$/,y=/^.[^:#\[\.,]*$/;function z(a,b,c){if(n.isFunction(b))return n.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return n.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(y.test(b))return n.filter(b,a,c);b=n.filter(b,a)}return n.grep(a,function(a){return n.inArray(a,b)>-1!==c})}n.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?n.find.matchesSelector(d,a)?[d]:[]:n.find.matches(a,n.grep(b,function(a){return 1===a.nodeType}))},n.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(n(a).filter(function(){for(b=0;e>b;b++)if(n.contains(d[b],this))return!0}));for(b=0;e>b;b++)n.find(a,d[b],c);return c=this.pushStack(e>1?n.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(z(this,a||[],!1))},not:function(a){return this.pushStack(z(this,a||[],!0))},is:function(a){return!!z(this,"string"==typeof a&&w.test(a)?n(a):a||[],!1).length}});var A,B=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,C=n.fn.init=function(a,b,c){var e,f;if(!a)return this;if(c=c||A,"string"==typeof a){if(e="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:B.exec(a),!e||!e[1]&&b)return!b||b.jquery?(b||c).find(a):this.constructor(b).find(a);if(e[1]){if(b=b instanceof n?b[0]:b,n.merge(this,n.parseHTML(e[1],b&&b.nodeType?b.ownerDocument||b:d,!0)),x.test(e[1])&&n.isPlainObject(b))for(e in b)n.isFunction(this[e])?this[e](b[e]):this.attr(e,b[e]);return this}if(f=d.getElementById(e[2]),f&&f.parentNode){if(f.id!==e[2])return A.find(a);this.length=1,this[0]=f}return this.context=d,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):n.isFunction(a)?"undefined"!=typeof c.ready?c.ready(a):a(n):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),n.makeArray(a,this))};C.prototype=n.fn,A=n(d);var D=/^(?:parents|prev(?:Until|All))/,E={children:!0,contents:!0,next:!0,prev:!0};n.fn.extend({has:function(a){var b,c=n(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(n.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=w.test(a)||"string"!=typeof a?n(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&n.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?n.uniqueSort(f):f)},index:function(a){return a?"string"==typeof a?n.inArray(this[0],n(a)):n.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(n.uniqueSort(n.merge(this.get(),n(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function F(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}n.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return u(a,"parentNode")},parentsUntil:function(a,b,c){return u(a,"parentNode",c)},next:function(a){return F(a,"nextSibling")},prev:function(a){return F(a,"previousSibling")},nextAll:function(a){return u(a,"nextSibling")},prevAll:function(a){return u(a,"previousSibling")},nextUntil:function(a,b,c){return u(a,"nextSibling",c)},prevUntil:function(a,b,c){return u(a,"previousSibling",c)},siblings:function(a){return v((a.parentNode||{}).firstChild,a)},children:function(a){return v(a.firstChild)},contents:function(a){return n.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:n.merge([],a.childNodes)}},function(a,b){n.fn[a]=function(c,d){var e=n.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=n.filter(d,e)),this.length>1&&(E[a]||(e=n.uniqueSort(e)),D.test(a)&&(e=e.reverse())),this.pushStack(e)}});var G=/\S+/g;function H(a){var b={};return n.each(a.match(G)||[],function(a,c){b[c]=!0}),b}n.Callbacks=function(a){a="string"==typeof a?H(a):n.extend({},a);var b,c,d,e,f=[],g=[],h=-1,i=function(){for(e=a.once,d=b=!0;g.length;h=-1){c=g.shift();while(++h-1)f.splice(c,1),h>=c&&h--}),this},has:function(a){return a?n.inArray(a,f)>-1:f.length>0},empty:function(){return f&&(f=[]),this},disable:function(){return e=g=[],f=c="",this},disabled:function(){return!f},lock:function(){return e=!0,c||j.disable(),this},locked:function(){return!!e},fireWith:function(a,c){return e||(c=c||[],c=[a,c.slice?c.slice():c],g.push(c),b||i()),this},fire:function(){return j.fireWith(this,arguments),this},fired:function(){return!!d}};return j},n.extend({Deferred:function(a){var b=[["resolve","done",n.Callbacks("once memory"),"resolved"],["reject","fail",n.Callbacks("once memory"),"rejected"],["notify","progress",n.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return n.Deferred(function(c){n.each(b,function(b,f){var g=n.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&n.isFunction(a.promise)?a.promise().progress(c.notify).done(c.resolve).fail(c.reject):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?n.extend(a,d):d}},e={};return d.pipe=d.then,n.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=e.call(arguments),d=c.length,f=1!==d||a&&n.isFunction(a.promise)?d:0,g=1===f?a:n.Deferred(),h=function(a,b,c){return function(d){b[a]=this,c[a]=arguments.length>1?e.call(arguments):d,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(d>1)for(i=new Array(d),j=new Array(d),k=new Array(d);d>b;b++)c[b]&&n.isFunction(c[b].promise)?c[b].promise().progress(h(b,j,i)).done(h(b,k,c)).fail(g.reject):--f;return f||g.resolveWith(k,c),g.promise()}});var I;n.fn.ready=function(a){return n.ready.promise().done(a),this},n.extend({isReady:!1,readyWait:1,holdReady:function(a){a?n.readyWait++:n.ready(!0)},ready:function(a){(a===!0?--n.readyWait:n.isReady)||(n.isReady=!0,a!==!0&&--n.readyWait>0||(I.resolveWith(d,[n]),n.fn.triggerHandler&&(n(d).triggerHandler("ready"),n(d).off("ready"))))}});function J(){d.addEventListener?(d.removeEventListener("DOMContentLoaded",K),a.removeEventListener("load",K)):(d.detachEvent("onreadystatechange",K),a.detachEvent("onload",K))}function K(){(d.addEventListener||"load"===a.event.type||"complete"===d.readyState)&&(J(),n.ready())}n.ready.promise=function(b){if(!I)if(I=n.Deferred(),"complete"===d.readyState||"loading"!==d.readyState&&!d.documentElement.doScroll)a.setTimeout(n.ready);else if(d.addEventListener)d.addEventListener("DOMContentLoaded",K),a.addEventListener("load",K);else{d.attachEvent("onreadystatechange",K),a.attachEvent("onload",K);var c=!1;try{c=null==a.frameElement&&d.documentElement}catch(e){}c&&c.doScroll&&!function f(){if(!n.isReady){try{c.doScroll("left")}catch(b){return a.setTimeout(f,50)}J(),n.ready()}}()}return I.promise(b)},n.ready.promise();var L;for(L in n(l))break;l.ownFirst="0"===L,l.inlineBlockNeedsLayout=!1,n(function(){var a,b,c,e;c=d.getElementsByTagName("body")[0],c&&c.style&&(b=d.createElement("div"),e=d.createElement("div"),e.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(e).appendChild(b),"undefined"!=typeof b.style.zoom&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",l.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(e))}),function(){var a=d.createElement("div");l.deleteExpando=!0;try{delete a.test}catch(b){l.deleteExpando=!1}a=null}();var M=function(a){var b=n.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b},N=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,O=/([A-Z])/g;function P(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(O,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:N.test(c)?n.parseJSON(c):c}catch(e){}n.data(a,b,c)}else c=void 0; +}return c}function Q(a){var b;for(b in a)if(("data"!==b||!n.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function R(a,b,d,e){if(M(a)){var f,g,h=n.expando,i=a.nodeType,j=i?n.cache:a,k=i?a[h]:a[h]&&h;if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||n.guid++:h),j[k]||(j[k]=i?{}:{toJSON:n.noop}),"object"!=typeof b&&"function"!=typeof b||(e?j[k]=n.extend(j[k],b):j[k].data=n.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[n.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[n.camelCase(b)])):f=g,f}}function S(a,b,c){if(M(a)){var d,e,f=a.nodeType,g=f?n.cache:a,h=f?a[n.expando]:n.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){n.isArray(b)?b=b.concat(n.map(b,n.camelCase)):b in d?b=[b]:(b=n.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!Q(d):!n.isEmptyObject(d))return}(c||(delete g[h].data,Q(g[h])))&&(f?n.cleanData([a],!0):l.deleteExpando||g!=g.window?delete g[h]:g[h]=void 0)}}}n.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?n.cache[a[n.expando]]:a[n.expando],!!a&&!Q(a)},data:function(a,b,c){return R(a,b,c)},removeData:function(a,b){return S(a,b)},_data:function(a,b,c){return R(a,b,c,!0)},_removeData:function(a,b){return S(a,b,!0)}}),n.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=n.data(f),1===f.nodeType&&!n._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=n.camelCase(d.slice(5)),P(f,d,e[d])));n._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){n.data(this,a)}):arguments.length>1?this.each(function(){n.data(this,a,b)}):f?P(f,a,n.data(f,a)):void 0},removeData:function(a){return this.each(function(){n.removeData(this,a)})}}),n.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=n._data(a,b),c&&(!d||n.isArray(c)?d=n._data(a,b,n.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=n.queue(a,b),d=c.length,e=c.shift(),f=n._queueHooks(a,b),g=function(){n.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return n._data(a,c)||n._data(a,c,{empty:n.Callbacks("once memory").add(function(){n._removeData(a,b+"queue"),n._removeData(a,c)})})}}),n.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},Z=/^(?:checkbox|radio)$/i,$=/<([\w:-]+)/,_=/^$|\/(?:java|ecma)script/i,aa=/^\s+/,ba="abbr|article|aside|audio|bdi|canvas|data|datalist|details|dialog|figcaption|figure|footer|header|hgroup|main|mark|meter|nav|output|picture|progress|section|summary|template|time|video";function ca(a){var b=ba.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}!function(){var a=d.createElement("div"),b=d.createDocumentFragment(),c=d.createElement("input");a.innerHTML="
    a",l.leadingWhitespace=3===a.firstChild.nodeType,l.tbody=!a.getElementsByTagName("tbody").length,l.htmlSerialize=!!a.getElementsByTagName("link").length,l.html5Clone="<:nav>"!==d.createElement("nav").cloneNode(!0).outerHTML,c.type="checkbox",c.checked=!0,b.appendChild(c),l.appendChecked=c.checked,a.innerHTML="",l.noCloneChecked=!!a.cloneNode(!0).lastChild.defaultValue,b.appendChild(a),c=d.createElement("input"),c.setAttribute("type","radio"),c.setAttribute("checked","checked"),c.setAttribute("name","t"),a.appendChild(c),l.checkClone=a.cloneNode(!0).cloneNode(!0).lastChild.checked,l.noCloneEvent=!!a.addEventListener,a[n.expando]=1,l.attributes=!a.getAttribute(n.expando)}();var da={option:[1,""],legend:[1,"
    ","
    "],area:[1,"",""],param:[1,"",""],thead:[1,"","
    "],tr:[2,"","
    "],col:[2,"","
    "],td:[3,"","
    "],_default:l.htmlSerialize?[0,"",""]:[1,"X
    ","
    "]};da.optgroup=da.option,da.tbody=da.tfoot=da.colgroup=da.caption=da.thead,da.th=da.td;function ea(a,b){var c,d,e=0,f="undefined"!=typeof a.getElementsByTagName?a.getElementsByTagName(b||"*"):"undefined"!=typeof a.querySelectorAll?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||n.nodeName(d,b)?f.push(d):n.merge(f,ea(d,b));return void 0===b||b&&n.nodeName(a,b)?n.merge([a],f):f}function fa(a,b){for(var c,d=0;null!=(c=a[d]);d++)n._data(c,"globalEval",!b||n._data(b[d],"globalEval"))}var ga=/<|&#?\w+;/,ha=/r;r++)if(g=a[r],g||0===g)if("object"===n.type(g))n.merge(q,g.nodeType?[g]:g);else if(ga.test(g)){i=i||p.appendChild(b.createElement("div")),j=($.exec(g)||["",""])[1].toLowerCase(),m=da[j]||da._default,i.innerHTML=m[1]+n.htmlPrefilter(g)+m[2],f=m[0];while(f--)i=i.lastChild;if(!l.leadingWhitespace&&aa.test(g)&&q.push(b.createTextNode(aa.exec(g)[0])),!l.tbody){g="table"!==j||ha.test(g)?""!==m[1]||ha.test(g)?0:i:i.firstChild,f=g&&g.childNodes.length;while(f--)n.nodeName(k=g.childNodes[f],"tbody")&&!k.childNodes.length&&g.removeChild(k)}n.merge(q,i.childNodes),i.textContent="";while(i.firstChild)i.removeChild(i.firstChild);i=p.lastChild}else q.push(b.createTextNode(g));i&&p.removeChild(i),l.appendChecked||n.grep(ea(q,"input"),ia),r=0;while(g=q[r++])if(d&&n.inArray(g,d)>-1)e&&e.push(g);else if(h=n.contains(g.ownerDocument,g),i=ea(p.appendChild(g),"script"),h&&fa(i),c){f=0;while(g=i[f++])_.test(g.type||"")&&c.push(g)}return i=null,p}!function(){var b,c,e=d.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(l[b]=c in a)||(e.setAttribute(c,"t"),l[b]=e.attributes[c].expando===!1);e=null}();var ka=/^(?:input|select|textarea)$/i,la=/^key/,ma=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,na=/^(?:focusinfocus|focusoutblur)$/,oa=/^([^.]*)(?:\.(.+)|)/;function pa(){return!0}function qa(){return!1}function ra(){try{return d.activeElement}catch(a){}}function sa(a,b,c,d,e,f){var g,h;if("object"==typeof b){"string"!=typeof c&&(d=d||c,c=void 0);for(h in b)sa(a,h,c,d,b[h],f);return a}if(null==d&&null==e?(e=c,d=c=void 0):null==e&&("string"==typeof c?(e=d,d=void 0):(e=d,d=c,c=void 0)),e===!1)e=qa;else if(!e)return a;return 1===f&&(g=e,e=function(a){return n().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=n.guid++)),a.each(function(){n.event.add(this,b,e,d,c)})}n.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=n.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return"undefined"==typeof n||a&&n.event.triggered===a.type?void 0:n.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(G)||[""],h=b.length;while(h--)f=oa.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=n.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=n.event.special[o]||{},l=n.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&n.expr.match.needsContext.test(e),namespace:p.join(".")},i),(m=g[o])||(m=g[o]=[],m.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,l):m.push(l),n.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n.hasData(a)&&n._data(a);if(r&&(k=r.events)){b=(b||"").match(G)||[""],j=b.length;while(j--)if(h=oa.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=n.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,m=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=m.length;while(f--)g=m[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(m.splice(f,1),g.selector&&m.delegateCount--,l.remove&&l.remove.call(a,g));i&&!m.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||n.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)n.event.remove(a,o+b[j],c,d,!0);n.isEmptyObject(k)&&(delete r.handle,n._removeData(a,"events"))}},trigger:function(b,c,e,f){var g,h,i,j,l,m,o,p=[e||d],q=k.call(b,"type")?b.type:b,r=k.call(b,"namespace")?b.namespace.split("."):[];if(i=m=e=e||d,3!==e.nodeType&&8!==e.nodeType&&!na.test(q+n.event.triggered)&&(q.indexOf(".")>-1&&(r=q.split("."),q=r.shift(),r.sort()),h=q.indexOf(":")<0&&"on"+q,b=b[n.expando]?b:new n.Event(q,"object"==typeof b&&b),b.isTrigger=f?2:3,b.namespace=r.join("."),b.rnamespace=b.namespace?new RegExp("(^|\\.)"+r.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=e),c=null==c?[b]:n.makeArray(c,[b]),l=n.event.special[q]||{},f||!l.trigger||l.trigger.apply(e,c)!==!1)){if(!f&&!l.noBubble&&!n.isWindow(e)){for(j=l.delegateType||q,na.test(j+q)||(i=i.parentNode);i;i=i.parentNode)p.push(i),m=i;m===(e.ownerDocument||d)&&p.push(m.defaultView||m.parentWindow||a)}o=0;while((i=p[o++])&&!b.isPropagationStopped())b.type=o>1?j:l.bindType||q,g=(n._data(i,"events")||{})[b.type]&&n._data(i,"handle"),g&&g.apply(i,c),g=h&&i[h],g&&g.apply&&M(i)&&(b.result=g.apply(i,c),b.result===!1&&b.preventDefault());if(b.type=q,!f&&!b.isDefaultPrevented()&&(!l._default||l._default.apply(p.pop(),c)===!1)&&M(e)&&h&&e[q]&&!n.isWindow(e)){m=e[h],m&&(e[h]=null),n.event.triggered=q;try{e[q]()}catch(s){}n.event.triggered=void 0,m&&(e[h]=m)}return b.result}},dispatch:function(a){a=n.event.fix(a);var b,c,d,f,g,h=[],i=e.call(arguments),j=(n._data(this,"events")||{})[a.type]||[],k=n.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=n.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,c=0;while((g=f.handlers[c++])&&!a.isImmediatePropagationStopped())a.rnamespace&&!a.rnamespace.test(g.namespace)||(a.handleObj=g,a.data=g.data,d=((n.event.special[g.origType]||{}).handle||g.handler).apply(f.elem,i),void 0!==d&&(a.result=d)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&("click"!==a.type||isNaN(a.button)||a.button<1))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(d=[],c=0;h>c;c++)f=b[c],e=f.selector+" ",void 0===d[e]&&(d[e]=f.needsContext?n(e,this).index(i)>-1:n.find(e,this,null,[i]).length),d[e]&&d.push(f);d.length&&g.push({elem:i,handlers:d})}return h]","i"),va=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:-]+)[^>]*)\/>/gi,wa=/\s*$/g,Aa=ca(d),Ba=Aa.appendChild(d.createElement("div"));function Ca(a,b){return n.nodeName(a,"table")&&n.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function Da(a){return a.type=(null!==n.find.attr(a,"type"))+"/"+a.type,a}function Ea(a){var b=ya.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function Fa(a,b){if(1===b.nodeType&&n.hasData(a)){var c,d,e,f=n._data(a),g=n._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)n.event.add(b,c,h[c][d])}g.data&&(g.data=n.extend({},g.data))}}function Ga(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!l.noCloneEvent&&b[n.expando]){e=n._data(b);for(d in e.events)n.removeEvent(b,d,e.handle);b.removeAttribute(n.expando)}"script"===c&&b.text!==a.text?(Da(b).text=a.text,Ea(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),l.html5Clone&&a.innerHTML&&!n.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&Z.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:"input"!==c&&"textarea"!==c||(b.defaultValue=a.defaultValue)}}function Ha(a,b,c,d){b=f.apply([],b);var e,g,h,i,j,k,m=0,o=a.length,p=o-1,q=b[0],r=n.isFunction(q);if(r||o>1&&"string"==typeof q&&!l.checkClone&&xa.test(q))return a.each(function(e){var f=a.eq(e);r&&(b[0]=q.call(this,e,f.html())),Ha(f,b,c,d)});if(o&&(k=ja(b,a[0].ownerDocument,!1,a,d),e=k.firstChild,1===k.childNodes.length&&(k=e),e||d)){for(i=n.map(ea(k,"script"),Da),h=i.length;o>m;m++)g=k,m!==p&&(g=n.clone(g,!0,!0),h&&n.merge(i,ea(g,"script"))),c.call(a[m],g,m);if(h)for(j=i[i.length-1].ownerDocument,n.map(i,Ea),m=0;h>m;m++)g=i[m],_.test(g.type||"")&&!n._data(g,"globalEval")&&n.contains(j,g)&&(g.src?n._evalUrl&&n._evalUrl(g.src):n.globalEval((g.text||g.textContent||g.innerHTML||"").replace(za,"")));k=e=null}return a}function Ia(a,b,c){for(var d,e=b?n.filter(b,a):a,f=0;null!=(d=e[f]);f++)c||1!==d.nodeType||n.cleanData(ea(d)),d.parentNode&&(c&&n.contains(d.ownerDocument,d)&&fa(ea(d,"script")),d.parentNode.removeChild(d));return a}n.extend({htmlPrefilter:function(a){return a.replace(va,"<$1>")},clone:function(a,b,c){var d,e,f,g,h,i=n.contains(a.ownerDocument,a);if(l.html5Clone||n.isXMLDoc(a)||!ua.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(Ba.innerHTML=a.outerHTML,Ba.removeChild(f=Ba.firstChild)),!(l.noCloneEvent&&l.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||n.isXMLDoc(a)))for(d=ea(f),h=ea(a),g=0;null!=(e=h[g]);++g)d[g]&&Ga(e,d[g]);if(b)if(c)for(h=h||ea(a),d=d||ea(f),g=0;null!=(e=h[g]);g++)Fa(e,d[g]);else Fa(a,f);return d=ea(f,"script"),d.length>0&&fa(d,!i&&ea(a,"script")),d=h=e=null,f},cleanData:function(a,b){for(var d,e,f,g,h=0,i=n.expando,j=n.cache,k=l.attributes,m=n.event.special;null!=(d=a[h]);h++)if((b||M(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)m[e]?n.event.remove(d,e):n.removeEvent(d,e,g.handle);j[f]&&(delete j[f],k||"undefined"==typeof d.removeAttribute?d[i]=void 0:d.removeAttribute(i),c.push(f))}}}),n.fn.extend({domManip:Ha,detach:function(a){return Ia(this,a,!0)},remove:function(a){return Ia(this,a)},text:function(a){return Y(this,function(a){return void 0===a?n.text(this):this.empty().append((this[0]&&this[0].ownerDocument||d).createTextNode(a))},null,a,arguments.length)},append:function(){return Ha(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ca(this,a);b.appendChild(a)}})},prepend:function(){return Ha(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ca(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return Ha(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return Ha(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&n.cleanData(ea(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&n.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return n.clone(this,a,b)})},html:function(a){return Y(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(ta,""):void 0;if("string"==typeof a&&!wa.test(a)&&(l.htmlSerialize||!ua.test(a))&&(l.leadingWhitespace||!aa.test(a))&&!da[($.exec(a)||["",""])[1].toLowerCase()]){a=n.htmlPrefilter(a);try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(n.cleanData(ea(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=[];return Ha(this,arguments,function(b){var c=this.parentNode;n.inArray(this,a)<0&&(n.cleanData(ea(this)),c&&c.replaceChild(b,this))},a)}}),n.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){n.fn[a]=function(a){for(var c,d=0,e=[],f=n(a),h=f.length-1;h>=d;d++)c=d===h?this:this.clone(!0),n(f[d])[b](c),g.apply(e,c.get());return this.pushStack(e)}});var Ja,Ka={HTML:"block",BODY:"block"};function La(a,b){var c=n(b.createElement(a)).appendTo(b.body),d=n.css(c[0],"display");return c.detach(),d}function Ma(a){var b=d,c=Ka[a];return c||(c=La(a,b),"none"!==c&&c||(Ja=(Ja||n("